You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2014/03/03 22:57:24 UTC
svn commit: r1573774 - in /hbase/trunk:
hbase-client/src/main/java/org/apache/hadoop/hbase/filter/
hbase-server/src/test/java/org/apache/hadoop/hbase/client/
hbase-server/src/test/java/org/apache/hadoop/hbase/filter/
hbase-server/src/test/java/org/apac...
Author: tedyu
Date: Mon Mar 3 21:57:23 2014
New Revision: 1573774
URL: http://svn.apache.org/r1573774
Log:
HBASE-10609 Remove filterKeyValue(Cell ignored) from FilterBase
Modified:
hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java
hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java
hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java?rev=1573774&r1=1573773&r2=1573774&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java Mon Mar 3 21:57:23 2014
@@ -98,6 +98,10 @@ public abstract class Filter {
* code is described below. This allows filters to filter only certain number of columns, then
* terminate without matching ever column.
*
+ * If filterRowKey returns true, filterKeyValue needs to be consistent with it.
+ *
+ * filterKeyValue can assume that filterRowKey has already been called for the row.
+ *
* If your filter returns <code>ReturnCode.NEXT_ROW</code>, it should return
* <code>ReturnCode.NEXT_ROW</code> until {@link #reset()} is called just in case the caller calls
* for the next row.
Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java?rev=1573774&r1=1573773&r2=1573774&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java Mon Mar 3 21:57:23 2014
@@ -73,17 +73,6 @@ public abstract class FilterBase extends
}
/**
- * Filters that dont filter by key value can inherit this implementation that
- * includes all Cells.
- *
- * @inheritDoc
- */
- @Override
- public ReturnCode filterKeyValue(Cell ignored) throws IOException {
- return ReturnCode.INCLUDE;
- }
-
- /**
* By default no transformation takes place
*
* @inheritDoc
Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java?rev=1573774&r1=1573773&r2=1573774&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java Mon Mar 3 21:57:23 2014
@@ -19,6 +19,7 @@
package org.apache.hadoop.hbase.filter;
+import java.io.IOException;
import java.util.ArrayList;
import org.apache.hadoop.classification.InterfaceAudience;
@@ -27,6 +28,7 @@ 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.Filter.ReturnCode;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import com.google.common.base.Preconditions;
@@ -57,6 +59,11 @@ public class KeyOnlyFilter extends Filte
return v.createKeyOnly(this.lenAsVal);
}
+ @Override
+ public ReturnCode filterKeyValue(Cell ignored) throws IOException {
+ return ReturnCode.INCLUDE;
+ }
+
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
Preconditions.checkArgument((filterArguments.size() == 0 || filterArguments.size() == 1),
"Expected: 0 or 1 but got: %s", filterArguments.size());
Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java?rev=1573774&r1=1573773&r2=1573774&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java Mon Mar 3 21:57:23 2014
@@ -22,9 +22,12 @@ 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.Cell;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
+import java.io.IOException;
import java.util.ArrayList;
/**
* Implementation of Filter interface that limits results to a specific page
@@ -57,6 +60,11 @@ public class PageFilter extends FilterBa
return pageSize;
}
+ @Override
+ public ReturnCode filterKeyValue(Cell ignored) throws IOException {
+ return ReturnCode.INCLUDE;
+ }
+
public boolean filterAllRemaining() {
return this.rowsAccepted >= this.pageSize;
}
Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java?rev=1573774&r1=1573773&r2=1573774&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java Mon Mar 3 21:57:23 2014
@@ -516,6 +516,10 @@ public class TestHCM {
syncBlockingFilter.set(true);
return false;
}
+ @Override
+ public ReturnCode filterKeyValue(Cell ignored) throws IOException {
+ return ReturnCode.INCLUDE;
+ }
public static Filter parseFrom(final byte [] pbBytes) throws DeserializationException{
return new BlockingFilter();
Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java?rev=1573774&r1=1573773&r2=1573774&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java Mon Mar 3 21:57:23 2014
@@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.client.Du
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
import org.apache.hadoop.hbase.filter.FilterList.Operator;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
@@ -598,6 +599,11 @@ public class TestFilter {
// always filter out rows
return true;
}
+
+ @Override
+ public ReturnCode filterKeyValue(Cell ignored) throws IOException {
+ return ReturnCode.INCLUDE;
+ }
}
/**
@@ -1985,6 +1991,11 @@ public class TestFilter {
public byte [] toByteArray() {return null;}
@Override
+ public ReturnCode filterKeyValue(Cell ignored) throws IOException {
+ return ReturnCode.INCLUDE;
+ }
+
+ @Override
public boolean filterRow() throws IOException {
ipcHandlerThread = Thread.currentThread();
try {
Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java?rev=1573774&r1=1573773&r2=1573774&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java Mon Mar 3 21:57:23 2014
@@ -419,7 +419,14 @@ public class TestFilterList {
Filter filterNoHint = new FilterBase() {
@Override
- public byte [] toByteArray() {return null;}
+ public byte [] toByteArray() {
+ return null;
+ }
+
+ @Override
+ public ReturnCode filterKeyValue(Cell ignored) throws IOException {
+ return ReturnCode.INCLUDE;
+ }
};
Filter filterMinHint = new FilterBase() {
Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java?rev=1573774&r1=1573773&r2=1573774&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java Mon Mar 3 21:57:23 2014
@@ -96,6 +96,7 @@ import org.apache.hadoop.hbase.exception
import org.apache.hadoop.hbase.filter.BinaryComparator;
import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.filter.FilterBase;
import org.apache.hadoop.hbase.filter.FilterList;
@@ -2639,6 +2640,10 @@ public class TestHRegion {
scan.setLoadColumnFamiliesOnDemand(true);
Filter bogusFilter = new FilterBase() {
@Override
+ public ReturnCode filterKeyValue(Cell ignored) throws IOException {
+ return ReturnCode.INCLUDE;
+ }
+ @Override
public boolean isFamilyEssential(byte[] name) {
return Bytes.equals(name, cf_first);
}