From 3b8bed11a6eddcbe1ea6af7007286b5bf9ba3185 Mon Sep 17 00:00:00 2001 From: Jonathan Gray Date: Mon, 5 Oct 2009 22:34:16 +0000 Subject: [PATCH] HBASE-1481 Add fast row key only scanning git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@822062 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 1 + bin/HBase.rb | 2 + .../hbase/filter/FirstKeyOnlyFilter.java | 66 +++++++++++++++++++ .../hadoop/hbase/io/HbaseObjectWritable.java | 1 + .../hadoop/hbase/mapreduce/RowCounter.java | 2 + .../hadoop/hbase/filter/TestFilter.java | 15 +++++ 6 files changed, 87 insertions(+) create mode 100644 src/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java diff --git a/CHANGES.txt b/CHANGES.txt index d1b5e4028d7..1481f193a68 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -97,6 +97,7 @@ Release 0.21.0 - Unreleased HBASE-1879 ReadOnly transactions generate WAL activity (Clint Morgan via Stack) HBASE-1875 Compression test utility HBASE-1832 Faster enable/disable/delete + HBASE-1481 Add fast row key only scanning OPTIMIZATIONS diff --git a/bin/HBase.rb b/bin/HBase.rb index 576d7a5dbb2..36852d771b9 100644 --- a/bin/HBase.rb +++ b/bin/HBase.rb @@ -18,6 +18,7 @@ import org.apache.hadoop.hbase.client.Get import org.apache.hadoop.hbase.client.Put import org.apache.hadoop.hbase.client.Scan import org.apache.hadoop.hbase.client.Delete +import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter import org.apache.hadoop.hbase.HConstants import org.apache.hadoop.hbase.io.hfile.Compression import org.apache.hadoop.hbase.HBaseConfiguration @@ -544,6 +545,7 @@ module HBase now = Time.now scan = Scan.new() scan.setCacheBlocks(false) + scan.setFilter(FirstKeyOnlyFilter.new()) s = @table.getScanner(scan) count = 0 i = s.iterator() diff --git a/src/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java b/src/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java new file mode 100644 index 00000000000..cc2402ea7ba --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java @@ -0,0 +1,66 @@ +/* + * Copyright 2009 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 org.apache.hadoop.hbase.KeyValue; + +import java.io.DataOutput; +import java.io.IOException; +import java.io.DataInput; + +/** + * A filter that will only return the first KV from each row. + *

+ * This filter can be used to more efficiently perform row count operations. + */ +public class FirstKeyOnlyFilter implements Filter { + private boolean foundKV = false; + + public FirstKeyOnlyFilter() { + } + + public void reset() { + foundKV = false; + } + + public boolean filterRowKey(byte[] buffer, int offset, int length) { + return false; + } + + public boolean filterAllRemaining() { + return false; + } + + public ReturnCode filterKeyValue(KeyValue v) { + if(foundKV) return ReturnCode.NEXT_ROW; + foundKV = true; + return ReturnCode.INCLUDE; + } + + public boolean filterRow() { + return false; + } + + public void write(DataOutput out) throws IOException { + } + + public void readFields(DataInput in) throws IOException { + } +} diff --git a/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java b/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java index 7405f83d538..d2e20f34381 100644 --- a/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java +++ b/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java @@ -149,6 +149,7 @@ public class HbaseObjectWritable implements Writable, Configurable { addToMap(QualifierFilter.class, code++); addToMap(SkipFilter.class, code++); addToMap(WritableByteArrayComparable.class, code++); + addToMap(FirstKeyOnlyFilter.class, code++); addToMap(Delete [].class, code++); } diff --git a/src/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java b/src/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java index 40e912b1688..90ffee36b84 100644 --- a/src/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java +++ b/src/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java @@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.mapreduce.Job; @@ -96,6 +97,7 @@ public class RowCounter { sb.append(args[i]); } Scan scan = new Scan(); + scan.setFilter(new FirstKeyOnlyFilter()); if (sb.length() > 0) { for (String columnName :sb.toString().split(" ")) { String [] fields = columnName.split(":"); diff --git a/src/test/org/apache/hadoop/hbase/filter/TestFilter.java b/src/test/org/apache/hadoop/hbase/filter/TestFilter.java index 983e32f4edf..aa8995ec6fb 100644 --- a/src/test/org/apache/hadoop/hbase/filter/TestFilter.java +++ b/src/test/org/apache/hadoop/hbase/filter/TestFilter.java @@ -789,6 +789,21 @@ public class TestFilter extends HBaseTestCase { } + public void testFirstKeyOnlyFilter() throws IOException { + Scan s = new Scan(); + s.setFilter(new FirstKeyOnlyFilter()); + // Expected KVs, the first KV from each of the remaining 6 rows + KeyValue [] kvs = { + new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), + new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), + new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]), + new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), + new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), + new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]) + }; + verifyScanFull(s, kvs); + } + public void testSingleColumnValueFilter() throws IOException { // From HBASE-1821