From 4d2ff7bda53b7503b6c5c24e7cd99931e13da219 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=BB=9D=E9=A1=B6?= Date: Tue, 27 Sep 2016 15:00:31 +0800 Subject: [PATCH] fix #EHB-511 Resolve client compatibility issue introduced by offheap changes Summary: As titled, resolve the compatibility issue, add back some interface/methods Test Plan: All UT should pass Reviewers: tianzhao.blj, hongxi.sy Reviewed By: hongxi.sy Subscribers: #hbase Differential Revision: http://phabricator.taobao.net/D5885 --- .../org/apache/hadoop/hbase/client/Result.java | 100 +++++++++++++++++++++ .../hadoop/hbase/filter/ColumnPrefixFilter.java | 25 ++++++ .../apache/hadoop/hbase/filter/CompareFilter.java | 12 +++ .../org/apache/hadoop/hbase/filter/Filter.java | 18 ++++ .../org/apache/hadoop/hbase/filter/FilterBase.java | 21 +++++ .../org/apache/hadoop/hbase/filter/FilterList.java | 26 ++++++ .../apache/hadoop/hbase/filter/FilterWrapper.java | 24 +++++ .../hadoop/hbase/filter/InclusiveStopFilter.java | 20 +++++ .../hadoop/hbase/filter/MultiRowRangeFilter.java | 47 ++++++++++ .../hbase/filter/MultipleColumnPrefixFilter.java | 14 +++ .../apache/hadoop/hbase/filter/PrefixFilter.java | 18 ++++ .../hadoop/hbase/filter/RandomRowFilter.java | 8 ++ .../org/apache/hadoop/hbase/filter/RowFilter.java | 11 +++ 13 files changed, 344 insertions(+) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java index 41d4ff5..2017212 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java @@ -133,6 +133,23 @@ public class Result implements CellScannable, CellScanner { } /** + * @deprecated Use {@link #create(List)} instead. + */ + @Deprecated + public Result(KeyValue[] cells) { + this(cells, null, false, false); + } + + /** + * @deprecated Use {@link #create(List)} instead. + */ + @Deprecated + public Result(List kvs) { + // TODO: Here we presume the passed in Cells are KVs. One day this won't always be so. + this(kvs.toArray(new Cell[kvs.size()]), null, false, false); + } + + /** * Instantiate a Result with the specified List of KeyValues. *
Note: You must ensure that the keyvalues are already sorted. * @param cells List of cells @@ -222,6 +239,24 @@ public class Result implements CellScannable, CellScanner { } /** + * Return an cells of a Result as an array of KeyValues + *

+ * WARNING do not use, expensive. This does an arraycopy of the cell[]'s value. + *

+ * Added to ease transition from 0.94 -> 0.96. + * @deprecated as of 0.96, use {@link #rawCells()} + * @return array of KeyValues, empty array if nothing in result. + */ + @Deprecated + public KeyValue[] raw() { + KeyValue[] kvs = new KeyValue[cells.length]; + for (int i = 0; i < kvs.length; i++) { + kvs[i] = KeyValueUtil.ensureKeyValue(cells[i]); + } + return kvs; + } + + /** * Create a sorted list of the Cell's in this result. * * Since HBase 0.20.5 this is equivalent to raw(). @@ -233,6 +268,28 @@ public class Result implements CellScannable, CellScanner { } /** + * Return an cells of a Result as an array of KeyValues + *

+ * WARNING do not use, expensive. This does an arraycopy of the cell[]'s value. + *

+ * Added to ease transition from 0.94 -> 0.96. + * @deprecated as of 0.96, use {@link #listCells()} + * @return all sorted List of KeyValues; can be null if no cells in the result + */ + @Deprecated + public List list() { + return isEmpty() ? null : Arrays.asList(raw()); + } + + /** + * @deprecated Use {@link #getColumnCells(byte[], byte[])} instead. + */ + @Deprecated + public List getColumn(byte[] family, byte[] qualifier) { + return KeyValueUtil.ensureKeyValues(getColumnCells(family, qualifier)); + } + + /** * Return the Cells for the specific column. The Cells are sorted in * the {@link CellComparator#COMPARATOR} order. That implies the first entry in * the list is the most recent column. If the query (Scan or Get) only @@ -337,6 +394,24 @@ public class Result implements CellScannable, CellScanner { } /** + * @deprecated Use {@link #getColumnLatestCell(byte[], byte[])} instead. + */ + @Deprecated + public KeyValue getColumnLatest(byte[] family, byte[] qualifier) { + return KeyValueUtil.ensureKeyValue(getColumnLatestCell(family, qualifier)); + } + + /** + * @deprecated Use {@link #getColumnLatestCell(byte[], int, int, byte[], int, int)} instead. + */ + @Deprecated + public KeyValue getColumnLatest(byte[] family, int foffset, int flength, byte[] qualifier, + int qoffset, int qlength) { + return KeyValueUtil + .ensureKeyValue(getColumnLatestCell(family, foffset, flength, qualifier, qoffset, qlength)); + } + + /** * The Cell for the most recent timestamp for a given column. * * @param family @@ -770,6 +845,31 @@ public class Result implements CellScannable, CellScanner { return size; } + /** + * for pre-0.94 users + * @deprecated + */ + public static long getWriteArraySize(Result[] results) { + long size = Bytes.SIZEOF_BYTE; // RESULT_VERSION + if (results == null || results.length == 0) { + size += Bytes.SIZEOF_INT; + return size; + } + + size += Bytes.SIZEOF_INT; // results.length + size += Bytes.SIZEOF_INT; // bufLen + for (Result result : results) { + size += Bytes.SIZEOF_INT; // either 0 or result.size() + if (result == null || result.isEmpty()) continue; + + for (KeyValue kv : result.raw()) { + size += Bytes.SIZEOF_INT; // kv.getLength(); + size += kv.getLength(); + } + } + + return size; + } /** * Does a deep comparison of two Results, down to the byte arrays. diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java index c314100..524739f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java @@ -76,6 +76,31 @@ public class ColumnPrefixFilter extends FilterBase { return v; } + /** + * @deprecated use {@link #filterColumn(Cell)} instead + */ + public ReturnCode filterColumn(byte[] buffer, int qualifierOffset, int qualifierLength) { + if (qualifierLength < prefix.length) { + int cmp = Bytes.compareTo(buffer, qualifierOffset, qualifierLength, this.prefix, 0, + qualifierLength); + if (cmp <= 0) { + return ReturnCode.SEEK_NEXT_USING_HINT; + } else { + return ReturnCode.NEXT_ROW; + } + } else { + int cmp = Bytes.compareTo(buffer, qualifierOffset, this.prefix.length, this.prefix, 0, + this.prefix.length); + if (cmp < 0) { + return ReturnCode.SEEK_NEXT_USING_HINT; + } else if (cmp > 0) { + return ReturnCode.NEXT_ROW; + } else { + return ReturnCode.INCLUDE; + } + } + } + public ReturnCode filterColumn(Cell cell) { int qualifierLength = cell.getQualifierLength(); if (qualifierLength < prefix.length) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java index 67d1ff5..1ed46f0 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java @@ -160,6 +160,18 @@ public abstract class CompareFilter extends FilterBase { } } + /** + * @deprecated reserved for backward compatible + */ + protected boolean doCompare(final CompareOp compareOp, final ByteArrayComparable comparator, + final byte[] data, final int offset, final int length) { + if (compareOp == CompareOp.NO_OP) { + return true; + } + int compareResult = comparator.compareTo(data, offset, length); + return compare(compareOp, compareResult); + } + // Override here explicitly as the method in super class FilterBase might do a KeyValue recreate. // See HBASE-12068 @Override diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java index 9923527..7566aa0 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java @@ -25,6 +25,7 @@ import java.util.List; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.exceptions.DeserializationException; /** @@ -151,6 +152,13 @@ public abstract class Filter { abstract public Cell transformCell(final Cell v) throws IOException; /** + * WARNING: please to not override this method. Instead override {@link #transformCell(Cell)}. + * This is for transition from 0.94 -> 0.96 + **/ + @Deprecated // use Cell transformCell(final Cell) + abstract public KeyValue transform(final KeyValue currentKV) throws IOException; + + /** * Return codes for filterValue(). */ @InterfaceAudience.Public @@ -216,6 +224,16 @@ public abstract class Filter { abstract public boolean filterRow() throws IOException; /** + * @param currentKV + * @return KeyValue which must be next seeked. return null if the filter is not sure which key to + * seek to next. + * @throws IOException + * @deprecated Function is Deprecated. Use {@link #getNextCellHint(Cell)} instead. + */ + @Deprecated + abstract public KeyValue getNextKeyHint(final KeyValue currentKV) throws IOException; + + /** * If the filter returns the match code SEEK_NEXT_USING_HINT, then it should also tell which is * the next key it must seek to. After receiving the match code SEEK_NEXT_USING_HINT, the * QueryMatcher would call this function to find out which key it must next seek to. diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java index 3bf04ff..78247d4 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java @@ -25,6 +25,7 @@ import java.util.List; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.KeyValue; /** * Abstract base class to help you implement new Filters. Common "ignore" or NOOP type @@ -174,4 +175,24 @@ public abstract class FilterBase extends Filter { boolean areSerializedFieldsEqual(Filter other) { return true; } + + /** + * WARNING: please to not override this method. Instead override {@link #transformCell(Cell)}. + *

+ * This is for transition from 0.94 -> 0.96 + */ + @Override + @Deprecated + public KeyValue transform(KeyValue currentKV) throws IOException { + return currentKV; + } + + /** + * This method is deprecated and you should override Cell getNextKeyHint(Cell) instead. + */ + @Override + @Deprecated + public KeyValue getNextKeyHint(KeyValue currentKV) throws IOException { + return null; + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java index 6205100..3fa1153 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java @@ -28,6 +28,8 @@ import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.FilterProtos; @@ -421,6 +423,12 @@ final public class FilterList extends Filter { } @Override + @Deprecated + public KeyValue getNextKeyHint(KeyValue currentKV) throws IOException { + return KeyValueUtil.ensureKeyValue(getNextCellHint((Cell) currentKV)); + } + + @Override public Cell getNextCellHint(Cell currentCell) throws IOException { Cell keyHint = null; if (operator == Operator.MUST_PASS_ALL) { @@ -485,4 +493,22 @@ final public class FilterList extends Filter { this.filters.size(), this.filters.subList(0, endIndex).toString()); } + + /** + * WARNING: please do not override this method. Instead override {@link #transformCell(Cell)}. + *

+ * When removing this, its body should be placed in transformCell. + *

+ * This is for transition from 0.94 -> 0.96 + */ + @Deprecated + @Override + public KeyValue transform(KeyValue v) throws IOException { + // transform() is expected to follow an inclusive filterKeyValue() immediately: + if (!v.equals(this.referenceCell)) { + throw new IllegalStateException( + "Reference Cell: " + this.referenceCell + " does not match: " + v); + } + return KeyValueUtil.ensureKeyValue(this.transformedCell); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java index 9b12d05..e125564 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java @@ -24,6 +24,8 @@ import java.util.List; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.FilterProtos; @@ -95,6 +97,16 @@ final public class FilterWrapper extends Filter { return this.filter.filterRow(); } + /** + * This method is deprecated and you should override Cell getNextKeyHint(Cell) instead. + */ + @Override + @Deprecated + public KeyValue getNextKeyHint(KeyValue currentKV) throws IOException { + // This will never get called. + return KeyValueUtil.ensureKeyValue(this.filter.getNextCellHint((Cell) currentKV)); + } + @Override public Cell getNextCellHint(Cell currentCell) throws IOException { return this.filter.getNextCellHint(currentCell); @@ -170,4 +182,16 @@ final public class FilterWrapper extends Filter { FilterWrapper other = (FilterWrapper)o; return this.filter.areSerializedFieldsEqual(other.filter); } + + /** + * WARNING: please to not override this method. Instead override {@link #transformCell(Cell)}. + *

+ * This is for transition from 0.94 -> 0.96 + */ + @Override + @Deprecated + public KeyValue transform(KeyValue currentKV) throws IOException { + // This will never get called. + return KeyValueUtil.ensureKeyValue(this.filter.transformCell(currentKV)); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java index fc8ee64..00da7ae 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java @@ -75,6 +75,26 @@ public class InclusiveStopFilter extends FilterBase { return done; } + /** + * @deprecated use {@link #filterRowKey(Cell)} instead + */ + public boolean filterRowKey(byte[] buffer, int offset, int length) { + if (buffer == null) { + // noinspection RedundantIfStatement + if (this.stopRowKey == null) { + return true; // filter... + } + return false; + } + // if stopRowKey is <= buffer, then true, filter row. + int cmp = Bytes.compareTo(stopRowKey, 0, stopRowKey.length, buffer, offset, length); + + if (cmp < 0) { + done = true; + } + return done; + } + public boolean filterAllRemaining() { return done; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java index c34d129..6cd54ab 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java @@ -129,6 +129,53 @@ public class MultiRowRangeFilter extends FilterBase { return false; } + /** + * @deprecated use {@link #filterRowKey(Cell)} instead + */ + @Override + public boolean filterRowKey(byte[] buffer, int offset, int length) { + // If it is the first time of running, calculate the current range index for + // the row key. If index is out of bound which happens when the start row + // user sets is after the largest stop row of the ranges, stop the scan. + // If row key is after the current range, find the next range and update index. + if (!initialized || !range.contains(buffer, offset, length)) { + byte[] rowkey = new byte[length]; + // we don't reuse the codes because of this array copy + System.arraycopy(buffer, offset, rowkey, 0, length); + index = getNextRangeIndex(rowkey); + if (index >= rangeList.size()) { + done = true; + currentReturnCode = ReturnCode.NEXT_ROW; + return false; + } + if(index != ROW_BEFORE_FIRST_RANGE) { + range = rangeList.get(index); + } else { + range = rangeList.get(0); + } + if (EXCLUSIVE) { + EXCLUSIVE = false; + currentReturnCode = ReturnCode.NEXT_ROW; + return false; + } + if (!initialized) { + if(index != ROW_BEFORE_FIRST_RANGE) { + currentReturnCode = ReturnCode.INCLUDE; + } else { + currentReturnCode = ReturnCode.SEEK_NEXT_USING_HINT; + } + initialized = true; + } else { + if (range.contains(buffer, offset, length)) { + currentReturnCode = ReturnCode.INCLUDE; + } else currentReturnCode = ReturnCode.SEEK_NEXT_USING_HINT; + } + } else { + currentReturnCode = ReturnCode.INCLUDE; + } + return false; + } + @Override public ReturnCode filterKeyValue(Cell ignored) { return currentReturnCode; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java index 02dfa71..415292b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.filter; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Comparator; import java.util.TreeSet; @@ -87,6 +88,10 @@ public class MultipleColumnPrefixFilter extends FilterBase { public ReturnCode filterColumn(Cell cell) { byte [] qualifier = CellUtil.cloneQualifier(cell); + return internalFilterColumn(qualifier); + } + + private ReturnCode internalFilterColumn(byte[] qualifier) { TreeSet lesserOrEqualPrefixes = (TreeSet) sortedPrefixes.headSet(qualifier, true); @@ -109,6 +114,15 @@ public class MultipleColumnPrefixFilter extends FilterBase { } } + /** + * @deprecated use {@link #filterColumn(Cell)} instead + */ + public ReturnCode filterColumn(byte[] buffer, int qualifierOffset, int qualifierLength) { + byte[] qualifier = + Arrays.copyOfRange(buffer, qualifierOffset, qualifierLength + qualifierOffset); + return internalFilterColumn(qualifier); + } + public static Filter createFilterFromArguments(ArrayList filterArguments) { byte [][] prefixes = new byte [filterArguments.size()][]; for (int i = 0 ; i < filterArguments.size(); i++) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java index 1704119..58b80ff 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java @@ -76,6 +76,24 @@ public class PrefixFilter extends FilterBase { return filterRow; } + /** + * @deprecated use {@link #filterRowKey(Cell)} instead + */ + public boolean filterRowKey(byte[] buffer, int offset, int length) { + if (buffer == null || this.prefix == null) return true; + if (length < prefix.length) return true; + // if they are equal, return false => pass row + // else return true, filter row + // if we are passed the prefix, set flag + int cmp = + Bytes.compareTo(buffer, offset, this.prefix.length, this.prefix, 0, this.prefix.length); + if ((!isReversed() && cmp > 0) || (isReversed() && cmp < 0)) { + passedPrefix = true; + } + filterRow = (cmp != 0); + return filterRow; + } + @Override public ReturnCode filterKeyValue(Cell v) { if (filterRow) return ReturnCode.NEXT_ROW; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java index 3a31215..703bc56 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java @@ -110,6 +110,14 @@ public class RandomRowFilter extends FilterBase { return filterOutRow; } + /** + * @deprecated use {@link #filterRowKey(Cell)} instead + */ + @Override + public boolean filterRowKey(byte[] data, int offset, int length) { + return filterRowKey(null); + } + @Override public void reset() { filterOutRow = false; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java index d0722f1..ac5e568 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java @@ -81,6 +81,17 @@ public class RowFilter extends CompareFilter { return this.filterOutRow; } + /** + * @deprecated use {@link #filterRowKey(Cell)} instead + */ + @Override + public boolean filterRowKey(byte[] data, int offset, int length) { + if (doCompare(this.compareOp, this.comparator, data, offset, length)) { + this.filterOutRow = true; + } + return this.filterOutRow; + } + @Override public boolean filterRow() { return this.filterOutRow; -- 1.8.3.1