You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by la...@apache.org on 2012/09/13 17:44:21 UTC

svn commit: r1384377 - in /hbase/trunk: hbase-common/src/main/java/org/apache/hadoop/hbase/ hbase-server/src/main/java/org/apache/hadoop/hbase/ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ hbase-server/src/test/java/org/apache/hadoo...

Author: larsh
Date: Thu Sep 13 15:44:21 2012
New Revision: 1384377

URL: http://svn.apache.org/viewvc?rev=1384377&view=rev
Log:
HBASE-6769 HRS.multi eats NoSuchColumnFamilyException (Elliott Clark)

Added:
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/FailedSanityCheckException.java
Modified:
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java

Modified: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java?rev=1384377&r1=1384376&r2=1384377&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java (original)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java Thu Sep 13 15:44:21 2012
@@ -39,6 +39,7 @@ public final class HConstants {
   public enum OperationStatusCode {
     NOT_RUN,
     SUCCESS,
+    BAD_FAMILY,
     SANITY_CHECK_FAILURE,
     FAILURE;
   }

Added: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/FailedSanityCheckException.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/FailedSanityCheckException.java?rev=1384377&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/FailedSanityCheckException.java (added)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/FailedSanityCheckException.java Thu Sep 13 15:44:21 2012
@@ -0,0 +1,49 @@
+/**
+ * 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;
+
+/**
+ * Exception thrown if a mutation fails sanity checks.
+ */
+public class FailedSanityCheckException extends DoNotRetryIOException {
+
+  private static final long serialVersionUID = 1788783640409186240L;
+
+  /**
+   * default constructor
+   */
+  public FailedSanityCheckException() {
+    super();
+  }
+
+  /**
+   * @param message
+   */
+  public FailedSanityCheckException(String message) {
+    super(message);
+  }
+
+  /**
+   * @param message
+   * @param cause
+   */
+  public FailedSanityCheckException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+}

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1384377&r1=1384376&r2=1384377&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Thu Sep 13 15:44:21 2012
@@ -73,6 +73,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.DroppedSnapshotException;
+import org.apache.hadoop.hbase.FailedSanityCheckException;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -2155,10 +2156,16 @@ public class HRegion implements HeapSize
           } else {
             prepareDelete((Delete) mutation);
           }
-        } catch (DoNotRetryIOException dnrioe) {
-          LOG.warn("No such column family in batch mutation", dnrioe);
+        } catch (NoSuchColumnFamilyException nscf) {
+          LOG.warn("No such column family in batch mutation", nscf);
           batchOp.retCodeDetails[lastIndexExclusive] = new OperationStatus(
-              OperationStatusCode.SANITY_CHECK_FAILURE, dnrioe.getMessage());
+              OperationStatusCode.BAD_FAMILY, nscf.getMessage());
+          lastIndexExclusive++;
+          continue;
+        } catch (FailedSanityCheckException fsce) {
+          LOG.warn("Batch Mutation did not pass sanity check", fsce);
+          batchOp.retCodeDetails[lastIndexExclusive] = new OperationStatus(
+              OperationStatusCode.SANITY_CHECK_FAILURE, fsce.getMessage());
           lastIndexExclusive++;
           continue;
         }
@@ -2731,7 +2738,7 @@ public class HRegion implements HeapSize
   }
 
   void checkTimestamps(final Map<byte[], List<KeyValue>> familyMap,
-      long now) throws DoNotRetryIOException {
+      long now) throws FailedSanityCheckException {
     if (timestampSlop == HConstants.LATEST_TIMESTAMP) {
       return;
     }
@@ -2740,7 +2747,7 @@ public class HRegion implements HeapSize
       for (KeyValue kv : kvs) {
         // see if the user-side TS is out of range. latest = server-side
         if (!kv.isLatestTimestamp() && kv.getTimestamp() > maxTs) {
-          throw new DoNotRetryIOException("Timestamp for KV out of range "
+          throw new FailedSanityCheckException("Timestamp for KV out of range "
               + kv + " (too.new=" + timestampSlop + ")");
         }
       }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1384377&r1=1384376&r2=1384377&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Thu Sep 13 15:44:21 2012
@@ -64,6 +64,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Chore;
 import org.apache.hadoop.hbase.ClockOutOfSyncException;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.FailedSanityCheckException;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
@@ -3922,10 +3923,27 @@ public class  HRegionServer implements C
 
       OperationStatus codes[] = region.batchMutate(mutationsWithLocks);
       for (i = 0; i < codes.length; i++) {
-        if (codes[i].getOperationStatusCode() != OperationStatusCode.SUCCESS) {
-          result = ResponseConverter.buildActionResult(
-            new DoNotRetryIOException(codes[i].getExceptionMsg()));
-          builder.setResult(i, result);
+        switch (codes[i].getOperationStatusCode()) {
+          case BAD_FAMILY:
+            result = ResponseConverter.buildActionResult(
+                new NoSuchColumnFamilyException(codes[i].getExceptionMsg()));
+            builder.setResult(i, result);
+            break;
+
+          case SANITY_CHECK_FAILURE:
+            result = ResponseConverter.buildActionResult(
+                new FailedSanityCheckException(codes[i].getExceptionMsg()));
+            builder.setResult(i, result);
+            break;
+
+          default:
+            result = ResponseConverter.buildActionResult(
+                new DoNotRetryIOException(codes[i].getExceptionMsg()));
+            builder.setResult(i, result);
+            break;
+
+          case SUCCESS:
+            break;
         }
       }
     } catch (IOException ie) {

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java?rev=1384377&r1=1384376&r2=1384377&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java Thu Sep 13 15:44:21 2012
@@ -66,6 +66,7 @@ import org.apache.hadoop.hbase.io.hfile.
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@@ -3683,6 +3684,25 @@ public class TestFromClientSide {
   }
 
   @Test
+  public void testPutNoCF() throws IOException {
+    final byte[] BAD_FAM = Bytes.toBytes("BAD_CF");
+    final byte[] VAL = Bytes.toBytes(100);
+    HTable table = TEST_UTIL.createTable(Bytes.toBytes("testPutNoCF"), new byte[][]{FAMILY});
+
+    boolean caughtNSCFE = false;
+
+    try {
+      Put p = new Put(ROW);
+      p.add(BAD_FAM, QUALIFIER, VAL);
+      table.put(p);
+    } catch (RetriesExhaustedWithDetailsException e) {
+      caughtNSCFE = e.getCause(0) instanceof NoSuchColumnFamilyException;
+    }
+    assertTrue("Should throw NoSuchColumnFamilyException", caughtNSCFE);
+
+  }
+
+  @Test
   public void testRowsPut() throws IOException {
     final byte[] CONTENTS_FAMILY = Bytes.toBytes("contents");
     final byte[] SMALL_FAMILY = Bytes.toBytes("smallfam");
@@ -4274,6 +4294,8 @@ public class TestFromClientSide {
     }
   }
 
+
+
   @Test
   public void testIncrement() throws Exception {
     LOG.info("Starting testIncrement");

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java?rev=1384377&r1=1384376&r2=1384377&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java Thu Sep 13 15:44:21 2012
@@ -37,6 +37,7 @@ import org.apache.hadoop.fs.FSDataOutput
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.FailedSanityCheckException;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestCase;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -575,7 +576,7 @@ public class TestHRegion extends HBaseTe
       boolean exception = false;
       try {
         this.region.put(p);
-      } catch (DoNotRetryIOException e) {
+      } catch (NoSuchColumnFamilyException e) {
         exception = true;
       }
       assertTrue(exception);
@@ -616,7 +617,7 @@ public class TestHRegion extends HBaseTe
       codes = this.region.put(puts);
       assertEquals(10, codes.length);
       for (int i = 0; i < 10; i++) {
-        assertEquals((i == 5) ? OperationStatusCode.SANITY_CHECK_FAILURE :
+        assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY :
           OperationStatusCode.SUCCESS, codes[i].getOperationStatusCode());
       }
       assertEquals(1, HLog.getSyncTime().count);
@@ -654,7 +655,7 @@ public class TestHRegion extends HBaseTe
       assertEquals(1, HLog.getSyncTime().count);
       codes = retFromThread.get();
       for (int i = 0; i < 10; i++) {
-        assertEquals((i == 5) ? OperationStatusCode.SANITY_CHECK_FAILURE :
+        assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY :
           OperationStatusCode.SUCCESS, codes[i].getOperationStatusCode());
       }
   
@@ -671,7 +672,7 @@ public class TestHRegion extends HBaseTe
       codes = region.batchMutate(putsAndLocks.toArray(new Pair[0]));
       LOG.info("...performed put");
       for (int i = 0; i < 10; i++) {
-        assertEquals((i == 5) ? OperationStatusCode.SANITY_CHECK_FAILURE :
+        assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY :
           OperationStatusCode.SUCCESS, codes[i].getOperationStatusCode());
       }
       // Make sure we didn't do an extra batch
@@ -687,6 +688,45 @@ public class TestHRegion extends HBaseTe
     }
   }
 
+  public void testBatchPutWithTsSlop() throws Exception {
+    byte[] b = Bytes.toBytes(getName());
+    byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
+    byte[] qual = Bytes.toBytes("qual");
+    byte[] val = Bytes.toBytes("val");
+
+    HBaseConfiguration conf = new HBaseConfiguration();
+
+
+    // add data with a timestamp that is too recent for range. Ensure assert
+    conf.setInt("hbase.hregion.keyvalue.timestamp.slop.millisecs", 1000);
+    this.region = initHRegion(b, getName(), conf, cf);
+
+    try{
+      HLog.getSyncTime(); // clear counter from prior tests
+      assertEquals(0, HLog.getSyncTime().count);
+
+      final Put[] puts = new Put[10];
+      for (int i = 0; i < 10; i++) {
+        puts[i] = new Put(Bytes.toBytes("row_" + i), Long.MAX_VALUE - 100);
+        puts[i].add(cf, qual, val);
+      }
+
+      OperationStatus[] codes = this.region.put(puts);
+      assertEquals(10, codes.length);
+      for (int i = 0; i < 10; i++) {
+        assertEquals(OperationStatusCode.SANITY_CHECK_FAILURE, codes[i]
+            .getOperationStatusCode());
+      }
+      assertEquals(0, HLog.getSyncTime().count);
+
+
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
+    }
+
+  }
+
   //////////////////////////////////////////////////////////////////////////////
   // checkAndMutate tests
   //////////////////////////////////////////////////////////////////////////////
@@ -1219,6 +1259,7 @@ public class TestHRegion extends HBaseTe
 
   }
 
+
   /**
    * Tests that there is server-side filtering for invalid timestamp upper
    * bound. Note that the timestamp lower bound is automatically handled for us
@@ -1234,6 +1275,7 @@ public class TestHRegion extends HBaseTe
     // add data with a timestamp that is too recent for range. Ensure assert
     conf.setInt("hbase.hregion.keyvalue.timestamp.slop.millisecs", 1000);
     this.region = initHRegion(tableName, method, conf, families);
+    boolean caughtExcep = false;
     try {
       try {
         // no TS specified == use latest. should not error
@@ -1244,9 +1286,11 @@ public class TestHRegion extends HBaseTe
             System.currentTimeMillis() + 2000,
             Bytes.toBytes("value")), false);
         fail("Expected IOE for TS out of configured timerange");
-      } catch (DoNotRetryIOException ioe) {
+      } catch (FailedSanityCheckException ioe) {
         LOG.debug("Received expected exception", ioe);
+        caughtExcep = true;
       }
+      assertTrue("Should catch FailedSanityCheckException", caughtExcep);
     } finally {
       HRegion.closeHRegion(this.region);
       this.region = null;