You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2011/12/05 03:32:28 UTC

svn commit: r1210313 - in /hbase/branches/0.90: CHANGES.txt src/main/java/org/apache/hadoop/hbase/io/hfile/InvalidHFileException.java src/main/java/org/apache/hadoop/hbase/regionserver/Store.java

Author: apurtell
Date: Mon Dec  5 02:32:27 2011
New Revision: 1210313

URL: http://svn.apache.org/viewvc?rev=1210313&view=rev
Log:
HBASE-4944. Optionally verify bulk loaded HFiles

Added:
    hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/io/hfile/InvalidHFileException.java
Modified:
    hbase/branches/0.90/CHANGES.txt
    hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java

Modified: hbase/branches/0.90/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/branches/0.90/CHANGES.txt?rev=1210313&r1=1210312&r2=1210313&view=diff
==============================================================================
--- hbase/branches/0.90/CHANGES.txt (original)
+++ hbase/branches/0.90/CHANGES.txt Mon Dec  5 02:32:27 2011
@@ -131,7 +131,7 @@ Release 0.90.5 - Unreleased
                (Jonathan Hsieh)
    HBASE-4806  Fix logging message in HbaseObjectWritable
                (Jonathan Hsieh via todd)
-
+   HBASE-4944  Optionally verify bulk loaded HFiles
 
   NEW FEATURE
    HBASE-4377  [hbck] Offline rebuild .META. from fs data only

Added: hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/io/hfile/InvalidHFileException.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/io/hfile/InvalidHFileException.java?rev=1210313&view=auto
==============================================================================
--- hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/io/hfile/InvalidHFileException.java (added)
+++ hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/io/hfile/InvalidHFileException.java Mon Dec  5 02:32:27 2011
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.io.hfile;
+
+import java.io.IOException;
+
+/**
+ * Thrown when an invalid HFile format is detected
+ */
+public class InvalidHFileException extends IOException {
+  private static final long serialVersionUID = 4660352028739861249L;
+
+  /** constructor */
+  public InvalidHFileException() {
+    super();
+  }
+
+  /**
+   * Constructor
+   * @param s message
+   */
+  public InvalidHFileException(String s) {
+    super(s);
+  }
+}
\ No newline at end of file

Modified: hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java?rev=1210313&r1=1210312&r2=1210313&view=diff
==============================================================================
--- hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java (original)
+++ hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java Mon Dec  5 02:32:27 2011
@@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.io.HeapSi
 import org.apache.hadoop.hbase.io.hfile.Compression;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.io.hfile.InvalidHFileException;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -106,6 +107,7 @@ public class Store implements HeapSize {
   private final String storeNameStr;
   private final boolean inMemory;
   private final int compactionKVMax;
+  private final boolean verifyBulkLoads;
 
   /*
    * List of store files inside this store. This is an immutable list that
@@ -203,6 +205,9 @@ public class Store implements HeapSize {
     this.compactRatio = conf.getFloat("hbase.hstore.compaction.ratio", 1.2F);
     this.compactionKVMax = conf.getInt("hbase.hstore.compaction.kv.max", 10);
 
+    this.verifyBulkLoads = conf.getBoolean("hbase.hstore.bulkload.verify",
+        false);
+
     if (Store.closeCheckInterval == 0) {
       Store.closeCheckInterval = conf.getInt(
           "hbase.hstore.close.check.interval", 10*1000*1000 /* 10 MB */);
@@ -320,9 +325,8 @@ public class Store implements HeapSize {
   }
 
   /**
-   * This throws a WrongRegionException if the bulkHFile does not fit in this
-   * region.
-   *
+   * This throws a WrongRegionException if the HFile does not fit in this
+   * region, or an InvalidHFileException if the HFile is not valid.
    */
   void assertBulkLoadHFileOk(Path srcPath) throws IOException {
     HFile.Reader reader  = null;
@@ -351,6 +355,34 @@ public class Store implements HeapSize {
             "Bulk load file " + srcPath.toString() + " does not fit inside region "
             + this.region);
       }
+
+      if (verifyBulkLoads) {
+        KeyValue prevKV = null;
+        HFileScanner scanner = reader.getScanner(false, false);
+        scanner.seekTo();
+        do {
+          KeyValue kv = scanner.getKeyValue();
+          if (prevKV != null) {
+            if (Bytes.compareTo(prevKV.getBuffer(), prevKV.getRowOffset(),
+                prevKV.getRowLength(), kv.getBuffer(), kv.getRowOffset(),
+                kv.getRowLength()) > 0) {
+              throw new InvalidHFileException("Previous row is greater than"
+                  + " current row: path=" + srcPath + " previous="
+                  + Bytes.toStringBinary(prevKV.getKey()) + " current="
+                  + Bytes.toStringBinary(kv.getKey()));
+            }
+            if (Bytes.compareTo(prevKV.getBuffer(), prevKV.getFamilyOffset(),
+                prevKV.getFamilyLength(), kv.getBuffer(), kv.getFamilyOffset(),
+                kv.getFamilyLength()) != 0) {
+              throw new InvalidHFileException("Previous key had different"
+                  + " family compared to current key: path=" + srcPath
+                  + " previous=" + Bytes.toStringBinary(prevKV.getFamily())
+                  + " current=" + Bytes.toStringBinary(kv.getFamily()));
+            }
+          }
+          prevKV = kv;
+        } while (scanner.next());
+      }
     } finally {
       if (reader != null) reader.close();
     }