You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2008/05/22 22:49:27 UTC
svn commit: r659249 [2/3] - in /hadoop/hbase/trunk: ./
src/java/org/apache/hadoop/hbase/client/
src/java/org/apache/hadoop/hbase/filter/
src/java/org/apache/hadoop/hbase/mapred/
src/java/org/apache/hadoop/hbase/regionserver/ src/test/org/apache/hadoop/...
Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java?rev=659249&r1=659248&r2=659249&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java Thu May 22 13:49:25 2008
@@ -1,318 +1,318 @@
-/**
- * Copyright 2007 The Apache Software Foundation
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.filter;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Map;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import java.util.Map.Entry;
-import java.util.regex.Pattern;
-
-import org.apache.hadoop.hbase.regionserver.HLogEdit;
-import org.apache.hadoop.hbase.util.Bytes;
-
-/**
- * Implementation of RowFilterInterface that can filter by rowkey regular
- * expression and/or individual column values (equals comparison only). Multiple
- * column filters imply an implicit conjunction of filter criteria.
- *
- * Note that column value filtering in this interface has been replaced by
- * {@link ColumnValueFilter}.
- */
-public class RegExpRowFilter implements RowFilterInterface {
-
- private Pattern rowKeyPattern = null;
- private String rowKeyRegExp = null;
-
- @Deprecated
- private Map<byte [], byte[]> equalsMap =
- new TreeMap<byte [], byte[]>(Bytes.BYTES_COMPARATOR);
- @Deprecated
- private Set<byte []> nullColumns =
- new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
-
- /**
- * Default constructor, filters nothing. Required though for RPC
- * deserialization.
- */
- public RegExpRowFilter() {
- super();
- }
-
- /**
- * Constructor that takes a row key regular expression to filter on.
- *
- * @param rowKeyRegExp
- */
- public RegExpRowFilter(final String rowKeyRegExp) {
- this.rowKeyRegExp = rowKeyRegExp;
- }
-
- /**
- * @deprecated Column filtering has been replaced by {@link ColumnValueFilter}
- * Constructor that takes a row key regular expression to filter on.
- *
- * @param rowKeyRegExp
- * @param columnFilter
- */
- @Deprecated
- public RegExpRowFilter(final String rowKeyRegExp,
- final Map<byte [], byte[]> columnFilter) {
- this.rowKeyRegExp = rowKeyRegExp;
- this.setColumnFilters(columnFilter);
- }
-
- /** {@inheritDoc} */
- @SuppressWarnings("unused")
- public void rowProcessed(boolean filtered, byte [] rowKey) {
- //doesn't care
- }
-
- /** {@inheritDoc} */
- public boolean processAlways() {
- return false;
- }
-
- /**
- * @deprecated Column filtering has been replaced by {@link ColumnValueFilter}
- * Specify a value that must be matched for the given column.
- *
- * @param colKey
- * the column to match on
- * @param value
- * the value that must equal the stored value.
- */
- @Deprecated
- public void setColumnFilter(final byte [] colKey, final byte[] value) {
- if (value == null) {
- nullColumns.add(colKey);
- } else {
- equalsMap.put(colKey, value);
- }
- }
-
- /**
- * @deprecated Column filtering has been replaced by {@link ColumnValueFilter}
- * Set column filters for a number of columns.
- *
- * @param columnFilter
- * Map of columns with value criteria.
- */
- @Deprecated
- public void setColumnFilters(final Map<byte [], byte[]> columnFilter) {
- if (null == columnFilter) {
- nullColumns.clear();
- equalsMap.clear();
- } else {
- for (Entry<byte [], byte[]> entry : columnFilter.entrySet()) {
- setColumnFilter(entry.getKey(), entry.getValue());
- }
- }
- }
-
- /**
- *
- * {@inheritDoc}
- */
- public void reset() {
- // Nothing to reset
- }
-
- /**
- *
- * {@inheritDoc}
- */
- public boolean filterAllRemaining() {
- return false;
- }
-
- /**
- *
- * {@inheritDoc}
- */
- public boolean filterRowKey(final byte [] rowKey) {
- return (filtersByRowKey() && rowKey != null)?
- !getRowKeyPattern().matcher(Bytes.toString(rowKey)).matches():
- false;
- }
-
- /**
- *
- * {@inheritDoc}
- */
- public boolean filterColumn(final byte [] rowKey, final byte [] colKey,
- final byte[] data) {
- if (filterRowKey(rowKey)) {
- return true;
- }
- if (filtersByColumnValue()) {
- byte[] filterValue = equalsMap.get(colKey);
- if (null != filterValue) {
- return !Arrays.equals(filterValue, data);
- }
- }
- if (nullColumns.contains(colKey)) {
- if (data != null && !HLogEdit.isDeleted(data)) {
- return true;
- }
- }
- return false;
- }
-
- /**
- *
- * {@inheritDoc}
- */
- public boolean filterRow(final SortedMap<byte [], byte[]> columns) {
- for (Entry<byte [], byte[]> col : columns.entrySet()) {
- if (nullColumns.contains(col.getKey())
- && !HLogEdit.isDeleted(col.getValue())) {
- return true;
- }
- }
- for (byte [] col : equalsMap.keySet()) {
- if (!columns.containsKey(col)) {
- return true;
- }
- }
- return false;
- }
-
- @Deprecated
- private boolean filtersByColumnValue() {
- return equalsMap != null && equalsMap.size() > 0;
- }
-
- private boolean filtersByRowKey() {
- return null != rowKeyPattern || null != rowKeyRegExp;
- }
-
- private String getRowKeyRegExp() {
- if (null == rowKeyRegExp && rowKeyPattern != null) {
- rowKeyRegExp = rowKeyPattern.toString();
- }
- return rowKeyRegExp;
- }
-
- private Pattern getRowKeyPattern() {
- if (rowKeyPattern == null && rowKeyRegExp != null) {
- rowKeyPattern = Pattern.compile(rowKeyRegExp);
- }
- return rowKeyPattern;
- }
-
- /**
- *
- * {@inheritDoc}
- */
- public void readFields(final DataInput in) throws IOException {
- boolean hasRowKeyPattern = in.readBoolean();
- if (hasRowKeyPattern) {
- rowKeyRegExp = in.readUTF();
- }
- // equals map
- equalsMap.clear();
- int size = in.readInt();
- for (int i = 0; i < size; i++) {
- byte [] key = Bytes.readByteArray(in);
- int len = in.readInt();
- byte[] value = null;
- if (len >= 0) {
- value = new byte[len];
- in.readFully(value);
- }
- setColumnFilter(key, value);
- }
- // nullColumns
- nullColumns.clear();
- size = in.readInt();
- for (int i = 0; i < size; i++) {
- setColumnFilter(Bytes.readByteArray(in), null);
- }
- }
-
- /**
- *
- * {@inheritDoc}
- */
- public void validate(final byte [][] columns) {
- Set<byte []> invalids = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
- for (byte [] colKey : getFilterColumns()) {
- boolean found = false;
- for (byte [] col : columns) {
- if (Bytes.equals(col, colKey)) {
- found = true;
- break;
- }
- }
- if (!found) {
- invalids.add(colKey);
- }
- }
-
- if (invalids.size() > 0) {
- throw new InvalidRowFilterException(String.format(
- "RowFilter contains criteria on columns %s not in %s", invalids,
- Arrays.toString(columns)));
- }
- }
-
- @Deprecated
- private Set<byte []> getFilterColumns() {
- Set<byte []> cols = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
- cols.addAll(equalsMap.keySet());
- cols.addAll(nullColumns);
- return cols;
- }
-
- /**
- *
- * {@inheritDoc}
- */
- public void write(final DataOutput out) throws IOException {
- if (!filtersByRowKey()) {
- out.writeBoolean(false);
- } else {
- out.writeBoolean(true);
- out.writeUTF(getRowKeyRegExp());
- }
-
- // equalsMap
- out.writeInt(equalsMap.size());
- for (Entry<byte [], byte[]> entry : equalsMap.entrySet()) {
- Bytes.writeByteArray(out, entry.getKey());
- byte[] value = entry.getValue();
- out.writeInt(value.length);
- out.write(value);
- }
-
- // null columns
- out.writeInt(nullColumns.size());
- for (byte [] col : nullColumns) {
- Bytes.writeByteArray(out, col);
- }
- }
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.filter;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.Map.Entry;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.hbase.regionserver.HLogEdit;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Implementation of RowFilterInterface that can filter by rowkey regular
+ * expression and/or individual column values (equals comparison only). Multiple
+ * column filters imply an implicit conjunction of filter criteria.
+ *
+ * Note that column value filtering in this interface has been replaced by
+ * {@link ColumnValueFilter}.
+ */
+public class RegExpRowFilter implements RowFilterInterface {
+
+ private Pattern rowKeyPattern = null;
+ private String rowKeyRegExp = null;
+
+ @Deprecated
+ private Map<byte [], byte[]> equalsMap =
+ new TreeMap<byte [], byte[]>(Bytes.BYTES_COMPARATOR);
+ @Deprecated
+ private Set<byte []> nullColumns =
+ new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
+
+ /**
+ * Default constructor, filters nothing. Required though for RPC
+ * deserialization.
+ */
+ public RegExpRowFilter() {
+ super();
+ }
+
+ /**
+ * Constructor that takes a row key regular expression to filter on.
+ *
+ * @param rowKeyRegExp
+ */
+ public RegExpRowFilter(final String rowKeyRegExp) {
+ this.rowKeyRegExp = rowKeyRegExp;
+ }
+
+ /**
+ * @deprecated Column filtering has been replaced by {@link ColumnValueFilter}
+ * Constructor that takes a row key regular expression to filter on.
+ *
+ * @param rowKeyRegExp
+ * @param columnFilter
+ */
+ @Deprecated
+ public RegExpRowFilter(final String rowKeyRegExp,
+ final Map<byte [], byte[]> columnFilter) {
+ this.rowKeyRegExp = rowKeyRegExp;
+ this.setColumnFilters(columnFilter);
+ }
+
+ /** {@inheritDoc} */
+ @SuppressWarnings("unused")
+ public void rowProcessed(boolean filtered, byte [] rowKey) {
+ //doesn't care
+ }
+
+ /** {@inheritDoc} */
+ public boolean processAlways() {
+ return false;
+ }
+
+ /**
+ * @deprecated Column filtering has been replaced by {@link ColumnValueFilter}
+ * Specify a value that must be matched for the given column.
+ *
+ * @param colKey
+ * the column to match on
+ * @param value
+ * the value that must equal the stored value.
+ */
+ @Deprecated
+ public void setColumnFilter(final byte [] colKey, final byte[] value) {
+ if (value == null) {
+ nullColumns.add(colKey);
+ } else {
+ equalsMap.put(colKey, value);
+ }
+ }
+
+ /**
+ * @deprecated Column filtering has been replaced by {@link ColumnValueFilter}
+ * Set column filters for a number of columns.
+ *
+ * @param columnFilter
+ * Map of columns with value criteria.
+ */
+ @Deprecated
+ public void setColumnFilters(final Map<byte [], byte[]> columnFilter) {
+ if (null == columnFilter) {
+ nullColumns.clear();
+ equalsMap.clear();
+ } else {
+ for (Entry<byte [], byte[]> entry : columnFilter.entrySet()) {
+ setColumnFilter(entry.getKey(), entry.getValue());
+ }
+ }
+ }
+
+ /**
+ *
+ * {@inheritDoc}
+ */
+ public void reset() {
+ // Nothing to reset
+ }
+
+ /**
+ *
+ * {@inheritDoc}
+ */
+ public boolean filterAllRemaining() {
+ return false;
+ }
+
+ /**
+ *
+ * {@inheritDoc}
+ */
+ public boolean filterRowKey(final byte [] rowKey) {
+ return (filtersByRowKey() && rowKey != null)?
+ !getRowKeyPattern().matcher(Bytes.toString(rowKey)).matches():
+ false;
+ }
+
+ /**
+ *
+ * {@inheritDoc}
+ */
+ public boolean filterColumn(final byte [] rowKey, final byte [] colKey,
+ final byte[] data) {
+ if (filterRowKey(rowKey)) {
+ return true;
+ }
+ if (filtersByColumnValue()) {
+ byte[] filterValue = equalsMap.get(colKey);
+ if (null != filterValue) {
+ return !Arrays.equals(filterValue, data);
+ }
+ }
+ if (nullColumns.contains(colKey)) {
+ if (data != null && !HLogEdit.isDeleted(data)) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ /**
+ *
+ * {@inheritDoc}
+ */
+ public boolean filterRow(final SortedMap<byte [], byte[]> columns) {
+ for (Entry<byte [], byte[]> col : columns.entrySet()) {
+ if (nullColumns.contains(col.getKey())
+ && !HLogEdit.isDeleted(col.getValue())) {
+ return true;
+ }
+ }
+ for (byte [] col : equalsMap.keySet()) {
+ if (!columns.containsKey(col)) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ @Deprecated
+ private boolean filtersByColumnValue() {
+ return equalsMap != null && equalsMap.size() > 0;
+ }
+
+ private boolean filtersByRowKey() {
+ return null != rowKeyPattern || null != rowKeyRegExp;
+ }
+
+ private String getRowKeyRegExp() {
+ if (null == rowKeyRegExp && rowKeyPattern != null) {
+ rowKeyRegExp = rowKeyPattern.toString();
+ }
+ return rowKeyRegExp;
+ }
+
+ private Pattern getRowKeyPattern() {
+ if (rowKeyPattern == null && rowKeyRegExp != null) {
+ rowKeyPattern = Pattern.compile(rowKeyRegExp);
+ }
+ return rowKeyPattern;
+ }
+
+ /**
+ *
+ * {@inheritDoc}
+ */
+ public void readFields(final DataInput in) throws IOException {
+ boolean hasRowKeyPattern = in.readBoolean();
+ if (hasRowKeyPattern) {
+ rowKeyRegExp = in.readUTF();
+ }
+ // equals map
+ equalsMap.clear();
+ int size = in.readInt();
+ for (int i = 0; i < size; i++) {
+ byte [] key = Bytes.readByteArray(in);
+ int len = in.readInt();
+ byte[] value = null;
+ if (len >= 0) {
+ value = new byte[len];
+ in.readFully(value);
+ }
+ setColumnFilter(key, value);
+ }
+ // nullColumns
+ nullColumns.clear();
+ size = in.readInt();
+ for (int i = 0; i < size; i++) {
+ setColumnFilter(Bytes.readByteArray(in), null);
+ }
+ }
+
+ /**
+ *
+ * {@inheritDoc}
+ */
+ public void validate(final byte [][] columns) {
+ Set<byte []> invalids = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
+ for (byte [] colKey : getFilterColumns()) {
+ boolean found = false;
+ for (byte [] col : columns) {
+ if (Bytes.equals(col, colKey)) {
+ found = true;
+ break;
+ }
+ }
+ if (!found) {
+ invalids.add(colKey);
+ }
+ }
+
+ if (invalids.size() > 0) {
+ throw new InvalidRowFilterException(String.format(
+ "RowFilter contains criteria on columns %s not in %s", invalids,
+ Arrays.toString(columns)));
+ }
+ }
+
+ @Deprecated
+ private Set<byte []> getFilterColumns() {
+ Set<byte []> cols = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
+ cols.addAll(equalsMap.keySet());
+ cols.addAll(nullColumns);
+ return cols;
+ }
+
+ /**
+ *
+ * {@inheritDoc}
+ */
+ public void write(final DataOutput out) throws IOException {
+ if (!filtersByRowKey()) {
+ out.writeBoolean(false);
+ } else {
+ out.writeBoolean(true);
+ out.writeUTF(getRowKeyRegExp());
+ }
+
+ // equalsMap
+ out.writeInt(equalsMap.size());
+ for (Entry<byte [], byte[]> entry : equalsMap.entrySet()) {
+ Bytes.writeByteArray(out, entry.getKey());
+ byte[] value = entry.getValue();
+ out.writeInt(value.length);
+ out.write(value);
+ }
+
+ // null columns
+ out.writeInt(nullColumns.size());
+ for (byte [] col : nullColumns) {
+ Bytes.writeByteArray(out, col);
+ }
+ }
}
\ No newline at end of file
Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowFilterInterface.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowFilterInterface.java?rev=659249&r1=659248&r2=659249&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowFilterInterface.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowFilterInterface.java Thu May 22 13:49:25 2008
@@ -1,115 +1,115 @@
-/**
- * Copyright 2007 The Apache Software Foundation
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.filter;
-
-import java.util.SortedMap;
-
-import org.apache.hadoop.io.Writable;
-
-/**
- *
- * Interface used for row-level filters applied to HRegion.HScanner scan
- * results during calls to next().
- */
-public interface RowFilterInterface extends Writable {
-
- /**
- * Resets the state of the filter. Used prior to the start of a Region scan.
- *
- */
- void reset();
-
- /**
- * Called to let filter know the final decision (to pass or filter) on a
- * given row. With out HScanner calling this, the filter does not know if a
- * row passed filtering even if it passed the row itself because other
- * filters may have failed the row. E.g. when this filter is a member of a
- * RowFilterSet with an OR operator.
- *
- * @see RowFilterSet
- * @param filtered
- * @param key
- */
- void rowProcessed(boolean filtered, byte [] key);
-
- /**
- * Returns whether or not the filter should always be processed in any
- * filtering call. This precaution is necessary for filters that maintain
- * state and need to be updated according to their response to filtering
- * calls (see WhileMatchRowFilter for an example). At times, filters nested
- * in RowFilterSets may or may not be called because the RowFilterSet
- * determines a result as fast as possible. Returning true for
- * processAlways() ensures that the filter will always be called.
- *
- * @return whether or not to always process the filter
- */
- boolean processAlways();
-
- /**
- * Determines if the filter has decided that all remaining results should be
- * filtered (skipped). This is used to prevent the scanner from scanning a
- * the rest of the HRegion when for sure the filter will exclude all
- * remaining rows.
- *
- * @return true if the filter intends to filter all remaining rows.
- */
- boolean filterAllRemaining();
-
- /**
- * Filters on just a row key. This is the first chance to stop a row.
- *
- * @param rowKey
- * @return true if given row key is filtered and row should not be processed.
- */
- boolean filterRowKey(final byte [] rowKey);
-
- /**
- * Filters on row key, column name, and column value. This will take individual columns out of a row,
- * but the rest of the row will still get through.
- *
- * @param rowKey row key to filter on.
- * @param colunmName column name to filter on
- * @param columnValue column value to filter on
- * @return true if row filtered and should not be processed.
- */
- boolean filterColumn(final byte [] rowKey, final byte [] colunmName,
- final byte[] columnValue);
-
- /**
- * Filter on the fully assembled row. This is the last chance to stop a row.
- *
- * @param columns
- * @return true if row filtered and should not be processed.
- */
- boolean filterRow(final SortedMap<byte [], byte[]> columns);
-
- /**
- * Validates that this filter applies only to a subset of the given columns.
- * This check is done prior to opening of scanner due to the limitation that
- * filtering of columns is dependent on the retrieval of those columns within
- * the HRegion. Criteria on columns that are not part of a scanner's column
- * list will be ignored. In the case of null value filters, all rows will pass
- * the filter. This behavior should be 'undefined' for the user and therefore
- * not permitted.
- *
- * @param columns
- */
- void validate(final byte [][] columns);
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.filter;
+
+import java.util.SortedMap;
+
+import org.apache.hadoop.io.Writable;
+
+/**
+ *
+ * Interface used for row-level filters applied to HRegion.HScanner scan
+ * results during calls to next().
+ */
+public interface RowFilterInterface extends Writable {
+
+ /**
+ * Resets the state of the filter. Used prior to the start of a Region scan.
+ *
+ */
+ void reset();
+
+ /**
+ * Called to let filter know the final decision (to pass or filter) on a
+ * given row. With out HScanner calling this, the filter does not know if a
+ * row passed filtering even if it passed the row itself because other
+ * filters may have failed the row. E.g. when this filter is a member of a
+ * RowFilterSet with an OR operator.
+ *
+ * @see RowFilterSet
+ * @param filtered
+ * @param key
+ */
+ void rowProcessed(boolean filtered, byte [] key);
+
+ /**
+ * Returns whether or not the filter should always be processed in any
+ * filtering call. This precaution is necessary for filters that maintain
+ * state and need to be updated according to their response to filtering
+ * calls (see WhileMatchRowFilter for an example). At times, filters nested
+ * in RowFilterSets may or may not be called because the RowFilterSet
+ * determines a result as fast as possible. Returning true for
+ * processAlways() ensures that the filter will always be called.
+ *
+ * @return whether or not to always process the filter
+ */
+ boolean processAlways();
+
+ /**
+ * Determines if the filter has decided that all remaining results should be
+ * filtered (skipped). This is used to prevent the scanner from scanning a
+ * the rest of the HRegion when for sure the filter will exclude all
+ * remaining rows.
+ *
+ * @return true if the filter intends to filter all remaining rows.
+ */
+ boolean filterAllRemaining();
+
+ /**
+ * Filters on just a row key. This is the first chance to stop a row.
+ *
+ * @param rowKey
+ * @return true if given row key is filtered and row should not be processed.
+ */
+ boolean filterRowKey(final byte [] rowKey);
+
+ /**
+ * Filters on row key, column name, and column value. This will take individual columns out of a row,
+ * but the rest of the row will still get through.
+ *
+ * @param rowKey row key to filter on.
+ * @param colunmName column name to filter on
+ * @param columnValue column value to filter on
+ * @return true if row filtered and should not be processed.
+ */
+ boolean filterColumn(final byte [] rowKey, final byte [] colunmName,
+ final byte[] columnValue);
+
+ /**
+ * Filter on the fully assembled row. This is the last chance to stop a row.
+ *
+ * @param columns
+ * @return true if row filtered and should not be processed.
+ */
+ boolean filterRow(final SortedMap<byte [], byte[]> columns);
+
+ /**
+ * Validates that this filter applies only to a subset of the given columns.
+ * This check is done prior to opening of scanner due to the limitation that
+ * filtering of columns is dependent on the retrieval of those columns within
+ * the HRegion. Criteria on columns that are not part of a scanner's column
+ * list will be ignored. In the case of null value filters, all rows will pass
+ * the filter. This behavior should be 'undefined' for the user and therefore
+ * not permitted.
+ *
+ * @param columns
+ */
+ void validate(final byte [][] columns);
}
\ No newline at end of file
Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java?rev=659249&r1=659248&r2=659249&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java Thu May 22 13:49:25 2008
@@ -1,231 +1,231 @@
-/**
- * Copyright 2007 The Apache Software Foundation
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.filter;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.SortedMap;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.io.ObjectWritable;
-
-/**
- * Implementation of RowFilterInterface that represents a set of RowFilters
- * which will be evaluated with a specified boolean operator MUST_PASS_ALL
- * (!AND) or MUST_PASS_ONE (!OR). Since you can use RowFilterSets as children
- * of RowFilterSet, you can create a hierarchy of filters to be evaluated.
- */
-public class RowFilterSet implements RowFilterInterface {
-
- /** set operator */
- public static enum Operator {
- /** !AND */
- MUST_PASS_ALL,
- /** !OR */
- MUST_PASS_ONE
- }
-
- private Operator operator = Operator.MUST_PASS_ALL;
- private Set<RowFilterInterface> filters = new HashSet<RowFilterInterface>();
-
- /**
- * Default constructor, filters nothing. Required though for RPC
- * deserialization.
- */
- public RowFilterSet() {
- super();
- }
-
- /**
- * Constructor that takes a set of RowFilters. The default operator
- * MUST_PASS_ALL is assumed.
- *
- * @param rowFilters
- */
- public RowFilterSet(final Set<RowFilterInterface> rowFilters) {
- this.filters = rowFilters;
- }
-
- /**
- * Constructor that takes a set of RowFilters and an operator.
- *
- * @param operator Operator to process filter set with.
- * @param rowFilters Set of row filters.
- */
- public RowFilterSet(final Operator operator,
- final Set<RowFilterInterface> rowFilters) {
- this.filters = rowFilters;
- this.operator = operator;
- }
-
- /** {@inheritDoc} */
- public void validate(final byte [][] columns) {
- for (RowFilterInterface filter : filters) {
- filter.validate(columns);
- }
- }
-
- /** {@inheritDoc} */
- public void reset() {
- for (RowFilterInterface filter : filters) {
- filter.reset();
- }
- }
-
- /** {@inheritDoc} */
- public void rowProcessed(boolean filtered, byte [] rowKey) {
- for (RowFilterInterface filter : filters) {
- filter.rowProcessed(filtered, rowKey);
- }
- }
-
- /** {@inheritDoc} */
- public boolean processAlways() {
- for (RowFilterInterface filter : filters) {
- if (filter.processAlways()) {
- return true;
- }
- }
- return false;
- }
-
- /** {@inheritDoc} */
- public boolean filterAllRemaining() {
- boolean result = operator == Operator.MUST_PASS_ONE;
- for (RowFilterInterface filter : filters) {
- if (operator == Operator.MUST_PASS_ALL) {
- if (filter.filterAllRemaining()) {
- return true;
- }
- } else if (operator == Operator.MUST_PASS_ONE) {
- if (!filter.filterAllRemaining()) {
- return false;
- }
- }
- }
- return result;
- }
-
- /** {@inheritDoc} */
- public boolean filterRowKey(final byte [] rowKey) {
- boolean resultFound = false;
- boolean result = operator == Operator.MUST_PASS_ONE;
- for (RowFilterInterface filter : filters) {
- if (!resultFound) {
- if (operator == Operator.MUST_PASS_ALL) {
- if (filter.filterAllRemaining() || filter.filterRowKey(rowKey)) {
- result = true;
- resultFound = true;
- }
- } else if (operator == Operator.MUST_PASS_ONE) {
- if (!filter.filterAllRemaining() && !filter.filterRowKey(rowKey)) {
- result = false;
- resultFound = true;
- }
- }
- } else if (filter.processAlways()) {
- filter.filterRowKey(rowKey);
- }
- }
- return result;
- }
-
- /** {@inheritDoc} */
- public boolean filterColumn(final byte [] rowKey, final byte [] colKey,
- final byte[] data) {
- boolean resultFound = false;
- boolean result = operator == Operator.MUST_PASS_ONE;
- for (RowFilterInterface filter : filters) {
- if (!resultFound) {
- if (operator == Operator.MUST_PASS_ALL) {
- if (filter.filterAllRemaining() ||
- filter.filterColumn(rowKey, colKey, data)) {
- result = true;
- resultFound = true;
- }
- } else if (operator == Operator.MUST_PASS_ONE) {
- if (!filter.filterAllRemaining() &&
- !filter.filterColumn(rowKey, colKey, data)) {
- result = false;
- resultFound = true;
- }
- }
- } else if (filter.processAlways()) {
- filter.filterColumn(rowKey, colKey, data);
- }
- }
- return result;
- }
-
- /** {@inheritDoc} */
- public boolean filterRow(final SortedMap<byte [], byte[]> columns) {
- boolean resultFound = false;
- boolean result = operator == Operator.MUST_PASS_ONE;
- for (RowFilterInterface filter : filters) {
- if (!resultFound) {
- if (operator == Operator.MUST_PASS_ALL) {
- if (filter.filterAllRemaining() || filter.filterRow(columns)) {
- result = true;
- resultFound = true;
- }
- } else if (operator == Operator.MUST_PASS_ONE) {
- if (!filter.filterAllRemaining() && !filter.filterRow(columns)) {
- result = false;
- resultFound = true;
- }
- }
- } else if (filter.processAlways()) {
- filter.filterRow(columns);
- }
- }
- return result;
- }
-
- /** {@inheritDoc} */
- public void readFields(final DataInput in) throws IOException {
- Configuration conf = new HBaseConfiguration();
- byte opByte = in.readByte();
- operator = Operator.values()[opByte];
- int size = in.readInt();
- if (size > 0) {
- filters = new HashSet<RowFilterInterface>();
- for (int i = 0; i < size; i++) {
- RowFilterInterface filter = (RowFilterInterface) ObjectWritable
- .readObject(in, conf);
- filters.add(filter);
- }
- }
- }
-
- /** {@inheritDoc} */
- public void write(final DataOutput out) throws IOException {
- Configuration conf = new HBaseConfiguration();
- out.writeByte(operator.ordinal());
- out.writeInt(filters.size());
- for (RowFilterInterface filter : filters) {
- ObjectWritable.writeObject(out, filter, RowFilterInterface.class, conf);
- }
- }
-
-}
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.filter;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.SortedMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.io.ObjectWritable;
+
+/**
+ * Implementation of RowFilterInterface that represents a set of RowFilters
+ * which will be evaluated with a specified boolean operator MUST_PASS_ALL
+ * (!AND) or MUST_PASS_ONE (!OR). Since you can use RowFilterSets as children
+ * of RowFilterSet, you can create a hierarchy of filters to be evaluated.
+ */
+public class RowFilterSet implements RowFilterInterface {
+
+ /** set operator */
+ public static enum Operator {
+ /** !AND */
+ MUST_PASS_ALL,
+ /** !OR */
+ MUST_PASS_ONE
+ }
+
+ private Operator operator = Operator.MUST_PASS_ALL;
+ private Set<RowFilterInterface> filters = new HashSet<RowFilterInterface>();
+
+ /**
+ * Default constructor, filters nothing. Required though for RPC
+ * deserialization.
+ */
+ public RowFilterSet() {
+ super();
+ }
+
+ /**
+ * Constructor that takes a set of RowFilters. The default operator
+ * MUST_PASS_ALL is assumed.
+ *
+ * @param rowFilters
+ */
+ public RowFilterSet(final Set<RowFilterInterface> rowFilters) {
+ this.filters = rowFilters;
+ }
+
+ /**
+ * Constructor that takes a set of RowFilters and an operator.
+ *
+ * @param operator Operator to process filter set with.
+ * @param rowFilters Set of row filters.
+ */
+ public RowFilterSet(final Operator operator,
+ final Set<RowFilterInterface> rowFilters) {
+ this.filters = rowFilters;
+ this.operator = operator;
+ }
+
+ /** {@inheritDoc} */
+ public void validate(final byte [][] columns) {
+ for (RowFilterInterface filter : filters) {
+ filter.validate(columns);
+ }
+ }
+
+ /** {@inheritDoc} */
+ public void reset() {
+ for (RowFilterInterface filter : filters) {
+ filter.reset();
+ }
+ }
+
+ /** {@inheritDoc} */
+ public void rowProcessed(boolean filtered, byte [] rowKey) {
+ for (RowFilterInterface filter : filters) {
+ filter.rowProcessed(filtered, rowKey);
+ }
+ }
+
+ /** {@inheritDoc} */
+ public boolean processAlways() {
+ for (RowFilterInterface filter : filters) {
+ if (filter.processAlways()) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ /** {@inheritDoc} */
+ public boolean filterAllRemaining() {
+ boolean result = operator == Operator.MUST_PASS_ONE;
+ for (RowFilterInterface filter : filters) {
+ if (operator == Operator.MUST_PASS_ALL) {
+ if (filter.filterAllRemaining()) {
+ return true;
+ }
+ } else if (operator == Operator.MUST_PASS_ONE) {
+ if (!filter.filterAllRemaining()) {
+ return false;
+ }
+ }
+ }
+ return result;
+ }
+
+ /** {@inheritDoc} */
+ public boolean filterRowKey(final byte [] rowKey) {
+ boolean resultFound = false;
+ boolean result = operator == Operator.MUST_PASS_ONE;
+ for (RowFilterInterface filter : filters) {
+ if (!resultFound) {
+ if (operator == Operator.MUST_PASS_ALL) {
+ if (filter.filterAllRemaining() || filter.filterRowKey(rowKey)) {
+ result = true;
+ resultFound = true;
+ }
+ } else if (operator == Operator.MUST_PASS_ONE) {
+ if (!filter.filterAllRemaining() && !filter.filterRowKey(rowKey)) {
+ result = false;
+ resultFound = true;
+ }
+ }
+ } else if (filter.processAlways()) {
+ filter.filterRowKey(rowKey);
+ }
+ }
+ return result;
+ }
+
+ /** {@inheritDoc} */
+ public boolean filterColumn(final byte [] rowKey, final byte [] colKey,
+ final byte[] data) {
+ boolean resultFound = false;
+ boolean result = operator == Operator.MUST_PASS_ONE;
+ for (RowFilterInterface filter : filters) {
+ if (!resultFound) {
+ if (operator == Operator.MUST_PASS_ALL) {
+ if (filter.filterAllRemaining() ||
+ filter.filterColumn(rowKey, colKey, data)) {
+ result = true;
+ resultFound = true;
+ }
+ } else if (operator == Operator.MUST_PASS_ONE) {
+ if (!filter.filterAllRemaining() &&
+ !filter.filterColumn(rowKey, colKey, data)) {
+ result = false;
+ resultFound = true;
+ }
+ }
+ } else if (filter.processAlways()) {
+ filter.filterColumn(rowKey, colKey, data);
+ }
+ }
+ return result;
+ }
+
+ /** {@inheritDoc} */
+ public boolean filterRow(final SortedMap<byte [], byte[]> columns) {
+ boolean resultFound = false;
+ boolean result = operator == Operator.MUST_PASS_ONE;
+ for (RowFilterInterface filter : filters) {
+ if (!resultFound) {
+ if (operator == Operator.MUST_PASS_ALL) {
+ if (filter.filterAllRemaining() || filter.filterRow(columns)) {
+ result = true;
+ resultFound = true;
+ }
+ } else if (operator == Operator.MUST_PASS_ONE) {
+ if (!filter.filterAllRemaining() && !filter.filterRow(columns)) {
+ result = false;
+ resultFound = true;
+ }
+ }
+ } else if (filter.processAlways()) {
+ filter.filterRow(columns);
+ }
+ }
+ return result;
+ }
+
+ /** {@inheritDoc} */
+ public void readFields(final DataInput in) throws IOException {
+ Configuration conf = new HBaseConfiguration();
+ byte opByte = in.readByte();
+ operator = Operator.values()[opByte];
+ int size = in.readInt();
+ if (size > 0) {
+ filters = new HashSet<RowFilterInterface>();
+ for (int i = 0; i < size; i++) {
+ RowFilterInterface filter = (RowFilterInterface) ObjectWritable
+ .readObject(in, conf);
+ filters.add(filter);
+ }
+ }
+ }
+
+ /** {@inheritDoc} */
+ public void write(final DataOutput out) throws IOException {
+ Configuration conf = new HBaseConfiguration();
+ out.writeByte(operator.ordinal());
+ out.writeInt(filters.size());
+ for (RowFilterInterface filter : filters) {
+ ObjectWritable.writeObject(out, filter, RowFilterInterface.class, conf);
+ }
+ }
+
+}
Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java?rev=659249&r1=659248&r2=659249&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java Thu May 22 13:49:25 2008
@@ -1,139 +1,139 @@
-/**
- * Copyright 2007 The Apache Software Foundation
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.filter;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.SortedMap;
-
-import org.apache.hadoop.hbase.util.Bytes;
-
-/**
- * Implementation of RowFilterInterface that filters out rows greater than or
- * equal to a specified rowKey.
- */
-public class StopRowFilter implements RowFilterInterface {
-
- private byte [] stopRowKey;
-
- /**
- * Default constructor, filters nothing. Required though for RPC
- * deserialization.
- */
- public StopRowFilter() {
- super();
- }
-
- /**
- * Constructor that takes a stopRowKey on which to filter
- *
- * @param stopRowKey rowKey to filter on.
- */
- public StopRowFilter(final byte [] stopRowKey) {
- this.stopRowKey = stopRowKey;
- }
-
- /**
- * An accessor for the stopRowKey
- *
- * @return the filter's stopRowKey
- */
- public byte [] getStopRowKey() {
- return this.stopRowKey;
- }
-
- /**
- *
- * {@inheritDoc}
- */
- public void validate(@SuppressWarnings("unused") final byte [][] columns) {
- // Doesn't filter columns
- }
-
- /**
- *
- * {@inheritDoc}
- */
- public void reset() {
- // Nothing to reset
- }
-
- /** {@inheritDoc} */
- @SuppressWarnings("unused")
- public void rowProcessed(boolean filtered, byte [] rowKey) {
- // Doesn't care
- }
-
- /** {@inheritDoc} */
- public boolean processAlways() {
- return false;
- }
-
- /** {@inheritDoc} */
- public boolean filterAllRemaining() {
- return false;
- }
-
- /** {@inheritDoc} */
- public boolean filterRowKey(final byte [] rowKey) {
- if (rowKey == null) {
- if (this.stopRowKey == null) {
- return true;
- }
- return false;
- }
- return Bytes.compareTo(stopRowKey, rowKey) <= 0;
- }
-
- /**
- * {@inheritDoc}
- *
- * Because StopRowFilter does not examine column information, this method
- * defaults to calling the rowKey-only version of filter.
- */
- public boolean filterColumn(@SuppressWarnings("unused") final byte [] rowKey,
- @SuppressWarnings("unused") final byte [] colKey,
- @SuppressWarnings("unused") final byte[] data) {
- return filterRowKey(rowKey);
- }
-
- /** {@inheritDoc}
- *
- * Because StopRowFilter does not examine column information, this method
- * defaults to calling filterAllRemaining().
- *
- * @param columns
- */
- public boolean filterRow(@SuppressWarnings("unused")
- final SortedMap<byte [], byte[]> columns) {
- return filterAllRemaining();
- }
-
- /** {@inheritDoc} */
- public void readFields(DataInput in) throws IOException {
- this.stopRowKey = Bytes.readByteArray(in);
- }
-
- /** {@inheritDoc} */
- public void write(DataOutput out) throws IOException {
- Bytes.writeByteArray(out, this.stopRowKey);
- }
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.filter;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.SortedMap;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Implementation of RowFilterInterface that filters out rows greater than or
+ * equal to a specified rowKey.
+ */
+public class StopRowFilter implements RowFilterInterface {
+
+ private byte [] stopRowKey;
+
+ /**
+ * Default constructor, filters nothing. Required though for RPC
+ * deserialization.
+ */
+ public StopRowFilter() {
+ super();
+ }
+
+ /**
+ * Constructor that takes a stopRowKey on which to filter
+ *
+ * @param stopRowKey rowKey to filter on.
+ */
+ public StopRowFilter(final byte [] stopRowKey) {
+ this.stopRowKey = stopRowKey;
+ }
+
+ /**
+ * An accessor for the stopRowKey
+ *
+ * @return the filter's stopRowKey
+ */
+ public byte [] getStopRowKey() {
+ return this.stopRowKey;
+ }
+
+ /**
+ *
+ * {@inheritDoc}
+ */
+ public void validate(@SuppressWarnings("unused") final byte [][] columns) {
+ // Doesn't filter columns
+ }
+
+ /**
+ *
+ * {@inheritDoc}
+ */
+ public void reset() {
+ // Nothing to reset
+ }
+
+ /** {@inheritDoc} */
+ @SuppressWarnings("unused")
+ public void rowProcessed(boolean filtered, byte [] rowKey) {
+ // Doesn't care
+ }
+
+ /** {@inheritDoc} */
+ public boolean processAlways() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
+ public boolean filterAllRemaining() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
+ public boolean filterRowKey(final byte [] rowKey) {
+ if (rowKey == null) {
+ if (this.stopRowKey == null) {
+ return true;
+ }
+ return false;
+ }
+ return Bytes.compareTo(stopRowKey, rowKey) <= 0;
+ }
+
+ /**
+ * {@inheritDoc}
+ *
+ * Because StopRowFilter does not examine column information, this method
+ * defaults to calling the rowKey-only version of filter.
+ */
+ public boolean filterColumn(@SuppressWarnings("unused") final byte [] rowKey,
+ @SuppressWarnings("unused") final byte [] colKey,
+ @SuppressWarnings("unused") final byte[] data) {
+ return filterRowKey(rowKey);
+ }
+
+ /** {@inheritDoc}
+ *
+ * Because StopRowFilter does not examine column information, this method
+ * defaults to calling filterAllRemaining().
+ *
+ * @param columns
+ */
+ public boolean filterRow(@SuppressWarnings("unused")
+ final SortedMap<byte [], byte[]> columns) {
+ return filterAllRemaining();
+ }
+
+ /** {@inheritDoc} */
+ public void readFields(DataInput in) throws IOException {
+ this.stopRowKey = Bytes.readByteArray(in);
+ }
+
+ /** {@inheritDoc} */
+ public void write(DataOutput out) throws IOException {
+ Bytes.writeByteArray(out, this.stopRowKey);
+ }
}
\ No newline at end of file
Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/WhileMatchRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/WhileMatchRowFilter.java?rev=659249&r1=659248&r2=659249&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/WhileMatchRowFilter.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/WhileMatchRowFilter.java Thu May 22 13:49:25 2008
@@ -1,151 +1,151 @@
-/**
- * Copyright 2007 The Apache Software Foundation
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.filter;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.SortedMap;
-
-
-/**
- * WhileMatchRowFilter is a wrapper filter that filters everything after the
- * first filtered row. Once the nested filter returns true for either of it's
- * filter(..) methods or filterNotNull(SortedMap<Text, byte[]>), this wrapper's
- * filterAllRemaining() will return true. All filtering methods will
- * thereafter defer to the result of filterAllRemaining().
- */
-public class WhileMatchRowFilter implements RowFilterInterface {
-
- private boolean filterAllRemaining = false;
- private RowFilterInterface filter;
-
- /**
- * Default constructor, filters nothing. Required though for RPC
- * deserialization.
- */
- public WhileMatchRowFilter() {
- super();
- }
-
- /**
- * Constructor
- * @param filter
- */
- public WhileMatchRowFilter(RowFilterInterface filter) {
- this.filter = filter;
- }
-
- /**
- * Returns the internal filter being wrapped
- *
- * @return the internal filter
- */
- public RowFilterInterface getInternalFilter() {
- return this.filter;
- }
-
- /** {@inheritDoc} */
- public void reset() {
- this.filterAllRemaining = false;
- this.filter.reset();
- }
-
- /** {@inheritDoc} */
- public boolean processAlways() {
- return true;
- }
-
- /**
- * Returns true once the nested filter has filtered out a row (returned true
- * on a call to one of it's filtering methods). Until then it returns false.
- *
- * @return true/false whether the nested filter has returned true on a filter
- * call.
- */
- public boolean filterAllRemaining() {
- return this.filterAllRemaining || this.filter.filterAllRemaining();
- }
-
- /** {@inheritDoc} */
- public boolean filterRowKey(final byte [] rowKey) {
- changeFAR(this.filter.filterRowKey(rowKey));
- return filterAllRemaining();
- }
-
- /** {@inheritDoc} */
- public boolean filterColumn(final byte [] rowKey, final byte [] colKey,
- final byte[] data) {
- changeFAR(this.filter.filterColumn(rowKey, colKey, data));
- return filterAllRemaining();
- }
-
- /** {@inheritDoc} */
- public boolean filterRow(final SortedMap<byte [], byte[]> columns) {
- changeFAR(this.filter.filterRow(columns));
- return filterAllRemaining();
- }
-
- /**
- * Change filterAllRemaining from false to true if value is true, otherwise
- * leave as is.
- *
- * @param value
- */
- private void changeFAR(boolean value) {
- this.filterAllRemaining = this.filterAllRemaining || value;
- }
-
- /** {@inheritDoc} */
- public void rowProcessed(boolean filtered, byte [] rowKey) {
- this.filter.rowProcessed(filtered, rowKey);
- }
-
- /** {@inheritDoc} */
- public void validate(final byte [][] columns) {
- this.filter.validate(columns);
- }
-
- /** {@inheritDoc} */
- public void readFields(DataInput in) throws IOException {
- String className = in.readUTF();
-
- try {
- this.filter = (RowFilterInterface)(Class.forName(className).
- newInstance());
- this.filter.readFields(in);
- } catch (InstantiationException e) {
- throw new RuntimeException("Failed to deserialize WhileMatchRowFilter.",
- e);
- } catch (IllegalAccessException e) {
- throw new RuntimeException("Failed to deserialize WhileMatchRowFilter.",
- e);
- } catch (ClassNotFoundException e) {
- throw new RuntimeException("Failed to deserialize WhileMatchRowFilter.",
- e);
- }
- }
-
- /** {@inheritDoc} */
- public void write(DataOutput out) throws IOException {
- out.writeUTF(this.filter.getClass().getName());
- this.filter.write(out);
- }
-}
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.filter;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.SortedMap;
+
+
+/**
+ * WhileMatchRowFilter is a wrapper filter that filters everything after the
+ * first filtered row. Once the nested filter returns true for either of it's
+ * filter(..) methods or filterNotNull(SortedMap<Text, byte[]>), this wrapper's
+ * filterAllRemaining() will return true. All filtering methods will
+ * thereafter defer to the result of filterAllRemaining().
+ */
+public class WhileMatchRowFilter implements RowFilterInterface {
+
+ private boolean filterAllRemaining = false;
+ private RowFilterInterface filter;
+
+ /**
+ * Default constructor, filters nothing. Required though for RPC
+ * deserialization.
+ */
+ public WhileMatchRowFilter() {
+ super();
+ }
+
+ /**
+ * Constructor
+ * @param filter
+ */
+ public WhileMatchRowFilter(RowFilterInterface filter) {
+ this.filter = filter;
+ }
+
+ /**
+ * Returns the internal filter being wrapped
+ *
+ * @return the internal filter
+ */
+ public RowFilterInterface getInternalFilter() {
+ return this.filter;
+ }
+
+ /** {@inheritDoc} */
+ public void reset() {
+ this.filterAllRemaining = false;
+ this.filter.reset();
+ }
+
+ /** {@inheritDoc} */
+ public boolean processAlways() {
+ return true;
+ }
+
+ /**
+ * Returns true once the nested filter has filtered out a row (returned true
+ * on a call to one of it's filtering methods). Until then it returns false.
+ *
+ * @return true/false whether the nested filter has returned true on a filter
+ * call.
+ */
+ public boolean filterAllRemaining() {
+ return this.filterAllRemaining || this.filter.filterAllRemaining();
+ }
+
+ /** {@inheritDoc} */
+ public boolean filterRowKey(final byte [] rowKey) {
+ changeFAR(this.filter.filterRowKey(rowKey));
+ return filterAllRemaining();
+ }
+
+ /** {@inheritDoc} */
+ public boolean filterColumn(final byte [] rowKey, final byte [] colKey,
+ final byte[] data) {
+ changeFAR(this.filter.filterColumn(rowKey, colKey, data));
+ return filterAllRemaining();
+ }
+
+ /** {@inheritDoc} */
+ public boolean filterRow(final SortedMap<byte [], byte[]> columns) {
+ changeFAR(this.filter.filterRow(columns));
+ return filterAllRemaining();
+ }
+
+ /**
+ * Change filterAllRemaining from false to true if value is true, otherwise
+ * leave as is.
+ *
+ * @param value
+ */
+ private void changeFAR(boolean value) {
+ this.filterAllRemaining = this.filterAllRemaining || value;
+ }
+
+ /** {@inheritDoc} */
+ public void rowProcessed(boolean filtered, byte [] rowKey) {
+ this.filter.rowProcessed(filtered, rowKey);
+ }
+
+ /** {@inheritDoc} */
+ public void validate(final byte [][] columns) {
+ this.filter.validate(columns);
+ }
+
+ /** {@inheritDoc} */
+ public void readFields(DataInput in) throws IOException {
+ String className = in.readUTF();
+
+ try {
+ this.filter = (RowFilterInterface)(Class.forName(className).
+ newInstance());
+ this.filter.readFields(in);
+ } catch (InstantiationException e) {
+ throw new RuntimeException("Failed to deserialize WhileMatchRowFilter.",
+ e);
+ } catch (IllegalAccessException e) {
+ throw new RuntimeException("Failed to deserialize WhileMatchRowFilter.",
+ e);
+ } catch (ClassNotFoundException e) {
+ throw new RuntimeException("Failed to deserialize WhileMatchRowFilter.",
+ e);
+ }
+ }
+
+ /** {@inheritDoc} */
+ public void write(DataOutput out) throws IOException {
+ out.writeUTF(this.filter.getClass().getName());
+ this.filter.write(out);
+ }
+}
Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java?rev=659249&r1=659248&r2=659249&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/TableOutputFormat.java Thu May 22 13:49:25 2008
@@ -1,111 +1,111 @@
-/**
- * Copyright 2007 The Apache Software Foundation
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.mapred;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.io.BatchUpdate;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.mapred.FileAlreadyExistsException;
-import org.apache.hadoop.mapred.InvalidJobConfException;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.FileOutputFormat;
-import org.apache.hadoop.mapred.RecordWriter;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.util.Progressable;
-
-/**
- * Convert Map/Reduce output and write it to an HBase table
- */
-public class TableOutputFormat extends
-FileOutputFormat<ImmutableBytesWritable, BatchUpdate> {
-
- /** JobConf parameter that specifies the output table */
- public static final String OUTPUT_TABLE = "hbase.mapred.outputtable";
- private final Log LOG = LogFactory.getLog(TableOutputFormat.class);
-
- /**
- * Convert Reduce output (key, value) to (HStoreKey, KeyedDataArrayWritable)
- * and write to an HBase table
- */
- protected class TableRecordWriter
- implements RecordWriter<ImmutableBytesWritable, BatchUpdate> {
- private HTable m_table;
-
- /**
- * Instantiate a TableRecordWriter with the HBase HClient for writing.
- *
- * @param table
- */
- public TableRecordWriter(HTable table) {
- m_table = table;
- }
-
- /** {@inheritDoc} */
- public void close(@SuppressWarnings("unused") Reporter reporter) {
- // Nothing to do.
- }
-
- /** {@inheritDoc} */
- public void write(@SuppressWarnings("unused") ImmutableBytesWritable key,
- BatchUpdate value) throws IOException {
- m_table.commit(value);
- }
- }
-
- /** {@inheritDoc} */
- @Override
- @SuppressWarnings("unchecked")
- public RecordWriter getRecordWriter(
- @SuppressWarnings("unused") FileSystem ignored,
- JobConf job,
- @SuppressWarnings("unused") String name,
- @SuppressWarnings("unused") Progressable progress) throws IOException {
-
- // expecting exactly one path
-
- String tableName = job.get(OUTPUT_TABLE);
- HTable table = null;
- try {
- table = new HTable(new HBaseConfiguration(job), tableName);
- } catch(IOException e) {
- LOG.error(e);
- throw e;
- }
- return new TableRecordWriter(table);
- }
-
- /** {@inheritDoc} */
- @Override
- @SuppressWarnings("unused")
- public void checkOutputSpecs(FileSystem ignored, JobConf job)
- throws FileAlreadyExistsException, InvalidJobConfException, IOException {
-
- String tableName = job.get(OUTPUT_TABLE);
- if(tableName == null) {
- throw new IOException("Must specify table name");
- }
- }
-}
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.mapred;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.io.BatchUpdate;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.mapred.FileAlreadyExistsException;
+import org.apache.hadoop.mapred.InvalidJobConfException;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.RecordWriter;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * Convert Map/Reduce output and write it to an HBase table
+ */
+public class TableOutputFormat extends
+FileOutputFormat<ImmutableBytesWritable, BatchUpdate> {
+
+ /** JobConf parameter that specifies the output table */
+ public static final String OUTPUT_TABLE = "hbase.mapred.outputtable";
+ private final Log LOG = LogFactory.getLog(TableOutputFormat.class);
+
+ /**
+ * Convert Reduce output (key, value) to (HStoreKey, KeyedDataArrayWritable)
+ * and write to an HBase table
+ */
+ protected class TableRecordWriter
+ implements RecordWriter<ImmutableBytesWritable, BatchUpdate> {
+ private HTable m_table;
+
+ /**
+ * Instantiate a TableRecordWriter with the HBase HClient for writing.
+ *
+ * @param table
+ */
+ public TableRecordWriter(HTable table) {
+ m_table = table;
+ }
+
+ /** {@inheritDoc} */
+ public void close(@SuppressWarnings("unused") Reporter reporter) {
+ // Nothing to do.
+ }
+
+ /** {@inheritDoc} */
+ public void write(@SuppressWarnings("unused") ImmutableBytesWritable key,
+ BatchUpdate value) throws IOException {
+ m_table.commit(value);
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ @SuppressWarnings("unchecked")
+ public RecordWriter getRecordWriter(
+ @SuppressWarnings("unused") FileSystem ignored,
+ JobConf job,
+ @SuppressWarnings("unused") String name,
+ @SuppressWarnings("unused") Progressable progress) throws IOException {
+
+ // expecting exactly one path
+
+ String tableName = job.get(OUTPUT_TABLE);
+ HTable table = null;
+ try {
+ table = new HTable(new HBaseConfiguration(job), tableName);
+ } catch(IOException e) {
+ LOG.error(e);
+ throw e;
+ }
+ return new TableRecordWriter(table);
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ @SuppressWarnings("unused")
+ public void checkOutputSpecs(FileSystem ignored, JobConf job)
+ throws FileAlreadyExistsException, InvalidJobConfException, IOException {
+
+ String tableName = job.get(OUTPUT_TABLE);
+ if(tableName == null) {
+ throw new IOException("Must specify table name");
+ }
+ }
+}