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
This commit is contained in:
parent
4fdfcaaf9f
commit
3b8bed11a6
|
@ -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
|
||||
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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.
|
||||
* <p>
|
||||
* 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 {
|
||||
}
|
||||
}
|
|
@ -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++);
|
||||
}
|
||||
|
|
|
@ -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(":");
|
||||
|
|
|
@ -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
|
||||
|
|
Loading…
Reference in New Issue