From 22ecb7a7b5da72e86951e88510f53efd19a503a0 Mon Sep 17 00:00:00 2001 From: tedyu Date: Fri, 18 Aug 2017 09:12:16 -0700 Subject: [PATCH] HBASE-18617 FuzzyRowKeyFilter should not modify the filter pairs (vinisha) --- .../hadoop/hbase/filter/FuzzyRowFilter.java | 22 ++++++++++----- .../filter/TestFuzzyRowFilterEndToEnd.java | 27 ++++++++++++------- 2 files changed, 33 insertions(+), 16 deletions(-) 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 20c66566516..2aa4857a34d 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 @@ -76,23 +76,31 @@ public class FuzzyRowFilter extends FilterBase { private RowTracker tracker; public FuzzyRowFilter(List> fuzzyKeysData) { - Pair p; - for (int i = 0; i < fuzzyKeysData.size(); i++) { - p = fuzzyKeysData.get(i); - if (p.getFirst().length != p.getSecond().length) { + List> fuzzyKeyDataCopy = new ArrayList<>(fuzzyKeysData.size()); + + for (Pair aFuzzyKeysData : fuzzyKeysData) { + if (aFuzzyKeysData.getFirst().length != aFuzzyKeysData.getSecond().length) { Pair readable = - new Pair<>(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 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(); } + private void preprocessSearchKey(Pair p) { if (!UNSAFE_UNALIGNED) { // do nothing 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 686c2a29c0b..b043e073ac6 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,6 @@ */ 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 org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -42,6 +35,7 @@ import org.apache.hadoop.hbase.filter.FilterList.Operator; import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.RegionScanner; +import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; import org.apache.hadoop.hbase.testclassification.FilterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -53,10 +47,16 @@ import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; - -import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; import org.junit.rules.TestName; +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; + /** */ @Category({ FilterTests.class, MediumTests.class }) @@ -140,6 +140,11 @@ public class TestFuzzyRowFilterEndToEnd { List> data = new ArrayList<>(); 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<>(fuzzyKey, mask)); FuzzyRowFilter filter = new FuzzyRowFilter(data); @@ -152,6 +157,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(name.getMethodName())); }