diff --git dev-support/test-patch.properties dev-support/test-patch.properties
index 490980a..94ce5a8 100644
--- dev-support/test-patch.properties
+++ dev-support/test-patch.properties
@@ -19,8 +19,9 @@ MAVEN_OPTS="${MAVEN_OPTS:-"-Xmx3100M"}"
# Please update the per-module test-patch.properties if you update this file.
OK_RELEASEAUDIT_WARNINGS=0
-# Allow two warnings. Javadoc complains about sun.misc.Unsafe use. See HBASE-7457
-OK_JAVADOC_WARNINGS=3
+# Allow four warnings. Javadoc complains about sun.misc.Unsafe use.
+# See HBASE-7457, HBASE-13761
+OK_JAVADOC_WARNINGS=4
MAX_LINE_LENGTH=100
diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
index 9b234de..2b835ef 100644
--- hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
+++ hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
@@ -1,25 +1,19 @@
/**
- * 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.
+ * 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.filter;
-import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
import java.util.List;
import org.apache.hadoop.hbase.Cell;
@@ -32,37 +26,28 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.UnsafeAccess;
import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.InvalidProtocolBufferException;
/**
- * Filters data based on fuzzy row key. Performs fast-forwards during scanning.
- * It takes pairs (row key, fuzzy info) to match row keys. Where fuzzy info is
- * a byte array with 0 or 1 as its values:
+ * This is optimized version of a standard FuzzyRowFilter Filters data based on fuzzy row key.
+ * Performs fast-forwards during scanning. It takes pairs (row key, fuzzy info) to match row keys.
+ * Where fuzzy info is a byte array with 0 or 1 as its values:
*
- *
- * 0 - means that this byte in provided row key is fixed, i.e. row key's byte at same position
- * must match
- *
- *
- * 1 - means that this byte in provided row key is NOT fixed, i.e. row key's byte at this
- * position can be different from the one in provided row key
- *
+ *
0 - means that this byte in provided row key is fixed, i.e. row key's byte at same position
+ * must match
+ *
1 - means that this byte in provided row key is NOT fixed, i.e. row key's byte at this
+ * position can be different from the one in provided row key
*
- *
- *
- * Example:
- * Let's assume row key format is userId_actionId_year_month. Length of userId is fixed
- * and is 4, length of actionId is 2 and year and month are 4 and 2 bytes long respectively.
- *
- * Let's assume that we need to fetch all users that performed certain action (encoded as "99")
- * in Jan of any year. Then the pair (row key, fuzzy info) would be the following:
- * row key = "????_99_????_01" (one can use any value instead of "?")
- * fuzzy info = "\x01\x01\x01\x01\x00\x00\x00\x00\x01\x01\x01\x01\x00\x00\x00"
- *
- * I.e. fuzzy info tells the matching mask is "????_99_????_01", where at ? can be any value.
- *
+ * Example: Let's assume row key format is userId_actionId_year_month. Length of userId is fixed and
+ * is 4, length of actionId is 2 and year and month are 4 and 2 bytes long respectively. Let's
+ * assume that we need to fetch all users that performed certain action (encoded as "99") in Jan of
+ * any year. Then the pair (row key, fuzzy info) would be the following: row key = "????_99_????_01"
+ * (one can use any value instead of "?") fuzzy info =
+ * "\x01\x01\x01\x01\x00\x00\x00\x00\x01\x01\x01\x01\x00\x00\x00" I.e. fuzzy info tells the matching
+ * mask is "????_99_????_01", where at ? can be any value.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@@ -70,81 +55,182 @@ public class FuzzyRowFilter extends FilterBase {
private List> fuzzyKeysData;
private boolean done = false;
+ /**
+ * The index of a last successfully found matching fuzzy string (in fuzzyKeysData). We will start
+ * matching next KV with this one. If they do not match then we will return back to the one-by-one
+ * iteration over fuzzyKeysData.
+ */
+ private int lastFoundIndex = -1;
+
+ /**
+ * Row tracker (keeps all next rows after SEEK_NEXT_USING_HINT was returned)
+ */
+ 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) {
- Pair readable = new Pair(
- Bytes.toStringBinary(p.getFirst()),
- Bytes.toStringBinary(p.getSecond()));
+ Pair readable =
+ new Pair(Bytes.toStringBinary(p.getFirst()), Bytes.toStringBinary(p
+ .getSecond()));
throw new IllegalArgumentException("Fuzzy pair lengths do not match: " + readable);
}
+ // update mask ( 0 -> -1 (0xff), 1 -> 0)
+ p.setSecond(preprocessMask(p.getSecond()));
+ preprocessSearchKey(p);
}
this.fuzzyKeysData = fuzzyKeysData;
+ this.tracker = new RowTracker();
}
- @Override
- public boolean filterRowKey(Cell cell) throws IOException {
- // Impl in FilterBase might do unnecessary copy for Off heap backed Cells.
- return false;
+ private void preprocessSearchKey(Pair p) {
+ if (UnsafeAccess.isAvailable() == false) {
+ // do nothing
+ return;
+ }
+ byte[] key = p.getFirst();
+ byte[] mask = p.getSecond();
+ for (int i = 0; i < mask.length; i++) {
+ // set non-fixed part of a search key to 0.
+ if (mask[i] == 0) key[i] = 0;
+ }
}
- // TODO: possible improvement: save which fuzzy row key to use when providing a hint
- @Override
- public ReturnCode filterKeyValue(Cell c) {
- // assigning "worst" result first and looking for better options
- SatisfiesCode bestOption = SatisfiesCode.NO_NEXT;
- for (Pair fuzzyData : fuzzyKeysData) {
- SatisfiesCode satisfiesCode = satisfies(isReversed(), c.getRowArray(), c.getRowOffset(),
- c.getRowLength(), fuzzyData.getFirst(), fuzzyData.getSecond());
- if (satisfiesCode == SatisfiesCode.YES) {
- return ReturnCode.INCLUDE;
+ /**
+ * We need to preprocess mask array, as since we treat 0's as unfixed positions and -1 (0xff) as
+ * fixed positions
+ * @param mask
+ * @return mask array
+ */
+ private byte[] preprocessMask(byte[] mask) {
+ if (UnsafeAccess.isAvailable() == false) {
+ // do nothing
+ return mask;
+ }
+ if (isPreprocessedMask(mask)) return mask;
+ for (int i = 0; i < mask.length; i++) {
+ if (mask[i] == 0) {
+ mask[i] = -1; // 0 -> -1
+ } else if (mask[i] == 1) {
+ mask[i] = 0;// 1 -> 0
}
+ }
+ return mask;
+ }
- if (satisfiesCode == SatisfiesCode.NEXT_EXISTS) {
- bestOption = SatisfiesCode.NEXT_EXISTS;
+ private boolean isPreprocessedMask(byte[] mask) {
+ for (int i = 0; i < mask.length; i++) {
+ if (mask[i] != -1 && mask[i] != 0) {
+ return false;
}
}
+ return true;
+ }
- if (bestOption == SatisfiesCode.NEXT_EXISTS) {
- return ReturnCode.SEEK_NEXT_USING_HINT;
+ @Override
+ public ReturnCode filterKeyValue(Cell c) {
+ final int startIndex = lastFoundIndex >= 0 ? lastFoundIndex : 0;
+ final int size = fuzzyKeysData.size();
+ for (int i = startIndex; i < size + startIndex; i++) {
+ final int index = i % size;
+ Pair fuzzyData = fuzzyKeysData.get(index);
+ SatisfiesCode satisfiesCode =
+ satisfies(isReversed(), c.getRowArray(), c.getRowOffset(), c.getRowLength(),
+ fuzzyData.getFirst(), fuzzyData.getSecond());
+ if (satisfiesCode == SatisfiesCode.YES) {
+ lastFoundIndex = index;
+ return ReturnCode.INCLUDE;
+ }
}
+ // NOT FOUND -> seek next using hint
+ lastFoundIndex = -1;
+ return ReturnCode.SEEK_NEXT_USING_HINT;
- // the only unhandled SatisfiesCode is NO_NEXT, i.e. we are done
- done = true;
- return ReturnCode.NEXT_ROW;
}
@Override
public Cell getNextCellHint(Cell currentCell) {
- byte[] nextRowKey = null;
- // Searching for the "smallest" row key that satisfies at least one fuzzy row key
- for (Pair fuzzyData : fuzzyKeysData) {
- byte[] nextRowKeyCandidate = getNextForFuzzyRule(isReversed(), currentCell.getRowArray(),
- currentCell.getRowOffset(), currentCell.getRowLength(), fuzzyData.getFirst(),
- fuzzyData.getSecond());
- if (nextRowKeyCandidate == null) {
- continue;
- }
- if (nextRowKey == null ||
- (reversed && Bytes.compareTo(nextRowKeyCandidate, nextRowKey) > 0) ||
- (!reversed && Bytes.compareTo(nextRowKeyCandidate, nextRowKey) < 0)) {
- nextRowKey = nextRowKeyCandidate;
+ boolean result = true;
+ if (tracker.needsUpdate()) {
+ result = tracker.updateTracker(currentCell);
+ }
+ if (result == false) {
+ done = true;
+ return null;
+ }
+ byte[] nextRowKey = tracker.nextRow();
+ // We need to compare nextRowKey with currentCell
+ int compareResult =
+ Bytes.compareTo(nextRowKey, 0, nextRowKey.length, currentCell.getRowArray(),
+ currentCell.getRowOffset(), currentCell.getRowLength());
+ if ((reversed && compareResult > 0) || (!reversed && compareResult < 0)) {
+ // This can happen when we have multilpe filters and some other filter
+ // returns next row with hint which is larger (smaller for reverse)
+ // than the current (really?)
+ result = tracker.updateTracker(currentCell);
+ if (result == false) {
+ done = true;
+ return null;
+ } else {
+ nextRowKey = tracker.nextRow();
}
}
+ return KeyValueUtil.createFirstOnRow(nextRowKey);
+ }
+
+ /**
+ * If we have multiple fuzzy keys, row tracker should improve overall performance It calculates
+ * all next rows (one per every fuzzy key), sort them accordingly (ascending for regular and
+ * descending for reverse). Next time getNextCellHint is called we check row tracker first and
+ * return next row from the tracker if it exists, if there are no rows in the tracker we update
+ * tracker with a current cell and return first row.
+ */
+ private class RowTracker {
+ private final List nextRows;
+ private int next = -1;
+
+ RowTracker() {
+ nextRows = new ArrayList();
+ }
+
+ boolean needsUpdate() {
+ return next == -1 || next == nextRows.size();
+ }
- if (!reversed && nextRowKey == null) {
- // Should never happen for forward scanners; logic in filterKeyValue should return NO_NEXT.
- // Can happen in reversed scanner when currentKV is just before the next possible match; in
- // this case, fall back on scanner simply calling KeyValueHeap.next()
- // TODO: is there a better way than throw exception? (stop the scanner?)
- throw new IllegalStateException("No next row key that satisfies fuzzy exists when"
- + " getNextKeyHint() is invoked." + " Filter: " + this.toString() + " currentKV: "
- + currentCell);
+ byte[] nextRow() {
+ if (next < 0 || next == nextRows.size()) return null;
+ return nextRows.get(next++);
+ }
+
+ boolean updateTracker(Cell currentCell) {
+ nextRows.clear();
+ for (Pair fuzzyData : fuzzyKeysData) {
+ byte[] nextRowKeyCandidate =
+ getNextForFuzzyRule(isReversed(), currentCell.getRowArray(),
+ currentCell.getRowOffset(), currentCell.getRowLength(), fuzzyData.getFirst(),
+ fuzzyData.getSecond());
+ if (nextRowKeyCandidate == null) {
+ continue;
+ }
+ nextRows.add(nextRowKeyCandidate);
+ }
+ // Sort all next row candidates
+ Collections.sort(nextRows, new Comparator() {
+ @Override
+ public int compare(byte[] o1, byte[] o2) {
+ if (reversed) {
+ return -Bytes.compareTo(o1, o2);
+ } else {
+ return Bytes.compareTo(o1, o2);
+ }
+ }
+ });
+ next = 0;
+ return nextRows.size() > 0;
}
- return nextRowKey == null ? null : KeyValueUtil.createFirstOnRow(nextRowKey);
}
@Override
@@ -155,9 +241,8 @@ public class FuzzyRowFilter extends FilterBase {
/**
* @return The filter serialized using pb
*/
- public byte [] toByteArray() {
- FilterProtos.FuzzyRowFilter.Builder builder =
- FilterProtos.FuzzyRowFilter.newBuilder();
+ public byte[] toByteArray() {
+ FilterProtos.FuzzyRowFilter.Builder builder = FilterProtos.FuzzyRowFilter.newBuilder();
for (Pair fuzzyData : fuzzyKeysData) {
BytesBytesPair.Builder bbpBuilder = BytesBytesPair.newBuilder();
bbpBuilder.setFirst(ByteStringer.wrap(fuzzyData.getFirst()));
@@ -173,8 +258,7 @@ public class FuzzyRowFilter extends FilterBase {
* @throws DeserializationException
* @see #toByteArray
*/
- public static FuzzyRowFilter parseFrom(final byte [] pbBytes)
- throws DeserializationException {
+ public static FuzzyRowFilter parseFrom(final byte[] pbBytes) throws DeserializationException {
FilterProtos.FuzzyRowFilter proto;
try {
proto = FilterProtos.FuzzyRowFilter.parseFrom(pbBytes);
@@ -182,7 +266,7 @@ public class FuzzyRowFilter extends FilterBase {
throw new DeserializationException(e);
}
int count = proto.getFuzzyKeysDataCount();
- ArrayList> fuzzyKeysData= new ArrayList>(count);
+ ArrayList> fuzzyKeysData = new ArrayList>(count);
for (int i = 0; i < count; ++i) {
BytesBytesPair current = proto.getFuzzyKeysData(i);
byte[] keyBytes = current.getFirst().toByteArray();
@@ -223,12 +307,90 @@ public class FuzzyRowFilter extends FilterBase {
@VisibleForTesting
static SatisfiesCode satisfies(boolean reverse, byte[] row, byte[] fuzzyKeyBytes,
- byte[] fuzzyKeyMeta) {
+ byte[] fuzzyKeyMeta) {
return satisfies(reverse, row, 0, row.length, fuzzyKeyBytes, fuzzyKeyMeta);
}
- private static SatisfiesCode satisfies(boolean reverse, byte[] row, int offset, int length,
- byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) {
+ static SatisfiesCode satisfies(boolean reverse, byte[] row, int offset, int length,
+ byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) {
+
+ if (UnsafeAccess.isAvailable() == false) {
+ return satisfiesNoUnsafe(reverse, row, offset, length, fuzzyKeyBytes, fuzzyKeyMeta);
+ }
+
+ if (row == null) {
+ // do nothing, let scan to proceed
+ return SatisfiesCode.YES;
+ }
+ length = Math.min(length, fuzzyKeyBytes.length);
+ int numWords = length / Bytes.SIZEOF_LONG;
+ int offsetAdj = offset + UnsafeAccess.BYTE_ARRAY_BASE_OFFSET;
+
+ int j = numWords << 3; // numWords * SIZEOF_LONG;
+
+ for (int i = 0; i < j; i += Bytes.SIZEOF_LONG) {
+
+ long fuzzyBytes =
+ UnsafeAccess.theUnsafe.getLong(fuzzyKeyBytes, UnsafeAccess.BYTE_ARRAY_BASE_OFFSET
+ + (long) i);
+ long fuzzyMeta =
+ UnsafeAccess.theUnsafe.getLong(fuzzyKeyMeta, UnsafeAccess.BYTE_ARRAY_BASE_OFFSET
+ + (long) i);
+ long rowValue = UnsafeAccess.theUnsafe.getLong(row, offsetAdj + (long) i);
+ if ((rowValue & fuzzyMeta) != (fuzzyBytes)) {
+ // We always return NEXT_EXISTS
+ return SatisfiesCode.NEXT_EXISTS;
+ }
+ }
+
+ int off = j;
+
+ if (length - off >= Bytes.SIZEOF_INT) {
+ int fuzzyBytes =
+ UnsafeAccess.theUnsafe.getInt(fuzzyKeyBytes, UnsafeAccess.BYTE_ARRAY_BASE_OFFSET
+ + (long) off);
+ int fuzzyMeta =
+ UnsafeAccess.theUnsafe.getInt(fuzzyKeyMeta, UnsafeAccess.BYTE_ARRAY_BASE_OFFSET
+ + (long) off);
+ int rowValue = UnsafeAccess.theUnsafe.getInt(row, offsetAdj + (long) off);
+ if ((rowValue & fuzzyMeta) != (fuzzyBytes)) {
+ // We always return NEXT_EXISTS
+ return SatisfiesCode.NEXT_EXISTS;
+ }
+ off += Bytes.SIZEOF_INT;
+ }
+
+ if (length - off >= Bytes.SIZEOF_SHORT) {
+ short fuzzyBytes =
+ UnsafeAccess.theUnsafe.getShort(fuzzyKeyBytes, UnsafeAccess.BYTE_ARRAY_BASE_OFFSET
+ + (long) off);
+ short fuzzyMeta =
+ UnsafeAccess.theUnsafe.getShort(fuzzyKeyMeta, UnsafeAccess.BYTE_ARRAY_BASE_OFFSET
+ + (long) off);
+ short rowValue = UnsafeAccess.theUnsafe.getShort(row, offsetAdj + (long) off);
+ if ((rowValue & fuzzyMeta) != (fuzzyBytes)) {
+ // We always return NEXT_EXISTS
+ // even if it does not (in this case getNextForFuzzyRule
+ // will return null)
+ return SatisfiesCode.NEXT_EXISTS;
+ }
+ off += Bytes.SIZEOF_SHORT;
+ }
+
+ if (length - off >= Bytes.SIZEOF_BYTE) {
+ int fuzzyBytes = fuzzyKeyBytes[off] & 0xff;
+ int fuzzyMeta = fuzzyKeyMeta[off] & 0xff;
+ int rowValue = row[offset + off] & 0xff;
+ if ((rowValue & fuzzyMeta) != (fuzzyBytes)) {
+ // We always return NEXT_EXISTS
+ return SatisfiesCode.NEXT_EXISTS;
+ }
+ }
+ return SatisfiesCode.YES;
+ }
+
+ static SatisfiesCode satisfiesNoUnsafe(boolean reverse, byte[] row, int offset,
+ int length, byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) {
if (row == null) {
// do nothing, let scan to proceed
return SatisfiesCode.YES;
@@ -265,12 +427,11 @@ public class FuzzyRowFilter extends FilterBase {
// bigger byte array that satisfies the rule we need to just increase this byte
// (see the code of getNextForFuzzyRule below) by one.
// Note: if non-fixed byte is already at biggest value, this doesn't allow us to say there's
- // bigger one that satisfies the rule as it can't be increased.
+ // bigger one that satisfies the rule as it can't be increased.
if (fuzzyKeyMeta[i] == 1 && !order.isMax(fuzzyKeyBytes[i])) {
nextRowKeyCandidateExists = true;
}
}
-
return SatisfiesCode.YES;
}
@@ -281,7 +442,7 @@ public class FuzzyRowFilter extends FilterBase {
@VisibleForTesting
static byte[] getNextForFuzzyRule(boolean reverse, byte[] row, byte[] fuzzyKeyBytes,
- byte[] fuzzyKeyMeta) {
+ byte[] fuzzyKeyMeta) {
return getNextForFuzzyRule(reverse, row, 0, row.length, fuzzyKeyBytes, fuzzyKeyMeta);
}
@@ -291,16 +452,20 @@ public class FuzzyRowFilter extends FilterBase {
public boolean lt(int lhs, int rhs) {
return lhs < rhs;
}
+
public boolean gt(int lhs, int rhs) {
return lhs > rhs;
}
+
public byte inc(byte val) {
// TODO: what about over/underflow?
return (byte) (val + 1);
}
+
public boolean isMax(byte val) {
return val == (byte) 0xff;
}
+
public byte min() {
return 0;
}
@@ -309,16 +474,20 @@ public class FuzzyRowFilter extends FilterBase {
public boolean lt(int lhs, int rhs) {
return lhs > rhs;
}
+
public boolean gt(int lhs, int rhs) {
return lhs < rhs;
}
+
public byte inc(byte val) {
// TODO: what about over/underflow?
return (byte) (val - 1);
}
+
public boolean isMax(byte val) {
return val == 0;
}
+
public byte min() {
return (byte) 0xFF;
}
@@ -330,33 +499,37 @@ public class FuzzyRowFilter extends FilterBase {
/** Returns true when {@code lhs < rhs}. */
public abstract boolean lt(int lhs, int rhs);
+
/** Returns true when {@code lhs > rhs}. */
public abstract boolean gt(int lhs, int rhs);
+
/** Returns {@code val} incremented by 1. */
public abstract byte inc(byte val);
+
/** Return true when {@code val} is the maximum value */
public abstract boolean isMax(byte val);
+
/** Return the minimum value according to this ordering scheme. */
public abstract byte min();
}
/**
- * @return greater byte array than given (row) which satisfies the fuzzy rule if it exists,
- * null otherwise
+ * @return greater byte array than given (row) which satisfies the fuzzy rule if it exists, null
+ * otherwise
*/
@VisibleForTesting
static byte[] getNextForFuzzyRule(boolean reverse, byte[] row, int offset, int length,
- byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) {
+ byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) {
// To find out the next "smallest" byte array that satisfies fuzzy rule and "greater" than
// the given one we do the following:
// 1. setting values on all "fixed" positions to the values from fuzzyKeyBytes
// 2. if during the first step given row did not increase, then we increase the value at
- // the first "non-fixed" position (where it is not maximum already)
+ // the first "non-fixed" position (where it is not maximum already)
// It is easier to perform this by using fuzzyKeyBytes copy and setting "non-fixed" position
// values than otherwise.
- byte[] result = Arrays.copyOf(fuzzyKeyBytes,
- length > fuzzyKeyBytes.length ? length : fuzzyKeyBytes.length);
+ byte[] result =
+ Arrays.copyOf(fuzzyKeyBytes, length > fuzzyKeyBytes.length ? length : fuzzyKeyBytes.length);
if (reverse && length > fuzzyKeyBytes.length) {
// we need trailing 0xff's instead of trailing 0x00's
for (int i = fuzzyKeyBytes.length; i < result.length; i++) {
@@ -368,13 +541,13 @@ public class FuzzyRowFilter extends FilterBase {
boolean increased = false;
for (int i = 0; i < result.length; i++) {
- if (i >= fuzzyKeyMeta.length || fuzzyKeyMeta[i] == 1) {
+ if (i >= fuzzyKeyMeta.length || fuzzyKeyMeta[i] == 0 /* non-fixed */) {
result[i] = row[offset + i];
if (!order.isMax(row[offset + i])) {
// this is "non-fixed" position and is not at max value, hence we can increase it
toInc = i;
}
- } else if (i < fuzzyKeyMeta.length && fuzzyKeyMeta[i] == 0) {
+ } else if (i < fuzzyKeyMeta.length && fuzzyKeyMeta[i] == -1 /* fixed */) {
if (order.lt((row[i + offset] & 0xFF), (fuzzyKeyBytes[i] & 0xFF))) {
// if setting value for any fixed position increased the original array,
// we are OK
@@ -400,7 +573,7 @@ public class FuzzyRowFilter extends FilterBase {
// Setting all "non-fixed" positions to zeroes to the right of the one we increased so
// that found "next" row key is the smallest possible
for (int i = toInc + 1; i < result.length; i++) {
- if (i >= fuzzyKeyMeta.length || fuzzyKeyMeta[i] == 1) {
+ if (i >= fuzzyKeyMeta.length || fuzzyKeyMeta[i] == 0 /* non-fixed */) {
result[i] = order.min();
}
}
@@ -410,20 +583,20 @@ public class FuzzyRowFilter extends FilterBase {
}
/**
- * @return true if and only if the fields of the filter that are serialized
- * are equal to the corresponding fields in other. Used for testing.
+ * @return true if and only if the fields of the filter that are serialized are equal to the
+ * corresponding fields in other. Used for testing.
*/
boolean areSerializedFieldsEqual(Filter o) {
if (o == this) return true;
if (!(o instanceof FuzzyRowFilter)) return false;
- FuzzyRowFilter other = (FuzzyRowFilter)o;
+ FuzzyRowFilter other = (FuzzyRowFilter) o;
if (this.fuzzyKeysData.size() != other.fuzzyKeysData.size()) return false;
for (int i = 0; i < fuzzyKeysData.size(); ++i) {
Pair thisData = this.fuzzyKeysData.get(i);
Pair otherData = other.fuzzyKeysData.get(i);
- if (!(Bytes.equals(thisData.getFirst(), otherData.getFirst())
- && Bytes.equals(thisData.getSecond(), otherData.getSecond()))) {
+ if (!(Bytes.equals(thisData.getFirst(), otherData.getFirst()) && Bytes.equals(
+ thisData.getSecond(), otherData.getSecond()))) {
return false;
}
}
diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/util/UnsafeAccess.java hbase-common/src/main/java/org/apache/hadoop/hbase/util/UnsafeAccess.java
new file mode 100644
index 0000000..b8a2242
--- /dev/null
+++ hbase-common/src/main/java/org/apache/hadoop/hbase/util/UnsafeAccess.java
@@ -0,0 +1,66 @@
+/**
+ * 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.util;
+
+import java.lang.reflect.Field;
+import java.nio.ByteOrder;
+import java.security.AccessController;
+import java.security.PrivilegedAction;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+import sun.misc.Unsafe;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class UnsafeAccess {
+
+ private static final Log LOG = LogFactory.getLog(UnsafeAccess.class);
+
+ public static final Unsafe theUnsafe;
+
+ /** The offset to the first element in a byte array. */
+ public static final int BYTE_ARRAY_BASE_OFFSET;
+
+ static {
+ theUnsafe = (Unsafe) AccessController.doPrivileged(new PrivilegedAction