You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2011/07/28 00:04:15 UTC

svn commit: r1151646 - in /hbase/branches/0.90: CHANGES.txt src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueSkipListSet.java src/test/java/org/apache/hadoop/hbase/regionserver/MemStoreScanPerformance.java

Author: apurtell
Date: Wed Jul 27 22:04:14 2011
New Revision: 1151646

URL: http://svn.apache.org/viewvc?rev=1151646&view=rev
Log:
HBASE-1938  Make in-memory table scanning faster; only iterator and benchmark changes applied

Added:
    hbase/branches/0.90/src/test/java/org/apache/hadoop/hbase/regionserver/MemStoreScanPerformance.java
Modified:
    hbase/branches/0.90/CHANGES.txt
    hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueSkipListSet.java

Modified: hbase/branches/0.90/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/branches/0.90/CHANGES.txt?rev=1151646&r1=1151645&r2=1151646&view=diff
==============================================================================
--- hbase/branches/0.90/CHANGES.txt (original)
+++ hbase/branches/0.90/CHANGES.txt Wed Jul 27 22:04:14 2011
@@ -173,6 +173,8 @@ Release 0.90.3 - May 19th, 2011
    HBASE-3813  Change RPC callQueue size from 'handlerCount *
                MAX_QUEUE_SIZE_PER_HANDLER;'
    HBASE-3860  HLog shouldn't create a new HBC when rolling
+   HBASE-1938  Make in-memory table scanning faster (nkeywal); only iterator
+               and benchmark changes applied
 
   TASK
    HBASE-3748  Add rolling of thrift/rest daemons to graceful_stop.sh script

Modified: hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueSkipListSet.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueSkipListSet.java?rev=1151646&r1=1151645&r2=1151646&view=diff
==============================================================================
--- hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueSkipListSet.java (original)
+++ hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueSkipListSet.java Wed Jul 27 22:04:14 2011
@@ -24,7 +24,6 @@ import org.apache.hadoop.hbase.KeyValue;
 import java.util.Collection;
 import java.util.Comparator;
 import java.util.Iterator;
-import java.util.Map;
 import java.util.NavigableSet;
 import java.util.SortedSet;
 import java.util.concurrent.ConcurrentNavigableMap;
@@ -55,37 +54,12 @@ class KeyValueSkipListSet implements Nav
     this.delegatee = m;
   }
 
-  /**
-   * Iterator that maps Iterator calls to return the value component of the
-   * passed-in Map.Entry Iterator.
-   */
-  static class MapEntryIterator implements Iterator<KeyValue> {
-    private final Iterator<Map.Entry<KeyValue, KeyValue>> iterator;
-
-    MapEntryIterator(final Iterator<Map.Entry<KeyValue, KeyValue>> i) {
-      this.iterator = i;
-    }
-
-    public boolean hasNext() {
-      return this.iterator.hasNext();
-    }
-
-    public KeyValue next() {
-      return this.iterator.next().getValue();
-    }
-
-    public void remove() {
-      this.iterator.remove();
-    }
-  }
-
   public KeyValue ceiling(KeyValue e) {
     throw new UnsupportedOperationException("Not implemented");
   }
 
   public Iterator<KeyValue> descendingIterator() {
-    return new MapEntryIterator(this.delegatee.descendingMap().entrySet().
-      iterator());
+    return this.delegatee.descendingMap().values().iterator();
   }
 
   public NavigableSet<KeyValue> descendingSet() {
@@ -110,7 +84,7 @@ class KeyValueSkipListSet implements Nav
   }
 
   public Iterator<KeyValue> iterator() {
-    return new MapEntryIterator(this.delegatee.entrySet().iterator());
+    return this.delegatee.values().iterator();
   }
 
   public KeyValue lower(KeyValue e) {

Added: hbase/branches/0.90/src/test/java/org/apache/hadoop/hbase/regionserver/MemStoreScanPerformance.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90/src/test/java/org/apache/hadoop/hbase/regionserver/MemStoreScanPerformance.java?rev=1151646&view=auto
==============================================================================
--- hbase/branches/0.90/src/test/java/org/apache/hadoop/hbase/regionserver/MemStoreScanPerformance.java (added)
+++ hbase/branches/0.90/src/test/java/org/apache/hadoop/hbase/regionserver/MemStoreScanPerformance.java Wed Jul 27 22:04:14 2011
@@ -0,0 +1,119 @@
+/*
+ * Copyright 2011 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 org.apache.hadoop.hbase.KeyValue;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+
+public class MemStoreScanPerformance {
+  private MemStore m;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+  }
+
+  @Before
+  public void setUp() throws Exception {
+  }
+
+  @After
+  public void tearDown() throws Exception {
+  }
+
+  @Test
+  public void testTenColumns() {
+    ReadWriteConsistencyControl.resetThreadReadPoint();
+    long totalScanTime = 0;
+    long totalRows = 0;
+    long totalLoadTime = 0;
+
+    for (int size = 50000; size < 160000; size += 25000) {
+      m = new MemStore();
+      long start = System.currentTimeMillis();
+      for (int i = 0; i < size; i++) {
+        byte[] row = format(i);
+        for (int j = 0; j < 10; j++) {
+          byte[] column = format(j);
+          this.m.add(new KeyValue(row, column, column));
+        }
+      }
+
+      KeyValue searched = new KeyValue(format(5), format(5), format(5));
+
+      totalLoadTime += (System.currentTimeMillis() - start);
+      System.out.println("Loaded in " + (System.currentTimeMillis() - start) +
+        " ms");
+      for (int i = 0; i < 10; ++i) {
+        try {
+          start = System.currentTimeMillis();
+          KeyValueScanner scanner = this.m.getScanners().get(0);
+
+          scanner.seek(searched);
+
+          while (true) {
+            KeyValue kv = scanner.next();
+            if (kv == null) break;
+            kv = scanner.peek();
+          }
+          totalScanTime += (System.currentTimeMillis() - start);
+          totalRows += size;
+          System.out.println("Scan with size " + size + ": " +
+            (System.currentTimeMillis() - start) + " ms");
+        } catch (IOException e) {
+          throw new Error(e);
+        }
+      }
+    }
+    System.out.println("Total load time: " + totalLoadTime + " ms (i.e:" +
+      (totalLoadTime / 1000L) + " seconds)");
+    System.out.println("Total scan time: " + totalScanTime + " ms (i.e:" +
+      (totalScanTime / 1000L) + " seconds)");
+    System.out.println("Rows scanned per seconds: " +
+      ((totalRows * 1000) / totalScanTime));
+    System.out.println("Rows loaded per seconds: " +
+      ((totalRows * 1000) / totalLoadTime));
+  }
+
+  private static byte[] format(final int number) {
+    byte[] b = new byte[10];
+    int d = Math.abs(number);
+    for (int i = b.length - 1; i >= 0; i--) {
+      b[i] = (byte) ((d % 10) + '0');
+      d /= 10;
+    }
+
+    return b;
+  }
+
+  public static void main(String[] args) {
+    org.junit.runner.JUnitCore.main(MemStoreScanPerformance.class.getName());
+  }
+}