You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by li...@apache.org on 2014/02/25 20:18:31 UTC

svn commit: r1571789 - in /hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase: HBaseTestingUtility.java regionserver/TestSeekOptimizations.java

Author: liyin
Date: Tue Feb 25 19:18:30 2014
New Revision: 1571789

URL: http://svn.apache.org/r1571789
Log:
[HBASE-10538] Fix mis-use of Scan in TestSeekOptimizations

Author: daviddeng

Summary: Make endRow always exclusive.

Test Plan: TestSeekOptimizations

Reviewers: manukranthk, liyintang, rshroff

Reviewed By: rshroff

CC: hbase-eng@

Differential Revision: https://phabricator.fb.com/D1180511

Task ID: 3761830

Modified:
    hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
    hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestSeekOptimizations.java

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java?rev=1571789&r1=1571788&r2=1571789&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java Tue Feb 25 19:18:30 2014
@@ -46,6 +46,8 @@ import java.util.Random;
 import java.util.Set;
 import java.util.UUID;
 
+import junit.framework.Assert;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Jdk14Logger;
@@ -79,8 +81,14 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.regionserver.MultiVersionConsistencyControl;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
-import org.apache.hadoop.hbase.util.*;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.RegionSplitter;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -122,7 +130,7 @@ public class HBaseTestingUtility {
 
   /** The root directory for all mini-cluster test data for this testing utility instance. */
   private File clusterTestBuildDir = null;
-  
+
   /** If there is a mini cluster running for this testing utility instance. */
   private boolean miniClusterRunning;
 
@@ -139,7 +147,7 @@ public class HBaseTestingUtility {
    * Default parent directory for test output.
    */
   public static final String DEFAULT_TEST_DIRECTORY = "target/build/data";
-  
+
   /** Filesystem URI used for map-reduce mini-cluster setup */
   private static String fsURI;
 
@@ -283,7 +291,7 @@ public class HBaseTestingUtility {
   /**
    * Shuts down instance created by call to {@link #startMiniDFSCluster(int)}
    * or does nothing.
-   * @throws IOException 
+   * @throws IOException
    * @throws Exception
    */
   public void shutdownMiniDFSCluster() throws IOException {
@@ -418,7 +426,7 @@ public class HBaseTestingUtility {
 
     // Don't leave here till we've done a successful scan of the .META.
     HTable t = null;
-    for (int i = 0; i < 10; ++i) { 
+    for (int i = 0; i < 10; ++i) {
       try {
         t = new HTable(this.conf, HConstants.META_TABLE_NAME);
         for (Result result : t.getScanner(new Scan())) {
@@ -984,7 +992,7 @@ public class HBaseTestingUtility {
     LOG.info("Starting mini mapreduce cluster...");
     // These are needed for the new and improved Map/Reduce framework
     Configuration c = getConfiguration();
-    
+
     setupClusterTestBuildDir();
     createDirsAndSetProperties();
 
@@ -1426,10 +1434,13 @@ REGION_LOOP:
     final int aLen = actual.size();
     final int minLen = Math.min(eLen, aLen);
 
-    int i;
-    for (i = 0; i < minLen
-        && KeyValue.COMPARATOR.compare(expected.get(i), actual.get(i)) == 0;
-        ++i) {}
+    int i = 0;
+    while (i < minLen) {
+      if (KeyValue.COMPARATOR.compare(expected.get(i), actual.get(i)) != 0) {
+        break;
+      }
+      i++;
+    }
 
     if (additionalMsg == null) {
       additionalMsg = "";
@@ -1439,10 +1450,9 @@ REGION_LOOP:
     }
 
     if (eLen != aLen || i != minLen) {
-      throw new AssertionError(
-          "Expected and actual KV arrays differ at position " + i + ": " +
-          safeGetAsStr(expected, i) + " (length " + eLen +") vs. " +
-          safeGetAsStr(actual, i) + " (length " + aLen + ")" + additionalMsg);
+      Assert.failNotEquals("KeyValue at position " + i + additionalMsg,
+          safeGetAsStr(expected, i) + " (length " + eLen + ")",
+          safeGetAsStr(actual, i) + " (length " + aLen + ")");
     }
   }
 
@@ -1550,7 +1560,7 @@ REGION_LOOP:
         + new Random().nextInt(MAX_RANDOM_PORT - MIN_RANDOM_PORT);
   }
 
-  /** 
+  /**
    * Returns a random free port and marks that port as taken. Not thread-safe. Expected to be
    * called from single-threaded test setup code/
    */
@@ -1687,14 +1697,14 @@ REGION_LOOP:
   }
 
   /**
-   * Sets the current thread name to the caller's method name. 
+   * Sets the current thread name to the caller's method name.
    */
   public static void setThreadNameFromMethod() {
     logMethodEntryAndSetThreadName(new Throwable().getStackTrace()[1].getMethodName());
   }
 
   /**
-   * Sets the current thread name to the caller's caller's method name. 
+   * Sets the current thread name to the caller's caller's method name.
    */
   public static void setThreadNameFromCallerMethod() {
     logMethodEntryAndSetThreadName(new Throwable().getStackTrace()[2].getMethodName());

Modified: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestSeekOptimizations.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestSeekOptimizations.java?rev=1571789&r1=1571788&r2=1571789&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestSeekOptimizations.java (original)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/regionserver/TestSeekOptimizations.java Tue Feb 25 19:18:30 2014
@@ -20,7 +20,8 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import static org.apache.hadoop.hbase.HBaseTestingUtility.assertKVListsEqual;
-import static org.apache.hadoop.hbase.regionserver.TestMultiColumnScanner.*;
+import static org.apache.hadoop.hbase.regionserver.TestMultiColumnScanner.FAMILY;
+import static org.apache.hadoop.hbase.regionserver.TestMultiColumnScanner.FAMILY_BYTES;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
@@ -44,9 +45,7 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.hfile.Compression;
-import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.After;
 import org.junit.Before;
@@ -198,13 +197,7 @@ public class TestSeekOptimizations {
     }
     scan.setMaxVersions(maxVersions);
     scan.setStartRow(rowBytes(startRow));
-
-    // Adjust for the fact that for multi-row queries the end row is exclusive.
-    {
-      final byte[] scannerStopRow =
-          rowBytes(endRow + (startRow != endRow ? 1 : 0));
-      scan.setStopRow(scannerStopRow);
-    }
+    scan.setStopRow(rowBytes(endRow < 0 ? endRow : (endRow + 1)));
 
     final long initialSeekCount = StoreFileScanner.getSeekCount();
     final InternalScanner scanner = region.getScanner(scan);