Repository: hbase
Updated Branches:
  refs/heads/branch-1 deeda60c6 -> 64bef6755


HBASE-18617 FuzzyRowKeyFilter should not modify the filter pairs (vinisha)


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/64bef675
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/64bef675
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/64bef675

Branch: refs/heads/branch-1
Commit: 64bef67558244d459661a3da312a747d100ea788
Parents: deeda60
Author: tedyu <yuzhih...@gmail.com>
Authored: Fri Aug 18 10:02:06 2017 -0700
Committer: tedyu <yuzhih...@gmail.com>
Committed: Fri Aug 18 10:02:06 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/filter/FuzzyRowFilter.java     | 21 ++++++++++-----
 .../filter/TestFuzzyRowFilterEndToEnd.java      | 27 +++++++++++++-------
 2 files changed, 32 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/64bef675/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
index 838e4bd..88fc17b 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
@@ -77,20 +77,27 @@ public class FuzzyRowFilter extends FilterBase {
   private RowTracker tracker;
 
   public FuzzyRowFilter(List<Pair<byte[], byte[]>> fuzzyKeysData) {
-    Pair<byte[], byte[]> p;
-    for (int i = 0; i < fuzzyKeysData.size(); i++) {
-      p = fuzzyKeysData.get(i);
-      if (p.getFirst().length != p.getSecond().length) {
+    List<Pair<byte[], byte[]>> fuzzyKeyDataCopy = new 
ArrayList<>(fuzzyKeysData.size());
+
+    for (Pair<byte[], byte[]> aFuzzyKeysData : fuzzyKeysData) {
+      if (aFuzzyKeysData.getFirst().length != 
aFuzzyKeysData.getSecond().length) {
         Pair<String, String> readable =
-            new Pair<String, String>(Bytes.toStringBinary(p.getFirst()), 
Bytes.toStringBinary(p
-                .getSecond()));
+          new Pair<>(Bytes.toStringBinary(aFuzzyKeysData.getFirst()), 
Bytes.toStringBinary(aFuzzyKeysData.getSecond()));
         throw new IllegalArgumentException("Fuzzy pair lengths do not match: " 
+ readable);
       }
+
+      Pair<byte[], byte[]> p = new Pair<>();
+      // create a copy of pair bytes so that they are not modified by the 
filter.
+      p.setFirst(Arrays.copyOf(aFuzzyKeysData.getFirst(), 
aFuzzyKeysData.getFirst().length));
+      p.setSecond(Arrays.copyOf(aFuzzyKeysData.getSecond(), 
aFuzzyKeysData.getSecond().length));
+
       // update mask ( 0 -> -1 (0xff), 1 -> 2)
       p.setSecond(preprocessMask(p.getSecond()));
       preprocessSearchKey(p);
+
+      fuzzyKeyDataCopy.add(p);
     }
-    this.fuzzyKeysData = fuzzyKeysData;
+    this.fuzzyKeysData = fuzzyKeyDataCopy;
     this.tracker = new RowTracker();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/64bef675/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilterEndToEnd.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilterEndToEnd.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilterEndToEnd.java
index fa738b2..fa0b77d 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilterEndToEnd.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilterEndToEnd.java
@@ -17,13 +17,7 @@
  */
 package org.apache.hadoop.hbase.filter;
 
-import static org.junit.Assert.assertEquals;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-
+import com.google.common.collect.Lists;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -33,11 +27,11 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.filter.FilterList.Operator;
 import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
@@ -53,7 +47,13 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
 
 /**
  */
@@ -136,6 +136,11 @@ public class TestFuzzyRowFilterEndToEnd {
     List<Pair<byte[], byte[]>> data = new ArrayList<Pair<byte[], byte[]>>();
     byte[] fuzzyKey = Bytes.toBytesBinary("\\x9B\\x00\\x044e");
     byte[] mask = new byte[] { 0, 0, 0, 0, 0 };
+
+    // copy the fuzzy key and mask to test HBASE-18617
+    byte[] copyFuzzyKey = Arrays.copyOf(fuzzyKey, fuzzyKey.length);
+    byte[] copyMask = Arrays.copyOf(mask, mask.length);
+
     data.add(new Pair<byte[], byte[]>(fuzzyKey, mask));
     FuzzyRowFilter filter = new FuzzyRowFilter(data);
 
@@ -148,6 +153,10 @@ public class TestFuzzyRowFilterEndToEnd {
       total++;
     }
     assertEquals(2, total);
+
+    assertEquals(true, Arrays.equals(copyFuzzyKey, fuzzyKey));
+    assertEquals(true, Arrays.equals(copyMask, mask));
+
     TEST_UTIL.deleteTable(TableName.valueOf(table));
   }
 

Reply via email to