hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jg...@apache.org
Subject svn commit: r822061 - in /hadoop/hbase/branches/0.20: ./ bin/ src/java/org/apache/hadoop/hbase/filter/ src/java/org/apache/hadoop/hbase/io/ src/java/org/apache/hadoop/hbase/mapreduce/ src/test/org/apache/hadoop/hbase/filter/
Date Mon, 05 Oct 2009 22:34:09 GMT
Author: jgray
Date: Mon Oct  5 22:34:08 2009
New Revision: 822061

URL: http://svn.apache.org/viewvc?rev=822061&view=rev
Log:
HBASE-1481  Add fast row key only scanning

Added:
    hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java
Modified:
    hadoop/hbase/branches/0.20/CHANGES.txt
    hadoop/hbase/branches/0.20/bin/HBase.rb
    hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java
    hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java
    hadoop/hbase/branches/0.20/src/test/org/apache/hadoop/hbase/filter/TestFilter.java

Modified: hadoop/hbase/branches/0.20/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/CHANGES.txt?rev=822061&r1=822060&r2=822061&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/CHANGES.txt (original)
+++ hadoop/hbase/branches/0.20/CHANGES.txt Mon Oct  5 22:34:08 2009
@@ -67,6 +67,7 @@
    HBASE-1879  ReadOnly transactions generate WAL activity (Clint Morgan via Stack)
    HBASE-1875  Compression test utility (elsif via Stack)
    HBASE-1832  Faster enable/disable/delete
+   HBASE-1481  Add fast row key only scanning
 
 Release 0.20.0 - Tue Sep  8 12:48:41 PDT 2009
 

Modified: hadoop/hbase/branches/0.20/bin/HBase.rb
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/bin/HBase.rb?rev=822061&r1=822060&r2=822061&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/bin/HBase.rb (original)
+++ hadoop/hbase/branches/0.20/bin/HBase.rb Mon Oct  5 22:34:08 2009
@@ -18,6 +18,7 @@
 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.BatchUpdate
 import org.apache.hadoop.hbase.io.RowResult
@@ -548,6 +549,7 @@
       now = Time.now
       scan = Scan.new()
       scan.setCacheBlocks(false)
+      scan.setFilter(FirstKeyOnlyFilter.new())
       s = @table.getScanner(scan)
       count = 0
       i = s.iterator()

Added: hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java?rev=822061&view=auto
==============================================================================
--- hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java
(added)
+++ hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java
Mon Oct  5 22:34:08 2009
@@ -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 {
+  }
+}

Modified: hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java?rev=822061&r1=822060&r2=822061&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java
(original)
+++ hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java
Mon Oct  5 22:34:08 2009
@@ -151,6 +151,7 @@
     addToMap(QualifierFilter.class, code++);
     addToMap(SkipFilter.class, code++);
     addToMap(WritableByteArrayComparable.class, code++);
+    addToMap(FirstKeyOnlyFilter.class, code++);
 
     addToMap(Delete [].class, code++);
   }

Modified: hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java?rev=822061&r1=822060&r2=822061&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java
(original)
+++ hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java
Mon Oct  5 22:34:08 2009
@@ -26,6 +26,7 @@
 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.mapreduce.Job;
 import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
@@ -96,6 +97,7 @@
     }
     Scan scan = new Scan();
     if (sb.length() > 0) scan.addColumns(sb.toString());
+    scan.setFilter(new FirstKeyOnlyFilter());
     // Second argument is the table name.
     TableMapReduceUtil.initTableMapperJob(tableName, scan,
       RowCounterMapper.class, ImmutableBytesWritable.class, Result.class, job);

Modified: hadoop/hbase/branches/0.20/src/test/org/apache/hadoop/hbase/filter/TestFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/src/test/org/apache/hadoop/hbase/filter/TestFilter.java?rev=822061&r1=822060&r2=822061&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/src/test/org/apache/hadoop/hbase/filter/TestFilter.java (original)
+++ hadoop/hbase/branches/0.20/src/test/org/apache/hadoop/hbase/filter/TestFilter.java Mon
Oct  5 22:34:08 2009
@@ -789,6 +789,21 @@
     
   }
   
+  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



Mime
View raw message