From 0d7a01cb8a1c15a8bfbb5814d2ae2c75eb8cf68f Mon Sep 17 00:00:00 2001 From: Michael Stack Date: Tue, 4 Dec 2007 17:07:22 +0000 Subject: [PATCH] HADOOP-2299 Support inclusive scans git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@601005 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 1 + .../hbase/filter/InclusiveStopRowFilter.java | 48 ++++++++++ .../hadoop/hbase/filter/StopRowFilter.java | 2 +- .../filter/TestInclusiveStopRowFilter.java | 93 +++++++++++++++++++ 4 files changed, 143 insertions(+), 1 deletion(-) create mode 100644 src/java/org/apache/hadoop/hbase/filter/InclusiveStopRowFilter.java create mode 100644 src/test/org/apache/hadoop/hbase/filter/TestInclusiveStopRowFilter.java diff --git a/CHANGES.txt b/CHANGES.txt index 3f0a1a4ac59..331d67fa5da 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -87,6 +87,7 @@ Trunk (unreleased changes) (Bryan Duxbury via Stack) HADOOP-2339 Delete command with no WHERE clause (Edward Yoon via Stack) + HADOOP-2299 Support inclusive scans (Bryan Duxbury via Stack) Release 0.15.1 diff --git a/src/java/org/apache/hadoop/hbase/filter/InclusiveStopRowFilter.java b/src/java/org/apache/hadoop/hbase/filter/InclusiveStopRowFilter.java new file mode 100644 index 00000000000..66fb2387780 --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/filter/InclusiveStopRowFilter.java @@ -0,0 +1,48 @@ +package org.apache.hadoop.hbase.filter; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.TreeMap; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.io.Text; + + +/* + * Subclass of StopRowFilter that filters rows > the stop row, + * making it include up to the last row but no further. + */ +public class InclusiveStopRowFilter extends StopRowFilter{ + /** + * Default constructor, filters nothing. Required though for RPC + * deserialization. + */ + public InclusiveStopRowFilter() {super();} + + /** + * Constructor that takes a stopRowKey on which to filter + * + * @param stopRowKey rowKey to filter on. + */ + public InclusiveStopRowFilter(final Text stopRowKey) { + super(stopRowKey); + } + + public boolean filter(final Text rowKey) { + if (rowKey == null) { + if (this.stopRowKey == null) { + return true; + } + return false; + } + boolean result = this.stopRowKey.compareTo(rowKey) < 0; + if (LOG.isDebugEnabled()) { + LOG.debug("Filter result for rowKey: " + rowKey + ". Result: " + + result); + } + return result; + } + +} \ No newline at end of file diff --git a/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java b/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java index be638140097..b5d8bebedf5 100644 --- a/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java +++ b/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java @@ -34,7 +34,7 @@ import org.apache.hadoop.io.Text; */ public class StopRowFilter implements RowFilterInterface { - private Text stopRowKey; + protected Text stopRowKey; static final Log LOG = LogFactory.getLog(StopRowFilter.class); diff --git a/src/test/org/apache/hadoop/hbase/filter/TestInclusiveStopRowFilter.java b/src/test/org/apache/hadoop/hbase/filter/TestInclusiveStopRowFilter.java new file mode 100644 index 00000000000..9eaf29f5cbe --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/filter/TestInclusiveStopRowFilter.java @@ -0,0 +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.io.Text; + +import junit.framework.TestCase; + +/** + * Tests the inclusive stop row filter + */ +public class TestInclusiveStopRowFilter extends TestCase { + private final Text STOP_ROW = new Text("stop_row"); + private final Text GOOD_ROW = new Text("good_row"); + private final Text PAST_STOP_ROW = new Text("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.filter(GOOD_ROW)); + assertFalse("Filtering on " + STOP_ROW, filter.filter(STOP_ROW)); + assertTrue("Filtering on " + PAST_STOP_ROW, filter.filter(PAST_STOP_ROW)); + + assertFalse("Filtering on " + GOOD_ROW, filter.filter(GOOD_ROW, null, + null)); + assertFalse("Filtering on " + STOP_ROW, filter.filter(STOP_ROW, null, null)); + assertTrue("Filtering on " + PAST_STOP_ROW, filter.filter(PAST_STOP_ROW, + null, null)); + + assertFalse("FilterAllRemaining", filter.filterAllRemaining()); + assertFalse("FilterNotNull", filter.filterNotNull(null)); + + assertFalse("Filter a null", filter.filter(null)); + } +}