You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jm...@apache.org on 2013/08/30 22:31:48 UTC
svn commit: r1519076 [2/6] - in /hbase/branches/0.95:
hbase-client/src/main/java/org/apache/hadoop/hbase/
hbase-client/src/main/java/org/apache/hadoop/hbase/client/
hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/
hbase-client/src...
Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java Fri Aug 30 20:31:44 2013
@@ -20,14 +20,18 @@
package org.apache.hadoop.hbase.filter;
import java.io.IOException;
+import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
+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;
+import org.apache.zookeeper.KeeperException.UnimplementedException;
import com.google.protobuf.InvalidProtocolBufferException;
@@ -40,7 +44,7 @@ import com.google.protobuf.InvalidProtoc
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
-public class FilterWrapper extends Filter {
+final public class FilterWrapper extends Filter {
Filter filter = null;
public FilterWrapper( Filter filter ) {
@@ -97,9 +101,23 @@ public class FilterWrapper extends Filte
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 {
- return this.filter.getNextKeyHint(currentKV);
+ return KeyValueUtil.ensureKeyValue(this.filter.getNextCellHint((Cell)currentKV));
+ }
+
+ /**
+ * Old filter wrapper descendants will implement KV getNextKeyHint(KV) so we should call it.
+ */
+ @Override
+ public Cell getNextCellHint(Cell currentKV) throws IOException {
+ // Old filters based off of this class will override KeyValue getNextKeyHint(KeyValue).
+ // Thus to maintain compatibility we need to call the old version.
+ return this.getNextKeyHint(KeyValueUtil.ensureKeyValue(currentKV));
}
@Override
@@ -108,13 +126,26 @@ public class FilterWrapper extends Filte
}
@Override
- public ReturnCode filterKeyValue(KeyValue v) throws IOException {
+ public ReturnCode filterKeyValue(Cell v) throws IOException {
return this.filter.filterKeyValue(v);
}
@Override
- public KeyValue transform(KeyValue v) throws IOException {
- return this.filter.transform(v);
+ public Cell transformCell(Cell v) throws IOException {
+ // Old filters based off of this class will override KeyValue transform(KeyValue).
+ // Thus to maintain compatibility we need to call the old version.
+ return transform(KeyValueUtil.ensureKeyValue(v));
+ }
+
+ /**
+ * 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 KeyValueUtil.ensureKeyValue(this.filter.transformCell(currentKV));
}
@Override
@@ -123,18 +154,31 @@ public class FilterWrapper extends Filte
}
@Override
- public void filterRow(List<KeyValue> kvs) throws IOException {
- //To fix HBASE-6429,
+ public void filterRowCells(List<Cell> kvs) throws IOException {
+ //To fix HBASE-6429,
//Filter with filterRow() returning true is incompatible with scan with limit
//1. hasFilterRow() returns true, if either filterRow() or filterRow(kvs) is implemented.
//2. filterRow() is merged with filterRow(kvs),
//so that to make all those row related filtering stuff in the same function.
- this.filter.filterRow(kvs);
+ this.filter.filterRowCells(kvs);
if (!kvs.isEmpty() && this.filter.filterRow()) {
kvs.clear();
}
}
+ /**
+ * WARNING: please to not override this method. Instead override {@link #transformCell(Cell)}.
+ *
+ * This is for transition from 0.94 -> 0.96
+ */
+ @Override
+ @Deprecated
+ public void filterRow(List<KeyValue> kvs) throws IOException {
+ // This is only used internally, marked InterfaceAudience.private, and not used anywhere.
+ // We can get away with not implementing this.
+ throw new UnsupportedOperationException("filterRow(List<KeyValue>) should never be called");
+ }
+
@Override
public boolean isFamilyEssential(byte[] name) throws IOException {
return filter.isFamilyEssential(name);
Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java Fri Aug 30 20:31:44 2013
@@ -18,15 +18,16 @@
*/
package org.apache.hadoop.hbase.filter;
-import com.google.common.base.Preconditions;
-import com.google.protobuf.InvalidProtocolBufferException;
+import java.util.ArrayList;
+
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
-import java.util.ArrayList;
+import com.google.common.base.Preconditions;
+import com.google.protobuf.InvalidProtocolBufferException;
/**
* A filter that will only return the first KV from each row.
@@ -45,7 +46,8 @@ public class FirstKeyOnlyFilter extends
foundKV = false;
}
- public ReturnCode filterKeyValue(KeyValue v) {
+ @Override
+ public ReturnCode filterKeyValue(Cell v) {
if(foundKV) return ReturnCode.NEXT_ROW;
foundKV = true;
return ReturnCode.INCLUDE;
Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java Fri Aug 30 20:31:44 2013
@@ -22,7 +22,9 @@ import com.google.protobuf.ByteString;
import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
+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.generated.FilterProtos;
import org.apache.hadoop.hbase.util.Bytes;
@@ -57,7 +59,8 @@ public class FirstKeyValueMatchingQualif
this.qualifiers = qualifiers;
}
- public ReturnCode filterKeyValue(KeyValue v) {
+ @Override
+ public ReturnCode filterKeyValue(Cell v) {
if (hasFoundKV()) {
return ReturnCode.NEXT_ROW;
} else if (hasOneMatchingQualifier(v)) {
@@ -66,9 +69,11 @@ public class FirstKeyValueMatchingQualif
return ReturnCode.INCLUDE;
}
- private boolean hasOneMatchingQualifier(KeyValue v) {
+ private boolean hasOneMatchingQualifier(Cell v) {
for (byte[] q : qualifiers) {
- if (v.matchingQualifier(q)) {
+ // TODO get rid of this by adding matching qualifier to interface.
+ KeyValue kv = KeyValueUtil.ensureKeyValue(v);
+ if (kv.matchingQualifier(q)) {
return true;
}
}
Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java Fri Aug 30 20:31:44 2013
@@ -21,7 +21,9 @@ import com.google.protobuf.ByteString;
import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
+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.generated.FilterProtos;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
@@ -72,8 +74,11 @@ public class FuzzyRowFilter extends Filt
// TODO: possible improvement: save which fuzzy row key to use when providing a hint
@Override
- public ReturnCode filterKeyValue(KeyValue kv) {
- byte[] rowKey = kv.getRow();
+ public ReturnCode filterKeyValue(Cell kv) {
+ // TODO add getRow() equivalent to Cell or change satisfies to take b[],o,l style args.
+ KeyValue v = KeyValueUtil.ensureKeyValue(kv);
+
+ byte[] rowKey = v.getRow();
// assigning "worst" result first and looking for better options
SatisfiesCode bestOption = SatisfiesCode.NO_NEXT;
for (Pair<byte[], byte[]> fuzzyData : fuzzyKeysData) {
@@ -98,8 +103,11 @@ public class FuzzyRowFilter extends Filt
}
@Override
- public KeyValue getNextKeyHint(KeyValue currentKV) {
- byte[] rowKey = currentKV.getRow();
+ public Cell getNextCellHint(Cell currentKV) {
+ // TODO make matching Column a cell method or CellUtil method.
+ KeyValue v = KeyValueUtil.ensureKeyValue(currentKV);
+
+ byte[] rowKey = v.getRow();
byte[] nextRowKey = null;
// Searching for the "smallest" row key that satisfies at least one fuzzy row key
for (Pair<byte[], byte[]> fuzzyData : fuzzyKeysData) {
Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java Fri Aug 30 20:31:44 2013
@@ -19,15 +19,18 @@
package org.apache.hadoop.hbase.filter;
-import com.google.common.base.Preconditions;
-import com.google.protobuf.InvalidProtocolBufferException;
+import java.util.ArrayList;
+
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
+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.generated.FilterProtos;
-import java.util.ArrayList;
+import com.google.common.base.Preconditions;
+import com.google.protobuf.InvalidProtocolBufferException;
/**
* A filter that will only return the key component of each KV (the value will
@@ -45,8 +48,13 @@ public class KeyOnlyFilter extends Filte
public KeyOnlyFilter(boolean lenAsVal) { this.lenAsVal = lenAsVal; }
@Override
- public KeyValue transform(KeyValue kv) {
- return kv.createKeyOnly(this.lenAsVal);
+ public Cell transformCell(Cell kv) {
+ // TODO Move to KeyValueUtil
+
+ // TODO make matching Column a cell method or CellUtil method.
+ KeyValue v = KeyValueUtil.ensureKeyValue(kv);
+
+ return v.createKeyOnly(this.lenAsVal);
}
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java Fri Aug 30 20:31:44 2013
@@ -21,6 +21,7 @@ import com.google.protobuf.ByteString;
import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
@@ -62,11 +63,11 @@ public class MultipleColumnPrefixFilter
}
@Override
- public ReturnCode filterKeyValue(KeyValue kv) {
- if (sortedPrefixes.size() == 0 || kv.getBuffer() == null) {
+ public ReturnCode filterKeyValue(Cell kv) {
+ if (sortedPrefixes.size() == 0 || kv.getQualifierArray() == null) {
return ReturnCode.INCLUDE;
} else {
- return filterColumn(kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength());
+ return filterColumn(kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength());
}
}
@@ -152,9 +153,10 @@ public class MultipleColumnPrefixFilter
return this.sortedPrefixes.equals(other.sortedPrefixes);
}
- public KeyValue getNextKeyHint(KeyValue kv) {
+ @Override
+ public Cell getNextCellHint(Cell kv) {
return KeyValue.createFirstOnRow(
- kv.getBuffer(), kv.getRowOffset(), kv.getRowLength(), kv.getBuffer(),
+ kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), kv.getFamilyArray(),
kv.getFamilyOffset(), kv.getFamilyLength(), hint, 0, hint.length);
}
Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java Fri Aug 30 20:31:44 2013
@@ -19,13 +19,13 @@
package org.apache.hadoop.hbase.filter;
-import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
+import com.google.protobuf.InvalidProtocolBufferException;
+
/**
* A binary comparator which lexicographically compares against the specified
* byte array using {@link org.apache.hadoop.hbase.util.Bytes#compareTo(byte[], byte[])}.
Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java Fri Aug 30 20:31:44 2013
@@ -44,7 +44,7 @@ import java.util.Stack;
* a filter object is constructed. This filter object is then wrapped
* in a scanner object which is then returned
* <p>
- * This class addresses the HBASE-4168 JIRA. More documentaton on this
+ * This class addresses the HBASE-4168 JIRA. More documentation on this
* Filter Language can be found at: https://issues.apache.org/jira/browse/HBASE-4176
*/
@InterfaceAudience.Public
@@ -242,8 +242,8 @@ public class ParseFilter {
}
try {
filterName = filterHashMap.get(filterName);
- Class c = Class.forName(filterName);
- Class[] argTypes = new Class [] {ArrayList.class};
+ Class<?> c = Class.forName(filterName);
+ Class<?>[] argTypes = new Class [] {ArrayList.class};
Method m = c.getDeclaredMethod("createFilterFromArguments", argTypes);
return (Filter) m.invoke(null,filterArguments);
} catch (ClassNotFoundException e) {
Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java Fri Aug 30 20:31:44 2013
@@ -19,17 +19,18 @@
package org.apache.hadoop.hbase.filter;
-import com.google.protobuf.InvalidProtocolBufferException;
+import java.io.IOException;
+import java.util.ArrayList;
+
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
-import java.io.IOException;
-import java.util.ArrayList;
+import com.google.protobuf.InvalidProtocolBufferException;
/**
* This filter is used to filter based on the column qualifier. It takes an
@@ -59,10 +60,10 @@ public class QualifierFilter extends Com
}
@Override
- public ReturnCode filterKeyValue(KeyValue v) {
+ public ReturnCode filterKeyValue(Cell v) {
int qualifierLength = v.getQualifierLength();
if (qualifierLength > 0) {
- if (doCompare(this.compareOp, this.comparator, v.getBuffer(),
+ if (doCompare(this.compareOp, this.comparator, v.getQualifierArray(),
v.getQualifierOffset(), qualifierLength)) {
return ReturnCode.SKIP;
}
@@ -71,7 +72,7 @@ public class QualifierFilter extends Com
}
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
- ArrayList arguments = CompareFilter.extractArguments(filterArguments);
+ ArrayList<?> arguments = CompareFilter.extractArguments(filterArguments);
CompareOp compareOp = (CompareOp)arguments.get(0);
ByteArrayComparable comparator = (ByteArrayComparable)arguments.get(1);
return new QualifierFilter(compareOp, comparator);
Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java Fri Aug 30 20:31:44 2013
@@ -19,14 +19,15 @@
package org.apache.hadoop.hbase.filter;
-import com.google.protobuf.InvalidProtocolBufferException;
+import java.util.Random;
+
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
-import java.util.Random;
+import com.google.protobuf.InvalidProtocolBufferException;
/**
* A filter that includes rows based on a chance.
@@ -71,7 +72,7 @@ public class RandomRowFilter extends Fil
}
@Override
- public ReturnCode filterKeyValue(KeyValue v) {
+ public ReturnCode filterKeyValue(Cell v) {
if (filterOutRow) {
return ReturnCode.NEXT_ROW;
}
Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java Fri Aug 30 20:31:44 2013
@@ -19,17 +19,18 @@
package org.apache.hadoop.hbase.filter;
-import com.google.protobuf.InvalidProtocolBufferException;
+import java.io.IOException;
+import java.util.ArrayList;
+
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
-import java.io.IOException;
-import java.util.ArrayList;
+import com.google.protobuf.InvalidProtocolBufferException;
/**
* This filter is used to filter based on the key. It takes an operator
@@ -65,7 +66,7 @@ public class RowFilter extends CompareFi
}
@Override
- public ReturnCode filterKeyValue(KeyValue v) {
+ public ReturnCode filterKeyValue(Cell v) {
if(this.filterOutRow) {
return ReturnCode.NEXT_ROW;
}
@@ -86,6 +87,7 @@ public class RowFilter extends CompareFi
}
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
+ @SuppressWarnings("rawtypes") // for arguments
ArrayList arguments = CompareFilter.extractArguments(filterArguments);
CompareOp compareOp = (CompareOp)arguments.get(0);
ByteArrayComparable comparator = (ByteArrayComparable)arguments.get(1);
Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java Fri Aug 30 20:31:44 2013
@@ -22,7 +22,9 @@ package org.apache.hadoop.hbase.filter;
import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
+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.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -100,10 +102,11 @@ public class SingleColumnValueExcludeFil
}
// Here we remove from row all key values from testing column
- public void filterRow(List<KeyValue> kvs) {
- Iterator it = kvs.iterator();
+ @Override
+ public void filterRowCells(List<Cell> kvs) {
+ Iterator<? extends Cell> it = kvs.iterator();
while (it.hasNext()) {
- KeyValue kv = (KeyValue)it.next();
+ KeyValue kv = KeyValueUtil.ensureKeyValue(it.next());
// If the current column is actually the tested column,
// we will skip it instead.
if (kv.matchingColumn(this.columnFamily, this.columnQualifier)) {
Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java Fri Aug 30 20:31:44 2013
@@ -26,7 +26,9 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
@@ -166,7 +168,11 @@ public class SingleColumnValueFilter ext
return columnQualifier;
}
- public ReturnCode filterKeyValue(KeyValue keyValue) {
+ @Override
+ public ReturnCode filterKeyValue(Cell c) {
+ // TODO get rid of this.
+ KeyValue keyValue = KeyValueUtil.ensureKeyValue(c);
+
// System.out.println("REMOVE KEY=" + keyValue.toString() + ", value=" + Bytes.toString(keyValue.getValue()));
if (this.matchedColumn) {
// We already found and matched the single column, all keys now pass
Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java Fri Aug 30 20:31:44 2013
@@ -23,7 +23,7 @@ import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
@@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.protobuf.
import com.google.protobuf.InvalidProtocolBufferException;
/**
- * A wrapper filter that filters an entire row if any of the KeyValue checks do
+ * A wrapper filter that filters an entire row if any of the Cell checks do
* not pass.
* <p>
* For example, if all columns in a row represent weights of different things,
@@ -45,7 +45,7 @@ import com.google.protobuf.InvalidProtoc
* new BinaryComparator(Bytes.toBytes(0))));
* </code>
* Any row which contained a column whose value was 0 will be filtered out
- * (since ValueFilter will not pass that KeyValue).
+ * (since ValueFilter will not pass that Cell).
* Without this filter, the other non-zero valued columns in the row would still
* be emitted.
*/
@@ -74,15 +74,15 @@ public class SkipFilter extends FilterBa
}
@Override
- public ReturnCode filterKeyValue(KeyValue v) throws IOException {
+ public ReturnCode filterKeyValue(Cell v) throws IOException {
ReturnCode c = filter.filterKeyValue(v);
changeFR(c != ReturnCode.INCLUDE);
return c;
}
@Override
- public KeyValue transform(KeyValue v) throws IOException {
- return filter.transform(v);
+ public Cell transformCell(Cell v) throws IOException {
+ return filter.transformCell(v);
}
public boolean filterRow() {
Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java Fri Aug 30 20:31:44 2013
@@ -21,7 +21,7 @@ import com.google.common.base.Preconditi
import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
@@ -88,7 +88,7 @@ public class TimestampsFilter extends Fi
}
@Override
- public ReturnCode filterKeyValue(KeyValue v) {
+ public ReturnCode filterKeyValue(Cell v) {
if (this.timestamps.contains(v.getTimestamp())) {
return ReturnCode.INCLUDE;
} else if (v.getTimestamp() < minTimeStamp) {
Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java Fri Aug 30 20:31:44 2013
@@ -22,7 +22,7 @@ package org.apache.hadoop.hbase.filter;
import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
@@ -58,8 +58,8 @@ public class ValueFilter extends Compare
}
@Override
- public ReturnCode filterKeyValue(KeyValue v) {
- if (doCompare(this.compareOp, this.comparator, v.getBuffer(),
+ public ReturnCode filterKeyValue(Cell v) {
+ if (doCompare(this.compareOp, this.comparator, v.getValueArray(),
v.getValueOffset(), v.getValueLength())) {
return ReturnCode.SKIP;
}
@@ -67,6 +67,7 @@ public class ValueFilter extends Compare
}
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
+ @SuppressWarnings("rawtypes") // for arguments
ArrayList arguments = CompareFilter.extractArguments(filterArguments);
CompareOp compareOp = (CompareOp)arguments.get(0);
ByteArrayComparable comparator = (ByteArrayComparable)arguments.get(1);
Modified: hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java (original)
+++ hbase/branches/0.95/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java Fri Aug 30 20:31:44 2013
@@ -23,7 +23,7 @@ import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
@@ -33,7 +33,7 @@ import com.google.protobuf.InvalidProtoc
/**
* A wrapper filter that returns true from {@link #filterAllRemaining()} as soon
* as the wrapped filters {@link Filter#filterRowKey(byte[], int, int)},
- * {@link Filter#filterKeyValue(org.apache.hadoop.hbase.KeyValue)},
+ * {@link Filter#filterKeyValue(org.apache.hadoop.hbase.Cell)},
* {@link org.apache.hadoop.hbase.filter.Filter#filterRow()} or
* {@link org.apache.hadoop.hbase.filter.Filter#filterAllRemaining()} methods
* returns true.
@@ -73,15 +73,15 @@ public class WhileMatchFilter extends Fi
}
@Override
- public ReturnCode filterKeyValue(KeyValue v) throws IOException {
+ public ReturnCode filterKeyValue(Cell v) throws IOException {
ReturnCode c = filter.filterKeyValue(v);
changeFAR(c != ReturnCode.INCLUDE);
return c;
}
@Override
- public KeyValue transform(KeyValue v) throws IOException {
- return filter.transform(v);
+ public Cell transformCell(Cell v) throws IOException {
+ return filter.transformCell(v);
}
@Override
Modified: hbase/branches/0.95/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestPayloadCarryingRpcController.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestPayloadCarryingRpcController.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestPayloadCarryingRpcController.java (original)
+++ hbase/branches/0.95/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestPayloadCarryingRpcController.java Fri Aug 30 20:31:44 2013
@@ -73,73 +73,73 @@ public class TestPayloadCarryingRpcContr
@Override
public byte[] getRowArray() {
- // TODO Auto-generated method stub
+ // unused
return null;
}
@Override
public int getRowOffset() {
- // TODO Auto-generated method stub
+ // unused
return 0;
}
@Override
public short getRowLength() {
- // TODO Auto-generated method stub
+ // unused
return 0;
}
@Override
public byte[] getFamilyArray() {
- // TODO Auto-generated method stub
+ // unused
return null;
}
@Override
public int getFamilyOffset() {
- // TODO Auto-generated method stub
+ // unused
return 0;
}
@Override
public byte getFamilyLength() {
- // TODO Auto-generated method stub
+ // unused
return 0;
}
@Override
public byte[] getQualifierArray() {
- // TODO Auto-generated method stub
+ // unused
return null;
}
@Override
public int getQualifierOffset() {
- // TODO Auto-generated method stub
+ // unused
return 0;
}
@Override
public int getQualifierLength() {
- // TODO Auto-generated method stub
+ // unused
return 0;
}
@Override
public long getTimestamp() {
- // TODO Auto-generated method stub
+ // unused
return 0;
}
@Override
public byte getTypeByte() {
- // TODO Auto-generated method stub
+ // unused
return 0;
}
@Override
public long getMvccVersion() {
- // TODO Auto-generated method stub
+ // unused
return 0;
}
@@ -160,19 +160,43 @@ public class TestPayloadCarryingRpcContr
@Override
public int getTagsOffset() {
- // TODO Auto-generated method stub
+ // unused
return 0;
}
@Override
public short getTagsLength() {
- // TODO Auto-generated method stub
+ // unused
return 0;
}
@Override
public byte[] getTagsArray() {
- // TODO Auto-generated method stub
+ // unused
+ return null;
+ }
+
+ @Override
+ public byte[] getValue() {
+ // unused
+ return null;
+ }
+
+ @Override
+ public byte[] getFamily() {
+ // unused
+ return null;
+ }
+
+ @Override
+ public byte[] getQualifier() {
+ // unused
+ return null;
+ }
+
+ @Override
+ public byte[] getRow() {
+ // unused
return null;
}
};
Modified: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java (original)
+++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java Fri Aug 30 20:31:44 2013
@@ -41,8 +41,10 @@ import org.apache.hadoop.classification.
* qualifier. Less intuitively, it will then treat the greater timestamp as the lesser value with
* the goal of sorting newer cells first.
* <p/>
- * This interface does not include methods that allocate new byte[]'s such as those used in client
- * or debugging code. These should be placed in a sub-interface or the {@link CellUtil} class.
+ * This interface should not include methods that allocate new byte[]'s such as those used in client
+ * or debugging code. These users should use the methods found in the {@link CellUtil} class.
+ * Currently for to minimize the impact of existing applications moving between 0.94 and 0.96, we
+ * include the costly helper methods marked as deprecated.
* <p/>
* Cell implements Comparable<Cell> which is only meaningful when comparing to other keys in the
* same table. It uses CellComparator which does not work on the -ROOT- and .META. tables.
@@ -182,5 +184,44 @@ public interface Cell {
* @return the total length of the tags in the Cell.
*/
short getTagsLength();
-
+
+ /**
+ * WARNING do not use, expensive. This gets an arraycopy of the cell's value.
+ *
+ * Added to ease transition from 0.94 -> 0.96.
+ *
+ * @deprecated as of 0.96, use {@link CellUtil#getValueArray(Cell)}
+ */
+ @Deprecated
+ byte[] getValue();
+
+ /**
+ * WARNING do not use, expensive. This gets an arraycopy of the cell's family.
+ *
+ * Added to ease transition from 0.94 -> 0.96.
+ *
+ * @deprecated as of 0.96, use {@link CellUtil#getFamilyArray(Cell)}
+ */
+ @Deprecated
+ byte[] getFamily();
+
+ /**
+ * WARNING do not use, expensive. This gets an arraycopy of the cell's qualifier.
+ *
+ * Added to ease transition from 0.94 -> 0.96.
+ *
+ * @deprecated as of 0.96, use {@link CellUtil#getQualifierArray(Cell)}
+ */
+ @Deprecated
+ byte[] getQualifier();
+
+ /**
+ * WARNING do not use, expensive. this gets an arraycopy of the cell's row.
+ *
+ * Added to ease transition from 0.94 -> 0.96.
+ *
+ * @deprecated as of 0.96, use {@link CellUtil#getRowByte(Cell, int)}
+ */
+ @Deprecated
+ byte[] getRow();
}
Modified: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java (original)
+++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java Fri Aug 30 20:31:44 2013
@@ -27,6 +27,7 @@ import java.util.NavigableMap;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.KeyValue.Type;
import org.apache.hadoop.hbase.util.ByteRange;
import org.apache.hadoop.hbase.util.Bytes;
@@ -257,6 +258,41 @@ public final class CellUtil {
right.getRowArray(), right.getRowOffset(), right.getRowLength());
}
+ public static boolean matchingRow(final Cell left, final byte[] buf) {
+ return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
+ buf, 0, buf.length);
+ }
+
+ public static boolean matchingFamily(final Cell left, final Cell right) {
+ return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
+ right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
+ }
+
+ public static boolean matchingFamily(final Cell left, final byte[] buf) {
+ return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
+ buf, 0, buf.length);
+ }
+
+ public static boolean matchingQualifier(final Cell left, final Cell right) {
+ return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(), left.getQualifierLength(),
+ right.getQualifierArray(), right.getQualifierOffset(), right.getQualifierLength());
+ }
+
+ public static boolean matchingQualifier(final Cell left, final byte[] buf) {
+ return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(), left.getQualifierLength(),
+ buf, 0, buf.length);
+ }
+
+
+ public static boolean matchingValue(final Cell left, final Cell right) {
+ return Bytes.equals(left.getValueArray(), left.getValueOffset(), left.getValueLength(),
+ right.getValueArray(), right.getValueOffset(), right.getValueLength());
+ }
+
+ public static boolean matchingValue(final Cell left, final byte[] buf) {
+ return Bytes.equals(left.getValueArray(), left.getValueOffset(), left.getValueLength(),
+ buf, 0, buf.length);
+ }
/**
* @return True if a delete type, a {@link KeyValue.Type#Delete} or
* a {KeyValue.Type#DeleteFamily} or a {@link KeyValue.Type#DeleteColumn}
@@ -266,6 +302,10 @@ public final class CellUtil {
return KeyValue.isDelete(cell.getTypeByte());
}
+ public static boolean isDeleteFamily(final Cell cell) {
+ return cell.getTypeByte() == Type.DeleteFamily.getCode();
+ }
+
/**
* @param cell
* @return Estimate of the <code>cell</code> size in bytes.
@@ -286,4 +326,4 @@ public final class CellUtil {
// Serialization is probably preceded by a length (it is in the KeyValueCodec at least).
Bytes.SIZEOF_INT;
}
-}
\ No newline at end of file
+}
Modified: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java (original)
+++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java Fri Aug 30 20:31:44 2013
@@ -26,7 +26,6 @@ import java.io.DataOutput;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
-import java.nio.BufferOverflowException;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Comparator;
@@ -36,7 +35,6 @@ import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
@@ -46,7 +44,10 @@ import org.apache.hadoop.io.RawComparato
import com.google.common.primitives.Longs;
/**
- * An HBase Key/Value. This is the fundamental HBase Type.
+ * An HBase Key/Value. This is the fundamental HBase Type.
+ * <p>
+ * HBase applications and users should use the Cell interface and avoid directly using KeyValue
+ * and member functions not defined in Cell.
* <p>
* If being used client-side, the primary methods to access individual fields are {@link #getRow()},
* {@link #getFamily()}, {@link #getQualifier()}, {@link #getTimestamp()}, and {@link #getValue()}.
@@ -66,8 +67,7 @@ import com.google.common.primitives.Long
* <code>Integer.MAX_SIZE</code>. The column does not contain the family/qualifier delimiter,
* {@link #COLUMN_FAMILY_DELIMITER}
*/
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
+@InterfaceAudience.Private
public class KeyValue implements Cell, HeapSize, Cloneable {
static final Log LOG = LogFactory.getLog(KeyValue.class);
@@ -472,6 +472,14 @@ public class KeyValue implements Cell, H
this.offset = 0;
}
+ public KeyValue(Cell c) {
+ this(c.getRowArray(), c.getRowOffset(), (int)c.getRowLength(),
+ c.getFamilyArray(), c.getFamilyOffset(), (int)c.getFamilyLength(),
+ c.getQualifierArray(), c.getQualifierOffset(), (int) c.getQualifierLength(),
+ c.getTimestamp(), Type.codeToType(c.getTypeByte()),
+ c.getValueArray(), c.getValueOffset(), c.getValueLength());
+ }
+
/**
* Create an empty byte[] representing a KeyValue
* All lengths are preset and can be filled in later.
@@ -1175,34 +1183,9 @@ public class KeyValue implements Cell, H
* save on allocations.
* @return Value in a new byte array.
*/
+ @Deprecated // use CellUtil.getValueArray()
public byte [] getValue() {
- int o = getValueOffset();
- int l = getValueLength();
- byte [] result = new byte[l];
- System.arraycopy(getBuffer(), o, result, 0, l);
- return result;
- }
-
- /**
- * Returns the value wrapped in a new <code>ByteBuffer</code>.
- *
- * @return the value
- */
- public ByteBuffer getValueAsByteBuffer() {
- return ByteBuffer.wrap(getBuffer(), getValueOffset(), getValueLength());
- }
-
- /**
- * Loads this object's value into the provided <code>ByteBuffer</code>.
- * <p>
- * Does not clear or flip the buffer.
- *
- * @param dst the buffer where to write the value
- *
- * @throws BufferOverflowException if there is insufficient space remaining in the buffer
- */
- public void loadValue(ByteBuffer dst) throws BufferOverflowException {
- dst.put(getBuffer(), getValueOffset(), getValueLength());
+ return CellUtil.getValueArray(this);
}
/**
@@ -1213,12 +1196,9 @@ public class KeyValue implements Cell, H
* lengths instead.
* @return Row in a new byte array.
*/
+ @Deprecated // use CellUtil.getRowArray()
public byte [] getRow() {
- int o = getRowOffset();
- short l = getRowLength();
- byte result[] = new byte[l];
- System.arraycopy(getBuffer(), o, result, 0, l);
- return result;
+ return CellUtil.getRowArray(this);
}
/**
@@ -1260,6 +1240,7 @@ public class KeyValue implements Cell, H
* a {KeyValue.Type#DeleteFamily} or a {@link KeyValue.Type#DeleteColumn}
* KeyValue type.
*/
+ @Deprecated // use CellUtil#isDelete
public boolean isDelete() {
return KeyValue.isDelete(getType());
}
@@ -1303,12 +1284,9 @@ public class KeyValue implements Cell, H
* lengths instead.
* @return Returns family. Makes a copy.
*/
+ @Deprecated // use CellUtil.getFamilyArray
public byte [] getFamily() {
- int o = getFamilyOffset();
- int l = getFamilyLength(o);
- byte [] result = new byte[l];
- System.arraycopy(this.bytes, o, result, 0, l);
- return result;
+ return CellUtil.getFamilyArray(this);
}
/**
@@ -1320,12 +1298,9 @@ public class KeyValue implements Cell, H
* Use {@link #getBuffer()} with appropriate offsets and lengths instead.
* @return Returns qualifier. Makes a copy.
*/
+ @Deprecated // use CellUtil.getQualifierArray
public byte [] getQualifier() {
- int o = getQualifierOffset();
- int l = getQualifierLength();
- byte [] result = new byte[l];
- System.arraycopy(this.bytes, o, result, 0, l);
- return result;
+ return CellUtil.getQualifierArray(this);
}
//---------------------------------------------------------------------------
@@ -1419,6 +1394,8 @@ public class KeyValue implements Cell, H
/**
* Creates a new KeyValue that only contains the key portion (the value is
* set to be null).
+ *
+ * TODO only used by KeyOnlyFilter -- move there.
* @param lenAsVal replace value with the actual value length (false=empty)
*/
public KeyValue createKeyOnly(boolean lenAsVal) {
@@ -2586,4 +2563,5 @@ public class KeyValue implements Cell, H
public byte[] getTagsArray() {
throw new UnsupportedOperationException("Not implememnted");
}
+
}
Modified: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTestUtil.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTestUtil.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTestUtil.java (original)
+++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueTestUtil.java Fri Aug 30 20:31:44 2013
@@ -80,11 +80,11 @@ public class KeyValueTestUtil {
* @param kvCollection2
* @return true if KeyValues from kvCollection2 are contained in kvCollection1
*/
- public static boolean containsIgnoreMvccVersion(Collection<KeyValue> kvCollection1,
- Collection<KeyValue> kvCollection2) {
- for (KeyValue kv1 : kvCollection1) {
+ public static boolean containsIgnoreMvccVersion(Collection<? extends Cell> kvCollection1,
+ Collection<? extends Cell> kvCollection2) {
+ for (Cell kv1 : kvCollection1) {
boolean found = false;
- for (KeyValue kv2 : kvCollection2) {
+ for (Cell kv2 : kvCollection2) {
if (CellComparator.equalsIgnoreMvccVersion(kv1, kv2)) found = true;
}
if (!found) return false;
Modified: hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java (original)
+++ hbase/branches/0.95/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java Fri Aug 30 20:31:44 2013
@@ -26,6 +26,7 @@ import java.util.Iterator;
import org.apache.commons.lang.NotImplementedException;
import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.io.compress.Compression;
@@ -73,7 +74,7 @@ public class EncodedDataBlock {
* @param headerSize header size of the block.
* @return Forwards sequential iterator.
*/
- public Iterator<KeyValue> getIterator(int headerSize) {
+ public Iterator<Cell> getIterator(int headerSize) {
final int rawSize = rawKVs.length;
byte[] encodedDataWithHeader = getEncodedData();
int bytesToSkip = headerSize + Bytes.SIZEOF_SHORT;
@@ -81,7 +82,7 @@ public class EncodedDataBlock {
bytesToSkip, encodedDataWithHeader.length - bytesToSkip);
final DataInputStream dis = new DataInputStream(bais);
- return new Iterator<KeyValue>() {
+ return new Iterator<Cell>() {
private ByteBuffer decompressedData = null;
@Override
@@ -93,7 +94,7 @@ public class EncodedDataBlock {
}
@Override
- public KeyValue next() {
+ public Cell next() {
if (decompressedData == null) {
try {
decompressedData = dataBlockEncoder.decodeKeyValues(
Modified: hbase/branches/0.95/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/BulkDeleteEndpoint.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/BulkDeleteEndpoint.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/BulkDeleteEndpoint.java (original)
+++ hbase/branches/0.95/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/BulkDeleteEndpoint.java Fri Aug 30 20:31:44 2013
@@ -26,10 +26,11 @@ import java.util.TreeSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Mutation;
@@ -132,9 +133,9 @@ public class BulkDeleteEndpoint extends
// filter and having necessary column(s).
scanner = region.getScanner(scan);
while (hasMore) {
- List<List<KeyValue>> deleteRows = new ArrayList<List<KeyValue>>(rowBatchSize);
+ List<List<Cell>> deleteRows = new ArrayList<List<Cell>>(rowBatchSize);
for (int i = 0; i < rowBatchSize; i++) {
- List<KeyValue> results = new ArrayList<KeyValue>();
+ List<Cell> results = new ArrayList<Cell>();
hasMore = scanner.next(results);
if (results.size() > 0) {
deleteRows.add(results);
@@ -147,7 +148,7 @@ public class BulkDeleteEndpoint extends
if (deleteRows.size() > 0) {
Mutation[] deleteArr = new Mutation[deleteRows.size()];
int i = 0;
- for (List<KeyValue> deleteRow : deleteRows) {
+ for (List<Cell> deleteRow : deleteRows) {
deleteArr[i++] = createDeleteMutation(deleteRow, deleteType, timestamp);
}
OperationStatus[] opStatus = region.batchMutate(deleteArr);
@@ -188,7 +189,7 @@ public class BulkDeleteEndpoint extends
done.run(result);
}
- private Delete createDeleteMutation(List<KeyValue> deleteRow, DeleteType deleteType,
+ private Delete createDeleteMutation(List<Cell> deleteRow, DeleteType deleteType,
Long timestamp) {
long ts;
if (timestamp == null) {
@@ -197,19 +198,19 @@ public class BulkDeleteEndpoint extends
ts = timestamp;
}
// We just need the rowkey. Get it from 1st KV.
- byte[] row = deleteRow.get(0).getRow();
+ byte[] row = CellUtil.getRowArray(deleteRow.get(0));
Delete delete = new Delete(row, ts);
if (deleteType == DeleteType.FAMILY) {
Set<byte[]> families = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
- for (KeyValue kv : deleteRow) {
- if (families.add(kv.getFamily())) {
- delete.deleteFamily(kv.getFamily(), ts);
+ for (Cell kv : deleteRow) {
+ if (families.add(CellUtil.getFamilyArray(kv))) {
+ delete.deleteFamily(CellUtil.getFamilyArray(kv), ts);
}
}
} else if (deleteType == DeleteType.COLUMN) {
Set<Column> columns = new HashSet<Column>();
- for (KeyValue kv : deleteRow) {
- Column column = new Column(kv.getFamily(), kv.getQualifier());
+ for (Cell kv : deleteRow) {
+ Column column = new Column(CellUtil.getFamilyArray(kv), CellUtil.getQualifierArray(kv));
if (columns.add(column)) {
// Making deleteColumns() calls more than once for the same cf:qualifier is not correct
// Every call to deleteColumns() will add a new KV to the familymap which will finally
@@ -224,14 +225,14 @@ public class BulkDeleteEndpoint extends
// the scan fetched will get deleted.
int noOfVersionsToDelete = 0;
if (timestamp == null) {
- for (KeyValue kv : deleteRow) {
- delete.deleteColumn(kv.getFamily(), kv.getQualifier(), kv.getTimestamp());
+ for (Cell kv : deleteRow) {
+ delete.deleteColumn(CellUtil.getFamilyArray(kv), CellUtil.getQualifierArray(kv), kv.getTimestamp());
noOfVersionsToDelete++;
}
} else {
Set<Column> columns = new HashSet<Column>();
- for (KeyValue kv : deleteRow) {
- Column column = new Column(kv.getFamily(), kv.getQualifier());
+ for (Cell kv : deleteRow) {
+ Column column = new Column(CellUtil.getFamilyArray(kv), CellUtil.getQualifierArray(kv));
// Only one version of particular column getting deleted.
if (columns.add(column)) {
delete.deleteColumn(column.family, column.qualifier, ts);
Modified: hbase/branches/0.95/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.java (original)
+++ hbase/branches/0.95/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.java Fri Aug 30 20:31:44 2013
@@ -18,12 +18,14 @@
package org.apache.hadoop.hbase.coprocessor.example;
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.Service;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
-import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
@@ -34,9 +36,9 @@ import org.apache.hadoop.hbase.protobuf.
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.util.Bytes;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.Service;
/**
* Sample coprocessor endpoint exposing a Service interface for counting rows and key values.
@@ -73,14 +75,14 @@ public class RowCountEndpoint extends Ex
InternalScanner scanner = null;
try {
scanner = env.getRegion().getScanner(scan);
- List<KeyValue> results = new ArrayList<KeyValue>();
+ List<Cell> results = new ArrayList<Cell>();
boolean hasMore = false;
byte[] lastRow = null;
long count = 0;
do {
hasMore = scanner.next(results);
- for (KeyValue kv : results) {
- byte[] currentRow = kv.getRow();
+ for (Cell kv : results) {
+ byte[] currentRow = CellUtil.getRowArray(kv);
if (lastRow == null || !Bytes.equals(lastRow, currentRow)) {
lastRow = currentRow;
count++;
@@ -113,12 +115,12 @@ public class RowCountEndpoint extends Ex
InternalScanner scanner = null;
try {
scanner = env.getRegion().getScanner(new Scan());
- List<KeyValue> results = new ArrayList<KeyValue>();
+ List<Cell> results = new ArrayList<Cell>();
boolean hasMore = false;
long count = 0;
do {
hasMore = scanner.next(results);
- for (KeyValue kv : results) {
+ for (Cell kv : results) {
count++;
}
results.clear();
Modified: hbase/branches/0.95/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestBulkDeleteProtocol.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestBulkDeleteProtocol.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestBulkDeleteProtocol.java (original)
+++ hbase/branches/0.95/hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestBulkDeleteProtocol.java Fri Aug 30 20:31:44 2013
@@ -25,11 +25,12 @@ import java.util.ArrayList;
import java.util.List;
import java.util.Map;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTable;
@@ -39,22 +40,18 @@ import org.apache.hadoop.hbase.client.Sc
import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest;
-import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse;
-import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteService;
import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.Builder;
import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType;
-import org.apache.hadoop.hbase.filter.FilterList;
-import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse;
+import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteService;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.filter.FilterList;
import org.apache.hadoop.hbase.filter.FilterList.Operator;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
import org.apache.hadoop.hbase.ipc.ServerRpcController;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.Ignore;
import org.junit.experimental.categories.Category;
@Category(MediumTests.class)
@@ -276,17 +273,17 @@ public class TestBulkDeleteProtocol {
scan.setMaxVersions();
for (Result result : ht.getScanner(scan)) {
assertEquals(3, result.getFamilyMap(FAMILY1).size());
- List<KeyValue> column = result.getColumn(FAMILY1, QUALIFIER1);
+ List<Cell> column = result.getColumn(FAMILY1, QUALIFIER1);
assertEquals(1, column.size());
- assertTrue(Bytes.equals("v1".getBytes(), column.get(0).getValue()));
+ assertTrue(CellUtil.matchingValue(column.get(0), "v1".getBytes()));
column = result.getColumn(FAMILY1, QUALIFIER2);
assertEquals(1, column.size());
- assertTrue(Bytes.equals("v1".getBytes(), column.get(0).getValue()));
+ assertTrue(CellUtil.matchingValue(column.get(0), "v1".getBytes()));
column = result.getColumn(FAMILY1, QUALIFIER3);
assertEquals(1, column.size());
- assertTrue(Bytes.equals("v1".getBytes(), column.get(0).getValue()));
+ assertTrue(CellUtil.matchingValue(column.get(0), "v1".getBytes()));
rows++;
}
assertEquals(100, rows);
@@ -330,10 +327,10 @@ public class TestBulkDeleteProtocol {
assertEquals(3, result.getFamilyMap(FAMILY1).size());
assertEquals(3, result.getColumn(FAMILY1, QUALIFIER1).size());
assertEquals(3, result.getColumn(FAMILY1, QUALIFIER2).size());
- List<KeyValue> column = result.getColumn(FAMILY1, QUALIFIER3);
+ List<Cell> column = result.getColumn(FAMILY1, QUALIFIER3);
assertEquals(2, column.size());
- assertTrue(Bytes.equals("v3".getBytes(), column.get(0).getValue()));
- assertTrue(Bytes.equals("v1".getBytes(), column.get(1).getValue()));
+ assertTrue(CellUtil.matchingValue(column.get(0), "v3".getBytes()));
+ assertTrue(CellUtil.matchingValue(column.get(1), "v1".getBytes()));
rows++;
}
assertEquals(100, rows);
@@ -410,14 +407,14 @@ public class TestBulkDeleteProtocol {
scan1.setMaxVersions();
for (Result res : ht.getScanner(scan1)) {
assertEquals(3, res.getFamilyMap(FAMILY1).size());
- List<KeyValue> column = res.getColumn(FAMILY1, QUALIFIER1);
+ List<Cell> column = res.getColumn(FAMILY1, QUALIFIER1);
assertEquals(2, column.size());
- assertTrue(Bytes.equals("v4".getBytes(), column.get(0).getValue()));
- assertTrue(Bytes.equals("v3".getBytes(), column.get(1).getValue()));
+ assertTrue(CellUtil.matchingValue(column.get(0), "v4".getBytes()));
+ assertTrue(CellUtil.matchingValue(column.get(1), "v3".getBytes()));
column = res.getColumn(FAMILY1, QUALIFIER2);
assertEquals(2, column.size());
- assertTrue(Bytes.equals("v4".getBytes(), column.get(0).getValue()));
- assertTrue(Bytes.equals("v3".getBytes(), column.get(1).getValue()));
+ assertTrue(CellUtil.matchingValue(column.get(0), "v4".getBytes()));
+ assertTrue(CellUtil.matchingValue(column.get(1), "v3".getBytes()));
assertEquals(4, res.getColumn(FAMILY1, QUALIFIER3).size());
rows++;
}
Modified: hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java (original)
+++ hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestBulkLoad.java Fri Aug 30 20:31:44 2013
@@ -18,6 +18,8 @@
*/
package org.apache.hadoop.hbase.mapreduce;
+import static org.junit.Assert.assertEquals;
+
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
@@ -30,6 +32,8 @@ import java.util.Set;
import org.apache.commons.lang.RandomStringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.IntegrationTestBase;
import org.apache.hadoop.hbase.IntegrationTestingUtility;
@@ -65,8 +69,6 @@ import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
-import static org.junit.Assert.assertEquals;
-
/**
* Test Bulk Load and MR on a distributed cluster.
* It starts an MR job that creates linked chains
@@ -468,7 +470,8 @@ public class IntegrationTestBulkLoad ext
for (Map.Entry<byte[], byte[]> entry : value.getFamilyMap(CHAIN_FAM).entrySet()) {
long chainId = Bytes.toLong(entry.getKey());
long next = Bytes.toLong(entry.getValue());
- long order = Bytes.toLong(value.getColumn(SORT_FAM, entry.getKey()).get(0).getValue());
+ Cell c = value.getColumn(SORT_FAM, entry.getKey()).get(0);
+ long order = Bytes.toLong(CellUtil.getValueArray(c));
context.write(new LinkKey(chainId, order), new LinkChain(longRk, next));
}
}
Modified: hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java (original)
+++ hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestImportTsv.java Fri Aug 30 20:31:44 2013
@@ -17,6 +17,11 @@
*/
package org.apache.hadoop.hbase.mapreduce;
+import static java.lang.String.format;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
import java.io.IOException;
import java.util.Arrays;
import java.util.Iterator;
@@ -31,6 +36,7 @@ import org.apache.hadoop.conf.Configurat
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -62,11 +68,6 @@ import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
-import static java.lang.String.format;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
/**
* Validate ImportTsv + LoadIncrementalHFiles on a distributed cluster.
*/
@@ -153,7 +154,7 @@ public class IntegrationTestImportTsv im
Iterator<KeyValue> expectedIt = simple_expected.iterator();
while (resultsIt.hasNext() && expectedIt.hasNext()) {
Result r = resultsIt.next();
- for (KeyValue actual : r.raw()) {
+ for (Cell actual : r.raw()) {
assertTrue(
"Ran out of expected values prematurely!",
expectedIt.hasNext());
Modified: hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java (original)
+++ hbase/branches/0.95/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java Fri Aug 30 20:31:44 2013
@@ -17,6 +17,9 @@
*/
package org.apache.hadoop.hbase.test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
import java.io.IOException;
import java.util.Random;
import java.util.Set;
@@ -24,13 +27,13 @@ import java.util.UUID;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
-import com.google.common.collect.Sets;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
@@ -38,7 +41,6 @@ import org.apache.hadoop.hbase.HTableDes
import org.apache.hadoop.hbase.IntegrationTestBase;
import org.apache.hadoop.hbase.IntegrationTestingUtility;
import org.apache.hadoop.hbase.IntegrationTests;
-import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
@@ -65,8 +67,7 @@ import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import com.google.common.collect.Sets;
/**
* A large test which loads a lot of data that has internal references, and
@@ -247,12 +248,12 @@ public class IntegrationTestLoadAndVerif
throws IOException, InterruptedException {
BytesWritable bwKey = new BytesWritable(key.get());
BytesWritable bwVal = new BytesWritable();
- for (KeyValue kv : value.list()) {
+ for (Cell kv : value.list()) {
if (Bytes.compareTo(TEST_QUALIFIER, 0, TEST_QUALIFIER.length,
- kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength()) == 0) {
+ kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength()) == 0) {
context.write(bwKey, EMPTY);
} else {
- bwVal.set(kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength());
+ bwVal.set(kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength());
context.write(bwVal, bwKey);
}
}
Modified: hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeCell.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeCell.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeCell.java (original)
+++ hbase/branches/0.95/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/decode/PrefixTreeCell.java Fri Aug 30 20:31:44 2013
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.codec.pr
import org.apache.hadoop.classification.InterfaceAudience;
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;
@@ -183,6 +184,26 @@ public class PrefixTreeCell implements C
return type.getCode();
}
+ /* Deprecated methods pushed into the Cell interface */
+ @Override
+ public byte[] getValue() {
+ return CellUtil.getValueArray(this);
+ }
+
+ @Override
+ public byte[] getFamily() {
+ return CellUtil.getFamilyArray(this);
+ }
+
+ @Override
+ public byte[] getQualifier() {
+ return CellUtil.getQualifierArray(this);
+ }
+
+ @Override
+ public byte[] getRow() {
+ return CellUtil.getRowArray(this);
+ }
/************************* helper methods *************************/
@@ -208,4 +229,5 @@ public class PrefixTreeCell implements C
public byte[] getTagsArray() {
throw new UnsupportedOperationException("Not implemented");
}
+
}
Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java Fri Aug 30 20:31:44 2013
@@ -15,27 +15,29 @@
package org.apache.hadoop.hbase.coprocessor;
+import java.io.IOException;
+import java.util.ArrayList;
import java.util.List;
import java.util.NavigableSet;
-import com.google.common.collect.ImmutableList;
-
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
@@ -48,8 +50,9 @@ import org.apache.hadoop.hbase.regionser
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.util.Pair;
+import org.mortbay.log.Log;
-import java.io.IOException;
+import com.google.common.collect.ImmutableList;
/**
* An abstract class that implements RegionObserver.
@@ -209,15 +212,56 @@ public abstract class BaseRegionObserver
}
@Override
- public void preGet(final ObserverContext<RegionCoprocessorEnvironment> e,
- final Get get, final List<KeyValue> results) throws IOException {
+ public void preGetOp(final ObserverContext<RegionCoprocessorEnvironment> e,
+ final Get get, final List<Cell> results) throws IOException {
+ // By default we are executing the deprecated preGet to support legacy RegionObservers
+ // We may use the results coming in and we may return the results going out.
+ List<KeyValue> kvs = new ArrayList<KeyValue>(results.size());
+ for (Cell c : results) {
+ kvs.add(KeyValueUtil.ensureKeyValue(c));
+ }
+ preGet(e, get, kvs);
+ results.clear();
+ results.addAll(kvs);
+ }
+
+ /**
+ * WARNING: please override preGetOp instead of this method. This is to maintain some
+ * compatibility and to ease the transition from 0.94 -> 0.96. It is super inefficient!
+ */
+ @Deprecated
+ @Override
+ public void preGet(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
+ final List<KeyValue> result)
+ throws IOException {
}
@Override
- public void postGet(final ObserverContext<RegionCoprocessorEnvironment> e,
- final Get get, final List<KeyValue> results) throws IOException {
+ public void postGetOp(final ObserverContext<RegionCoprocessorEnvironment> e,
+ final Get get, final List<Cell> results) throws IOException {
+ // By default we are executing the deprecated preGet to support legacy RegionObservers
+ // We may use the results coming in and we may return the results going out.
+ List<KeyValue> kvs = new ArrayList<KeyValue>(results.size());
+ for (Cell c : results) {
+ kvs.add(KeyValueUtil.ensureKeyValue(c));
+ }
+ postGet(e, get, kvs);
+ results.clear();
+ results.addAll(kvs);
+ }
+
+ /**
+ * WARNING: please override postGetOp instead of this method. This is to maintain some
+ * compatibility and to ease the transition from 0.94 -> 0.96. It is super inefficient!
+ */
+ @Deprecated
+ @Override
+ public void postGet(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
+ final List<KeyValue> result)
+ throws IOException {
}
+
@Override
public boolean preExists(final ObserverContext<RegionCoprocessorEnvironment> e,
final Get get, final boolean exists) throws IOException {
Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java Fri Aug 30 20:31:44 2013
@@ -21,6 +21,7 @@ import java.util.NavigableSet;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
@@ -429,6 +430,15 @@ public interface RegionObserver extends
* is not bypassed.
* @throws IOException if an error occurred on the coprocessor
*/
+ void preGetOp(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
+ final List<Cell> result)
+ throws IOException;
+
+ /**
+ * WARNING: please override preGetOp instead of this method. This is to maintain some
+ * compatibility and to ease the transition from 0.94 -> 0.96.
+ */
+ @Deprecated
void preGet(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
final List<KeyValue> result)
throws IOException;
@@ -443,6 +453,15 @@ public interface RegionObserver extends
* @param result the result to return to the client, modify as necessary
* @throws IOException if an error occurred on the coprocessor
*/
+ void postGetOp(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
+ final List<Cell> result)
+ throws IOException;
+
+ /**
+ * WARNING: please override postGetOp instead of this method. This is to maintain some
+ * compatibility and to ease the transition from 0.94 -> 0.96.
+ */
+ @Deprecated
void postGet(final ObserverContext<RegionCoprocessorEnvironment> c, final Get get,
final List<KeyValue> result)
throws IOException;
Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/GroupingTableMap.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/GroupingTableMap.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/GroupingTableMap.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapred/GroupingTableMap.java Fri Aug 30 20:31:44 2013
@@ -20,11 +20,12 @@ package org.apache.hadoop.hbase.mapred;
import java.io.IOException;
import java.util.ArrayList;
-import java.util.Map;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.MapReduceBase;
@@ -115,12 +116,12 @@ implements TableMap<ImmutableBytesWritab
ArrayList<byte[]> foundList = new ArrayList<byte[]>();
int numCols = columns.length;
if (numCols > 0) {
- for (KeyValue value: r.list()) {
- byte [] column = KeyValue.makeColumn(value.getFamily(),
- value.getQualifier());
+ for (Cell value: r.list()) {
+ byte [] column = KeyValue.makeColumn(CellUtil.getFamilyArray(value),
+ CellUtil.getQualifierArray(value));
for (int i = 0; i < numCols; i++) {
if (Bytes.equals(column, columns[i])) {
- foundList.add(value.getValue());
+ foundList.add(CellUtil.getValueArray(value));
break;
}
}
Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CellCounter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CellCounter.java?rev=1519076&r1=1519075&r2=1519076&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CellCounter.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/CellCounter.java Fri Aug 30 20:31:44 2013
@@ -26,8 +26,9 @@ import org.apache.hadoop.classification.
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.filter.CompareFilter;
@@ -115,9 +116,9 @@ public class CellCounter {
context.getCounter(Counters.ROWS).increment(1);
context.write(new Text("Total ROWS"), new IntWritable(1));
- for (KeyValue value : values.list()) {
- currentRowKey = Bytes.toStringBinary(value.getRow());
- String thisRowFamilyName = Bytes.toStringBinary(value.getFamily());
+ for (Cell value : values.list()) {
+ currentRowKey = Bytes.toStringBinary(CellUtil.getRowArray(value));
+ String thisRowFamilyName = Bytes.toStringBinary(CellUtil.getFamilyArray(value));
if (thisRowFamilyName != null &&
!thisRowFamilyName.equals(currentFamilyName)) {
currentFamilyName = thisRowFamilyName;
@@ -127,7 +128,7 @@ public class CellCounter {
context.write(new Text(thisRowFamilyName), new IntWritable(1));
}
String thisRowQualifierName = thisRowFamilyName + separator
- + Bytes.toStringBinary(value.getQualifier());
+ + Bytes.toStringBinary(CellUtil.getQualifierArray(value));
if (thisRowQualifierName != null &&
!thisRowQualifierName.equals(currentQualifierName)) {
currentQualifierName = thisRowQualifierName;