hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From mberto...@apache.org
Subject svn commit: r1466418 - in /hbase/branches/0.94/src: main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java
Date Wed, 10 Apr 2013 10:36:56 GMT
Author: mbertozzi
Date: Wed Apr 10 10:36:56 2013
New Revision: 1466418

URL: http://svn.apache.org/r1466418
Log:
HBASE-8313 Add Bloom filter testing for HFileOutputFormat

Modified:
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java?rev=1466418&r1=1466417&r2=1466418&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
(original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
Wed Apr 10 10:36:56 2013
@@ -123,7 +123,7 @@ public class StoreFile extends SchemaCon
       Bytes.toBytes("EXCLUDE_FROM_MINOR_COMPACTION");
 
   /** Bloom filter Type in FileInfo */
-  static final byte[] BLOOM_FILTER_TYPE_KEY =
+  public static final byte[] BLOOM_FILTER_TYPE_KEY =
       Bytes.toBytes("BLOOM_FILTER_TYPE");
 
   /** Delete Family Count in FileInfo */

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java?rev=1466418&r1=1466417&r2=1466418&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java Wed
Apr 10 10:36:56 2013
@@ -61,6 +61,7 @@ import org.apache.hadoop.hbase.io.encodi
 import org.apache.hadoop.hbase.io.hfile.ChecksumUtil;
 import org.apache.hadoop.hbase.io.hfile.Compression;
 import org.apache.hadoop.hbase.io.hfile.Compression.Algorithm;
+import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.ServerManager;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -2257,4 +2258,56 @@ public class HBaseTestingUtility {
     return result;
   }
 
+  /**
+   * Create a set of column descriptors with the combination of compression,
+   * encoding, bloom codecs available.
+   * @return the list of column descriptors
+   */
+  public static List<HColumnDescriptor> generateColumnDescriptors() {
+    return generateColumnDescriptors("");
+  }
+
+  /**
+   * Create a set of column descriptors with the combination of compression,
+   * encoding, bloom codecs available.
+   * @param prefix family names prefix
+   * @return the list of column descriptors
+   */
+  public static List<HColumnDescriptor> generateColumnDescriptors(final String prefix)
{
+    List<HColumnDescriptor> htds = new ArrayList<HColumnDescriptor>();
+    long familyId = 0;
+    for (Compression.Algorithm compressionType: getSupportedCompressionAlgorithms()) {
+      for (DataBlockEncoding encodingType: DataBlockEncoding.values()) {
+        for (StoreFile.BloomType bloomType: StoreFile.BloomType.values()) {
+          String name = String.format("%s-cf-!@#&-%d!@#", prefix, familyId);
+          HColumnDescriptor htd = new HColumnDescriptor(name);
+          htd.setCompressionType(compressionType);
+          htd.setDataBlockEncoding(encodingType);
+          htd.setBloomFilterType(bloomType);
+          htds.add(htd);
+          familyId++;
+        }
+      }
+    }
+    return htds;
+  }
+
+  /**
+   * Get supported compression algorithms.
+   * @return supported compression algorithms.
+   */
+  public static Compression.Algorithm[] getSupportedCompressionAlgorithms() {
+    String[] allAlgos = HFile.getSupportedCompressionAlgorithms();
+    List<Compression.Algorithm> supportedAlgos = new ArrayList<Compression.Algorithm>();
+    for (String algoName : allAlgos) {
+      try {
+        Compression.Algorithm algo = Compression.getCompressionAlgorithmByName(algoName);
+        algo.getCompressor();
+        supportedAlgos.add(algo);
+      } catch (Throwable t) {
+        // this algo is not available
+      }
+    }
+    return supportedAlgos.toArray(new Compression.Algorithm[0]);
+  }
 }

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java?rev=1466418&r1=1466417&r2=1466418&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java
(original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java
Wed Apr 10 10:36:56 2013
@@ -33,6 +33,8 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeSet;
 import java.util.concurrent.Callable;
 import java.util.Random;
 
@@ -57,6 +59,7 @@ import org.apache.hadoop.hbase.io.hfile.
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
 import org.apache.hadoop.hbase.regionserver.Store;
+import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.TimeRangeTracker;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -553,29 +556,23 @@ public class TestHFileOutputFormat  {
   }
 
   /**
-   * Test that {@link HFileOutputFormat} RecordWriter uses compression settings
-   * from the column family descriptor
+   * Test that {@link HFileOutputFormat} RecordWriter uses compression and
+   * bloom filter settings from the column family descriptor
    */
   @Test
-  public void testColumnFamilyCompression() throws Exception {
+  public void testColumnFamilySettings() throws Exception {
     Configuration conf = new Configuration(this.util.getConfiguration());
     RecordWriter<ImmutableBytesWritable, KeyValue> writer = null;
     TaskAttemptContext context = null;
-    Path dir =
-        util.getDataTestDir("testColumnFamilyCompression");
+    Path dir = util.getDataTestDir("testColumnFamilySettings");
 
+    // Setup table descriptor
     HTable table = Mockito.mock(HTable.class);
-
-    Map<String, Compression.Algorithm> configuredCompression =
-      new HashMap<String, Compression.Algorithm>();
-    Compression.Algorithm[] supportedAlgos = getSupportedCompressionAlgorithms();
-
-    int familyIndex = 0;
-    for (byte[] family : FAMILIES) {
-      configuredCompression.put(Bytes.toString(family),
-                                supportedAlgos[familyIndex++ % supportedAlgos.length]);
+    HTableDescriptor htd = new HTableDescriptor(TABLE_NAME);
+    Mockito.doReturn(htd).when(table).getTableDescriptor();
+    for (HColumnDescriptor hcd: this.util.generateColumnDescriptors()) {
+      htd.addFamily(hcd);
     }
-    setupMockColumnFamilies(table, configuredCompression);
 
     // set up the table to return some mock keys
     setupMockStartKeys(table);
@@ -594,75 +591,45 @@ public class TestHFileOutputFormat  {
       writer = hof.getRecordWriter(context);
 
       // write out random rows
-      writeRandomKeyValues(writer, context, ROWSPERSPLIT);
+      writeRandomKeyValues(writer, context, htd.getFamiliesKeys(), ROWSPERSPLIT);
       writer.close(context);
 
       // Make sure that a directory was created for every CF
-      FileSystem fileSystem = dir.getFileSystem(conf);
+      FileSystem fs = dir.getFileSystem(conf);
 
       // commit so that the filesystem has one directory per column family
       hof.getOutputCommitter(context).commitTask(context);
       hof.getOutputCommitter(context).commitJob(context);
-      for (byte[] family : FAMILIES) {
-        String familyStr = new String(family);
-        boolean found = false;
-        for (FileStatus f : fileSystem.listStatus(dir)) {
-
-          if (Bytes.toString(family).equals(f.getPath().getName())) {
-            // we found a matching directory
-            found = true;
-
-            // verify that the compression on this file matches the configured
-            // compression
-            Path dataFilePath = fileSystem.listStatus(f.getPath())[0].getPath();
-            Reader reader = HFile.createReader(fileSystem, dataFilePath,
-                new CacheConfig(conf));
-            reader.loadFileInfo();
-            assertEquals("Incorrect compression used for column family " + familyStr
-                         + "(reader: " + reader + ")",
-                         configuredCompression.get(familyStr), reader.getCompressionAlgorithm());
-            break;
-          }
-        }
-
-        if (!found) {
-          fail("HFile for column family " + familyStr + " not found");
-        }
+      FileStatus[] families = FSUtils.listStatus(fs, dir, new FSUtils.FamilyDirFilter(fs));
+      assertEquals(htd.getFamilies().size(), families.length);
+      for (FileStatus f : families) {
+        String familyStr = f.getPath().getName();
+        HColumnDescriptor hcd = htd.getFamily(Bytes.toBytes(familyStr));
+        // verify that the compression on this file matches the configured
+        // compression
+        Path dataFilePath = fs.listStatus(f.getPath())[0].getPath();
+        Reader reader = HFile.createReader(fs, dataFilePath, new CacheConfig(conf));
+        Map<byte[], byte[]> fileInfo = reader.loadFileInfo();
+
+        byte[] bloomFilter = fileInfo.get(StoreFile.BLOOM_FILTER_TYPE_KEY);
+        if (bloomFilter == null) bloomFilter = Bytes.toBytes("NONE");
+        assertEquals("Incorrect bloom filter used for column family " + familyStr +
+          "(reader: " + reader + ")",
+          hcd.getBloomFilterType(), StoreFile.BloomType.valueOf(Bytes.toString(bloomFilter)));
+        assertEquals("Incorrect compression used for column family " + familyStr +
+          "(reader: " + reader + ")", hcd.getCompression(), reader.getCompressionAlgorithm());
       }
-
     } finally {
       dir.getFileSystem(conf).delete(dir, true);
     }
   }
 
-
-  /**
-   * @return
-   */
-  private Compression.Algorithm[] getSupportedCompressionAlgorithms() {
-    String[] allAlgos = HFile.getSupportedCompressionAlgorithms();
-    List<Compression.Algorithm> supportedAlgos = Lists.newArrayList();
-
-    for (String algoName : allAlgos) {
-      try {
-        Compression.Algorithm algo = Compression.getCompressionAlgorithmByName(algoName);
-        algo.getCompressor();
-        supportedAlgos.add(algo);
-      }catch (Exception e) {
-        // this algo is not available
-      }
-    }
-
-    return supportedAlgos.toArray(new Compression.Algorithm[0]);
-  }
-
-
   /**
    * Write random values to the writer assuming a table created using
    * {@link #FAMILIES} as column family descriptors
    */
-  private void writeRandomKeyValues(RecordWriter<ImmutableBytesWritable, KeyValue>
writer, TaskAttemptContext context,
-      int numRows)
+  private void writeRandomKeyValues(RecordWriter<ImmutableBytesWritable, KeyValue>
writer,
+      TaskAttemptContext context, Set<byte[]> families, int numRows)
       throws IOException, InterruptedException {
     byte keyBytes[] = new byte[Bytes.SIZEOF_INT];
     int valLength = 10;
@@ -678,7 +645,7 @@ public class TestHFileOutputFormat  {
       random.nextBytes(valBytes);
       ImmutableBytesWritable key = new ImmutableBytesWritable(keyBytes);
 
-      for (byte[] family : TestHFileOutputFormat.FAMILIES) {
+      for (byte[] family : families) {
         KeyValue kv = new KeyValue(keyBytes, family,
             PerformanceEvaluation.QUALIFIER_NAME, valBytes);
         writer.write(key, kv);



Mime
View raw message