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 [3/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/regionserver/HStoreScanner.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HStoreScanner.java?rev=659249&r1=659248&r2=659249&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HStoreScanner.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HStoreScanner.java Thu May 22 13:49:25 2008
@@ -1,273 +1,273 @@
-/**
- * Copyright 2008 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.regionserver;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.SortedMap;
-import java.util.TreeMap;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HStoreKey;
-import org.apache.hadoop.hbase.filter.RowFilterInterface;
-import org.apache.hadoop.hbase.util.Bytes;
-
-/**
- * Scanner scans both the memcache and the HStore
- */
-class HStoreScanner implements InternalScanner {
- static final Log LOG = LogFactory.getLog(HStoreScanner.class);
-
- private InternalScanner[] scanners;
- private TreeMap<byte [], byte []>[] resultSets;
- private HStoreKey[] keys;
- private boolean wildcardMatch = false;
- private boolean multipleMatchers = false;
- private RowFilterInterface dataFilter;
- private HStore store;
-
- /** Create an Scanner with a handle on the memcache and HStore files. */
- @SuppressWarnings("unchecked")
- HStoreScanner(HStore store, byte [][] targetCols, byte [] firstRow,
- long timestamp, RowFilterInterface filter)
- throws IOException {
- this.store = store;
- this.dataFilter = filter;
- if (null != dataFilter) {
- dataFilter.reset();
- }
- this.scanners = new InternalScanner[2];
- this.resultSets = new TreeMap[scanners.length];
- this.keys = new HStoreKey[scanners.length];
-
- try {
- scanners[0] = store.memcache.getScanner(timestamp, targetCols, firstRow);
- scanners[1] = new StoreFileScanner(store, timestamp, targetCols, firstRow);
- for (int i = 0; i < scanners.length; i++) {
- if (scanners[i].isWildcardScanner()) {
- this.wildcardMatch = true;
- }
- if (scanners[i].isMultipleMatchScanner()) {
- this.multipleMatchers = true;
- }
- }
- } catch(IOException e) {
- for (int i = 0; i < this.scanners.length; i++) {
- if(scanners[i] != null) {
- closeScanner(i);
- }
- }
- throw e;
- }
-
- // Advance to the first key in each scanner.
- // All results will match the required column-set and scanTime.
- for (int i = 0; i < scanners.length; i++) {
- keys[i] = new HStoreKey();
- resultSets[i] = new TreeMap<byte [], byte []>(Bytes.BYTES_COMPARATOR);
- if(scanners[i] != null && !scanners[i].next(keys[i], resultSets[i])) {
- closeScanner(i);
- }
- }
- }
-
- /** @return true if the scanner is a wild card scanner */
- public boolean isWildcardScanner() {
- return wildcardMatch;
- }
-
- /** @return true if the scanner is a multiple match scanner */
- public boolean isMultipleMatchScanner() {
- return multipleMatchers;
- }
-
- /** {@inheritDoc} */
- public boolean next(HStoreKey key, SortedMap<byte [], byte[]> results)
- throws IOException {
-
- // Filtered flag is set by filters. If a cell has been 'filtered out'
- // -- i.e. it is not to be returned to the caller -- the flag is 'true'.
- boolean filtered = true;
- boolean moreToFollow = true;
- while (filtered && moreToFollow) {
- // Find the lowest-possible key.
- byte [] chosenRow = null;
- long chosenTimestamp = -1;
- for (int i = 0; i < this.keys.length; i++) {
- if (scanners[i] != null &&
- (chosenRow == null ||
- (Bytes.compareTo(keys[i].getRow(), chosenRow) < 0) ||
- ((Bytes.compareTo(keys[i].getRow(), chosenRow) == 0) &&
- (keys[i].getTimestamp() > chosenTimestamp)))) {
- chosenRow = keys[i].getRow();
- chosenTimestamp = keys[i].getTimestamp();
- }
- }
-
- // Filter whole row by row key?
- filtered = dataFilter != null? dataFilter.filterRowKey(chosenRow) : false;
-
- // Store the key and results for each sub-scanner. Merge them as
- // appropriate.
- if (chosenTimestamp >= 0 && !filtered) {
- // Here we are setting the passed in key with current row+timestamp
- key.setRow(chosenRow);
- key.setVersion(chosenTimestamp);
- key.setColumn(HConstants.EMPTY_BYTE_ARRAY);
- // Keep list of deleted cell keys within this row. We need this
- // because as we go through scanners, the delete record may be in an
- // early scanner and then the same record with a non-delete, non-null
- // value in a later. Without history of what we've seen, we'll return
- // deleted values. This List should not ever grow too large since we
- // are only keeping rows and columns that match those set on the
- // scanner and which have delete values. If memory usage becomes a
- // problem, could redo as bloom filter.
- List<HStoreKey> deletes = new ArrayList<HStoreKey>();
- for (int i = 0; i < scanners.length && !filtered; i++) {
- while ((scanners[i] != null
- && !filtered
- && moreToFollow)
- && (Bytes.compareTo(keys[i].getRow(), chosenRow) == 0)) {
- // If we are doing a wild card match or there are multiple
- // matchers per column, we need to scan all the older versions of
- // this row to pick up the rest of the family members
- if (!wildcardMatch
- && !multipleMatchers
- && (keys[i].getTimestamp() != chosenTimestamp)) {
- break;
- }
-
- // NOTE: We used to do results.putAll(resultSets[i]);
- // but this had the effect of overwriting newer
- // values with older ones. So now we only insert
- // a result if the map does not contain the key.
- HStoreKey hsk = new HStoreKey(key.getRow(), HConstants.EMPTY_BYTE_ARRAY,
- key.getTimestamp());
- for (Map.Entry<byte [], byte[]> e : resultSets[i].entrySet()) {
- hsk.setColumn(e.getKey());
- if (HLogEdit.isDeleted(e.getValue())) {
- if (!deletes.contains(hsk)) {
- // Key changes as we cycle the for loop so add a copy to
- // the set of deletes.
- deletes.add(new HStoreKey(hsk));
- }
- } else if (!deletes.contains(hsk) &&
- !filtered &&
- moreToFollow &&
- !results.containsKey(e.getKey())) {
- if (dataFilter != null) {
- // Filter whole row by column data?
- filtered =
- dataFilter.filterColumn(chosenRow, e.getKey(), e.getValue());
- if (filtered) {
- results.clear();
- break;
- }
- }
- results.put(e.getKey(), e.getValue());
- }
- }
- resultSets[i].clear();
- if (!scanners[i].next(keys[i], resultSets[i])) {
- closeScanner(i);
- }
- }
- }
- }
-
- for (int i = 0; i < scanners.length; i++) {
- // If the current scanner is non-null AND has a lower-or-equal
- // row label, then its timestamp is bad. We need to advance it.
- while ((scanners[i] != null) &&
- (Bytes.compareTo(keys[i].getRow(), chosenRow) <= 0)) {
- resultSets[i].clear();
- if (!scanners[i].next(keys[i], resultSets[i])) {
- closeScanner(i);
- }
- }
- }
-
- moreToFollow = chosenTimestamp >= 0;
-
- if (dataFilter != null) {
- if (dataFilter.filterAllRemaining()) {
- moreToFollow = false;
- }
- }
-
- if (results.size() <= 0 && !filtered) {
- // There were no results found for this row. Marked it as
- // 'filtered'-out otherwise we will not move on to the next row.
- filtered = true;
- }
- }
-
- // If we got no results, then there is no more to follow.
- if (results == null || results.size() <= 0) {
- moreToFollow = false;
- }
-
- // Make sure scanners closed if no more results
- if (!moreToFollow) {
- for (int i = 0; i < scanners.length; i++) {
- if (null != scanners[i]) {
- closeScanner(i);
- }
- }
- }
-
- return moreToFollow;
- }
-
- /** Shut down a single scanner */
- void closeScanner(int i) {
- try {
- try {
- scanners[i].close();
- } catch (IOException e) {
- LOG.warn(store.storeName + " failed closing scanner " + i, e);
- }
- } finally {
- scanners[i] = null;
- keys[i] = null;
- resultSets[i] = null;
- }
- }
-
- /** {@inheritDoc} */
- public void close() {
- for(int i = 0; i < scanners.length; i++) {
- if(scanners[i] != null) {
- closeScanner(i);
- }
- }
- }
-
- public Iterator<Map.Entry<HStoreKey, SortedMap<byte [], byte[]>>> iterator() {
- throw new UnsupportedOperationException("Unimplemented serverside. " +
- "next(HStoreKey, StortedMap(...) is more efficient");
- }
-}
+/**
+ * Copyright 2008 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.regionserver;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HStoreKey;
+import org.apache.hadoop.hbase.filter.RowFilterInterface;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Scanner scans both the memcache and the HStore
+ */
+class HStoreScanner implements InternalScanner {
+ static final Log LOG = LogFactory.getLog(HStoreScanner.class);
+
+ private InternalScanner[] scanners;
+ private TreeMap<byte [], byte []>[] resultSets;
+ private HStoreKey[] keys;
+ private boolean wildcardMatch = false;
+ private boolean multipleMatchers = false;
+ private RowFilterInterface dataFilter;
+ private HStore store;
+
+ /** Create an Scanner with a handle on the memcache and HStore files. */
+ @SuppressWarnings("unchecked")
+ HStoreScanner(HStore store, byte [][] targetCols, byte [] firstRow,
+ long timestamp, RowFilterInterface filter)
+ throws IOException {
+ this.store = store;
+ this.dataFilter = filter;
+ if (null != dataFilter) {
+ dataFilter.reset();
+ }
+ this.scanners = new InternalScanner[2];
+ this.resultSets = new TreeMap[scanners.length];
+ this.keys = new HStoreKey[scanners.length];
+
+ try {
+ scanners[0] = store.memcache.getScanner(timestamp, targetCols, firstRow);
+ scanners[1] = new StoreFileScanner(store, timestamp, targetCols, firstRow);
+ for (int i = 0; i < scanners.length; i++) {
+ if (scanners[i].isWildcardScanner()) {
+ this.wildcardMatch = true;
+ }
+ if (scanners[i].isMultipleMatchScanner()) {
+ this.multipleMatchers = true;
+ }
+ }
+ } catch(IOException e) {
+ for (int i = 0; i < this.scanners.length; i++) {
+ if(scanners[i] != null) {
+ closeScanner(i);
+ }
+ }
+ throw e;
+ }
+
+ // Advance to the first key in each scanner.
+ // All results will match the required column-set and scanTime.
+ for (int i = 0; i < scanners.length; i++) {
+ keys[i] = new HStoreKey();
+ resultSets[i] = new TreeMap<byte [], byte []>(Bytes.BYTES_COMPARATOR);
+ if(scanners[i] != null && !scanners[i].next(keys[i], resultSets[i])) {
+ closeScanner(i);
+ }
+ }
+ }
+
+ /** @return true if the scanner is a wild card scanner */
+ public boolean isWildcardScanner() {
+ return wildcardMatch;
+ }
+
+ /** @return true if the scanner is a multiple match scanner */
+ public boolean isMultipleMatchScanner() {
+ return multipleMatchers;
+ }
+
+ /** {@inheritDoc} */
+ public boolean next(HStoreKey key, SortedMap<byte [], byte[]> results)
+ throws IOException {
+
+ // Filtered flag is set by filters. If a cell has been 'filtered out'
+ // -- i.e. it is not to be returned to the caller -- the flag is 'true'.
+ boolean filtered = true;
+ boolean moreToFollow = true;
+ while (filtered && moreToFollow) {
+ // Find the lowest-possible key.
+ byte [] chosenRow = null;
+ long chosenTimestamp = -1;
+ for (int i = 0; i < this.keys.length; i++) {
+ if (scanners[i] != null &&
+ (chosenRow == null ||
+ (Bytes.compareTo(keys[i].getRow(), chosenRow) < 0) ||
+ ((Bytes.compareTo(keys[i].getRow(), chosenRow) == 0) &&
+ (keys[i].getTimestamp() > chosenTimestamp)))) {
+ chosenRow = keys[i].getRow();
+ chosenTimestamp = keys[i].getTimestamp();
+ }
+ }
+
+ // Filter whole row by row key?
+ filtered = dataFilter != null? dataFilter.filterRowKey(chosenRow) : false;
+
+ // Store the key and results for each sub-scanner. Merge them as
+ // appropriate.
+ if (chosenTimestamp >= 0 && !filtered) {
+ // Here we are setting the passed in key with current row+timestamp
+ key.setRow(chosenRow);
+ key.setVersion(chosenTimestamp);
+ key.setColumn(HConstants.EMPTY_BYTE_ARRAY);
+ // Keep list of deleted cell keys within this row. We need this
+ // because as we go through scanners, the delete record may be in an
+ // early scanner and then the same record with a non-delete, non-null
+ // value in a later. Without history of what we've seen, we'll return
+ // deleted values. This List should not ever grow too large since we
+ // are only keeping rows and columns that match those set on the
+ // scanner and which have delete values. If memory usage becomes a
+ // problem, could redo as bloom filter.
+ List<HStoreKey> deletes = new ArrayList<HStoreKey>();
+ for (int i = 0; i < scanners.length && !filtered; i++) {
+ while ((scanners[i] != null
+ && !filtered
+ && moreToFollow)
+ && (Bytes.compareTo(keys[i].getRow(), chosenRow) == 0)) {
+ // If we are doing a wild card match or there are multiple
+ // matchers per column, we need to scan all the older versions of
+ // this row to pick up the rest of the family members
+ if (!wildcardMatch
+ && !multipleMatchers
+ && (keys[i].getTimestamp() != chosenTimestamp)) {
+ break;
+ }
+
+ // NOTE: We used to do results.putAll(resultSets[i]);
+ // but this had the effect of overwriting newer
+ // values with older ones. So now we only insert
+ // a result if the map does not contain the key.
+ HStoreKey hsk = new HStoreKey(key.getRow(), HConstants.EMPTY_BYTE_ARRAY,
+ key.getTimestamp());
+ for (Map.Entry<byte [], byte[]> e : resultSets[i].entrySet()) {
+ hsk.setColumn(e.getKey());
+ if (HLogEdit.isDeleted(e.getValue())) {
+ if (!deletes.contains(hsk)) {
+ // Key changes as we cycle the for loop so add a copy to
+ // the set of deletes.
+ deletes.add(new HStoreKey(hsk));
+ }
+ } else if (!deletes.contains(hsk) &&
+ !filtered &&
+ moreToFollow &&
+ !results.containsKey(e.getKey())) {
+ if (dataFilter != null) {
+ // Filter whole row by column data?
+ filtered =
+ dataFilter.filterColumn(chosenRow, e.getKey(), e.getValue());
+ if (filtered) {
+ results.clear();
+ break;
+ }
+ }
+ results.put(e.getKey(), e.getValue());
+ }
+ }
+ resultSets[i].clear();
+ if (!scanners[i].next(keys[i], resultSets[i])) {
+ closeScanner(i);
+ }
+ }
+ }
+ }
+
+ for (int i = 0; i < scanners.length; i++) {
+ // If the current scanner is non-null AND has a lower-or-equal
+ // row label, then its timestamp is bad. We need to advance it.
+ while ((scanners[i] != null) &&
+ (Bytes.compareTo(keys[i].getRow(), chosenRow) <= 0)) {
+ resultSets[i].clear();
+ if (!scanners[i].next(keys[i], resultSets[i])) {
+ closeScanner(i);
+ }
+ }
+ }
+
+ moreToFollow = chosenTimestamp >= 0;
+
+ if (dataFilter != null) {
+ if (dataFilter.filterAllRemaining()) {
+ moreToFollow = false;
+ }
+ }
+
+ if (results.size() <= 0 && !filtered) {
+ // There were no results found for this row. Marked it as
+ // 'filtered'-out otherwise we will not move on to the next row.
+ filtered = true;
+ }
+ }
+
+ // If we got no results, then there is no more to follow.
+ if (results == null || results.size() <= 0) {
+ moreToFollow = false;
+ }
+
+ // Make sure scanners closed if no more results
+ if (!moreToFollow) {
+ for (int i = 0; i < scanners.length; i++) {
+ if (null != scanners[i]) {
+ closeScanner(i);
+ }
+ }
+ }
+
+ return moreToFollow;
+ }
+
+ /** Shut down a single scanner */
+ void closeScanner(int i) {
+ try {
+ try {
+ scanners[i].close();
+ } catch (IOException e) {
+ LOG.warn(store.storeName + " failed closing scanner " + i, e);
+ }
+ } finally {
+ scanners[i] = null;
+ keys[i] = null;
+ resultSets[i] = null;
+ }
+ }
+
+ /** {@inheritDoc} */
+ public void close() {
+ for(int i = 0; i < scanners.length; i++) {
+ if(scanners[i] != null) {
+ closeScanner(i);
+ }
+ }
+ }
+
+ public Iterator<Map.Entry<HStoreKey, SortedMap<byte [], byte[]>>> iterator() {
+ throw new UnsupportedOperationException("Unimplemented serverside. " +
+ "next(HStoreKey, StortedMap(...) is more efficient");
+ }
+}
Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestInclusiveStopRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestInclusiveStopRowFilter.java?rev=659249&r1=659248&r2=659249&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestInclusiveStopRowFilter.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestInclusiveStopRowFilter.java Thu May 22 13:49:25 2008
@@ -1,93 +1,93 @@
-/**
- * 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.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-
-import org.apache.hadoop.hbase.util.Bytes;
-
-import junit.framework.TestCase;
-
-/**
- * Tests the inclusive stop row filter
- */
-public class TestInclusiveStopRowFilter extends TestCase {
- private final byte [] STOP_ROW = Bytes.toBytes("stop_row");
- private final byte [] GOOD_ROW = Bytes.toBytes("good_row");
- private final byte [] PAST_STOP_ROW = Bytes.toBytes("zzzzzz");
-
- RowFilterInterface mainFilter;
-
- /** {@inheritDoc} */
- @Override
- protected void setUp() throws Exception {
- super.setUp();
- mainFilter = new InclusiveStopRowFilter(STOP_ROW);
- }
-
- /**
- * Tests identification of the stop row
- * @throws Exception
- */
- public void testStopRowIdentification() throws Exception {
- stopRowTests(mainFilter);
- }
-
- /**
- * Tests serialization
- * @throws Exception
- */
- public void testSerialization() throws Exception {
- // Decompose mainFilter to bytes.
- ByteArrayOutputStream stream = new ByteArrayOutputStream();
- DataOutputStream out = new DataOutputStream(stream);
- mainFilter.write(out);
- out.close();
- byte[] buffer = stream.toByteArray();
-
- // Recompose mainFilter.
- DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
- RowFilterInterface newFilter = new InclusiveStopRowFilter();
- newFilter.readFields(in);
-
- // Ensure the serialization preserved the filter by running a full test.
- stopRowTests(newFilter);
- }
-
- private void stopRowTests(RowFilterInterface filter) throws Exception {
- assertFalse("Filtering on " + GOOD_ROW, filter.filterRowKey(GOOD_ROW));
- assertFalse("Filtering on " + STOP_ROW, filter.filterRowKey(STOP_ROW));
- assertTrue("Filtering on " + PAST_STOP_ROW, filter.filterRowKey(PAST_STOP_ROW));
-
- assertFalse("Filtering on " + GOOD_ROW, filter.filterColumn(GOOD_ROW, null,
- null));
- assertFalse("Filtering on " + STOP_ROW, filter.filterColumn(STOP_ROW, null, null));
- assertTrue("Filtering on " + PAST_STOP_ROW, filter.filterColumn(PAST_STOP_ROW,
- null, null));
-
- assertFalse("FilterAllRemaining", filter.filterAllRemaining());
- assertFalse("FilterNotNull", filter.filterRow(null));
-
- assertFalse("Filter a null", filter.filterRowKey(null));
- }
-}
+/**
+ * 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.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+import junit.framework.TestCase;
+
+/**
+ * Tests the inclusive stop row filter
+ */
+public class TestInclusiveStopRowFilter extends TestCase {
+ private final byte [] STOP_ROW = Bytes.toBytes("stop_row");
+ private final byte [] GOOD_ROW = Bytes.toBytes("good_row");
+ private final byte [] PAST_STOP_ROW = Bytes.toBytes("zzzzzz");
+
+ RowFilterInterface mainFilter;
+
+ /** {@inheritDoc} */
+ @Override
+ protected void setUp() throws Exception {
+ super.setUp();
+ mainFilter = new InclusiveStopRowFilter(STOP_ROW);
+ }
+
+ /**
+ * Tests identification of the stop row
+ * @throws Exception
+ */
+ public void testStopRowIdentification() throws Exception {
+ stopRowTests(mainFilter);
+ }
+
+ /**
+ * Tests serialization
+ * @throws Exception
+ */
+ public void testSerialization() throws Exception {
+ // Decompose mainFilter to bytes.
+ ByteArrayOutputStream stream = new ByteArrayOutputStream();
+ DataOutputStream out = new DataOutputStream(stream);
+ mainFilter.write(out);
+ out.close();
+ byte[] buffer = stream.toByteArray();
+
+ // Recompose mainFilter.
+ DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
+ RowFilterInterface newFilter = new InclusiveStopRowFilter();
+ newFilter.readFields(in);
+
+ // Ensure the serialization preserved the filter by running a full test.
+ stopRowTests(newFilter);
+ }
+
+ private void stopRowTests(RowFilterInterface filter) throws Exception {
+ assertFalse("Filtering on " + GOOD_ROW, filter.filterRowKey(GOOD_ROW));
+ assertFalse("Filtering on " + STOP_ROW, filter.filterRowKey(STOP_ROW));
+ assertTrue("Filtering on " + PAST_STOP_ROW, filter.filterRowKey(PAST_STOP_ROW));
+
+ assertFalse("Filtering on " + GOOD_ROW, filter.filterColumn(GOOD_ROW, null,
+ null));
+ assertFalse("Filtering on " + STOP_ROW, filter.filterColumn(STOP_ROW, null, null));
+ assertTrue("Filtering on " + PAST_STOP_ROW, filter.filterColumn(PAST_STOP_ROW,
+ null, null));
+
+ assertFalse("FilterAllRemaining", filter.filterAllRemaining());
+ assertFalse("FilterNotNull", filter.filterRow(null));
+
+ assertFalse("Filter a null", filter.filterRowKey(null));
+ }
+}
Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestPageRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestPageRowFilter.java?rev=659249&r1=659248&r2=659249&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestPageRowFilter.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestPageRowFilter.java Thu May 22 13:49:25 2008
@@ -1,99 +1,99 @@
-/**
- * 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.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-
-import org.apache.hadoop.hbase.util.Bytes;
-
-
-import junit.framework.TestCase;
-
-/**
- * Tests for the page row filter
- */
-public class TestPageRowFilter extends TestCase {
-
- RowFilterInterface mainFilter;
- final int ROW_LIMIT = 3;
-
- /** {@inheritDoc} */
- @Override
- protected void setUp() throws Exception {
- super.setUp();
- mainFilter = new PageRowFilter(ROW_LIMIT);
- }
-
- /**
- * test page size filter
- * @throws Exception
- */
- public void testPageSize() throws Exception {
- pageSizeTests(mainFilter);
- }
-
- /**
- * Test filter serialization
- * @throws Exception
- */
- public void testSerialization() throws Exception {
- // Decompose mainFilter to bytes.
- ByteArrayOutputStream stream = new ByteArrayOutputStream();
- DataOutputStream out = new DataOutputStream(stream);
- mainFilter.write(out);
- out.close();
- byte[] buffer = stream.toByteArray();
-
- // Recompose mainFilter.
- DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
- RowFilterInterface newFilter = new PageRowFilter();
- newFilter.readFields(in);
-
- // Ensure the serialization preserved the filter by running a full test.
- pageSizeTests(newFilter);
- }
-
- private void pageSizeTests(RowFilterInterface filter) throws Exception {
- testFiltersBeyondPageSize(filter, ROW_LIMIT);
- // Test reset works by going in again.
- filter.reset();
- testFiltersBeyondPageSize(filter, ROW_LIMIT);
- }
-
- private void testFiltersBeyondPageSize(final RowFilterInterface filter,
- final int pageSize) {
- for (int i = 0; i < (pageSize * 2); i++) {
- byte [] row = Bytes.toBytes(Integer.toString(i));
- boolean filterOut = filter.filterRowKey(row);
- if (!filterOut) {
- assertFalse("Disagrees with 'filter'", filter.filterAllRemaining());
- } else {
- // Once we have all for a page, calls to filterAllRemaining should
- // stay true.
- assertTrue("Disagrees with 'filter'", filter.filterAllRemaining());
- assertTrue(i >= pageSize);
- }
- filter.rowProcessed(filterOut, row);
- }
- }
-}
+/**
+ * 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.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+
+import junit.framework.TestCase;
+
+/**
+ * Tests for the page row filter
+ */
+public class TestPageRowFilter extends TestCase {
+
+ RowFilterInterface mainFilter;
+ final int ROW_LIMIT = 3;
+
+ /** {@inheritDoc} */
+ @Override
+ protected void setUp() throws Exception {
+ super.setUp();
+ mainFilter = new PageRowFilter(ROW_LIMIT);
+ }
+
+ /**
+ * test page size filter
+ * @throws Exception
+ */
+ public void testPageSize() throws Exception {
+ pageSizeTests(mainFilter);
+ }
+
+ /**
+ * Test filter serialization
+ * @throws Exception
+ */
+ public void testSerialization() throws Exception {
+ // Decompose mainFilter to bytes.
+ ByteArrayOutputStream stream = new ByteArrayOutputStream();
+ DataOutputStream out = new DataOutputStream(stream);
+ mainFilter.write(out);
+ out.close();
+ byte[] buffer = stream.toByteArray();
+
+ // Recompose mainFilter.
+ DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
+ RowFilterInterface newFilter = new PageRowFilter();
+ newFilter.readFields(in);
+
+ // Ensure the serialization preserved the filter by running a full test.
+ pageSizeTests(newFilter);
+ }
+
+ private void pageSizeTests(RowFilterInterface filter) throws Exception {
+ testFiltersBeyondPageSize(filter, ROW_LIMIT);
+ // Test reset works by going in again.
+ filter.reset();
+ testFiltersBeyondPageSize(filter, ROW_LIMIT);
+ }
+
+ private void testFiltersBeyondPageSize(final RowFilterInterface filter,
+ final int pageSize) {
+ for (int i = 0; i < (pageSize * 2); i++) {
+ byte [] row = Bytes.toBytes(Integer.toString(i));
+ boolean filterOut = filter.filterRowKey(row);
+ if (!filterOut) {
+ assertFalse("Disagrees with 'filter'", filter.filterAllRemaining());
+ } else {
+ // Once we have all for a page, calls to filterAllRemaining should
+ // stay true.
+ assertTrue("Disagrees with 'filter'", filter.filterAllRemaining());
+ assertTrue(i >= pageSize);
+ }
+ filter.rowProcessed(filterOut, row);
+ }
+ }
+}
Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java?rev=659249&r1=659248&r2=659249&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java Thu May 22 13:49:25 2008
@@ -1,196 +1,196 @@
-/**
- * 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.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.UnsupportedEncodingException;
-import java.util.Map;
-import java.util.TreeMap;
-
-import junit.framework.TestCase;
-
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.regionserver.HLogEdit;
-import org.apache.hadoop.hbase.util.Bytes;
-
-/**
- * Tests for regular expression row filter
- */
-public class TestRegExpRowFilter extends TestCase {
- TreeMap<byte [], byte []> colvalues;
- RowFilterInterface mainFilter;
- final char FIRST_CHAR = 'a';
- final char LAST_CHAR = 'e';
- final String HOST_PREFIX = "org.apache.site-";
- static byte [] GOOD_BYTES = null;
-
- static {
- try {
- GOOD_BYTES = "abc".getBytes(HConstants.UTF8_ENCODING);
- } catch (UnsupportedEncodingException e) {
- fail();
- }
- }
- /** {@inheritDoc} */
- @Override
- protected void setUp() throws Exception {
- super.setUp();
- this.colvalues = new TreeMap<byte [], byte[]>(Bytes.BYTES_COMPARATOR);
- for (char c = FIRST_CHAR; c < LAST_CHAR; c++) {
- colvalues.put(Bytes.toBytes(new String(new char [] {c})), GOOD_BYTES);
- }
- this.mainFilter = new RegExpRowFilter(HOST_PREFIX + ".*", colvalues);
- }
-
- /**
- * Tests filtering using a regex on the row key
- * @throws Exception
- */
- public void testRegexOnRow() throws Exception {
- regexRowTests(mainFilter);
- }
-
- /**
- * Tests filtering using a regex on row and colum
- * @throws Exception
- */
- public void testRegexOnRowAndColumn() throws Exception {
- regexRowColumnTests(mainFilter);
- }
-
- /**
- * Only return values that are not null
- * @throws Exception
- */
- public void testFilterNotNull() throws Exception {
- filterNotNullTests(mainFilter);
- }
-
- /**
- * Test serialization
- * @throws Exception
- */
- public void testSerialization() throws Exception {
- // Decompose mainFilter to bytes.
- ByteArrayOutputStream stream = new ByteArrayOutputStream();
- DataOutputStream out = new DataOutputStream(stream);
- mainFilter.write(out);
- out.close();
- byte[] buffer = stream.toByteArray();
-
- // Recompose filter.
- DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
- RowFilterInterface newFilter = new RegExpRowFilter();
- newFilter.readFields(in);
-
- // Ensure the serialization preserved the filter by running all test.
- regexRowTests(newFilter);
- newFilter.reset();
- regexRowColumnTests(newFilter);
- newFilter.reset();
- filterNotNullTests(newFilter);
- }
-
- private void regexRowTests(RowFilterInterface filter) throws Exception {
- for (char c = FIRST_CHAR; c <= LAST_CHAR; c++) {
- byte [] t = createRow(c);
- assertFalse("Failed with characer " + c, filter.filterRowKey(t));
- }
- String yahooSite = "com.yahoo.www";
- assertTrue("Failed with character " +
- yahooSite, filter.filterRowKey(Bytes.toBytes(yahooSite)));
- }
-
- private void regexRowColumnTests(RowFilterInterface filter)
- throws UnsupportedEncodingException {
-
- for (char c = FIRST_CHAR; c <= LAST_CHAR; c++) {
- byte [] t = createRow(c);
- for (Map.Entry<byte [], byte []> e: this.colvalues.entrySet()) {
- assertFalse("Failed on " + c,
- filter.filterColumn(t, e.getKey(), e.getValue()));
- }
- }
- // Try a row and column I know will pass.
- char c = 'c';
- byte [] r = createRow(c);
- byte [] col = Bytes.toBytes(Character.toString(c));
- assertFalse("Failed with character " + c,
- filter.filterColumn(r, col, GOOD_BYTES));
-
- // Do same but with bad bytes.
- assertTrue("Failed with character " + c,
- filter.filterColumn(r, col, "badbytes".getBytes(HConstants.UTF8_ENCODING)));
-
- // Do with good bytes but bad column name. Should not filter out.
- assertFalse("Failed with character " + c,
- filter.filterColumn(r, Bytes.toBytes("badcolumn"), GOOD_BYTES));
-
- // Good column, good bytes but bad row.
- assertTrue("Failed with character " + c,
- filter.filterColumn(Bytes.toBytes("bad row"),
- Bytes.toBytes("badcolumn"), GOOD_BYTES));
- }
-
- private void filterNotNullTests(RowFilterInterface filter) throws Exception {
- // Modify the filter to expect certain columns to be null:
- // Expecting a row WITH columnKeys: a-d, WITHOUT columnKey: e
- ((RegExpRowFilter)filter).setColumnFilter(new byte [] {LAST_CHAR}, null);
-
- char secondToLast = (char)(LAST_CHAR - 1);
- char thirdToLast = (char)(LAST_CHAR - 2);
-
- // Modify the row to be missing an expected columnKey (d)
- colvalues.remove(new byte [] {(byte)secondToLast});
-
- // Try a row that is missing an expected columnKey.
- // Testing row with columnKeys: a-c
- assertTrue("Failed with last columnKey " + thirdToLast, filter.
- filterRow(colvalues));
-
- // Try a row that has all expected columnKeys, and NO null-expected
- // columnKeys.
- // Testing row with columnKeys: a-d
- colvalues.put(new byte [] {(byte)secondToLast}, GOOD_BYTES);
- assertFalse("Failed with last columnKey " + secondToLast, filter.
- filterRow(colvalues));
-
- // Try a row that has all expected columnKeys AND a null-expected columnKey.
- // Testing row with columnKeys: a-e
- colvalues.put(new byte [] {LAST_CHAR}, GOOD_BYTES);
- assertTrue("Failed with last columnKey " + LAST_CHAR, filter.
- filterRow(colvalues));
-
- // Try a row that has all expected columnKeys and a null-expected columnKey
- // that maps to a null value.
- // Testing row with columnKeys: a-e, e maps to null
- colvalues.put(new byte [] {LAST_CHAR},
- HLogEdit.deleteBytes.get());
- assertFalse("Failed with last columnKey " + LAST_CHAR + " mapping to null.",
- filter.filterRow(colvalues));
- }
-
- private byte [] createRow(final char c) {
- return Bytes.toBytes(HOST_PREFIX + Character.toString(c));
- }
-}
+/**
+ * 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.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.UnsupportedEncodingException;
+import java.util.Map;
+import java.util.TreeMap;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.regionserver.HLogEdit;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Tests for regular expression row filter
+ */
+public class TestRegExpRowFilter extends TestCase {
+ TreeMap<byte [], byte []> colvalues;
+ RowFilterInterface mainFilter;
+ final char FIRST_CHAR = 'a';
+ final char LAST_CHAR = 'e';
+ final String HOST_PREFIX = "org.apache.site-";
+ static byte [] GOOD_BYTES = null;
+
+ static {
+ try {
+ GOOD_BYTES = "abc".getBytes(HConstants.UTF8_ENCODING);
+ } catch (UnsupportedEncodingException e) {
+ fail();
+ }
+ }
+ /** {@inheritDoc} */
+ @Override
+ protected void setUp() throws Exception {
+ super.setUp();
+ this.colvalues = new TreeMap<byte [], byte[]>(Bytes.BYTES_COMPARATOR);
+ for (char c = FIRST_CHAR; c < LAST_CHAR; c++) {
+ colvalues.put(Bytes.toBytes(new String(new char [] {c})), GOOD_BYTES);
+ }
+ this.mainFilter = new RegExpRowFilter(HOST_PREFIX + ".*", colvalues);
+ }
+
+ /**
+ * Tests filtering using a regex on the row key
+ * @throws Exception
+ */
+ public void testRegexOnRow() throws Exception {
+ regexRowTests(mainFilter);
+ }
+
+ /**
+ * Tests filtering using a regex on row and colum
+ * @throws Exception
+ */
+ public void testRegexOnRowAndColumn() throws Exception {
+ regexRowColumnTests(mainFilter);
+ }
+
+ /**
+ * Only return values that are not null
+ * @throws Exception
+ */
+ public void testFilterNotNull() throws Exception {
+ filterNotNullTests(mainFilter);
+ }
+
+ /**
+ * Test serialization
+ * @throws Exception
+ */
+ public void testSerialization() throws Exception {
+ // Decompose mainFilter to bytes.
+ ByteArrayOutputStream stream = new ByteArrayOutputStream();
+ DataOutputStream out = new DataOutputStream(stream);
+ mainFilter.write(out);
+ out.close();
+ byte[] buffer = stream.toByteArray();
+
+ // Recompose filter.
+ DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
+ RowFilterInterface newFilter = new RegExpRowFilter();
+ newFilter.readFields(in);
+
+ // Ensure the serialization preserved the filter by running all test.
+ regexRowTests(newFilter);
+ newFilter.reset();
+ regexRowColumnTests(newFilter);
+ newFilter.reset();
+ filterNotNullTests(newFilter);
+ }
+
+ private void regexRowTests(RowFilterInterface filter) throws Exception {
+ for (char c = FIRST_CHAR; c <= LAST_CHAR; c++) {
+ byte [] t = createRow(c);
+ assertFalse("Failed with characer " + c, filter.filterRowKey(t));
+ }
+ String yahooSite = "com.yahoo.www";
+ assertTrue("Failed with character " +
+ yahooSite, filter.filterRowKey(Bytes.toBytes(yahooSite)));
+ }
+
+ private void regexRowColumnTests(RowFilterInterface filter)
+ throws UnsupportedEncodingException {
+
+ for (char c = FIRST_CHAR; c <= LAST_CHAR; c++) {
+ byte [] t = createRow(c);
+ for (Map.Entry<byte [], byte []> e: this.colvalues.entrySet()) {
+ assertFalse("Failed on " + c,
+ filter.filterColumn(t, e.getKey(), e.getValue()));
+ }
+ }
+ // Try a row and column I know will pass.
+ char c = 'c';
+ byte [] r = createRow(c);
+ byte [] col = Bytes.toBytes(Character.toString(c));
+ assertFalse("Failed with character " + c,
+ filter.filterColumn(r, col, GOOD_BYTES));
+
+ // Do same but with bad bytes.
+ assertTrue("Failed with character " + c,
+ filter.filterColumn(r, col, "badbytes".getBytes(HConstants.UTF8_ENCODING)));
+
+ // Do with good bytes but bad column name. Should not filter out.
+ assertFalse("Failed with character " + c,
+ filter.filterColumn(r, Bytes.toBytes("badcolumn"), GOOD_BYTES));
+
+ // Good column, good bytes but bad row.
+ assertTrue("Failed with character " + c,
+ filter.filterColumn(Bytes.toBytes("bad row"),
+ Bytes.toBytes("badcolumn"), GOOD_BYTES));
+ }
+
+ private void filterNotNullTests(RowFilterInterface filter) throws Exception {
+ // Modify the filter to expect certain columns to be null:
+ // Expecting a row WITH columnKeys: a-d, WITHOUT columnKey: e
+ ((RegExpRowFilter)filter).setColumnFilter(new byte [] {LAST_CHAR}, null);
+
+ char secondToLast = (char)(LAST_CHAR - 1);
+ char thirdToLast = (char)(LAST_CHAR - 2);
+
+ // Modify the row to be missing an expected columnKey (d)
+ colvalues.remove(new byte [] {(byte)secondToLast});
+
+ // Try a row that is missing an expected columnKey.
+ // Testing row with columnKeys: a-c
+ assertTrue("Failed with last columnKey " + thirdToLast, filter.
+ filterRow(colvalues));
+
+ // Try a row that has all expected columnKeys, and NO null-expected
+ // columnKeys.
+ // Testing row with columnKeys: a-d
+ colvalues.put(new byte [] {(byte)secondToLast}, GOOD_BYTES);
+ assertFalse("Failed with last columnKey " + secondToLast, filter.
+ filterRow(colvalues));
+
+ // Try a row that has all expected columnKeys AND a null-expected columnKey.
+ // Testing row with columnKeys: a-e
+ colvalues.put(new byte [] {LAST_CHAR}, GOOD_BYTES);
+ assertTrue("Failed with last columnKey " + LAST_CHAR, filter.
+ filterRow(colvalues));
+
+ // Try a row that has all expected columnKeys and a null-expected columnKey
+ // that maps to a null value.
+ // Testing row with columnKeys: a-e, e maps to null
+ colvalues.put(new byte [] {LAST_CHAR},
+ HLogEdit.deleteBytes.get());
+ assertFalse("Failed with last columnKey " + LAST_CHAR + " mapping to null.",
+ filter.filterRow(colvalues));
+ }
+
+ private byte [] createRow(final char c) {
+ return Bytes.toBytes(HOST_PREFIX + Character.toString(c));
+ }
+}
Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRowFilterSet.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRowFilterSet.java?rev=659249&r1=659248&r2=659249&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRowFilterSet.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRowFilterSet.java Thu May 22 13:49:25 2008
@@ -1,187 +1,187 @@
-/**
- * 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.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.UnsupportedEncodingException;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.TreeMap;
-
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.util.Bytes;
-
-
-import junit.framework.TestCase;
-
-/**
- * Tests filter sets
- */
-public class TestRowFilterSet extends TestCase {
-
- RowFilterInterface filterMPALL;
- RowFilterInterface filterMPONE;
- static final int MAX_PAGES = 5;
- final char FIRST_CHAR = 'a';
- final char LAST_CHAR = 'e';
- TreeMap<byte [], byte[]> colvalues;
- static byte[] GOOD_BYTES = null;
- static byte[] BAD_BYTES = null;
-
- static {
- try {
- GOOD_BYTES = "abc".getBytes(HConstants.UTF8_ENCODING);
- BAD_BYTES = "def".getBytes(HConstants.UTF8_ENCODING);
- } catch (UnsupportedEncodingException e) {
- fail();
- }
- }
-
- /** {@inheritDoc} */
- @Override
- protected void setUp() throws Exception {
- super.setUp();
-
- colvalues = new TreeMap<byte [], byte[]>(Bytes.BYTES_COMPARATOR);
- for (char c = FIRST_CHAR; c < LAST_CHAR; c++) {
- colvalues.put(new byte [] {(byte)c}, GOOD_BYTES);
- }
-
- Set<RowFilterInterface> filters = new HashSet<RowFilterInterface>();
- filters.add(new PageRowFilter(MAX_PAGES));
- filters.add(new RegExpRowFilter(".*regex.*", colvalues));
- filters.add(new WhileMatchRowFilter(new StopRowFilter(Bytes.toBytes("yyy"))));
- filters.add(new WhileMatchRowFilter(new RegExpRowFilter(".*match.*")));
- filterMPALL = new RowFilterSet(RowFilterSet.Operator.MUST_PASS_ALL,
- filters);
- filterMPONE = new RowFilterSet(RowFilterSet.Operator.MUST_PASS_ONE,
- filters);
- }
-
- /**
- * Test "must pass one"
- * @throws Exception
- */
- public void testMPONE() throws Exception {
- MPONETests(filterMPONE);
- }
-
- /**
- * Test "must pass all"
- * @throws Exception
- */
- public void testMPALL() throws Exception {
- MPALLTests(filterMPALL);
- }
-
- /**
- * Test serialization
- * @throws Exception
- */
- public void testSerialization() throws Exception {
- // Decompose filterMPALL to bytes.
- ByteArrayOutputStream stream = new ByteArrayOutputStream();
- DataOutputStream out = new DataOutputStream(stream);
- filterMPALL.write(out);
- out.close();
- byte[] buffer = stream.toByteArray();
-
- // Recompose filterMPALL.
- DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
- RowFilterInterface newFilter = new RowFilterSet();
- newFilter.readFields(in);
-
- // Ensure the serialization preserved the filter by running a full test.
- MPALLTests(newFilter);
- }
-
- private void MPONETests(RowFilterInterface filter) throws Exception {
- // A row that shouldn't cause any filters to return true.
- RFSAssertion(filter, "regex_match", false);
-
- // A row that should cause the WhileMatchRowFilter to filter all remaining.
- RFSAssertion(filter, "regex_only", false);
-
- // Make sure the overall filterAllRemaining is unchanged (correct for
- // MUST_PASS_ONE).
- assertFalse(filter.filterAllRemaining());
-
- // A row that should cause the RegExpRowFilter to fail and the
- // StopRowFilter to filter all remaining.
- RFSAssertion(filter, "yyy_match", false);
-
- // Accept several more rows such that PageRowFilter will exceed its limit.
- for (int i=0; i<=MAX_PAGES-3; i++)
- filter.rowProcessed(false, Bytes.toBytes("unimportant_key"));
-
- // A row that should cause the RegExpRowFilter to filter this row, making
- // all the filters return true and thus the RowFilterSet as well.
- RFSAssertion(filter, "bad_column", true);
-
- // Make sure the overall filterAllRemaining is unchanged (correct for
- // MUST_PASS_ONE).
- assertFalse(filter.filterAllRemaining());
- }
-
- private void MPALLTests(RowFilterInterface filter) throws Exception {
- // A row that shouldn't cause any filters to return true.
- RFSAssertion(filter, "regex_match", false);
-
- // A row that should cause WhileMatchRowFilter to filter all remaining.
- RFSAssertion(filter, "regex_only", true);
-
- // Make sure the overall filterAllRemaining is changed (correct for
- // MUST_PASS_ALL).
- RFSAssertReset(filter);
-
- // A row that should cause the RegExpRowFilter to fail and the
- // StopRowFilter to filter all remaining.
- RFSAssertion(filter, "yyy_match", true);
-
- // Make sure the overall filterAllRemaining is changed (correct for
- // MUST_PASS_ALL).
- RFSAssertReset(filter);
-
- // A row that should cause the RegExpRowFilter to fail.
- boolean filtered = filter.filterColumn(Bytes.toBytes("regex_match"),
- new byte [] { FIRST_CHAR }, BAD_BYTES);
- assertTrue("Filtering on 'regex_match' and bad column data.", filtered);
- filterMPALL.rowProcessed(filtered, Bytes.toBytes("regex_match"));
- }
-
- private void RFSAssertion(RowFilterInterface filter, String toTest,
- boolean assertTrue) throws Exception {
- byte [] testText = Bytes.toBytes(toTest);
- boolean filtered = filter.filterRowKey(testText);
- assertTrue("Filtering on '" + toTest + "'",
- assertTrue? filtered : !filtered);
- filter.rowProcessed(filtered, testText);
- }
-
- private void RFSAssertReset(RowFilterInterface filter) throws Exception{
- assertTrue(filter.filterAllRemaining());
- // Reset for continued testing
- filter.reset();
- assertFalse(filter.filterAllRemaining());
- }
-}
+/**
+ * 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.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.UnsupportedEncodingException;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.TreeMap;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.util.Bytes;
+
+
+import junit.framework.TestCase;
+
+/**
+ * Tests filter sets
+ */
+public class TestRowFilterSet extends TestCase {
+
+ RowFilterInterface filterMPALL;
+ RowFilterInterface filterMPONE;
+ static final int MAX_PAGES = 5;
+ final char FIRST_CHAR = 'a';
+ final char LAST_CHAR = 'e';
+ TreeMap<byte [], byte[]> colvalues;
+ static byte[] GOOD_BYTES = null;
+ static byte[] BAD_BYTES = null;
+
+ static {
+ try {
+ GOOD_BYTES = "abc".getBytes(HConstants.UTF8_ENCODING);
+ BAD_BYTES = "def".getBytes(HConstants.UTF8_ENCODING);
+ } catch (UnsupportedEncodingException e) {
+ fail();
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override
+ protected void setUp() throws Exception {
+ super.setUp();
+
+ colvalues = new TreeMap<byte [], byte[]>(Bytes.BYTES_COMPARATOR);
+ for (char c = FIRST_CHAR; c < LAST_CHAR; c++) {
+ colvalues.put(new byte [] {(byte)c}, GOOD_BYTES);
+ }
+
+ Set<RowFilterInterface> filters = new HashSet<RowFilterInterface>();
+ filters.add(new PageRowFilter(MAX_PAGES));
+ filters.add(new RegExpRowFilter(".*regex.*", colvalues));
+ filters.add(new WhileMatchRowFilter(new StopRowFilter(Bytes.toBytes("yyy"))));
+ filters.add(new WhileMatchRowFilter(new RegExpRowFilter(".*match.*")));
+ filterMPALL = new RowFilterSet(RowFilterSet.Operator.MUST_PASS_ALL,
+ filters);
+ filterMPONE = new RowFilterSet(RowFilterSet.Operator.MUST_PASS_ONE,
+ filters);
+ }
+
+ /**
+ * Test "must pass one"
+ * @throws Exception
+ */
+ public void testMPONE() throws Exception {
+ MPONETests(filterMPONE);
+ }
+
+ /**
+ * Test "must pass all"
+ * @throws Exception
+ */
+ public void testMPALL() throws Exception {
+ MPALLTests(filterMPALL);
+ }
+
+ /**
+ * Test serialization
+ * @throws Exception
+ */
+ public void testSerialization() throws Exception {
+ // Decompose filterMPALL to bytes.
+ ByteArrayOutputStream stream = new ByteArrayOutputStream();
+ DataOutputStream out = new DataOutputStream(stream);
+ filterMPALL.write(out);
+ out.close();
+ byte[] buffer = stream.toByteArray();
+
+ // Recompose filterMPALL.
+ DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
+ RowFilterInterface newFilter = new RowFilterSet();
+ newFilter.readFields(in);
+
+ // Ensure the serialization preserved the filter by running a full test.
+ MPALLTests(newFilter);
+ }
+
+ private void MPONETests(RowFilterInterface filter) throws Exception {
+ // A row that shouldn't cause any filters to return true.
+ RFSAssertion(filter, "regex_match", false);
+
+ // A row that should cause the WhileMatchRowFilter to filter all remaining.
+ RFSAssertion(filter, "regex_only", false);
+
+ // Make sure the overall filterAllRemaining is unchanged (correct for
+ // MUST_PASS_ONE).
+ assertFalse(filter.filterAllRemaining());
+
+ // A row that should cause the RegExpRowFilter to fail and the
+ // StopRowFilter to filter all remaining.
+ RFSAssertion(filter, "yyy_match", false);
+
+ // Accept several more rows such that PageRowFilter will exceed its limit.
+ for (int i=0; i<=MAX_PAGES-3; i++)
+ filter.rowProcessed(false, Bytes.toBytes("unimportant_key"));
+
+ // A row that should cause the RegExpRowFilter to filter this row, making
+ // all the filters return true and thus the RowFilterSet as well.
+ RFSAssertion(filter, "bad_column", true);
+
+ // Make sure the overall filterAllRemaining is unchanged (correct for
+ // MUST_PASS_ONE).
+ assertFalse(filter.filterAllRemaining());
+ }
+
+ private void MPALLTests(RowFilterInterface filter) throws Exception {
+ // A row that shouldn't cause any filters to return true.
+ RFSAssertion(filter, "regex_match", false);
+
+ // A row that should cause WhileMatchRowFilter to filter all remaining.
+ RFSAssertion(filter, "regex_only", true);
+
+ // Make sure the overall filterAllRemaining is changed (correct for
+ // MUST_PASS_ALL).
+ RFSAssertReset(filter);
+
+ // A row that should cause the RegExpRowFilter to fail and the
+ // StopRowFilter to filter all remaining.
+ RFSAssertion(filter, "yyy_match", true);
+
+ // Make sure the overall filterAllRemaining is changed (correct for
+ // MUST_PASS_ALL).
+ RFSAssertReset(filter);
+
+ // A row that should cause the RegExpRowFilter to fail.
+ boolean filtered = filter.filterColumn(Bytes.toBytes("regex_match"),
+ new byte [] { FIRST_CHAR }, BAD_BYTES);
+ assertTrue("Filtering on 'regex_match' and bad column data.", filtered);
+ filterMPALL.rowProcessed(filtered, Bytes.toBytes("regex_match"));
+ }
+
+ private void RFSAssertion(RowFilterInterface filter, String toTest,
+ boolean assertTrue) throws Exception {
+ byte [] testText = Bytes.toBytes(toTest);
+ boolean filtered = filter.filterRowKey(testText);
+ assertTrue("Filtering on '" + toTest + "'",
+ assertTrue? filtered : !filtered);
+ filter.rowProcessed(filtered, testText);
+ }
+
+ private void RFSAssertReset(RowFilterInterface filter) throws Exception{
+ assertTrue(filter.filterAllRemaining());
+ // Reset for continued testing
+ filter.reset();
+ assertFalse(filter.filterAllRemaining());
+ }
+}
Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java?rev=659249&r1=659248&r2=659249&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java Thu May 22 13:49:25 2008
@@ -1,93 +1,93 @@
-/**
- * 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.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-
-import org.apache.hadoop.hbase.util.Bytes;
-
-import junit.framework.TestCase;
-
-/**
- * Tests the stop row filter
- */
-public class TestStopRowFilter extends TestCase {
- private final byte [] STOP_ROW = Bytes.toBytes("stop_row");
- private final byte [] GOOD_ROW = Bytes.toBytes("good_row");
- private final byte [] PAST_STOP_ROW = Bytes.toBytes("zzzzzz");
-
- RowFilterInterface mainFilter;
-
- /** {@inheritDoc} */
- @Override
- protected void setUp() throws Exception {
- super.setUp();
- mainFilter = new StopRowFilter(STOP_ROW);
- }
-
- /**
- * Tests identification of the stop row
- * @throws Exception
- */
- public void testStopRowIdentification() throws Exception {
- stopRowTests(mainFilter);
- }
-
- /**
- * Tests serialization
- * @throws Exception
- */
- public void testSerialization() throws Exception {
- // Decompose mainFilter to bytes.
- ByteArrayOutputStream stream = new ByteArrayOutputStream();
- DataOutputStream out = new DataOutputStream(stream);
- mainFilter.write(out);
- out.close();
- byte[] buffer = stream.toByteArray();
-
- // Recompose mainFilter.
- DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
- RowFilterInterface newFilter = new StopRowFilter();
- newFilter.readFields(in);
-
- // Ensure the serialization preserved the filter by running a full test.
- stopRowTests(newFilter);
- }
-
- private void stopRowTests(RowFilterInterface filter) throws Exception {
- assertFalse("Filtering on " + GOOD_ROW, filter.filterRowKey(GOOD_ROW));
- assertTrue("Filtering on " + STOP_ROW, filter.filterRowKey(STOP_ROW));
- assertTrue("Filtering on " + PAST_STOP_ROW, filter.filterRowKey(PAST_STOP_ROW));
-
- assertFalse("Filtering on " + GOOD_ROW, filter.filterColumn(GOOD_ROW, null,
- null));
- assertTrue("Filtering on " + STOP_ROW, filter.filterColumn(STOP_ROW, null, null));
- assertTrue("Filtering on " + PAST_STOP_ROW, filter.filterColumn(PAST_STOP_ROW,
- null, null));
-
- assertFalse("FilterAllRemaining", filter.filterAllRemaining());
- assertFalse("FilterNotNull", filter.filterRow(null));
-
- assertFalse("Filter a null", filter.filterRowKey(null));
- }
-}
+/**
+ * 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.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+import junit.framework.TestCase;
+
+/**
+ * Tests the stop row filter
+ */
+public class TestStopRowFilter extends TestCase {
+ private final byte [] STOP_ROW = Bytes.toBytes("stop_row");
+ private final byte [] GOOD_ROW = Bytes.toBytes("good_row");
+ private final byte [] PAST_STOP_ROW = Bytes.toBytes("zzzzzz");
+
+ RowFilterInterface mainFilter;
+
+ /** {@inheritDoc} */
+ @Override
+ protected void setUp() throws Exception {
+ super.setUp();
+ mainFilter = new StopRowFilter(STOP_ROW);
+ }
+
+ /**
+ * Tests identification of the stop row
+ * @throws Exception
+ */
+ public void testStopRowIdentification() throws Exception {
+ stopRowTests(mainFilter);
+ }
+
+ /**
+ * Tests serialization
+ * @throws Exception
+ */
+ public void testSerialization() throws Exception {
+ // Decompose mainFilter to bytes.
+ ByteArrayOutputStream stream = new ByteArrayOutputStream();
+ DataOutputStream out = new DataOutputStream(stream);
+ mainFilter.write(out);
+ out.close();
+ byte[] buffer = stream.toByteArray();
+
+ // Recompose mainFilter.
+ DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
+ RowFilterInterface newFilter = new StopRowFilter();
+ newFilter.readFields(in);
+
+ // Ensure the serialization preserved the filter by running a full test.
+ stopRowTests(newFilter);
+ }
+
+ private void stopRowTests(RowFilterInterface filter) throws Exception {
+ assertFalse("Filtering on " + GOOD_ROW, filter.filterRowKey(GOOD_ROW));
+ assertTrue("Filtering on " + STOP_ROW, filter.filterRowKey(STOP_ROW));
+ assertTrue("Filtering on " + PAST_STOP_ROW, filter.filterRowKey(PAST_STOP_ROW));
+
+ assertFalse("Filtering on " + GOOD_ROW, filter.filterColumn(GOOD_ROW, null,
+ null));
+ assertTrue("Filtering on " + STOP_ROW, filter.filterColumn(STOP_ROW, null, null));
+ assertTrue("Filtering on " + PAST_STOP_ROW, filter.filterColumn(PAST_STOP_ROW,
+ null, null));
+
+ assertFalse("FilterAllRemaining", filter.filterAllRemaining());
+ assertFalse("FilterNotNull", filter.filterRow(null));
+
+ assertFalse("Filter a null", filter.filterRowKey(null));
+ }
+}
Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java?rev=659249&r1=659248&r2=659249&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java Thu May 22 13:49:25 2008
@@ -1,150 +1,150 @@
-/**
- * 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.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-
-import org.apache.hadoop.hbase.util.Bytes;
-
-import junit.framework.TestCase;
-
-
-/**
- * Tests for the while-match filter
- */
-public class TestWhileMatchRowFilter extends TestCase {
-
- WhileMatchRowFilter wmStopRowFilter;
- WhileMatchRowFilter wmRegExpRowFilter;
-
- /** {@inheritDoc} */
- @Override
- protected void setUp() throws Exception {
- super.setUp();
- wmStopRowFilter = new WhileMatchRowFilter(new StopRowFilter(
- Bytes.toBytes("s")));
- wmRegExpRowFilter = new WhileMatchRowFilter(new RegExpRowFilter(
- ".*regex.*"));
- }
-
- /**
- * Tests while match stop row
- * @throws Exception
- */
- public void testWhileMatchStopRow() throws Exception {
- whileMatchStopRowTests(wmStopRowFilter);
- }
-
- /**
- * Tests while match regex
- * @throws Exception
- */
- public void testWhileMatchRegExp() throws Exception {
- whileMatchRegExpTests(wmRegExpRowFilter);
- }
-
- /**
- * Tests serialization
- * @throws Exception
- */
- public void testSerialization() throws Exception {
- // Decompose wmRegExpRowFilter to bytes.
- ByteArrayOutputStream stream = new ByteArrayOutputStream();
- DataOutputStream out = new DataOutputStream(stream);
- wmRegExpRowFilter.write(out);
- out.close();
- byte[] buffer = stream.toByteArray();
-
- // Recompose wmRegExpRowFilter.
- DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
- WhileMatchRowFilter newFilter = new WhileMatchRowFilter();
- newFilter.readFields(in);
-
- // Ensure the serialization preserved the filter by running a full test.
- whileMatchRegExpTests(newFilter);
- }
-
- private void whileMatchStopRowTests(WhileMatchRowFilter filter) throws
- Exception {
- RowFilterInterface innerFilter = filter.getInternalFilter();
- String toTest;
-
- // Test cases that should pass the row
- toTest = "apples";
- assertFalse("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest)));
- assertFalse("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes(
- toTest)));
-
- // Test cases that should fail the row
- toTest = "tuna";
- assertTrue("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest)));
- assertTrue("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes(
- toTest)));
-
- // The difference in switch
- assertTrue("filter: filterAllRemaining", filter.filterAllRemaining());
- assertFalse("innerFilter: filterAllRemaining pre-reset",
- innerFilter.filterAllRemaining());
-
- // Test resetting
- filter.reset();
- assertFalse("filter: filterAllRemaining post-reset",
- filter.filterAllRemaining());
-
- // Test filterNotNull for functionality only (no switch-cases)
- assertFalse("filter: filterNotNull", filter.filterRow(null));
- }
-
- private void whileMatchRegExpTests(WhileMatchRowFilter filter) throws
- Exception {
- RowFilterInterface innerFilter = filter.getInternalFilter();
- String toTest;
-
- // Test cases that should pass the row
- toTest = "regex_match";
- assertFalse("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest)));
- assertFalse("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes(
- toTest)));
-
- // Test cases that should fail the row
- toTest = "not_a_match";
- assertTrue("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest)));
- assertTrue("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes(
- toTest)));
-
- // The difference in switch
- assertTrue("filter: filterAllRemaining", filter.filterAllRemaining());
- assertFalse("innerFilter: filterAllRemaining pre-reset",
- innerFilter.filterAllRemaining());
-
- // Test resetting
- filter.reset();
- assertFalse("filter: filterAllRemaining post-reset",
- filter.filterAllRemaining());
-
- // Test filter(Text, Text, byte[]) for functionality only (no switch-cases)
- toTest = "asdf_regex_hjkl";
- assertFalse("filter: '" + toTest + "'", filter.filterColumn(Bytes.toBytes(toTest),
- null, null));
- }
-}
+/**
+ * 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.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+import junit.framework.TestCase;
+
+
+/**
+ * Tests for the while-match filter
+ */
+public class TestWhileMatchRowFilter extends TestCase {
+
+ WhileMatchRowFilter wmStopRowFilter;
+ WhileMatchRowFilter wmRegExpRowFilter;
+
+ /** {@inheritDoc} */
+ @Override
+ protected void setUp() throws Exception {
+ super.setUp();
+ wmStopRowFilter = new WhileMatchRowFilter(new StopRowFilter(
+ Bytes.toBytes("s")));
+ wmRegExpRowFilter = new WhileMatchRowFilter(new RegExpRowFilter(
+ ".*regex.*"));
+ }
+
+ /**
+ * Tests while match stop row
+ * @throws Exception
+ */
+ public void testWhileMatchStopRow() throws Exception {
+ whileMatchStopRowTests(wmStopRowFilter);
+ }
+
+ /**
+ * Tests while match regex
+ * @throws Exception
+ */
+ public void testWhileMatchRegExp() throws Exception {
+ whileMatchRegExpTests(wmRegExpRowFilter);
+ }
+
+ /**
+ * Tests serialization
+ * @throws Exception
+ */
+ public void testSerialization() throws Exception {
+ // Decompose wmRegExpRowFilter to bytes.
+ ByteArrayOutputStream stream = new ByteArrayOutputStream();
+ DataOutputStream out = new DataOutputStream(stream);
+ wmRegExpRowFilter.write(out);
+ out.close();
+ byte[] buffer = stream.toByteArray();
+
+ // Recompose wmRegExpRowFilter.
+ DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
+ WhileMatchRowFilter newFilter = new WhileMatchRowFilter();
+ newFilter.readFields(in);
+
+ // Ensure the serialization preserved the filter by running a full test.
+ whileMatchRegExpTests(newFilter);
+ }
+
+ private void whileMatchStopRowTests(WhileMatchRowFilter filter) throws
+ Exception {
+ RowFilterInterface innerFilter = filter.getInternalFilter();
+ String toTest;
+
+ // Test cases that should pass the row
+ toTest = "apples";
+ assertFalse("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest)));
+ assertFalse("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes(
+ toTest)));
+
+ // Test cases that should fail the row
+ toTest = "tuna";
+ assertTrue("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest)));
+ assertTrue("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes(
+ toTest)));
+
+ // The difference in switch
+ assertTrue("filter: filterAllRemaining", filter.filterAllRemaining());
+ assertFalse("innerFilter: filterAllRemaining pre-reset",
+ innerFilter.filterAllRemaining());
+
+ // Test resetting
+ filter.reset();
+ assertFalse("filter: filterAllRemaining post-reset",
+ filter.filterAllRemaining());
+
+ // Test filterNotNull for functionality only (no switch-cases)
+ assertFalse("filter: filterNotNull", filter.filterRow(null));
+ }
+
+ private void whileMatchRegExpTests(WhileMatchRowFilter filter) throws
+ Exception {
+ RowFilterInterface innerFilter = filter.getInternalFilter();
+ String toTest;
+
+ // Test cases that should pass the row
+ toTest = "regex_match";
+ assertFalse("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest)));
+ assertFalse("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes(
+ toTest)));
+
+ // Test cases that should fail the row
+ toTest = "not_a_match";
+ assertTrue("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest)));
+ assertTrue("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes(
+ toTest)));
+
+ // The difference in switch
+ assertTrue("filter: filterAllRemaining", filter.filterAllRemaining());
+ assertFalse("innerFilter: filterAllRemaining pre-reset",
+ innerFilter.filterAllRemaining());
+
+ // Test resetting
+ filter.reset();
+ assertFalse("filter: filterAllRemaining post-reset",
+ filter.filterAllRemaining());
+
+ // Test filter(Text, Text, byte[]) for functionality only (no switch-cases)
+ toTest = "asdf_regex_hjkl";
+ assertFalse("filter: '" + toTest + "'", filter.filterColumn(Bytes.toBytes(toTest),
+ null, null));
+ }
+}