You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by la...@apache.org on 2011/02/03 12:40:07 UTC

svn commit: r1066785 - in /hbase/trunk: CHANGES.txt src/main/java/org/apache/hadoop/hbase/regionserver/Store.java src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java

Author: larsgeorge
Date: Thu Feb  3 11:40:07 2011
New Revision: 1066785

URL: http://svn.apache.org/viewvc?rev=1066785&view=rev
Log:
HBASE-3492  NPE while splitting table with empty column family store

Modified:
    hbase/trunk/CHANGES.txt
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java

Modified: hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/trunk/CHANGES.txt?rev=1066785&r1=1066784&r2=1066785&view=diff
==============================================================================
--- hbase/trunk/CHANGES.txt (original)
+++ hbase/trunk/CHANGES.txt Thu Feb  3 11:40:07 2011
@@ -38,6 +38,7 @@ Release 0.91.0 - Unreleased
                row are the same
    HBASE-3416  For intra-row scanning, the update readers notification resets
                the query matcher and can lead to incorrect behavior
+   HBASE-3492  NPE while splitting table with empty column family store
 
 
   IMPROVEMENTS

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java?rev=1066785&r1=1066784&r2=1066785&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java Thu Feb  3 11:40:07 2011
@@ -725,17 +725,17 @@ public class Store implements HeapSize {
    * @param dir
    * @throws IOException
    */
-  private static long getLowestTimestamp(FileSystem fs, 
+  private static long getLowestTimestamp(FileSystem fs,
       final List<StoreFile> candidates) throws IOException {
     long minTs = Long.MAX_VALUE;
     if (candidates.isEmpty()) {
-      return minTs; 
+      return minTs;
     }
     Path[] p = new Path[candidates.size()];
     for (int i = 0; i < candidates.size(); ++i) {
       p[i] = candidates.get(i).getPath();
     }
-    
+
     FileStatus[] stats = fs.listStatus(p);
     if (stats == null || stats.length == 0) {
       return minTs;
@@ -756,13 +756,13 @@ public class Store implements HeapSize {
         return false;
       }
     }
-    
+
     List<StoreFile> candidates = new ArrayList<StoreFile>(this.storefiles);
 
     // exclude files above the max compaction threshold
     // except: save all references. we MUST compact them
     int pos = 0;
-    while (pos < candidates.size() && 
+    while (pos < candidates.size() &&
            candidates.get(pos).getReader().length() > this.maxCompactSize &&
            !candidates.get(pos).isReference()) ++pos;
     candidates.subList(0, pos).clear();
@@ -868,7 +868,7 @@ public class Store implements HeapSize {
       // do not compact old files above a configurable threshold
       // save all references. we MUST compact them
       int pos = 0;
-      while (pos < filesToCompact.size() && 
+      while (pos < filesToCompact.size() &&
              filesToCompact.get(pos).getReader().length() > maxCompactSize &&
              !filesToCompact.get(pos).isReference()) ++pos;
       filesToCompact.subList(0, pos).clear();
@@ -878,7 +878,7 @@ public class Store implements HeapSize {
       LOG.debug(this.storeNameStr + ": no store files to compact");
       return filesToCompact;
     }
-    
+
     // major compact on user action or age (caveat: we have too many files)
     boolean majorcompaction = (forcemajor || isMajorCompaction(filesToCompact))
       && filesToCompact.size() < this.maxFilesToCompact;
@@ -891,7 +891,7 @@ public class Store implements HeapSize {
       int start = 0;
       double r = this.compactRatio;
 
-      /* TODO: add sorting + unit test back in when HBASE-2856 is fixed 
+      /* TODO: add sorting + unit test back in when HBASE-2856 is fixed
       // Sort files by size to correct when normal skew is altered by bulk load.
       Collections.sort(filesToCompact, StoreFile.Comparators.FILE_SIZE);
        */
@@ -1320,10 +1320,11 @@ public class Store implements HeapSize {
     this.lock.readLock().lock();
     try {
       // sanity checks
-      if (!force) {
-        if (storeSize < this.desiredMaxFileSize || this.storefiles.isEmpty()) {
-          return null;
-        }
+      if (this.storefiles.isEmpty()) {
+        return null;
+      }
+      if (!force && storeSize < this.desiredMaxFileSize) {
+        return null;
       }
 
       if (this.region.getRegionInfo().isMetaRegion()) {

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java?rev=1066785&r1=1066784&r2=1066785&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java Thu Feb  3 11:40:07 2011
@@ -61,7 +61,7 @@ import org.apache.hadoop.hbase.util.Manu
 import com.google.common.base.Joiner;
 
 /**
- * Test class fosr the Store
+ * Test class for the Store
  */
 public class TestStore extends TestCase {
   public static final Log LOG = LogFactory.getLog(TestStore.class);
@@ -630,4 +630,15 @@ public class TestStore extends TestCase 
     result = HBaseTestingUtility.getFromStoreFile(store, get);
     assertTrue(result.size()==0);
   }
+
+  /**
+   * Test for HBASE-3492 - Test split on empty colfam (no store files).
+   *
+   * @throws IOException When the IO operations fail.
+   */
+  public void testSplitWithEmptyColFam() throws IOException {
+    init(this.getName());
+    assertNull(store.checkSplit(false));
+    assertNull(store.checkSplit(true));
+  }
 }