You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jm...@apache.org on 2013/02/14 14:35:59 UTC

svn commit: r1446173 [5/5] - in /hbase/branches/hbase-7290v2: ./ bin/ hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/ hbase-common/src/main/java/org/apache/hadoop/hbase/util/ hbase-common/src/main/java/org/apache/hadoop/hbase/util/test/...

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java?rev=1446173&r1=1446172&r2=1446173&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java Thu Feb 14 13:35:54 2013
@@ -207,7 +207,6 @@ public class TestEndToEndSplitTransactio
     HTable table;
     byte[] tableName, family;
     HBaseAdmin admin;
-    HTable metaTable;
     HRegionServer rs;
 
     RegionSplitter(HTable table) throws IOException {
@@ -216,7 +215,6 @@ public class TestEndToEndSplitTransactio
       this.family = table.getTableDescriptor().getFamiliesKeys().iterator().next();
       admin = TEST_UTIL.getHBaseAdmin();
       rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0);
-      metaTable = new HTable(conf, HConstants.META_TABLE_NAME);
     }
 
     public void run() {
@@ -247,14 +245,14 @@ public class TestEndToEndSplitTransactio
           addData(start);
           addData(mid);
 
-          flushAndBlockUntilDone(region.getRegionName());
-          compactAndBlockUntilDone(region.getRegionName());
+          flushAndBlockUntilDone(admin, rs, region.getRegionName());
+          compactAndBlockUntilDone(admin, rs, region.getRegionName());
 
           log("Initiating region split for:" + region.getRegionNameAsString());
           try {
             admin.split(region.getRegionName(), splitPoint);
             //wait until the split is complete
-            blockUntilRegionSplit(50000, region.getRegionName(), true);
+            blockUntilRegionSplit(conf, 50000, region.getRegionName(), true);
 
           } catch (NotServingRegionException ex) {
             //ignore
@@ -262,10 +260,6 @@ public class TestEndToEndSplitTransactio
         }
       } catch (Throwable ex) {
         this.ex = ex;
-      } finally {
-        if (metaTable != null) {
-          IOUtils.closeQuietly(metaTable);
-        }
       }
     }
 
@@ -278,106 +272,6 @@ public class TestEndToEndSplitTransactio
       }
       table.flushCommits();
     }
-
-    void flushAndBlockUntilDone(byte[] regionName) throws IOException, InterruptedException {
-      log("flushing region: " + Bytes.toStringBinary(regionName));
-      admin.flush(regionName);
-      log("blocking until flush is complete: " + Bytes.toStringBinary(regionName));
-      Threads.sleepWithoutInterrupt(500);
-      while (rs.cacheFlusher.getFlushQueueSize() > 0) {
-        Threads.sleep(50);
-      }
-    }
-
-    void compactAndBlockUntilDone(byte[] regionName) throws IOException,
-      InterruptedException {
-      log("Compacting region: " + Bytes.toStringBinary(regionName));
-      admin.majorCompact(regionName);
-      log("blocking until compaction is complete: " + Bytes.toStringBinary(regionName));
-      Threads.sleepWithoutInterrupt(500);
-      while (rs.compactSplitThread.getCompactionQueueSize() > 0) {
-        Threads.sleep(50);
-      }
-    }
-
-    /** bloks until the region split is complete in META and region server opens the daughters */
-    void blockUntilRegionSplit(long timeout, final byte[] regionName, boolean waitForDaughters)
-        throws IOException, InterruptedException {
-      long start = System.currentTimeMillis();
-      log("blocking until region is split:" +  Bytes.toStringBinary(regionName));
-      HRegionInfo daughterA = null, daughterB = null;
-
-      while (System.currentTimeMillis() - start < timeout) {
-        Result result = getRegionRow(regionName);
-        if (result == null) {
-          break;
-        }
-
-        HRegionInfo region = HRegionInfo.getHRegionInfo(result);
-        if(region.isSplitParent()) {
-          log("found parent region: " + region.toString());
-          PairOfSameType<HRegionInfo> pair = HRegionInfo.getDaughterRegions(result);
-          daughterA = pair.getFirst();
-          daughterB = pair.getSecond();
-          break;
-        }
-        sleep(100);
-      }
-
-      //if we are here, this means the region split is complete or timed out
-      if (waitForDaughters) {
-        long rem = timeout - (System.currentTimeMillis() - start);
-        blockUntilRegionIsInMeta(rem, daughterA.getRegionName());
-
-        rem = timeout - (System.currentTimeMillis() - start);
-        blockUntilRegionIsInMeta(rem, daughterB.getRegionName());
-
-        rem = timeout - (System.currentTimeMillis() - start);
-        blockUntilRegionIsOpenedByRS(rem, daughterA.getRegionName());
-
-        rem = timeout - (System.currentTimeMillis() - start);
-        blockUntilRegionIsOpenedByRS(rem, daughterB.getRegionName());
-      }
-    }
-
-    Result getRegionRow(byte[] regionName) throws IOException {
-      Get get = new Get(regionName);
-      return metaTable.get(get);
-    }
-
-    void blockUntilRegionIsInMeta(long timeout, byte[] regionName)
-        throws IOException, InterruptedException {
-      log("blocking until region is in META: " + Bytes.toStringBinary(regionName));
-      long start = System.currentTimeMillis();
-      while (System.currentTimeMillis() - start < timeout) {
-        Result result = getRegionRow(regionName);
-        if (result != null) {
-          HRegionInfo info = HRegionInfo.getHRegionInfo(result);
-          if (info != null && !info.isOffline()) {
-            log("found region in META: " + Bytes.toStringBinary(regionName));
-            break;
-          }
-        }
-        sleep(10);
-      }
-    }
-
-    void blockUntilRegionIsOpenedByRS(long timeout, byte[] regionName)
-      throws IOException, InterruptedException {
-      log("blocking until region is opened by region server: " + Bytes.toStringBinary(regionName));
-      long start = System.currentTimeMillis();
-      while (System.currentTimeMillis() - start < timeout) {
-        List<HRegion> regions = rs.getOnlineRegions(tableName);
-        for (HRegion region : regions) {
-          if (Bytes.compareTo(region.getRegionName(), regionName) == 0) {
-            log("found region open in RS: " + Bytes.toStringBinary(regionName));
-            return;
-          }
-        }
-        sleep(10);
-      }
-    }
-
   }
 
   /**
@@ -484,5 +378,118 @@ public class TestEndToEndSplitTransactio
     LOG.info(msg);
   }
 
+  /* some utility methods for split tests */
+
+  public static void flushAndBlockUntilDone(HBaseAdmin admin, HRegionServer rs, byte[] regionName)
+      throws IOException, InterruptedException {
+    log("flushing region: " + Bytes.toStringBinary(regionName));
+    admin.flush(regionName);
+    log("blocking until flush is complete: " + Bytes.toStringBinary(regionName));
+    Threads.sleepWithoutInterrupt(500);
+    while (rs.cacheFlusher.getFlushQueueSize() > 0) {
+      Threads.sleep(50);
+    }
+  }
+
+  public static void compactAndBlockUntilDone(HBaseAdmin admin, HRegionServer rs, byte[] regionName)
+      throws IOException, InterruptedException {
+    log("Compacting region: " + Bytes.toStringBinary(regionName));
+    admin.majorCompact(regionName);
+    log("blocking until compaction is complete: " + Bytes.toStringBinary(regionName));
+    Threads.sleepWithoutInterrupt(500);
+    while (rs.compactSplitThread.getCompactionQueueSize() > 0) {
+      Threads.sleep(50);
+    }
+  }
+
+  /** Blocks until the region split is complete in META and region server opens the daughters */
+  public static void blockUntilRegionSplit(Configuration conf, long timeout,
+      final byte[] regionName, boolean waitForDaughters)
+      throws IOException, InterruptedException {
+    long start = System.currentTimeMillis();
+    log("blocking until region is split:" +  Bytes.toStringBinary(regionName));
+    HRegionInfo daughterA = null, daughterB = null;
+    HTable metaTable = new HTable(conf, HConstants.META_TABLE_NAME);
+
+    try {
+      while (System.currentTimeMillis() - start < timeout) {
+        Result result = getRegionRow(metaTable, regionName);
+        if (result == null) {
+          break;
+        }
+
+        HRegionInfo region = HRegionInfo.getHRegionInfo(result);
+        if(region.isSplitParent()) {
+          log("found parent region: " + region.toString());
+          PairOfSameType<HRegionInfo> pair = HRegionInfo.getDaughterRegions(result);
+          daughterA = pair.getFirst();
+          daughterB = pair.getSecond();
+          break;
+        }
+        Threads.sleep(100);
+      }
+
+      //if we are here, this means the region split is complete or timed out
+      if (waitForDaughters) {
+        long rem = timeout - (System.currentTimeMillis() - start);
+        blockUntilRegionIsInMeta(metaTable, rem, daughterA);
+
+        rem = timeout - (System.currentTimeMillis() - start);
+        blockUntilRegionIsInMeta(metaTable, rem, daughterB);
+
+        rem = timeout - (System.currentTimeMillis() - start);
+        blockUntilRegionIsOpened(conf, rem, daughterA);
+
+        rem = timeout - (System.currentTimeMillis() - start);
+        blockUntilRegionIsOpened(conf, rem, daughterB);
+      }
+    } finally {
+      IOUtils.closeQuietly(metaTable);
+    }
+  }
+
+  public static Result getRegionRow(HTable metaTable, byte[] regionName) throws IOException {
+    Get get = new Get(regionName);
+    return metaTable.get(get);
+  }
+
+  public static void blockUntilRegionIsInMeta(HTable metaTable, long timeout, HRegionInfo hri)
+      throws IOException, InterruptedException {
+    log("blocking until region is in META: " + hri.getRegionNameAsString());
+    long start = System.currentTimeMillis();
+    while (System.currentTimeMillis() - start < timeout) {
+      Result result = getRegionRow(metaTable, hri.getRegionName());
+      if (result != null) {
+        HRegionInfo info = HRegionInfo.getHRegionInfo(result);
+        if (info != null && !info.isOffline()) {
+          log("found region in META: " + hri.getRegionNameAsString());
+          break;
+        }
+      }
+      Threads.sleep(10);
+    }
+  }
+
+  public static void blockUntilRegionIsOpened(Configuration conf, long timeout, HRegionInfo hri)
+      throws IOException, InterruptedException {
+    log("blocking until region is opened for reading:" + hri.getRegionNameAsString());
+    long start = System.currentTimeMillis();
+    HTable table = new HTable(conf, hri.getTableName());
+
+    try {
+      Get get = new Get(hri.getStartKey());
+      while (System.currentTimeMillis() - start < timeout) {
+        try {
+          table.get(get);
+          break;
+        } catch(IOException ex) {
+          //wait some more
+        }
+        Threads.sleep(10);
+      }
+    } finally {
+      IOUtils.closeQuietly(table);
+    }
+  }
 }
 

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java?rev=1446173&r1=1446172&r2=1446173&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java Thu Feb 14 13:35:54 2013
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -519,6 +520,41 @@ public class TestHRegion extends HBaseTe
     }
   }
 
+  public void testAppendWithReadOnlyTable() throws Exception {
+    byte[] TABLE = Bytes.toBytes("readOnlyTable");
+    this.region = initHRegion(TABLE, getName(), conf, true, Bytes.toBytes("somefamily"));
+    boolean exceptionCaught = false;
+    Append append = new Append(Bytes.toBytes("somerow"));
+    append.add(Bytes.toBytes("somefamily"), Bytes.toBytes("somequalifier"), 
+        Bytes.toBytes("somevalue"));
+    try {
+      region.append(append, false);
+    } catch (IOException e) {
+      exceptionCaught = true;
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
+    }
+    assertTrue(exceptionCaught == true);
+  }
+
+  public void testIncrWithReadOnlyTable() throws Exception {
+    byte[] TABLE = Bytes.toBytes("readOnlyTable");
+    this.region = initHRegion(TABLE, getName(), conf, true, Bytes.toBytes("somefamily"));
+    boolean exceptionCaught = false;    
+    Increment inc = new Increment(Bytes.toBytes("somerow"));
+    inc.addColumn(Bytes.toBytes("somefamily"), Bytes.toBytes("somequalifier"), 1L);
+    try {
+      region.increment(inc, false);
+    } catch (IOException e) {
+      exceptionCaught = true;
+    } finally {
+      HRegion.closeHRegion(this.region);
+      this.region = null;
+    }
+    assertTrue(exceptionCaught == true);
+  }
+
   private void deleteColumns(HRegion r, String value, String keyPrefix)
   throws IOException {
     InternalScanner scanner = buildScanner(keyPrefix, value, r);
@@ -3119,7 +3155,7 @@ public class TestHRegion extends HBaseTe
     byte[] tableName = Bytes.toBytes(method);
     byte[] family = Bytes.toBytes("family");
     this.region = initHRegion(tableName, Bytes.toBytes("x"), Bytes.toBytes("z"), method,
-        conf, family);
+        conf, false, family);
     try {
       byte[] rowNotServed = Bytes.toBytes("a");
       Get g = new Get(rowNotServed);
@@ -3221,7 +3257,7 @@ public class TestHRegion extends HBaseTe
       }
       //before compaction
       HStore store = (HStore) region.getStore(fam1);
-      List<StoreFile> storeFiles = store.getStorefiles();
+      Collection<StoreFile> storeFiles = store.getStorefiles();
       for (StoreFile storefile : storeFiles) {
         StoreFile.Reader reader = storefile.getReader();
         reader.loadFileInfo();
@@ -3841,7 +3877,22 @@ public class TestHRegion extends HBaseTe
   public static HRegion initHRegion (byte [] tableName, String callingMethod,
       Configuration conf, byte [] ... families)
     throws IOException{
-    return initHRegion(tableName, null, null, callingMethod, conf, families);
+    return initHRegion(tableName, null, null, callingMethod, conf, false, families);
+  }
+
+  /**
+   * @param tableName
+   * @param callingMethod
+   * @param conf
+   * @param isReadOnly
+   * @param families
+   * @throws IOException
+   * @return A region on which you must call {@link HRegion#closeHRegion(HRegion)} when done.
+   */
+  public static HRegion initHRegion (byte [] tableName, String callingMethod,
+      Configuration conf, boolean isReadOnly, byte [] ... families)
+    throws IOException{
+    return initHRegion(tableName, null, null, callingMethod, conf, isReadOnly, families);
   }
 
   /**
@@ -3850,14 +3901,16 @@ public class TestHRegion extends HBaseTe
    * @param stopKey
    * @param callingMethod
    * @param conf
+   * @param isReadOnly
    * @param families
    * @throws IOException
    * @return A region on which you must call {@link HRegion#closeHRegion(HRegion)} when done.
    */
   private static HRegion initHRegion(byte[] tableName, byte[] startKey, byte[] stopKey,
-      String callingMethod, Configuration conf, byte[]... families)
+      String callingMethod, Configuration conf, boolean isReadOnly, byte[]... families)
       throws IOException {
     HTableDescriptor htd = new HTableDescriptor(tableName);
+    htd.setReadOnly(isReadOnly);
     for(byte [] family : families) {
       htd.addFamily(new HColumnDescriptor(family));
     }

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java?rev=1446173&r1=1446172&r2=1446173&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java Thu Feb 14 13:35:54 2013
@@ -28,7 +28,6 @@ import java.util.List;
 import java.util.TreeMap;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.CompoundConfiguration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -251,4 +250,40 @@ public class TestRegionSplitPolicy {
         Bytes.toString(policy.getSplitPoint()));
   }
 
+  @Test
+  public void testDelimitedKeyPrefixRegionSplitPolicy() throws IOException {
+    HTableDescriptor myHtd = new HTableDescriptor();
+    myHtd.setValue(HTableDescriptor.SPLIT_POLICY,
+        DelimitedKeyPrefixRegionSplitPolicy.class.getName());
+    myHtd.setValue(DelimitedKeyPrefixRegionSplitPolicy.DELIMITER_KEY, ",");
+
+    HRegion myMockRegion = Mockito.mock(HRegion.class);
+    Mockito.doReturn(myHtd).when(myMockRegion).getTableDesc();
+    Mockito.doReturn(stores).when(myMockRegion).getStores();
+
+    HStore mockStore = Mockito.mock(HStore.class);
+    Mockito.doReturn(2000L).when(mockStore).getSize();
+    Mockito.doReturn(true).when(mockStore).canSplit();
+    Mockito.doReturn(Bytes.toBytes("ab,cd")).when(mockStore).getSplitPoint();
+    stores.put(new byte[] { 1 }, mockStore);
+
+    DelimitedKeyPrefixRegionSplitPolicy policy = (DelimitedKeyPrefixRegionSplitPolicy) RegionSplitPolicy
+        .create(myMockRegion, conf);
+
+    assertEquals("ab", Bytes.toString(policy.getSplitPoint()));
+
+    Mockito.doReturn(true).when(myMockRegion).shouldForceSplit();
+    Mockito.doReturn(Bytes.toBytes("efg,h")).when(myMockRegion)
+        .getExplicitSplitPoint();
+
+    policy = (DelimitedKeyPrefixRegionSplitPolicy) RegionSplitPolicy
+        .create(myMockRegion, conf);
+
+    assertEquals("efg", Bytes.toString(policy.getSplitPoint()));
+
+    Mockito.doReturn(Bytes.toBytes("ijk")).when(myMockRegion)
+    .getExplicitSplitPoint();
+    assertEquals("ijk", Bytes.toString(policy.getSplitPoint()));
+  }
+
 }

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java?rev=1446173&r1=1446172&r2=1446173&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java Thu Feb 14 13:35:54 2013
@@ -163,16 +163,9 @@ public class TestSplitTransaction {
    * Pass a reference store
    */
   @Test public void testPrepareWithRegionsWithReference() throws IOException {
-    // create a mock that will act as a reference StoreFile
-    StoreFile storeFileMock  = Mockito.mock(StoreFile.class);
-    when(storeFileMock.isReference()).thenReturn(true);
-
-    // add the mock to the parent stores
     HStore storeMock = Mockito.mock(HStore.class);
-    List<StoreFile> storeFileList = new ArrayList<StoreFile>(1);
-    storeFileList.add(storeFileMock);
-    when(storeMock.getStorefiles()).thenReturn(storeFileList);
-    when(storeMock.close()).thenReturn(ImmutableList.copyOf(storeFileList));
+    when(storeMock.hasReferences()).thenReturn(true);
+    when(storeMock.close()).thenReturn(ImmutableList.<StoreFile>of());
     this.parent.stores.put(Bytes.toBytes(""), storeMock);
 
     SplitTransaction st = new SplitTransaction(this.parent, GOOD_SPLIT_ROW);
@@ -258,7 +251,7 @@ public class TestSplitTransaction {
     // Start transaction.
     SplitTransaction st = prepareGOOD_SPLIT_ROW();
     SplitTransaction spiedUponSt = spy(st);
-    when(spiedUponSt.createDaughterRegion(spiedUponSt.getSecondDaughter(), null)).
+    when(spiedUponSt.createDaughterRegion(spiedUponSt.getSecondDaughter())).
       thenThrow(new MockedFailedDaughterCreation());
     // Run the execute.  Look at what it returns.
     boolean expectedException = false;

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java?rev=1446173&r1=1446172&r2=1446173&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java Thu Feb 14 13:35:54 2013
@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.lang.ref.SoftReference;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
@@ -269,14 +270,16 @@ public class TestStore extends TestCase 
   }
   
   private static long getLowestTimeStampFromFS(FileSystem fs, 
-      final List<StoreFile> candidates) throws IOException {
+      final Collection<StoreFile> candidates) throws IOException {
     long minTs = Long.MAX_VALUE;
     if (candidates.isEmpty()) {
       return minTs; 
     }
     Path[] p = new Path[candidates.size()];
-    for (int i = 0; i < candidates.size(); ++i) {
-      p[i] = candidates.get(i).getPath();
+    int i = 0;
+    for (StoreFile sf : candidates) {
+      p[i] = sf.getPath();
+      ++i;
     }
     
     FileStatus[] stats = fs.listStatus(p);
@@ -305,7 +308,7 @@ public class TestStore extends TestCase 
     flush(1);
     // Now put in place an empty store file.  Its a little tricky.  Have to
     // do manually with hacked in sequence id.
-    StoreFile f = this.store.getStorefiles().get(0);
+    StoreFile f = this.store.getStorefiles().iterator().next();
     Path storedir = f.getPath().getParent();
     long seqid = f.getMaxSequenceId();
     Configuration c = HBaseConfiguration.create();

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java?rev=1446173&r1=1446172&r2=1446173&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java Thu Feb 14 13:35:54 2013
@@ -699,13 +699,14 @@ public class TestStoreFile extends HBase
 
   public void testSeqIdComparator() {
     assertOrdering(StoreFile.Comparators.SEQ_ID,
-        mockStoreFile(true, 1000, -1, "/foo/123"),
-        mockStoreFile(true, 1000, -1, "/foo/126"),
-        mockStoreFile(true, 2000, -1, "/foo/126"),
-        mockStoreFile(false, -1, 1, "/foo/1"),
-        mockStoreFile(false, -1, 3, "/foo/2"),
-        mockStoreFile(false, -1, 5, "/foo/2"),
-        mockStoreFile(false, -1, 5, "/foo/3"));
+        mockStoreFile(true,  100,   1000, -1, "/foo/123"),
+        mockStoreFile(true,  100,   1000, -1, "/foo/124"),
+        mockStoreFile(true,  99,    1000, -1, "/foo/126"),
+        mockStoreFile(true,  98,    2000, -1, "/foo/126"),
+        mockStoreFile(false, 3453, -1,     1, "/foo/1"),
+        mockStoreFile(false, 2,    -1,     3, "/foo/2"),
+        mockStoreFile(false, 1000, -1,     5, "/foo/2"),
+        mockStoreFile(false, 76,   -1,     5, "/foo/3"));
   }
 
   /**
@@ -724,9 +725,17 @@ public class TestStoreFile extends HBase
   /**
    * Create a mock StoreFile with the given attributes.
    */
-  private StoreFile mockStoreFile(boolean bulkLoad, long bulkTimestamp,
-      long seqId, String path) {
+  private StoreFile mockStoreFile(boolean bulkLoad,
+                                  long size,
+                                  long bulkTimestamp,
+                                  long seqId,
+                                  String path) {
     StoreFile mock = Mockito.mock(StoreFile.class);
+    StoreFile.Reader reader = Mockito.mock(StoreFile.Reader.class);
+
+    Mockito.doReturn(size).when(reader).length();
+
+    Mockito.doReturn(reader).when(mock).getReader();
     Mockito.doReturn(bulkLoad).when(mock).isBulkLoadResult();
     Mockito.doReturn(bulkTimestamp).when(mock).getBulkLoadTimestamp();
     Mockito.doReturn(seqId).when(mock).getMaxSequenceId();

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestOpenRegionHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestOpenRegionHandler.java?rev=1446173&r1=1446172&r2=1446173&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestOpenRegionHandler.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/handler/TestOpenRegionHandler.java Thu Feb 14 13:35:54 2013
@@ -184,6 +184,33 @@ public class TestOpenRegionHandler {
     assertEquals(EventType.RS_ZK_REGION_FAILED_OPEN, rt.getEventType());
   }
   
+  @Test
+  public void testTransitionToFailedOpenEvenIfCleanupFails() throws Exception {
+    Server server = new MockServer(HTU);
+    RegionServerServices rsServices = new MockRegionServerServices();
+    // Create it OFFLINE, which is what it expects
+    ZKAssign.createNodeOffline(server.getZooKeeper(), TEST_HRI, server.getServerName());
+    // Create the handler
+    OpenRegionHandler handler = new OpenRegionHandler(server, rsServices, TEST_HRI, TEST_HTD) {
+      @Override
+      boolean updateMeta(HRegion r) {
+        return false;
+      };
 
+      @Override
+      void cleanupFailedOpen(HRegion region) throws IOException {
+        throw new IOException("FileSystem got closed.");
+      }
+    };
+    rsServices.getRegionsInTransitionInRS().put(TEST_HRI.getEncodedNameAsBytes(), Boolean.TRUE);
+    try {
+      handler.process();
+    } catch (Exception e) {
+      // Ignore the IOException that we have thrown from cleanupFailedOpen
+    }
+    RegionTransition rt = RegionTransition.parseFrom(ZKAssign.getData(server.getZooKeeper(),
+        TEST_HRI.getEncodedName()));
+    assertEquals(EventType.RS_ZK_REGION_FAILED_OPEN, rt.getEventType());
+  }
 }
 

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java?rev=1446173&r1=1446172&r2=1446173&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java Thu Feb 14 13:35:54 2013
@@ -1076,7 +1076,7 @@ public class TestHLogSplit {
       }
       Path tableDir = new Path(HBASEDIR, new String(TABLE_NAME));
       Path regionDir = new Path(tableDir, REGIONS.get(0));
-      Path recoveredEdits = new Path(regionDir, HLogSplitter.RECOVERED_EDITS);
+      Path recoveredEdits = new Path(regionDir, HLog.RECOVERED_EDITS_DIR);
       String region = "juliet";
       Path julietLog = new Path(HLOGDIR, HLOG_FILE_PREFIX + ".juliet");
       try {

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java?rev=1446173&r1=1446172&r2=1446173&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java Thu Feb 14 13:35:54 2013
@@ -268,10 +268,10 @@ public class TestWALReplay {
     HLog wal3 = createWAL(this.conf);
     wal3.setSequenceNumber(wal2.getSequenceNumber());
     try {
-      final HRegion region = new HRegion(basedir, wal3, this.fs, this.conf, hri,
-        htd, null);
-      long seqid = region.initialize();
-      assertTrue(seqid > wal3.getSequenceNumber());
+      long wal3SeqId = wal3.getSequenceNumber();
+      HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal3);
+      long seqid = region.getOpenSeqNum();
+      assertTrue(seqid > wal3SeqId);
 
       // TODO: Scan all.
       region.close();
@@ -325,9 +325,10 @@ public class TestWALReplay {
       public Object run() throws Exception {
         runWALSplit(newConf);
         HLog wal2 = createWAL(newConf);
-        HRegion region2 = new HRegion(basedir, wal2, FileSystem.get(newConf),
-          newConf, hri, htd, null);
-        long seqid2 = region2.initialize();
+
+        HRegion region2 = HRegion.openHRegion(newConf, FileSystem.get(newConf),
+          hbaseRootDir, hri, htd, wal2);
+        long seqid2 = region2.getOpenSeqNum();
         assertTrue(seqid2 > -1);
 
         // I can't close wal1.  Its been appropriated when we split.
@@ -365,8 +366,8 @@ public class TestWALReplay {
     // of the families during the load of edits so its seqid is not same as
     // others to test we do right thing when different seqids.
     HLog wal = createWAL(this.conf);
-    HRegion region = new HRegion(basedir, wal, this.fs, this.conf, hri, htd, null);
-    long seqid = region.initialize();
+    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal);
+    long seqid = region.getOpenSeqNum();
     // HRegionServer usually does this. It knows the largest seqid across all regions.
     wal.setSequenceNumber(seqid);
     boolean first = true;
@@ -390,8 +391,8 @@ public class TestWALReplay {
     wal.close();
     runWALSplit(this.conf);
     HLog wal2 = createWAL(this.conf);
-    HRegion region2 = new HRegion(basedir, wal2, this.fs, this.conf, hri, htd, null);
-    long seqid2 = region2.initialize();
+    HRegion region2 = HRegion.openHRegion(conf, this.fs, hbaseRootDir, hri, htd, wal2);
+    long seqid2 = region2.getOpenSeqNum();
     // HRegionServer usually does this. It knows the largest seqid across all regions.
     wal2.setSequenceNumber(seqid2);
     assertTrue(seqid + result.size() < seqid2);
@@ -482,8 +483,8 @@ public class TestWALReplay {
     // of the families during the load of edits so its seqid is not same as
     // others to test we do right thing when different seqids.
     HLog wal = createWAL(this.conf);
-    HRegion region = new HRegion(basedir, wal, this.fs, this.conf, hri, htd, null);
-    long seqid = region.initialize();
+    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal);
+    long seqid = region.getOpenSeqNum();
     // HRegionServer usually does this. It knows the largest seqid across all regions.
     wal.setSequenceNumber(seqid);
     for (HColumnDescriptor hcd: htd.getFamilies()) {
@@ -518,8 +519,8 @@ public class TestWALReplay {
     // Let us try to split and recover
     runWALSplit(this.conf);
     HLog wal2 = createWAL(this.conf);
-    HRegion region2 = new HRegion(basedir, wal2, this.fs, this.conf, hri, htd, null);
-    long seqid2 = region2.initialize();
+    HRegion region2 = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal2);
+    long seqid2 = region2.getOpenSeqNum();
     // HRegionServer usually does this. It knows the largest seqid across all regions.
     wal2.setSequenceNumber(seqid2);
     assertTrue(seqid + result.size() < seqid2);
@@ -638,9 +639,9 @@ public class TestWALReplay {
 
     // Mock the HLog
     MockHLog wal = createMockWAL(this.conf);
-    
-    HRegion region = new HRegion(basedir, wal, this.fs, this.conf, hri, htd, null);
-    long seqid = region.initialize();
+
+    HRegion region = HRegion.openHRegion(this.conf, this.fs, hbaseRootDir, hri, htd, wal);
+    long seqid = region.getOpenSeqNum();
     // HRegionServer usually does this. It knows the largest seqid across all
     // regions.
     wal.setSequenceNumber(seqid);

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java?rev=1446173&r1=1446172&r2=1446173&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java Thu Feb 14 13:35:54 2013
@@ -37,6 +37,7 @@ import java.util.Map.Entry;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 
+import org.apache.commons.io.IOUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -71,11 +72,12 @@ import org.apache.hadoop.hbase.master.Re
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.TestEndToEndSplitTransaction;
 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter;
-import org.apache.hadoop.hbase.util.HBaseFsck.PrintingErrorReporter;
-import org.apache.hadoop.hbase.util.HBaseFsck.TableInfo;
 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
 import org.apache.hadoop.hbase.util.HBaseFsck.HbckInfo;
+import org.apache.hadoop.hbase.util.HBaseFsck.PrintingErrorReporter;
+import org.apache.hadoop.hbase.util.HBaseFsck.TableInfo;
 import org.apache.hadoop.hbase.util.hbck.HFileCorruptionChecker;
 import org.apache.hadoop.hbase.util.hbck.HbckTestingUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKAssign;
@@ -402,7 +404,7 @@ public class TestHBaseFsck {
       deleteTable(table);
     }
   }
-  
+
   @Test
   public void testHbckFixOrphanTable() throws Exception {
     String table = "tableInfo";
@@ -411,31 +413,31 @@ public class TestHBaseFsck {
     try {
       setupTable(table);
       HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
-      
+
       Path hbaseTableDir = new Path(conf.get(HConstants.HBASE_DIR) + "/" + table );
       fs = hbaseTableDir.getFileSystem(conf);
       FileStatus status = FSTableDescriptors.getTableInfoPath(fs, hbaseTableDir);
       tableinfo = status.getPath();
       fs.rename(tableinfo, new Path("/.tableinfo"));
-      
+
       //to report error if .tableinfo is missing.
-      HBaseFsck hbck = doFsck(conf, false); 
+      HBaseFsck hbck = doFsck(conf, false);
       assertErrors(hbck, new ERROR_CODE[] { ERROR_CODE.NO_TABLEINFO_FILE });
-      
+
       // fix OrphanTable with default .tableinfo (htd not yet cached on master)
       hbck = doFsck(conf, true);
       assertNoErrors(hbck);
       status = null;
       status = FSTableDescriptors.getTableInfoPath(fs, hbaseTableDir);
       assertNotNull(status);
-      
+
       HTableDescriptor htd = admin.getTableDescriptor(table.getBytes());
       htd.setValue("NOT_DEFAULT", "true");
       admin.disableTable(table);
       admin.modifyTable(table.getBytes(), htd);
       admin.enableTable(table);
       fs.delete(status.getPath(), true);
-      
+
       // fix OrphanTable with cache
       htd = admin.getTableDescriptor(table.getBytes()); // warms up cached htd on master
       hbck = doFsck(conf, true);
@@ -1194,6 +1196,7 @@ public class TestHBaseFsck {
   @Test
   public void testLingeringSplitParent() throws Exception {
     String table = "testLingeringSplitParent";
+    HTable meta = null;
     try {
       setupTable(table);
       assertEquals(ROWKEYS.length, countRows());
@@ -1207,7 +1210,7 @@ public class TestHBaseFsck {
         Bytes.toBytes("C"), true, true, false);
 
       // Create a new meta entry to fake it as a split parent.
-      HTable meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getName());
+      meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getName());
       HRegionInfo hri = location.getRegionInfo();
 
       HRegionInfo a = new HRegionInfo(tbl.getTableName(),
@@ -1260,6 +1263,119 @@ public class TestHBaseFsck {
       assertEquals(ROWKEYS.length, countRows());
     } finally {
       deleteTable(table);
+      IOUtils.closeQuietly(meta);
+    }
+  }
+
+  /**
+   * Tests that LINGERING_SPLIT_PARENT is not erroneously reported for
+   * valid cases where the daughters are there.
+   */
+  @Test
+  public void testValidLingeringSplitParent() throws Exception {
+    String table = "testLingeringSplitParent";
+    HTable meta = null;
+    try {
+      setupTable(table);
+      assertEquals(ROWKEYS.length, countRows());
+
+      // make sure data in regions, if in hlog only there is no data loss
+      TEST_UTIL.getHBaseAdmin().flush(table);
+      HRegionLocation location = tbl.getRegionLocation("B");
+
+      meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getName());
+      HRegionInfo hri = location.getRegionInfo();
+
+      // do a regular split
+      HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
+      byte[] regionName = location.getRegionInfo().getRegionName();
+      admin.split(location.getRegionInfo().getRegionName(), Bytes.toBytes("BM"));
+      TestEndToEndSplitTransaction.blockUntilRegionSplit(
+          TEST_UTIL.getConfiguration(), 60000, regionName, true);
+
+      // TODO: fixHdfsHoles does not work against splits, since the parent dir lingers on
+      // for some time until children references are deleted. HBCK erroneously sees this as
+      // overlapping regions
+      HBaseFsck hbck = doFsck(conf, true, true, false, false, false, true, true, true, null);
+      assertErrors(hbck, new ERROR_CODE[] {}); //no LINGERING_SPLIT_PARENT reported
+
+      // assert that the split META entry is still there.
+      Get get = new Get(hri.getRegionName());
+      Result result = meta.get(get);
+      assertNotNull(result);
+      assertNotNull(HRegionInfo.getHRegionInfo(result));
+
+      assertEquals(ROWKEYS.length, countRows());
+
+      // assert that we still have the split regions
+      assertEquals(tbl.getStartKeys().length, SPLITS.length + 1 + 1); //SPLITS + 1 is # regions pre-split.
+      assertNoErrors(doFsck(conf, false));
+    } finally {
+      deleteTable(table);
+      IOUtils.closeQuietly(meta);
+    }
+  }
+
+  /**
+   * Split crashed after write to META finished for the parent region, but
+   * failed to write daughters (pre HBASE-7721 codebase)
+   */
+  @Test
+  public void testSplitDaughtersNotInMeta() throws Exception {
+    String table = "testSplitdaughtersNotInMeta";
+    HTable meta = null;
+    try {
+      setupTable(table);
+      assertEquals(ROWKEYS.length, countRows());
+
+      // make sure data in regions, if in hlog only there is no data loss
+      TEST_UTIL.getHBaseAdmin().flush(table);
+      HRegionLocation location = tbl.getRegionLocation("B");
+
+      meta = new HTable(conf, HTableDescriptor.META_TABLEDESC.getName());
+      HRegionInfo hri = location.getRegionInfo();
+
+      // do a regular split
+      HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
+      byte[] regionName = location.getRegionInfo().getRegionName();
+      admin.split(location.getRegionInfo().getRegionName(), Bytes.toBytes("BM"));
+      TestEndToEndSplitTransaction.blockUntilRegionSplit(
+          TEST_UTIL.getConfiguration(), 60000, regionName, true);
+
+      PairOfSameType<HRegionInfo> daughters = HRegionInfo.getDaughterRegions(meta.get(new Get(regionName)));
+
+      // Delete daughter regions from meta, but not hdfs, unassign it.
+      Map<HRegionInfo, ServerName> hris = tbl.getRegionLocations();
+      undeployRegion(admin, hris.get(daughters.getFirst()), daughters.getFirst());
+      undeployRegion(admin, hris.get(daughters.getSecond()), daughters.getSecond());
+
+      meta.delete(new Delete(daughters.getFirst().getRegionName()));
+      meta.delete(new Delete(daughters.getSecond().getRegionName()));
+      meta.flushCommits();
+
+      HBaseFsck hbck = doFsck(conf, false);
+      assertErrors(hbck, new ERROR_CODE[] {ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
+          ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN}); //no LINGERING_SPLIT_PARENT
+
+      // now fix it. The fix should not revert the region split, but add daughters to META
+      hbck = doFsck(conf, true, true, false, false, false, false, false, false, null);
+      assertErrors(hbck, new ERROR_CODE[] {ERROR_CODE.NOT_IN_META_OR_DEPLOYED,
+          ERROR_CODE.NOT_IN_META_OR_DEPLOYED, ERROR_CODE.HOLE_IN_REGION_CHAIN});
+
+      // assert that the split META entry is still there.
+      Get get = new Get(hri.getRegionName());
+      Result result = meta.get(get);
+      assertNotNull(result);
+      assertNotNull(HRegionInfo.getHRegionInfo(result));
+
+      assertEquals(ROWKEYS.length, countRows());
+
+      // assert that we still have the split regions
+      assertEquals(tbl.getStartKeys().length, SPLITS.length + 1 + 1); //SPLITS + 1 is # regions pre-split.
+      assertNoErrors(doFsck(conf, false)); //should be fixed by now
+    } finally {
+      deleteTable(table);
+      IOUtils.closeQuietly(meta);
     }
   }
 

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitCalculator.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitCalculator.java?rev=1446173&r1=1446172&r2=1446173&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitCalculator.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionSplitCalculator.java Thu Feb 14 13:35:54 2013
@@ -26,6 +26,7 @@ import java.util.Collection;
 import java.util.Comparator;
 import java.util.List;
 import java.util.SortedSet;
+import java.util.UUID;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -39,21 +40,21 @@ import org.junit.experimental.categories
 
 @Category(SmallTests.class)
 public class TestRegionSplitCalculator {
-  final static Log LOG = LogFactory.getLog(TestRegionSplitCalculator.class);
+  private static final Log LOG = LogFactory.getLog(TestRegionSplitCalculator.class);
 
   /**
    * This is range uses a user specified start and end keys. It also has an
-   * extra time based tiebreaker so that different ranges with the same
-   * start/end key pair count as different regions.
+   * extra tiebreaker so that different ranges with the same start/end key pair
+   * count as different regions.
    */
   static class SimpleRange implements KeyRange {
     byte[] start, end;
-    long tiebreaker;
+    UUID tiebreaker;
 
     SimpleRange(byte[] start, byte[] end) {
       this.start = start;
       this.end = end;
-      this.tiebreaker = System.nanoTime();
+      this.tiebreaker = UUID.randomUUID();
     }
 
     @Override
@@ -108,9 +109,9 @@ public class TestRegionSplitCalculator {
     // we display this way because the last end key should be displayed as well.
     StringBuilder sb = new StringBuilder();
     for (byte[] k : splits) {
-      sb.append(Bytes.toString(k) + ":\t");
+      sb.append(Bytes.toString(k)).append(":\t");
       for (SimpleRange r : regions.get(k)) {
-        sb.append(r.toString() + "\t");
+        sb.append(r.toString()).append("\t");
       }
       sb.append("\n");
     }
@@ -147,7 +148,7 @@ public class TestRegionSplitCalculator {
     LOG.info("Empty");
     String res = dump(sc.getSplits(), regions);
     checkDepths(sc.getSplits(), regions);
-    assertEquals(res, "");
+    assertEquals("", res);
   }
 
   @Test
@@ -161,7 +162,7 @@ public class TestRegionSplitCalculator {
     LOG.info("Single edge");
     String res = dump(sc.getSplits(), regions);
     checkDepths(sc.getSplits(), regions, 1, 0);
-    assertEquals(res, "A:\t[A, B]\t\n" + "B:\t\n");
+    assertEquals("A:\t[A, B]\t\n" + "B:\t\n", res);
   }
 
   @Test
@@ -175,7 +176,7 @@ public class TestRegionSplitCalculator {
     LOG.info("Single empty edge");
     String res = dump(sc.getSplits(), regions);
     checkDepths(sc.getSplits(), regions, 1);
-    assertEquals(res, "A:\t[A, A]\t\n");
+    assertEquals("A:\t[A, A]\t\n", res);
   }
 
   @Test
@@ -193,8 +194,8 @@ public class TestRegionSplitCalculator {
     LOG.info("AC covers AB, BC");
     String res = dump(sc.getSplits(), regions);
     checkDepths(sc.getSplits(), regions, 2, 2, 0);
-    assertEquals(res, "A:\t[A, B]\t[A, C]\t\n" + "B:\t[A, C]\t[B, C]\t\n"
-        + "C:\t\n");
+    assertEquals("A:\t[A, B]\t[A, C]\t\n" + "B:\t[A, C]\t[B, C]\t\n"
+        + "C:\t\n", res);
   }
 
   @Test
@@ -212,8 +213,8 @@ public class TestRegionSplitCalculator {
     LOG.info("AB, BD covers BC");
     String res = dump(sc.getSplits(), regions);
     checkDepths(sc.getSplits(), regions, 1, 2, 1, 0);
-    assertEquals(res, "A:\t[A, B]\t\n" + "B:\t[B, C]\t[B, D]\t\n"
-        + "C:\t[B, D]\t\n" + "D:\t\n");
+    assertEquals("A:\t[A, B]\t\n" + "B:\t[B, C]\t[B, D]\t\n"
+        + "C:\t[B, D]\t\n" + "D:\t\n", res);
   }
 
   @Test
@@ -231,8 +232,8 @@ public class TestRegionSplitCalculator {
     LOG.info("Hole between C and E");
     String res = dump(sc.getSplits(), regions);
     checkDepths(sc.getSplits(), regions, 1, 1, 0, 1, 0);
-    assertEquals(res, "A:\t[A, B]\t\n" + "B:\t[B, C]\t\n" + "C:\t\n"
-        + "E:\t[E, F]\t\n" + "F:\t\n");
+    assertEquals("A:\t[A, B]\t\n" + "B:\t[B, C]\t\n" + "C:\t\n"
+        + "E:\t[E, F]\t\n" + "F:\t\n", res);
   }
 
   @Test
@@ -248,8 +249,8 @@ public class TestRegionSplitCalculator {
     LOG.info("AC and BD overlap but share no start/end keys");
     String res = dump(sc.getSplits(), regions);
     checkDepths(sc.getSplits(), regions, 1, 2, 1, 0);
-    assertEquals(res, "A:\t[A, C]\t\n" + "B:\t[A, C]\t[B, D]\t\n"
-        + "C:\t[B, D]\t\n" + "D:\t\n");
+    assertEquals("A:\t[A, C]\t\n" + "B:\t[A, C]\t[B, D]\t\n"
+        + "C:\t[B, D]\t\n" + "D:\t\n", res);
   }
 
   @Test
@@ -265,7 +266,7 @@ public class TestRegionSplitCalculator {
     LOG.info("AC and AB overlap in the beginning");
     String res = dump(sc.getSplits(), regions);
     checkDepths(sc.getSplits(), regions, 2, 1, 0);
-    assertEquals(res, "A:\t[A, B]\t[A, C]\t\n" + "B:\t[A, C]\t\n" + "C:\t\n");
+    assertEquals("A:\t[A, B]\t[A, C]\t\n" + "B:\t[A, C]\t\n" + "C:\t\n", res);
   }
 
   @Test
@@ -281,13 +282,15 @@ public class TestRegionSplitCalculator {
     LOG.info("AC and BC overlap in the end");
     String res = dump(sc.getSplits(), regions);
     checkDepths(sc.getSplits(), regions, 1, 2, 0);
-    assertEquals(res, "A:\t[A, C]\t\n" + "B:\t[A, C]\t[B, C]\t\n" + "C:\t\n");
+    assertEquals("A:\t[A, C]\t\n" + "B:\t[A, C]\t[B, C]\t\n" + "C:\t\n", res);
   }
 
   @Test
   public void testSplitCalculatorEq() {
     SimpleRange a = new SimpleRange(Bytes.toBytes("A"), Bytes.toBytes("C"));
     SimpleRange b = new SimpleRange(Bytes.toBytes("A"), Bytes.toBytes("C"));
+
+    LOG.info(a.tiebreaker + " - " + b.tiebreaker);
     RegionSplitCalculator<SimpleRange> sc = new RegionSplitCalculator<SimpleRange>(
         cmp);
     sc.add(a);
@@ -297,7 +300,7 @@ public class TestRegionSplitCalculator {
     LOG.info("AC and AC overlap completely");
     String res = dump(sc.getSplits(), regions);
     checkDepths(sc.getSplits(), regions, 2, 0);
-    assertEquals(res, "A:\t[A, C]\t[A, C]\t\n" + "C:\t\n");
+    assertEquals("A:\t[A, C]\t[A, C]\t\n" + "C:\t\n", res);
   }
 
   @Test
@@ -311,7 +314,7 @@ public class TestRegionSplitCalculator {
     LOG.info("CA is backwards");
     String res = dump(sc.getSplits(), regions);
     checkDepths(sc.getSplits(), regions); // expect nothing
-    assertEquals(res, "");
+    assertEquals("", res);
   }
 
   @Test
@@ -331,11 +334,11 @@ public class TestRegionSplitCalculator {
     LOG.info("Something fairly complex");
     String res = dump(sc.getSplits(), regions);
     checkDepths(sc.getSplits(), regions, 3, 3, 3, 1, 2, 0, 1, 0, 1, 0);
-    assertEquals(res, "A:\t[A, Am]\t[A, B]\t[A, C]\t\n"
+    assertEquals("A:\t[A, Am]\t[A, B]\t[A, C]\t\n"
         + "Am:\t[A, B]\t[A, C]\t[Am, C]\t\n"
         + "B:\t[A, C]\t[Am, C]\t[B, E]\t\n" + "C:\t[B, E]\t\n"
         + "D:\t[B, E]\t[D, E]\t\n" + "E:\t\n" + "F:\t[F, G]\t\n" + "G:\t\n"
-        + "H:\t[H, I]\t\n" + "I:\t\n");
+        + "H:\t[H, I]\t\n" + "I:\t\n", res);
   }
 
   @Test
@@ -350,8 +353,8 @@ public class TestRegionSplitCalculator {
     LOG.info("Special cases -- empty");
     String res = dump(sc.getSplits(), regions);
     checkDepths(sc.getSplits(), regions, 1, 1, 1, 0);
-    assertEquals(res, ":\t[, A]\t\n" + "A:\t[A, B]\t\n" + "B:\t[B, ]\t\n"
-        + "null:\t\n");
+    assertEquals(":\t[, A]\t\n" + "A:\t[A, B]\t\n" + "B:\t[B, ]\t\n"
+        + "null:\t\n", res);
   }
 
   @Test
@@ -382,8 +385,8 @@ public class TestRegionSplitCalculator {
 
     SimpleRange r1 = bigRanges.get(1);
     SimpleRange r2 = bigRanges.get(2);
-    assertEquals(Bytes.toString(r1.start), "A");
-    assertEquals(Bytes.toString(r2.start), "A");
+    assertEquals("A", Bytes.toString(r1.start));
+    assertEquals("A", Bytes.toString(r2.start));
     String r1e = Bytes.toString(r1.end);
     String r2e = Bytes.toString(r2.end);
     assertTrue((r1e.equals("C") && r2e.equals("E"))

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperACL.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperACL.java?rev=1446173&r1=1446172&r2=1446173&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperACL.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperACL.java Thu Feb 14 13:35:54 2013
@@ -86,7 +86,6 @@ public class TestZooKeeperACL {
     zkw = new ZooKeeperWatcher(
       new Configuration(TEST_UTIL.getConfiguration()),
         TestZooKeeper.class.getName(), null);
-    ZKUtil.waitForZKConnectionIfAuthenticating(zkw);
   }
 
   /**

Modified: hbase/branches/hbase-7290v2/pom.xml
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/pom.xml?rev=1446173&r1=1446172&r2=1446173&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/pom.xml (original)
+++ hbase/branches/hbase-7290v2/pom.xml Thu Feb 14 13:35:54 2013
@@ -42,7 +42,7 @@
   <version>0.95-SNAPSHOT</version>
   <name>HBase</name>
   <description>
-    Apache HBase&#153; is the &amp;lt;a href="http://hadoop.apache.org"&amp;rt;Hadoop&lt;/a&amp;rt; database. Use it when you need
+    Apache HBase™ is the &amp;lt;a href="http://hadoop.apache.org"&amp;rt;Hadoop&lt;/a&amp;rt; database. Use it when you need
     random, realtime read/write access to your Big Data.
     This project's goal is the hosting of very large tables -- billions of rows X millions of columns -- atop clusters
     of commodity hardware.
@@ -56,6 +56,7 @@
     <module>hbase-common</module>
     <module>hbase-it</module>
     <module>hbase-examples</module>
+    <module>hbase-prefix-tree</module>
   </modules>
   <scm>
     <connection>scm:svn:http://svn.apache.org/repos/asf/hbase/trunk</connection>
@@ -124,6 +125,14 @@
       <organizationUrl>http://www.intel.com</organizationUrl>
     </developer>
     <developer>
+      <id>ddas</id>
+      <name>Devaraj Das</name>
+      <email>ddas@apache.org</email>
+      <timezone>-8</timezone>
+      <organization>Hortonworks</organization>
+      <organizationUrl>http://www.hortonworks.com</organizationUrl>
+    </developer>
+    <developer>
       <id>dmeil</id>
       <name>Doug Meil</name>
       <email>dmeil@apache.org</email>
@@ -265,7 +274,7 @@
       <email>yuzhihong@gmail.com</email>
       <timezone>-8</timezone>
       <organization>Hortonworks</organization>
-      <organizationUrl>http://www.ebay.com</organizationUrl>
+      <organizationUrl>http://www.hortonworks.com</organizationUrl>
     </developer>
     <developer>
       <id>todd</id>
@@ -979,6 +988,14 @@
         <scope>test</scope>
       </dependency>
       <dependency>
+        <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-prefix-tree</artifactId>
+        <version>${project.version}</version>
+        <!-- unfortunately, runtime scope causes Eclipse to give compile time access which isn't
+          needed, however it is apparently needed to run things within Eclipse -->
+        <scope>runtime</scope>
+      </dependency>
+      <dependency>
         <artifactId>hbase-examples</artifactId>
         <groupId>org.apache.hbase</groupId>
         <version>${project.version}</version>

Modified: hbase/branches/hbase-7290v2/src/docbkx/book.xml
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/src/docbkx/book.xml?rev=1446173&r1=1446172&r2=1446173&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/src/docbkx/book.xml (original)
+++ hbase/branches/hbase-7290v2/src/docbkx/book.xml Thu Feb 14 13:35:54 2013
@@ -1488,8 +1488,9 @@ if (!b) {
 	   <section xml:id="client.connections"><title>Connections</title>
            <para>For connection configuration information, see <xref linkend="client_dependencies" />.
          </para>
-         <para><link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html">HTable</link>
-instances are not thread-safe.  When creating HTable instances, it is advisable to use the same <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HBaseConfiguration">HBaseConfiguration</link>
+         <para><emphasis><link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTable.html">HTable</link>
+                 instances are not thread-safe</emphasis>.  Only one thread use an instance of HTable at any given
+             time.  When creating HTable instances, it is advisable to use the same <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/HBaseConfiguration">HBaseConfiguration</link>
 instance.  This will ensure sharing of ZooKeeper and socket instances to the RegionServers
 which is usually what you want.  For example, this is preferred:
 		<programlisting>HBaseConfiguration conf = HBaseConfiguration.create();
@@ -1505,7 +1506,16 @@ HTable table2 = new HTable(conf2, "myTab
           </para>
           <section xml:id="client.connection.pooling"><title>Connection Pooling</title>
             <para>For applications which require high-end multithreaded access (e.g., web-servers or application servers that may serve many application threads
-            in a single JVM), see <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTablePool.html">HTablePool</link>.
+            in a single JVM), one solution is <link xlink:href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/HTablePool.html">HTablePool</link>.
+            But as written currently, it is difficult to control client resource consumption when using HTablePool.
+            </para>
+            <para>
+                Another solution is to precreate an <classname>HConnection</classname> using
+                <programlisting>HConnectionManager.createConnection(Configuration)</programlisting> as
+                well as an <classname>ExecutorService</classname>; then use the
+                <programlisting>HTable(byte[], HConnection, ExecutorService)</programlisting>
+                constructor to create <classname>HTable</classname> instances on demand.
+                This construction is very lightweight and resources are controlled/shared if you go this route.
             </para>
           </section>
    	  </section>
@@ -2763,7 +2773,7 @@ If you just want to know if some tables 
 in only specific tables. For example the following command would only attempt to check table
 TableFoo and TableBar. The benefit is that hbck will run in less time.
 <programlisting>
-$ ./bin/hbase/ hbck TableFoo TableBar
+$ ./bin/hbase hbck TableFoo TableBar
 </programlisting>
 	</section>
 	<section><title>Inconsistencies</title>
@@ -2896,7 +2906,7 @@ integrity options.
 Finally, there are safeguards to limit repairs to only specific tables. For example the following
 command would only attempt to check and repair table TableFoo and TableBar.
 <programlisting>
-$ ./bin/hbase/ hbck -repair TableFoo TableBar
+$ ./bin/hbase hbck -repair TableFoo TableBar
 </programlisting>
 	<section><title>Special cases: Meta is not properly assigned</title>
 There are a few special cases that hbck can handle as well.

Modified: hbase/branches/hbase-7290v2/src/docbkx/ops_mgt.xml
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/src/docbkx/ops_mgt.xml?rev=1446173&r1=1446172&r2=1446173&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/src/docbkx/ops_mgt.xml (original)
+++ hbase/branches/hbase-7290v2/src/docbkx/ops_mgt.xml Thu Feb 14 13:35:54 2013
@@ -251,9 +251,12 @@ row10	c1	c2
        <title>WALPlayer</title>
        <para>WALPlayer is a utility to replay WAL files into HBase.
        </para>
-       <para>The WAL can be replayed for a set of tables or all tables, and a timerange can be provided (in milliseconds). The WAL is filtered to this set of tables. The output can optionally be mapped to another set of tables.
+       <para>The WAL can be replayed for a set of tables or all tables, and a
+           timerange can be provided (in milliseconds). The WAL is filtered to
+           this set of tables. The output can optionally be mapped to another set of tables.
        </para>
-       <para>WALPlayer can also generate HFiles for later bulk importing, in that case only a single table and no mapping can be specified.
+       <para>WALPlayer can also generate HFiles for later bulk importing, in that case
+           only a single table and no mapping can be specified.
        </para>
        <para>Invoke via:
 <programlisting>$ bin/hbase org.apache.hadoop.hbase.mapreduce.WALPlayer [options] &lt;wal inputdir&gt; &lt;tables&gt; [&lt;tableMappings>]&gt;
@@ -263,18 +266,41 @@ row10	c1	c2
 <programlisting>$ bin/hbase org.apache.hadoop.hbase.mapreduce.WALPlayer /backuplogdir oldTable1,oldTable2 newTable1,newTable2
 </programlisting>
        </para>
+       <para>
+           WALPlayer, by default, runs as a mapreduce job.  To NOT run WALPlayer as a mapreduce job on your cluster,
+           force it to run all in the local process by adding the flags <code>-Dmapred.job.tracker=local</code> on the command line.
+       </para>
     </section>
     <section xml:id="rowcounter">
-       <title>RowCounter</title>
-       <para>RowCounter is a mapreduce job to count all the rows of a table.  This is a good utility to use
-           as a sanity check to ensure that HBase can read all the blocks of a table if there are any concerns of metadata inconsistency.
-           It will run the mapreduce all in a single process but it will run faster if you have a MapReduce cluster in place for it to
-           exploit.
+       <title>RowCounter and CellCounter</title>
+       <para><ulink url="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/RowCounter.html">RowCounter</ulink> is a
+       mapreduce job to count all the rows of a table.  This is a good utility to use as a sanity check to ensure that HBase can read
+       all the blocks of a table if there are any concerns of metadata inconsistency. It will run the mapreduce all in a single
+       process but it will run faster if you have a MapReduce cluster in place for it to exploit.
 <programlisting>$ bin/hbase org.apache.hadoop.hbase.mapreduce.RowCounter &lt;tablename&gt; [&lt;column1&gt; &lt;column2&gt;...]
 </programlisting>
        </para>
-       <para>Note:  caching for the input Scan is configured via <code>hbase.client.scanner.caching</code> in the job configuration.
+       <para>Note: caching for the input Scan is configured via <code>hbase.client.scanner.caching</code> in the job configuration.
+       </para>
+       <para>HBase ships another diagnostic mapreduce job called
+         <ulink url="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/CellCounter.html">CellCounter</ulink>. Like
+         RowCounter, it gathers more fine-grained statistics about your table. The statistics gathered by RowCounter are more fine-grained
+         and include:
+         <itemizedlist>
+           <listitem>Total number of rows in the table.</listitem>
+           <listitem>Total number of CFs across all rows.</listitem>
+           <listitem>Total qualifiers across all rows.</listitem>
+           <listitem>Total occurrence of each CF.</listitem>
+           <listitem>Total occurrence of each qualifier.</listitem>
+           <listitem>Total number of versions of each qualifier.</listitem>
+         </itemizedlist>
+       </para>
+       <para>The program allows you to limit the scope of the run. Provide a row regex or prefix to limit the rows to analyze. Use
+         <code>hbase.mapreduce.scan.column.family</code> to specify scanning a single column family.
+         <programlisting>$ bin/hbase org.apache.hadoop.hbase.mapreduce.CellCounter &lt;tablename&gt; &lt;outputDir&gt; [regex or prefix]</programlisting>
        </para>
+       <para>Note: just like RowCounter, caching for the input Scan is configured via <code>hbase.client.scanner.caching</code> in the
+       job configuration. </para>
     </section>
 
     </section>  <!--  tools -->

Modified: hbase/branches/hbase-7290v2/src/site/resources/images/hbase_logo.png
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/src/site/resources/images/hbase_logo.png?rev=1446173&r1=1446172&r2=1446173&view=diff
==============================================================================
Binary files - no diff available.

Modified: hbase/branches/hbase-7290v2/src/site/resources/images/hbase_logo.svg
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/src/site/resources/images/hbase_logo.svg?rev=1446173&r1=1446172&r2=1446173&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/src/site/resources/images/hbase_logo.svg (original)
+++ hbase/branches/hbase-7290v2/src/site/resources/images/hbase_logo.svg Thu Feb 14 13:35:54 2013
@@ -1,41 +1,78 @@
-<?xml version="1.0" encoding="utf-8"?>
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
 <!-- Generator: Adobe Illustrator 15.1.0, SVG Export Plug-In . SVG Version: 6.00 Build 0)  -->
-<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN" "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd">
-<svg version="1.1" id="Layer_1" xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink" x="0px" y="0px"
-	 width="792px" height="612px" viewBox="0 0 792 612" enable-background="new 0 0 792 612" xml:space="preserve">
-<path fill="#BA160C" d="M233.586,371.672h-9.895v-51.583h9.895V371.672L233.586,371.672z M223.691,307.6v-19.465h9.895V307.6
-	H223.691z M223.691,371.672h-9.896v-32.117h-63.584v32.117h-19.466v-83.537h19.466v31.954h55.128h8.457h9.896V371.672
-	L223.691,371.672z M223.691,288.135h-9.896V307.6h9.896V288.135z"/>
-<path fill="#BA160C" d="M335.939,329.334c6.812,4.218,10.219,10.652,10.219,19.303c0,6.272-2,11.571-6.002,15.897
-	c-4.325,4.758-10.165,7.137-17.519,7.137h-28.629v-19.465h28.629c2.812,0,4.218-2.109,4.218-6.327c0-4.216-1.406-6.325-4.218-6.325
-	h-28.629v-19.303h27.17c2.811,0,4.217-2.109,4.217-6.327c0-4.216-1.406-6.326-4.217-6.326h-27.17v-19.464h27.17
-	c7.353,0,13.192,2.379,17.519,7.137c3.892,4.325,5.839,9.625,5.839,15.896C344.536,318.954,341.67,325.009,335.939,329.334z
-	 M294.008,371.672h-52.312v-51.42h19.466h5.259h27.588v19.303h-32.847v12.652h32.847V371.672L294.008,371.672z M294.008,307.599
-	h-32.847v0h-19.466v-19.465h52.312V307.599z"/>
-<path fill="#878888" d="M355.123,266.419v-8.92h14.532v-5.353c0-1.932-0.644-2.899-1.933-2.899h-12.6v-8.919h12.6
-	c3.223,0,5.836,1.164,7.842,3.494c2.007,2.33,3.011,5.104,3.011,8.325v26.463h-8.921v-12.19H355.123L355.123,266.419z
-	 M473.726,278.61h-29.587c-3.469,0-6.417-1.152-8.845-3.458c-2.429-2.304-3.642-5.191-3.642-8.659v-14.049
-	c0-3.47,1.213-6.356,3.642-8.662c2.428-2.304,5.376-3.455,8.845-3.455h29.587v8.919h-29.587c-2.378,0-3.567,1.066-3.567,3.197
-	v14.049c0,2.131,1.189,3.196,3.567,3.196h29.587V278.61L473.726,278.61z M567.609,278.61h-8.996v-14.718h-22.895v14.718h-8.92
-	v-38.282h8.92v14.644h22.895v-14.644h8.996V278.61L567.609,278.61z M661.494,249.247h-31.889v5.725h29.807v8.92h-29.807v5.797
-	h31.814v8.92h-40.735v-38.282h40.809V249.247z M355.123,240.328v8.919h-12.674c-1.239,0-1.858,0.967-1.858,2.899v5.353h5.575h2.435
-	h6.522v8.92h-6.522h-2.435h-5.575v12.19h-8.92v-26.463c0-3.221,1.004-5.996,3.011-8.325c2.006-2.33,4.596-3.494,7.768-3.494H355.123
-	L355.123,240.328z M254.661,266.122v-8.92h13.083c1.288,0,1.933-1.313,1.933-3.939c0-2.676-0.645-4.015-1.933-4.015h-13.083v-8.919
-	h13.083c3.32,0,5.995,1.363,8.028,4.088c1.883,2.478,2.825,5.425,2.825,8.846c0,3.419-0.942,6.342-2.825,8.771
-	c-2.033,2.725-4.708,4.088-8.028,4.088H254.661z M177.649,278.61h-8.92v-12.19h-14.532v-8.92h14.532v-5.353
-	c0-1.932-0.644-2.899-1.932-2.899h-12.6v-8.919h12.6c3.222,0,5.835,1.164,7.842,3.494c2.007,2.33,3.01,5.104,3.01,8.325V278.61
-	L177.649,278.61z M254.661,240.328v8.919h-15.016v7.954h15.016v8.92h-15.016v12.488h-8.92v-38.282H254.661z M154.198,266.419h-7.604
-	h-1.354h-5.575v12.19h-8.92v-26.463c0-3.221,1.004-5.996,3.01-8.325c2.007-2.33,4.597-3.494,7.768-3.494h12.674v8.919h-12.674
-	c-1.239,0-1.858,0.967-1.858,2.899v5.353h5.575h1.354h7.604V266.419z"/>
-<path fill="#BA160C" d="M456.325,371.672H436.86V345.07h-31.094h-0.618v-19.466h0.618h31.094v-11.68
-	c0-4.216-1.406-6.324-4.218-6.324h-27.494v-19.465h27.494c7.03,0,12.733,2.541,17.114,7.623c4.379,5.083,6.569,11.139,6.569,18.167
-	V371.672z M405.148,345.07h-19.547h-12.165v26.602h-19.466v-57.748c0-7.028,2.19-13.083,6.569-18.167
-	c4.379-5.083,10.03-7.623,16.952-7.623h27.656V307.6h-27.656c-2.704,0-4.055,2.108-4.055,6.324v11.68h12.165h19.547V345.07z"/>
-<path fill="#BA160C" d="M564.329,345.88c0,7.03-2.109,13.031-6.327,18.006c-4.541,5.19-10.273,7.786-17.193,7.786h-72.02v-19.465
-	h72.02c2.704,0,4.055-2.109,4.055-6.327c0-4.216-1.352-6.325-4.055-6.325h-52.394c-6.92,0-12.652-2.596-17.193-7.787
-	c-4.327-4.865-6.49-10.813-6.49-17.843c0-7.028,2.218-13.083,6.651-18.167c4.434-5.083,10.112-7.623,17.032-7.623h72.021v19.464
-	h-72.021c-2.703,0-4.055,2.109-4.055,6.326c0,4.109,1.352,6.164,4.055,6.164h52.394c6.92,0,12.652,2.596,17.193,7.787
-	C562.22,332.85,564.329,338.852,564.329,345.88z"/>
-<polygon fill="#BA160C" points="661.494,307.599 591.906,307.599 591.906,320.089 656.952,320.089 656.952,339.555 591.906,339.555 
-	591.906,352.207 661.331,352.207 661.331,371.672 572.44,371.672 572.44,288.135 661.494,288.135 "/>
-</svg>
+
+<svg
+   xmlns:dc="http://purl.org/dc/elements/1.1/"
+   xmlns:cc="http://creativecommons.org/ns#"
+   xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"
+   xmlns:svg="http://www.w3.org/2000/svg"
+   xmlns="http://www.w3.org/2000/svg"
+   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+   version="1.1"
+   id="Layer_1"
+   x="0px"
+   y="0px"
+   width="792px"
+   height="612px"
+   viewBox="0 0 792 612"
+   enable-background="new 0 0 792 612"
+   xml:space="preserve"
+   inkscape:version="0.48.4 r9939"
+   sodipodi:docname="hbase_banner_logo.png"
+   inkscape:export-filename="hbase_logo_filledin.png"
+   inkscape:export-xdpi="90"
+   inkscape:export-ydpi="90"><metadata
+   id="metadata3285"><rdf:RDF><cc:Work
+       rdf:about=""><dc:format>image/svg+xml</dc:format><dc:type
+         rdf:resource="http://purl.org/dc/dcmitype/StillImage" /><dc:title></dc:title></cc:Work></rdf:RDF></metadata><defs
+   id="defs3283" /><sodipodi:namedview
+   pagecolor="#ffffff"
+   bordercolor="#666666"
+   borderopacity="1"
+   objecttolerance="10"
+   gridtolerance="10"
+   guidetolerance="10"
+   inkscape:pageopacity="0"
+   inkscape:pageshadow="2"
+   inkscape:window-width="1131"
+   inkscape:window-height="715"
+   id="namedview3281"
+   showgrid="false"
+   inkscape:zoom="4.3628026"
+   inkscape:cx="328.98554"
+   inkscape:cy="299.51695"
+   inkscape:window-x="752"
+   inkscape:window-y="456"
+   inkscape:window-maximized="0"
+   inkscape:current-layer="Layer_1" />
+<path
+   d="m 233.586,371.672 -9.895,0 0,-51.583 9.895,0 0,51.583 z m -9.77344,-51.59213 -0.12156,-31.94487 9.895,0 -0.0405,31.98539 z m -0.12156,51.59213 -9.896,0 0,-32.117 -63.584,0 0,32.117 -19.466,0 0,-83.537 19.466,0 0,31.954 55.128,0 8.457,0 9.896,0 0,51.583 z m 0,-83.537 -9.896,0 0,31.98539 10.01756,-0.0405 z"
+   id="path3269"
+   inkscape:connector-curvature="0"
+   style="fill:#ba160c"
+   sodipodi:nodetypes="cccccccccccccccccccccccccccccc" />
+<path
+   d="m 335.939,329.334 c 6.812,4.218 10.219,10.652 10.219,19.303 0,6.272 -2,11.571 -6.002,15.897 -4.325,4.758 -10.165,7.137 -17.519,7.137 l -28.629,0 0,-19.465 28.629,0 c 2.812,0 4.218,-2.109 4.218,-6.327 0,-4.216 -1.406,-6.325 -4.218,-6.325 l -28.629,0 0,-19.303 27.17,0 c 2.811,0 4.217,-2.109 4.217,-6.327 0,-4.216 -1.406,-6.326 -4.217,-6.326 l -27.17,0 0,-19.464 27.17,0 c 7.353,0 13.192,2.379 17.519,7.137 3.892,4.325 5.839,9.625 5.839,15.896 0,7.787 -2.866,13.842 -8.597,18.167 z m -41.931,42.338 -52.312,0 0,-51.42 19.466,0 5.259,0 27.588,0 0,19.303 -32.847,0 0,12.652 32.847,0 0,19.465 z m 0,-64.073 -32.847,0 0.0405,13.24974 -19.466,-0.48623 -0.0405,-32.22851 52.312,0 0,19.465 z"
+   id="path3271"
+   inkscape:connector-curvature="0"
+   style="fill:#ba160c"
+   sodipodi:nodetypes="cscsccsssccsssccscsccccccccccccccccccccc" />
+<path
+   d="M355.123,266.419v-8.92h14.532v-5.353c0-1.932-0.644-2.899-1.933-2.899h-12.6v-8.919h12.6  c3.223,0,5.836,1.164,7.842,3.494c2.007,2.33,3.011,5.104,3.011,8.325v26.463h-8.921v-12.19H355.123L355.123,266.419z   M473.726,278.61h-29.587c-3.469,0-6.417-1.152-8.845-3.458c-2.429-2.304-3.642-5.191-3.642-8.659v-14.049  c0-3.47,1.213-6.356,3.642-8.662c2.428-2.304,5.376-3.455,8.845-3.455h29.587v8.919h-29.587c-2.378,0-3.567,1.066-3.567,3.197  v14.049c0,2.131,1.189,3.196,3.567,3.196h29.587V278.61L473.726,278.61z M567.609,278.61h-8.996v-14.718h-22.895v14.718h-8.92  v-38.282h8.92v14.644h22.895v-14.644h8.996V278.61L567.609,278.61z M661.494,249.247h-31.889v5.725h29.807v8.92h-29.807v5.797  h31.814v8.92h-40.735v-38.282h40.809V249.247z M355.123,240.328v8.919h-12.674c-1.239,0-1.858,0.967-1.858,2.899v5.353h5.575h2.435  h6.522v8.92h-6.522h-2.435h-5.575v12.19h-8.92v-26.463c0-3.221,1.004-5.996,3.011-8.325c2.006-2.33,4.596-3.494,7.768-3.494H355.123  L355.123,240.328z M254.661,266.122v-8.92h13.083c1.
 288,0,1.933-1.313,1.933-3.939c0-2.676-0.645-4.015-1.933-4.015h-13.083v-8.919  h13.083c3.32,0,5.995,1.363,8.028,4.088c1.883,2.478,2.825,5.425,2.825,8.846c0,3.419-0.942,6.342-2.825,8.771  c-2.033,2.725-4.708,4.088-8.028,4.088H254.661z M177.649,278.61h-8.92v-12.19h-14.532v-8.92h14.532v-5.353  c0-1.932-0.644-2.899-1.932-2.899h-12.6v-8.919h12.6c3.222,0,5.835,1.164,7.842,3.494c2.007,2.33,3.01,5.104,3.01,8.325V278.61  L177.649,278.61z M254.661,240.328v8.919h-15.016v7.954h15.016v8.92h-15.016v12.488h-8.92v-38.282H254.661z M154.198,266.419h-7.604  h-1.354h-5.575v12.19h-8.92v-26.463c0-3.221,1.004-5.996,3.01-8.325c2.007-2.33,4.597-3.494,7.768-3.494h12.674v8.919h-12.674  c-1.239,0-1.858,0.967-1.858,2.899v5.353h5.575h1.354h7.604V266.419z"
+   id="path3273"
+   style="fill:#666666"
+   fill="#878888" />
+<path
+   fill="#BA160C"
+   d="M456.325,371.672H436.86V345.07h-31.094h-0.618v-19.466h0.618h31.094v-11.68  c0-4.216-1.406-6.324-4.218-6.324h-27.494v-19.465h27.494c7.03,0,12.733,2.541,17.114,7.623c4.379,5.083,6.569,11.139,6.569,18.167  V371.672z M405.148,345.07h-19.547h-12.165v26.602h-19.466v-57.748c0-7.028,2.19-13.083,6.569-18.167  c4.379-5.083,10.03-7.623,16.952-7.623h27.656V307.6h-27.656c-2.704,0-4.055,2.108-4.055,6.324v11.68h12.165h19.547V345.07z"
+   id="path3275" />
+<path
+   fill="#BA160C"
+   d="M564.329,345.88c0,7.03-2.109,13.031-6.327,18.006c-4.541,5.19-10.273,7.786-17.193,7.786h-72.02v-19.465  h72.02c2.704,0,4.055-2.109,4.055-6.327c0-4.216-1.352-6.325-4.055-6.325h-52.394c-6.92,0-12.652-2.596-17.193-7.787  c-4.327-4.865-6.49-10.813-6.49-17.843c0-7.028,2.218-13.083,6.651-18.167c4.434-5.083,10.112-7.623,17.032-7.623h72.021v19.464  h-72.021c-2.703,0-4.055,2.109-4.055,6.326c0,4.109,1.352,6.164,4.055,6.164h52.394c6.92,0,12.652,2.596,17.193,7.787  C562.22,332.85,564.329,338.852,564.329,345.88z"
+   id="path3277" />
+<polygon
+   fill="#BA160C"
+   points="661.494,307.599 591.906,307.599 591.906,320.089 656.952,320.089 656.952,339.555 591.906,339.555   591.906,352.207 661.331,352.207 661.331,371.672 572.44,371.672 572.44,288.135 661.494,288.135 "
+   id="polygon3279" />
+</svg>
\ No newline at end of file

Modified: hbase/branches/hbase-7290v2/src/site/xdoc/index.xml
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/src/site/xdoc/index.xml?rev=1446173&r1=1446172&r2=1446173&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/src/site/xdoc/index.xml (original)
+++ hbase/branches/hbase-7290v2/src/site/xdoc/index.xml Thu Feb 14 13:35:54 2013
@@ -65,7 +65,9 @@ Apache HBase is an open-source, distribu
    </p>
  </section>
      <section name="News">
+         <p>April 11th, 2013 <a href="http://www.meetup.com/hbaseusergroup/events/103587852/">HBase Meetup at AdRoll</a> in San Francisco</p>
          <p>February 28th, 2013 <a href="http://www.meetup.com/hbaseusergroup/events/96584102/">HBase Meetup at Intel Mission Campus</a></p>
+         <p>February 19th, 2013 <a href="http://www.meetup.com/hackathon/events/103633042/">Developers PowWow</a> at HortonWorks' new digs</p>
          <p>January 23rd, 2013 <a href="http://www.meetup.com/hbaseusergroup/events/91381312/">HBase Meetup at WibiData World HQ!</a></p>
             <p>December 4th, 2012 <a href="http://www.meetup.com/hackathon/events/90536432/">0.96 Bug Squashing and Testing Hackathon</a> at Cloudera, SF.</p>
             <p>October 29th, 2012 <a href="http://www.meetup.com/hbaseusergroup/events/82791572/">HBase User Group Meetup</a> at Wize Commerce in San Mateo.</p>