You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2012/09/29 06:06:38 UTC

svn commit: r1391750 - in /hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/io/hfile: HFile.java HFileBlockIndex.java

Author: tedyu
Date: Sat Sep 29 04:06:38 2012
New Revision: 1391750

URL: http://svn.apache.org/viewvc?rev=1391750&view=rev
Log:
HBASE-6871 HFileBlockIndex Write Error in HFile V2 due to incorrect split into intermediate index blocks (Mikhail)


Modified:
    hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
    hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java?rev=1391750&r1=1391749&r2=1391750&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java (original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java Sat Sep 29 04:06:38 2012
@@ -40,6 +40,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.KeyComparator;
 import org.apache.hadoop.hbase.io.HbaseMapWritable;
@@ -49,6 +50,7 @@ import org.apache.hadoop.hbase.util.FSUt
 import org.apache.hadoop.io.RawComparator;
 import org.apache.hadoop.io.Writable;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 
 /**
@@ -241,6 +243,13 @@ public class HFile {
   public static abstract class WriterFactory {
     protected Configuration conf;
     protected CacheConfig cacheConf;
+    protected FileSystem fs;
+    protected Path path;
+    protected FSDataOutputStream ostream;
+    protected int blockSize = HColumnDescriptor.DEFAULT_BLOCKSIZE;
+    protected Compression.Algorithm compression =
+      HFile.DEFAULT_COMPRESSION_ALGORITHM;
+    protected KeyComparator comparator;
 
     WriterFactory(Configuration conf, CacheConfig cacheConf) {
       this.conf = conf;
@@ -265,6 +274,27 @@ public class HFile {
     public abstract Writer createWriter(final FSDataOutputStream ostream,
         final int blockSize, final Compression.Algorithm compress,
         final KeyComparator c) throws IOException;
+
+    public WriterFactory withBlockSize(int blockSize) {
+      this.blockSize = blockSize;
+      return this;
+    }
+    public WriterFactory withPath(FileSystem fs, Path path) {
+      Preconditions.checkNotNull(fs);
+      Preconditions.checkNotNull(path);
+      this.fs = fs;
+      this.path = path;
+      return this;
+    }
+
+    public Writer create() throws IOException {
+      if ((path != null ? 1 : 0) + (ostream != null ? 1 : 0) != 1) {
+        throw new AssertionError("Please specify exactly one of " +
+            "filesystem/path or path");
+      }
+      return createWriter(fs, path, blockSize,
+          compression, comparator);
+    }
   }
 
   /** The configuration key for HFile version to use for new files */

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java?rev=1391750&r1=1391749&r2=1391750&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java (original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java Sat Sep 29 04:06:38 2012
@@ -718,7 +718,7 @@ public class HFileBlockIndex {
      * @throws IOException
      */
     public long writeIndexBlocks(FSDataOutputStream out) throws IOException {
-      if (curInlineChunk.getNumEntries() != 0) {
+      if (curInlineChunk != null && curInlineChunk.getNumEntries() != 0) {
         throw new IOException("Trying to write a multi-level block index, " +
             "but are " + curInlineChunk.getNumEntries() + " entries in the " +
             "last inline chunk.");
@@ -729,9 +729,11 @@ public class HFileBlockIndex {
       byte[] midKeyMetadata = numLevels > 1 ? rootChunk.getMidKeyMetadata()
           : null;
 
-      while (rootChunk.getRootSize() > maxChunkSize) {
-        rootChunk = writeIntermediateLevel(out, rootChunk);
-        numLevels += 1;
+      if (curInlineChunk != null) {
+        while (rootChunk.getRootSize() > maxChunkSize) {
+          rootChunk = writeIntermediateLevel(out, rootChunk);
+          numLevels += 1;
+        }
       }
 
       // write the root level
@@ -890,11 +892,18 @@ public class HFileBlockIndex {
      */
     @Override
     public boolean shouldWriteBlock(boolean closing) {
-      if (singleLevelOnly)
+      if (singleLevelOnly) {
         throw new UnsupportedOperationException(INLINE_BLOCKS_NOT_ALLOWED);
+      }
 
-      if (curInlineChunk.getNumEntries() == 0)
+      if (curInlineChunk == null) {
+        throw new IllegalStateException("curInlineChunk is null; has shouldWriteBlock been " +
+            "called with closing=true and then called again?");
+      }
+
+      if (curInlineChunk.getNumEntries() == 0) {
         return false;
+      }
 
       // We do have some entries in the current inline chunk.
       if (closing) {
@@ -904,7 +913,7 @@ public class HFileBlockIndex {
 
           expectNumLevels(1);
           rootChunk = curInlineChunk;
-          curInlineChunk = new BlockIndexChunk();
+          curInlineChunk = null;  // Disallow adding any more index entries.
           return false;
         }