You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jg...@apache.org on 2009/10/22 20:53:33 UTC

svn commit: r828815 - in /hadoop/hbase/branches/0.20: CHANGES.txt src/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java src/test/org/apache/hadoop/hbase/regionserver/TestKeyValueHeap.java

Author: jgray
Date: Thu Oct 22 18:53:33 2009
New Revision: 828815

URL: http://svn.apache.org/viewvc?rev=828815&view=rev
Log:
HBASE-1927  Scanners not closed properly in certain circumstances

Modified:
    hadoop/hbase/branches/0.20/CHANGES.txt
    hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java
    hadoop/hbase/branches/0.20/src/test/org/apache/hadoop/hbase/regionserver/TestKeyValueHeap.java

Modified: hadoop/hbase/branches/0.20/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/CHANGES.txt?rev=828815&r1=828814&r2=828815&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/CHANGES.txt (original)
+++ hadoop/hbase/branches/0.20/CHANGES.txt Thu Oct 22 18:53:33 2009
@@ -19,6 +19,7 @@
    HBASE-1895  HConstants.MAX_ROW_LENGTH is incorrectly 64k, should be 32k
    HBASE-1925  IllegalAccessError: Has not been initialized (getMaxSequenceId)
    HBASE-1929  If hbase-default.xml is not in CP, zk session timeout is 10 secs!
+   HBASE-1927  Scanners not closed properly in certain circumstances
 
   IMPROVEMENTS
    HBASE-1899  Use scanner caching in shell count

Modified: hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java?rev=828815&r1=828814&r2=828815&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java (original)
+++ hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java Thu Oct 22 18:53:33 2009
@@ -46,7 +46,8 @@
   private KVScannerComparator comparator;
 
   /**
-   * Constructor
+   * Constructor.  This KeyValueHeap will handle closing of passed in 
+   * KeyValueScanners.
    * @param scanners
    * @param comparator
    */
@@ -57,6 +58,8 @@
     for (KeyValueScanner scanner : scanners) {
       if (scanner.peek() != null) {
         this.heap.add(scanner);
+      } else {
+        scanner.close();
       }
     }
     this.current = heap.poll();

Modified: hadoop/hbase/branches/0.20/src/test/org/apache/hadoop/hbase/regionserver/TestKeyValueHeap.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/src/test/org/apache/hadoop/hbase/regionserver/TestKeyValueHeap.java?rev=828815&r1=828814&r2=828815&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/src/test/org/apache/hadoop/hbase/regionserver/TestKeyValueHeap.java (original)
+++ hadoop/hbase/branches/0.20/src/test/org/apache/hadoop/hbase/regionserver/TestKeyValueHeap.java Thu Oct 22 18:53:33 2009
@@ -160,12 +160,49 @@
     }
     
   }
+  
+  public void testScannerLeak() {
+    // Test for unclosed scanners (HBASE-1927)
+    
+    List<KeyValue> l1 = new ArrayList<KeyValue>();
+    l1.add(new KeyValue(row1, fam1, col5, data));
+    l1.add(new KeyValue(row2, fam1, col1, data));
+    l1.add(new KeyValue(row2, fam1, col2, data));
+    scanners.add(new Scanner(l1));
+
+    List<KeyValue> l2 = new ArrayList<KeyValue>();
+    l2.add(new KeyValue(row1, fam1, col1, data));
+    l2.add(new KeyValue(row1, fam1, col2, data));
+    scanners.add(new Scanner(l2));
+
+    List<KeyValue> l3 = new ArrayList<KeyValue>();
+    l3.add(new KeyValue(row1, fam1, col3, data));
+    l3.add(new KeyValue(row1, fam1, col4, data));
+    l3.add(new KeyValue(row1, fam2, col1, data));
+    l3.add(new KeyValue(row1, fam2, col2, data));
+    l3.add(new KeyValue(row2, fam1, col3, data));
+    scanners.add(new Scanner(l3));
+    
+    List<KeyValue> l4 = new ArrayList<KeyValue>();
+    scanners.add(new Scanner(l4));
+
+    //Creating KeyValueHeap
+    KeyValueHeap kvh =
+      new KeyValueHeap(scanners.toArray(new Scanner[0]), KeyValue.COMPARATOR);
+    
+    while(kvh.next() != null);
+    
+    for(Scanner scanner : scanners) {
+      assertTrue(scanner.isClosed());
+    }
+  }
 
   private class Scanner implements KeyValueScanner {
     private Set<KeyValue> scan =
       new TreeSet<KeyValue>((Comparator)KeyValue.COMPARATOR);
     private Iterator<KeyValue> iter;
     private KeyValue current;
+    private boolean closed = false;
 
     public Scanner(List<KeyValue> list) {
       Collections.sort(list, (Comparator)KeyValue.COMPARATOR);
@@ -189,7 +226,13 @@
       return oldCurrent;
     }
 
-    public void close(){}
+    public void close(){
+      closed = true;
+    }
+    
+    public boolean isClosed() {
+      return closed;
+    }
     
     public boolean seek(KeyValue seekKv) {
       while(iter.hasNext()){