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/20 18:48:20 UTC

svn commit: r1448318 [2/2] - in /hbase/branches/hbase-7290v2: ./ hbase-common/src/main/java/org/apache/hadoop/hbase/util/ hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/ hbase-it/src/test/java/org/apache/hadoop/hbase/ hbase-se...

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java?rev=1448318&r1=1448317&r2=1448318&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java Wed Feb 20 17:48:19 2013
@@ -115,8 +115,7 @@ public class TestFromClientSide3 {
     HConnection conn = HConnectionManager.getConnection(TEST_UTIL
         .getConfiguration());
     HRegionLocation loc = table.getRegionLocation(row, true);
-    AdminProtocol server = conn.getAdmin(loc.getHostname(), loc
-        .getPort());
+    AdminProtocol server = conn.getAdmin(loc.getServerName());
     byte[] regName = loc.getRegionInfo().getRegionName();
 
     for (int i = 0; i < nFlushes; i++) {
@@ -163,8 +162,7 @@ public class TestFromClientSide3 {
     // Verify we have multiple store files.
     HRegionLocation loc = hTable.getRegionLocation(row, true);
     byte[] regionName = loc.getRegionInfo().getRegionName();
-    AdminProtocol server = connection.getAdmin(
-      loc.getHostname(), loc.getPort());
+    AdminProtocol server = connection.getAdmin(loc.getServerName());
     assertTrue(ProtobufUtil.getStoreFiles(
       server, regionName, FAMILY).size() > 1);
 
@@ -177,7 +175,7 @@ public class TestFromClientSide3 {
       loc = hTable.getRegionLocation(row, true);
       if (!loc.getRegionInfo().isOffline()) {
         regionName = loc.getRegionInfo().getRegionName();
-        server = connection.getAdmin(loc.getHostname(), loc.getPort());
+        server = connection.getAdmin(loc.getServerName());
         if (ProtobufUtil.getStoreFiles(
             server, regionName, FAMILY).size() <= 1) {
           break;
@@ -211,7 +209,7 @@ public class TestFromClientSide3 {
     Thread.sleep(10 * 1000);
     loc = hTable.getRegionLocation(row, true);
     regionName = loc.getRegionInfo().getRegionName();
-    server = connection.getAdmin(loc.getHostname(), loc.getPort());
+    server = connection.getAdmin(loc.getServerName());
     int sfCount = ProtobufUtil.getStoreFiles(
       server, regionName, FAMILY).size();
     assertTrue(sfCount > 1);
@@ -236,8 +234,7 @@ public class TestFromClientSide3 {
       loc = hTable.getRegionLocation(row, true);
       regionName = loc.getRegionInfo().getRegionName();
       try {
-        server = connection.getAdmin(loc.getHostname(), loc
-            .getPort());
+        server = connection.getAdmin(loc.getServerName());
         if (ProtobufUtil.getStoreFiles(
             server, regionName, FAMILY).size() < sfCount) {
           break;

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java?rev=1448318&r1=1448317&r2=1448318&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHCM.java Wed Feb 20 17:48:19 2013
@@ -151,8 +151,8 @@ public class TestHCM {
 
     final int nextPort = conn.getCachedLocation(TABLE_NAME, ROW).getPort() + 1;
     HRegionLocation loc = conn.getCachedLocation(TABLE_NAME, ROW);
-    conn.updateCachedLocation(loc.getRegionInfo(), loc, "127.0.0.1", nextPort,
-      HConstants.LATEST_TIMESTAMP);
+    conn.updateCachedLocation(loc.getRegionInfo(), loc, new ServerName("127.0.0.1", nextPort,
+      HConstants.LATEST_TIMESTAMP), HConstants.LATEST_TIMESTAMP);
     Assert.assertEquals(conn.getCachedLocation(TABLE_NAME, ROW).getPort(), nextPort);
 
     conn.forceDeleteCachedLocation(TABLE_NAME.clone(), ROW.clone());
@@ -349,34 +349,34 @@ public class TestHCM {
     HRegionLocation location = conn.getCachedLocation(TABLE_NAME2, ROW);
     assertNotNull(location);
 
-    HRegionLocation anySource = new HRegionLocation(location.getRegionInfo(),
-        location.getHostname(), location.getPort() - 1);
+    HRegionLocation anySource = new HRegionLocation(location.getRegionInfo(), new ServerName(
+        location.getHostname(), location.getPort() - 1, 0L), HConstants.NO_SEQNUM);
 
     // Same server as already in cache reporting - overwrites any value despite seqNum.
     int nextPort = location.getPort() + 1;
     conn.updateCachedLocation(location.getRegionInfo(), location,
-        "127.0.0.1", nextPort, location.getSeqNum() - 1);
+        new ServerName("127.0.0.1", nextPort, 0), location.getSeqNum() - 1);
     location = conn.getCachedLocation(TABLE_NAME2, ROW);
     Assert.assertEquals(nextPort, location.getPort());
 
     // No source specified - same.
     nextPort = location.getPort() + 1;
     conn.updateCachedLocation(location.getRegionInfo(), location,
-        "127.0.0.1", nextPort, location.getSeqNum() - 1);
+        new ServerName("127.0.0.1", nextPort, 0), location.getSeqNum() - 1);
     location = conn.getCachedLocation(TABLE_NAME2, ROW);
     Assert.assertEquals(nextPort, location.getPort());
 
     // Higher seqNum - overwrites lower seqNum.
     nextPort = location.getPort() + 1;
     conn.updateCachedLocation(location.getRegionInfo(), anySource,
-        "127.0.0.1", nextPort, location.getSeqNum() + 1);
+        new ServerName("127.0.0.1", nextPort, 0), location.getSeqNum() + 1);
     location = conn.getCachedLocation(TABLE_NAME2, ROW);
     Assert.assertEquals(nextPort, location.getPort());
 
     // Lower seqNum - does not overwrite higher seqNum.
     nextPort = location.getPort() + 1;
     conn.updateCachedLocation(location.getRegionInfo(), anySource,
-        "127.0.0.1", nextPort, location.getSeqNum() - 1);
+        new ServerName("127.0.0.1", nextPort, 0), location.getSeqNum() - 1);
     location = conn.getCachedLocation(TABLE_NAME2, ROW);
     Assert.assertEquals(nextPort - 1, location.getPort());
   }

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java?rev=1448318&r1=1448317&r2=1448318&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java Wed Feb 20 17:48:19 2013
@@ -72,7 +72,7 @@ public class TestMultiParallel {
     UTIL.startMiniCluster(slaves);
     HTable t = UTIL.createTable(Bytes.toBytes(TEST_TABLE), Bytes.toBytes(FAMILY));
     UTIL.createMultiRegions(t, Bytes.toBytes(FAMILY));
-    UTIL.waitTableAvailable(Bytes.toBytes(TEST_TABLE), 15 * 1000);
+    UTIL.waitTableEnabled(Bytes.toBytes(TEST_TABLE));
     t.close();
   }
 

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java?rev=1448318&r1=1448317&r2=1448318&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java Wed Feb 20 17:48:19 2013
@@ -637,7 +637,7 @@ public class TestClassLoading {
 
   private void waitForTable(byte[] name) throws InterruptedException, IOException {
     // First wait until all regions are online
-    TEST_UTIL.waitTableEnabled(name, 5000);
+    TEST_UTIL.waitTableEnabled(name);
     // Now wait a bit longer for the coprocessor hosts to load the CPs
     Thread.sleep(1000);
   }

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java?rev=1448318&r1=1448317&r2=1448318&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java Wed Feb 20 17:48:19 2013
@@ -439,7 +439,7 @@ public class TestHFileOutputFormat  {
         LOG.info("Waiting for table to disable");
       }
       admin.enableTable(TABLE_NAME);
-      util.waitTableAvailable(TABLE_NAME, 30000);
+      util.waitTableAvailable(TABLE_NAME);
       assertEquals("Data should remain after reopening of regions",
           tableDigestBefore, util.checksumRows(table));
     } finally {

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java?rev=1448318&r1=1448317&r2=1448318&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java Wed Feb 20 17:48:19 2013
@@ -153,7 +153,7 @@ public class TestLoadIncrementalHFiles {
     admin.createTable(htd, SPLIT_KEYS);
 
     HTable table = new HTable(util.getConfiguration(), TABLE);
-    util.waitTableAvailable(TABLE, 30000);
+    util.waitTableEnabled(TABLE);
     LoadIncrementalHFiles loader = new LoadIncrementalHFiles(util.getConfiguration(), useSecure);
     loader.doBulkLoad(dir, table);
 
@@ -184,7 +184,7 @@ public class TestLoadIncrementalHFiles {
     admin.createTable(htd, SPLIT_KEYS);
 
     HTable table = new HTable(util.getConfiguration(), TABLE);
-    util.waitTableAvailable(TABLE, 30000);
+    util.waitTableEnabled(TABLE);
     LoadIncrementalHFiles loader = new LoadIncrementalHFiles(
       util.getConfiguration());
 

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java?rev=1448318&r1=1448317&r2=1448318&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java Wed Feb 20 17:48:19 2013
@@ -37,10 +37,12 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.LargeTests;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.client.ClientProtocol;
 import org.apache.hadoop.hbase.client.HConnection;
@@ -269,7 +271,7 @@ public class TestLoadIncrementalHFilesSp
     Mockito.doNothing().when(c).close();
     // Make it so we return a particular location when asked.
     final HRegionLocation loc = new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO,
-        "example.org", 1234);
+        new ServerName("example.org", 1234, 0), HConstants.NO_SEQNUM);
     Mockito.when(c.getRegionLocation((byte[]) Mockito.any(),
         (byte[]) Mockito.any(), Mockito.anyBoolean())).
       thenReturn(loc);
@@ -278,7 +280,7 @@ public class TestLoadIncrementalHFilesSp
     ClientProtocol hri = Mockito.mock(ClientProtocol.class);
     Mockito.when(hri.bulkLoadHFile((RpcController)Mockito.any(), (BulkLoadHFileRequest)Mockito.any())).
       thenThrow(new ServiceException(new IOException("injecting bulk load error")));
-    Mockito.when(c.getClient(Mockito.anyString(), Mockito.anyInt())).
+    Mockito.when(c.getClient(Mockito.any(ServerName.class))).
       thenReturn(hri);
     return c;
   }

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestSecureLoadIncrementalHFiles.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestSecureLoadIncrementalHFiles.java?rev=1448318&r1=1448317&r2=1448318&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestSecureLoadIncrementalHFiles.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestSecureLoadIncrementalHFiles.java Wed Feb 20 17:48:19 2013
@@ -49,7 +49,7 @@ public class TestSecureLoadIncrementalHF
     util.startMiniCluster();
 
     // Wait for the ACL table to become available
-    util.waitTableAvailable(AccessControlLists.ACL_TABLE_NAME, 5000);
+    util.waitTableEnabled(AccessControlLists.ACL_TABLE_NAME);
   }
 
 }

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestSecureLoadIncrementalHFilesSplitRecovery.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestSecureLoadIncrementalHFilesSplitRecovery.java?rev=1448318&r1=1448317&r2=1448318&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestSecureLoadIncrementalHFilesSplitRecovery.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestSecureLoadIncrementalHFilesSplitRecovery.java Wed Feb 20 17:48:19 2013
@@ -54,7 +54,7 @@ public class TestSecureLoadIncrementalHF
     util.startMiniCluster();
 
     // Wait for the ACL table to become available
-    util.waitTableAvailable(AccessControlLists.ACL_TABLE_NAME, 5000);
+    util.waitTableEnabled(AccessControlLists.ACL_TABLE_NAME);
   }
 
   //Disabling this test as it does not work in secure mode

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java?rev=1448318&r1=1448317&r2=1448318&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java Wed Feb 20 17:48:19 2013
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.client.Cl
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
@@ -499,4 +500,9 @@ class MockRegionServer implements AdminP
     // TODO Auto-generated method stub
     return null;
   }
+
+  @Override
+  public ExecutorService getExecutorService() {
+    return null;
+  }
 }

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java?rev=1448318&r1=1448317&r2=1448318&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartCluster.java Wed Feb 20 17:48:19 2013
@@ -97,7 +97,7 @@ public class TestRestartCluster {
       UTIL.createTable(TABLE, FAMILY);
     }
     for(byte [] TABLE : TABLES) {
-      UTIL.waitTableAvailable(TABLE, 30000);
+      UTIL.waitTableEnabled(TABLE);
     }
 
     List<HRegionInfo> allRegions =
@@ -128,7 +128,7 @@ public class TestRestartCluster {
       } catch(TableExistsException tee) {
         LOG.info("Table already exists as expected");
       }
-      UTIL.waitTableAvailable(TABLE, 30000);
+      UTIL.waitTableAvailable(TABLE);
     }
   }
 

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java?rev=1448318&r1=1448317&r2=1448318&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java Wed Feb 20 17:48:19 2013
@@ -589,10 +589,9 @@ public class TestCompaction extends HBas
     HStore store = (HStore) r.getStore(COLUMN_FAMILY);
 
     Collection<StoreFile> storeFiles = store.getStorefiles();
-    Compactor tool = store.compactionPolicy.getCompactor();
+    Compactor tool = store.compactor;
 
-    List<Path> newFiles =
-      tool.compact(storeFiles, false);
+    List<Path> newFiles = tool.compact(storeFiles, false);
 
     // Now lets corrupt the compacted file.
     FileSystem fs = FileSystem.get(conf);

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java?rev=1448318&r1=1448317&r2=1448318&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java Wed Feb 20 17:48:19 2013
@@ -288,7 +288,7 @@ public class TestDefaultCompactSelection
     compactEquals(sfCreate(100,50,23,12,12), true, 23, 12, 12);
     conf.setLong(HConstants.MAJOR_COMPACTION_PERIOD, 1);
     conf.setFloat("hbase.hregion.majorcompaction.jitter", 0);
-    store.compactionPolicy.updateConfiguration();
+    store.compactionPolicy.setConf(conf);
     try {
       // trigger an aged major compaction
       compactEquals(sfCreate(50,25,12,12), 50, 25, 12, 12);
@@ -321,7 +321,7 @@ public class TestDefaultCompactSelection
      */
     // set an off-peak compaction threshold
     this.conf.setFloat("hbase.hstore.compaction.ratio.offpeak", 5.0F);
-    store.compactionPolicy.updateConfiguration();
+    store.compactionPolicy.setConf(this.conf);
     // Test with and without the flag.
     compactEquals(sfCreate(999, 50, 12, 12, 1), false, true, 50, 12, 12, 1);
     compactEquals(sfCreate(999, 50, 12, 12, 1), 12, 12, 1);

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java?rev=1448318&r1=1448317&r2=1448318&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java Wed Feb 20 17:48:19 2013
@@ -165,8 +165,7 @@ public class TestHRegionServerBulkLoad {
           public Void call() throws Exception {
             LOG.debug("compacting " + location + " for row "
                 + Bytes.toStringBinary(row));
-            AdminProtocol server = connection.getAdmin(
-              location.getHostname(), location.getPort());
+            AdminProtocol server = connection.getAdmin(location.getServerName());
             CompactRegionRequest request =
               RequestConverter.buildCompactRegionRequest(
                 location.getRegionInfo().getRegionName(), true, null);

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java?rev=1448318&r1=1448317&r2=1448318&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java Wed Feb 20 17:48:19 2013
@@ -568,6 +568,7 @@ public class TestStoreScanner extends Te
       assertEquals(kvs[14], results.get(5));
       assertEquals(kvs[15], results.get(6));
       assertEquals(7, results.size());
+      scanner.close();
     }finally{
     EnvironmentEdgeManagerTestHelper.reset();
     }

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/PerfTestCompactionPolicies.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/PerfTestCompactionPolicies.java?rev=1448318&r1=1448317&r2=1448318&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/PerfTestCompactionPolicies.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/PerfTestCompactionPolicies.java Wed Feb 20 17:48:19 2013
@@ -26,8 +26,11 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.KeyValue.KVComparator;
 import org.apache.hadoop.hbase.regionserver.HStore;
+import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -61,11 +64,11 @@ public class PerfTestCompactionPolicies 
   @Parameterized.Parameters
   public static Collection<Object[]> data() {
     return Arrays.asList(new Object[][] {
-        {new DefaultCompactionPolicy(), 3, 2, 1.2f},
-        {new DefaultCompactionPolicy(), 4, 2, 1.2f},
-        {new DefaultCompactionPolicy(), 5, 2, 1.2f},
-        {new DefaultCompactionPolicy(), 4, 2, 1.3f},
-        {new DefaultCompactionPolicy(), 4, 2, 1.4f},
+        {DefaultCompactionPolicy.class, 3, 2, 1.2f},
+        {DefaultCompactionPolicy.class, 4, 2, 1.2f},
+        {DefaultCompactionPolicy.class, 5, 2, 1.2f},
+        {DefaultCompactionPolicy.class, 4, 2, 1.3f},
+        {DefaultCompactionPolicy.class, 4, 2, 1.4f},
 
     });
   }
@@ -77,7 +80,8 @@ public class PerfTestCompactionPolicies 
    * @param min The min number of files to compact
    * @param ratio The ratio that files must be under to be compacted.
    */
-  public PerfTestCompactionPolicies(CompactionPolicy cp, int max, int min, float ratio) {
+  public PerfTestCompactionPolicies(Class<? extends CompactionPolicy> cpClass,
+      int max, int min, float ratio) {
     this.max = max;
     this.min = min;
     this.ratio = ratio;
@@ -86,11 +90,7 @@ public class PerfTestCompactionPolicies 
     org.apache.log4j.Logger.getLogger(CompactionConfiguration.class).
         setLevel(org.apache.log4j.Level.ERROR);
 
-    org.apache.log4j.Logger.getLogger(cp.getClass()).
-        setLevel(org.apache.log4j.Level.ERROR);
-
-    this.cp = cp;
-
+    org.apache.log4j.Logger.getLogger(cpClass).setLevel(org.apache.log4j.Level.ERROR);
 
     Configuration configuration = HBaseConfiguration.create();
 
@@ -99,11 +99,10 @@ public class PerfTestCompactionPolicies 
     configuration.setInt("hbase.hstore.compaction.min", min);
     configuration.setFloat("hbase.hstore.compaction.ratio", ratio);
 
-    cp.store = createMockStore();
-
-    //Now set the conf.
-    cp.setConf(configuration);
-
+    HStore store = createMockStore();
+    this.cp = ReflectionUtils.instantiateWithCustomCtor(cpClass.getName(),
+        new Class[] { Configuration.class, StoreConfigInformation.class },
+        new Object[] { configuration, store });
 
     //Used for making paths
     random = new Random(42);

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java?rev=1448318&r1=1448317&r2=1448318&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessControlFilter.java Wed Feb 20 17:48:19 2013
@@ -74,7 +74,7 @@ public class TestAccessControlFilter {
     conf.set("hbase.superuser", conf.get("hbase.superuser", "") +
         String.format(",%s.hfs.0,%s.hfs.1,%s.hfs.2", baseuser, baseuser, baseuser));
     TEST_UTIL.startMiniCluster();
-    TEST_UTIL.waitTableAvailable(AccessControlLists.ACL_TABLE_NAME, 5000);
+    TEST_UTIL.waitTableEnabled(AccessControlLists.ACL_TABLE_NAME);
 
     ADMIN = User.createUserForTesting(conf, "admin", new String[]{"supergroup"});
     READER = User.createUserForTesting(conf, "reader", new String[0]);

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java?rev=1448318&r1=1448317&r2=1448318&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java Wed Feb 20 17:48:19 2013
@@ -140,7 +140,7 @@ public class TestAccessController {
       Coprocessor.PRIORITY_HIGHEST, 1, conf);
 
     // Wait for the ACL table to become available
-    TEST_UTIL.waitTableAvailable(AccessControlLists.ACL_TABLE_NAME, 5000);
+    TEST_UTIL.waitTableEnabled(AccessControlLists.ACL_TABLE_NAME);
 
     // create a set of test users
     SUPERUSER = User.createUserForTesting(conf, "admin", new String[] { "supergroup" });
@@ -824,7 +824,7 @@ public class TestAccessController {
 
       HTable table = new HTable(conf, tableName);
       try {
-        TEST_UTIL.waitTableAvailable(tableName, 30000);
+        TEST_UTIL.waitTableEnabled(tableName);
         LoadIncrementalHFiles loader = new LoadIncrementalHFiles(conf);
         loader.doBulkLoad(loadPath, table);
       } finally {

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java?rev=1448318&r1=1448317&r2=1448318&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestTablePermissions.java Wed Feb 20 17:48:19 2013
@@ -93,7 +93,7 @@ public class TestTablePermissions {
     UTIL.startMiniCluster();
 
     // Wait for the ACL table to become available
-    UTIL.waitTableAvailable(AccessControlLists.ACL_TABLE_NAME, 5000);
+    UTIL.waitTableEnabled(AccessControlLists.ACL_TABLE_NAME);
 
     ZKW = new ZooKeeperWatcher(UTIL.getConfiguration(),
       "TestTablePermissions", ABORTABLE);

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java?rev=1448318&r1=1448317&r2=1448318&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java Wed Feb 20 17:48:19 2013
@@ -28,6 +28,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.catalog.CatalogTracker;
+import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.regionserver.CompactionRequestor;
@@ -172,4 +173,9 @@ public class MockRegionServerServices im
     // TODO Auto-generated method stub
     return null;
   }
+
+  @Override
+  public ExecutorService getExecutorService() {
+    return null;
+  }
 }

Modified: hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java?rev=1448318&r1=1448317&r2=1448318&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java (original)
+++ hbase/branches/hbase-7290v2/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java Wed Feb 20 17:48:19 2013
@@ -19,7 +19,10 @@ package org.apache.hadoop.hbase.util;
 // this is deliberately not in the o.a.h.h.regionserver package
 // in order to make sure all required classes/method are available
 
+import static org.junit.Assert.assertEquals;
+
 import java.io.IOException;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -59,7 +62,12 @@ import org.junit.experimental.categories
 
 import static org.junit.Assert.*;
 
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
 @Category(MediumTests.class)
+@RunWith(Parameterized.class)
 public class TestCoprocessorScanPolicy {
   final Log LOG = LogFactory.getLog(getClass());
   protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@@ -67,7 +75,6 @@ public class TestCoprocessorScanPolicy {
   private static final byte[] Q = Bytes.toBytes("qual");
   private static final byte[] R = Bytes.toBytes("row");
 
-
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     Configuration conf = TEST_UTIL.getConfiguration();
@@ -81,9 +88,22 @@ public class TestCoprocessorScanPolicy {
     TEST_UTIL.shutdownMiniCluster();
   }
 
+  @Parameters
+  public static Collection<Object[]> parameters() {
+    return HBaseTestingUtility.BOOLEAN_PARAMETERIZED;
+  }
+
+  public TestCoprocessorScanPolicy(boolean parallelSeekEnable) {
+    TEST_UTIL.getMiniHBaseCluster().getConf()
+        .setBoolean(StoreScanner.STORESCANNER_PARALLEL_SEEK_ENABLE, parallelSeekEnable);
+  }
+
   @Test
   public void testBaseCases() throws Exception {
     byte[] tableName = Bytes.toBytes("baseCases");
+    if (TEST_UTIL.getHBaseAdmin().tableExists(tableName)) {
+      TEST_UTIL.deleteTable(tableName);
+    }
     HTable t = TEST_UTIL.createTable(tableName, F, 1);
     // set the version override to 2
     Put p = new Put(R);
@@ -130,6 +150,9 @@ public class TestCoprocessorScanPolicy {
   @Test
   public void testTTL() throws Exception {
     byte[] tableName = Bytes.toBytes("testTTL");
+    if (TEST_UTIL.getHBaseAdmin().tableExists(tableName)) {
+      TEST_UTIL.deleteTable(tableName);
+    }
     HTableDescriptor desc = new HTableDescriptor(tableName);
     HColumnDescriptor hcd = new HColumnDescriptor(F)
     .setMaxVersions(10)

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=1448318&r1=1448317&r2=1448318&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 Wed Feb 20 17:48:19 2013
@@ -505,7 +505,7 @@ public class TestHBaseFsck {
     HConnection connection = admin.getConnection();
     for (ServerName hsi : regionServers) {
       AdminProtocol server =
-        connection.getAdmin(hsi.getHostname(), hsi.getPort());
+        connection.getAdmin(hsi);
 
       // list all online regions from this region server
       List<HRegionInfo> regions = ProtobufUtil.getOnlineRegions(server);

Modified: hbase/branches/hbase-7290v2/src/docbkx/configuration.xml
URL: http://svn.apache.org/viewvc/hbase/branches/hbase-7290v2/src/docbkx/configuration.xml?rev=1448318&r1=1448317&r2=1448318&view=diff
==============================================================================
--- hbase/branches/hbase-7290v2/src/docbkx/configuration.xml (original)
+++ hbase/branches/hbase-7290v2/src/docbkx/configuration.xml Wed Feb 20 17:48:19 2013
@@ -221,18 +221,48 @@ to ensure well-formedness of your docume
         xlink:href="http://hadoop.apache.org">Hadoop</link><indexterm>
             <primary>Hadoop</primary>
           </indexterm></title>
-         <note><title>Please read all of this section</title>
-         <para>Please read this section to the end.  Up front we
-         wade through the weeds of Hadoop versions.  Later we talk of what you must do in HBase
-         to make it work w/ a particular Hadoop version.</para>
-         </note>
-
-          <para>
-        HBase will lose data unless it is running on an HDFS that has a durable
-        <code>sync</code> implementation. Hadoop 0.20.2, Hadoop 0.20.203.0, and Hadoop 0.20.204.0
-	DO NOT have this attribute.
-        Currently only Hadoop versions 0.20.205.x or any release in excess of this
-        version -- this includes hadoop 1.0.0 -- have a working, durable sync
+         <para>Selecting a Hadoop version is critical for your HBase deployment. Below table shows some information about what versions of Hadoop are supported by various HBase versions. Based on the version of HBase, you should select the most appropriate version of Hadoop. We are not in the Hadoop distro selection business. You can use Hadoop distributions from Apache, or learn about vendor distributions of Hadoop at <link xlink:href="http://wiki.apache.org/hadoop/Distributions%20and%20Commercial%20Support"/></para>
+         <para>
+	     <table>
+		 <title>Hadoop version support matrix</title>
+		 <tgroup cols='4' align='left' colsep='1' rowsep='1'><colspec colname='c1' align='left'/><colspec colname='c2' align='center'/><colspec colname='c3' align='center'/><colspec colname='c4' align='center'/>
+         <thead>
+	     <row><entry>               </entry><entry>HBase-0.92.x</entry><entry>HBase-0.94.x</entry><entry>HBase-0.96</entry></row>
+	     </thead><tbody>
+         <row><entry>Hadoop-0.20.205</entry><entry>S</entry>          <entry>S</entry>           <entry>X</entry></row>
+         <row><entry>Hadoop-0.22.x  </entry><entry>S</entry>          <entry>S</entry>           <entry>X</entry></row>
+         <row><entry>Hadoop-1.0.x   </entry><entry>S</entry>          <entry>S</entry>           <entry>S</entry></row>
+         <row><entry>Hadoop-1.1.x   </entry><entry>NT</entry>         <entry>S</entry>           <entry>S</entry></row>
+         <row><entry>Hadoop-0.23.x  </entry><entry>X</entry>          <entry>S</entry>           <entry>NT</entry></row>
+         <row><entry>Hadoop-2.x     </entry><entry>X</entry>          <entry>S</entry>           <entry>S</entry></row>
+		 </tbody></tgroup></table>
+
+        Where
+		<simplelist type='vert' columns='1'>
+		<member>S = supported and tested,</member>
+		<member>X = not supported,</member>
+		<member>NT = it should run, but not tested enough.</member>
+		</simplelist>
+        </para>
+        <para>
+	Because HBase depends on Hadoop, it bundles an instance of the Hadoop jar under its <filename>lib</filename> directory. The bundled jar is ONLY for use in standalone mode. In distributed mode, it is <emphasis>critical</emphasis> that the version of Hadoop that is out on your cluster match what is under HBase. Replace the hadoop jar found in the HBase lib directory with the hadoop jar you are running on your cluster to avoid version mismatch issues. Make sure you replace the jar in HBase everywhere on your cluster. Hadoop version mismatch issues have various manifestations but often all looks like its hung up. 
+    </para>
+    <section xml:id="hadoop.hbase-0.94">
+	<title>Apache HBase 0.92 and 0.94</title>
+	<para>HBase 0.92 and 0.94 versions can work with Hadoop versions, 0.20.205, 0.22.x, 1.0.x, and 1.1.x. HBase-0.94 can additionally work with Hadoop-0.23.x and 2.x, but you may have to recompile the code using the specific maven profile (see top level pom.xml)</para>
+   </section>
+
+    <section xml:id="hadoop.hbase-0.96">
+	<title>Apache HBase 0.96</title>
+	<para>Apache HBase 0.96.0 requires Apache Hadoop 1.x at a minimum, and it can run equally well on hadoop-2.0.
+	As of Apache HBase 0.96.x, Apache Hadoop 1.0.x at least is required. We will no longer run properly on older Hadoops such as 0.20.205 or branch-0.20-append. Do not move to Apache HBase 0.96.x if you cannot upgrade your Hadoop<footnote><para>See <link xlink:href="http://search-hadoop.com/m/7vFVx4EsUb2">HBase, mail # dev - DISCUSS: Have hbase require at least hadoop 1.0.0 in hbase 0.96.0?</link></para></footnote>.</para>
+   </section>
+
+    <section xml:id="hadoop.older.versions">
+	<title>Hadoop versions 0.20.x - 1.x</title>
+	<para>
+     HBase will lose data unless it is running on an HDFS that has a durable
+        <code>sync</code> implementation.  DO NOT use Hadoop 0.20.2, Hadoop 0.20.203.0, and Hadoop 0.20.204.0 which DO NOT have this attribute. Currently only Hadoop versions 0.20.205.x or any release in excess of this version -- this includes hadoop-1.0.0 -- have a working, durable sync
           <footnote>
           <para>The Cloudera blog post <link xlink:href="http://www.cloudera.com/blog/2012/01/an-update-on-apache-hadoop-1-0/">An update on Apache Hadoop 1.0</link>
           by Charles Zedlweski has a nice exposition on how all the Hadoop versions relate.
@@ -252,73 +282,13 @@ to ensure well-formedness of your docume
         </programlisting>
         You will have to restart your cluster after making this edit.  Ignore the chicken-little
         comment you'll find in the <filename>hdfs-default.xml</filename> in the
-        description for the <varname>dfs.support.append</varname> configuration; it says it is not enabled because there
-        are <quote>... bugs in the 'append code' and is not supported in any production
-        cluster.</quote>. This comment is stale, from another era, and while I'm sure there
-        are bugs, the sync/append code has been running
-        in production at large scale deploys and is on
-        by default in the offerings of hadoop by commercial vendors
-        <footnote><para>Until recently only the
-        <link xlink:href="http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-append/">branch-0.20-append</link>
-        branch had a working sync but no official release was ever made from this branch.
-        You had to build it yourself. Michael Noll wrote a detailed blog,
-        <link xlink:href="http://www.michael-noll.com/blog/2011/04/14/building-an-hadoop-0-20-x-version-for-hbase-0-90-2/">Building
-        an Hadoop 0.20.x version for Apache HBase 0.90.2</link>, on how to build an
-    Hadoop from branch-0.20-append.  Recommended.</para></footnote>
-    <footnote><para>Praveen Kumar has written
-            a complimentary article,
-            <link xlink:href="http://praveen.kumar.in/2011/06/20/building-hadoop-and-hbase-for-hbase-maven-application-development/">Building Hadoop and HBase for HBase Maven application development</link>.
-</para></footnote><footnote>Cloudera have <varname>dfs.support.append</varname> set to true by default.</footnote>.
-        Please use the most up-to-date Hadoop possible.</para>
-   <note><title>Apache HBase 0.96.0 requires Apache Hadoop 1.0.0 at a minimum</title>
-   <para>As of Apache HBase 0.96.x, Apache Hadoop 1.0.x at least is required.  We will no
-   longer run properly on older Hadoops such as <filename>0.20.205</filename> or <filename>branch-0.20-append</filename>.
-   Do not move to Apache HBase 0.96.x if you cannot upgrade your Hadoop<footnote><para>See <link xlink:href="http://search-hadoop.com/m/7vFVx4EsUb2">HBase, mail # dev - DISCUSS: Have hbase require at least hadoop 1.0.0 in hbase 0.96.0?</link></para></footnote>.</para>
-   <para>Apache HBase 0.96.0 runs on Apache Hadoop 2.0.
-   </para>
-   </note>
-
-<para>Or use the
-    <link xlink:href="http://www.cloudera.com/">Cloudera</link> or
-    <link xlink:href="http://www.mapr.com/">MapR</link> distributions.
-    Cloudera' <link xlink:href="http://archive.cloudera.com/docs/">CDH3</link>
-    is Apache Hadoop 0.20.x plus patches including all of the
-    <link xlink:href="http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-append/">branch-0.20-append</link>
-    additions needed to add a durable sync. Use the released, most recent version of CDH3.  In CDH, append
-    support is enabled by default so you do not need to make the above mentioned edits to
-    <filename>hdfs-site.xml</filename> or to <filename>hbase-site.xml</filename>.</para>
-    <para>
-    <link xlink:href="http://www.mapr.com/">MapR</link>
-    includes a commercial, reimplementation of HDFS.
-    It has a durable sync as well as some other interesting features that are not
-    yet in Apache Hadoop.  Their <link xlink:href="http://www.mapr.com/products/mapr-editions/m3-edition">M3</link>
-    product is free to use and unlimited.
-    </para>
-
-        <para>Because HBase depends on Hadoop, it bundles an instance of the
-        Hadoop jar under its <filename>lib</filename> directory. The bundled jar is ONLY for use in standalone mode.
-        In distributed mode, it is <emphasis>critical</emphasis> that the version of Hadoop that is out
-        on your cluster match what is under HBase.  Replace the hadoop jar found in the HBase
-        <filename>lib</filename> directory with the hadoop jar you are running on
-        your cluster to avoid version mismatch issues. Make sure you
-        replace the jar in HBase everywhere on your cluster.  Hadoop version
-        mismatch issues have various manifestations but often all looks like
-        its hung up.</para>
-    <note xml:id="bigtop"><title>Packaging and Apache BigTop</title>
-        <para><link xlink:href="http://bigtop.apache.org">Apache Bigtop</link>
-            is an umbrella for packaging and tests of the Apache Hadoop
-            ecosystem, including Apache HBase. Bigtop performs testing at various
-            levels (packaging, platform, runtime, upgrade, etc...), developed by a
-            community, with a focus on the system as a whole, rather than individual
-            projects. We recommend installing Apache HBase packages as provided by a
-            Bigtop release rather than rolling your own piecemeal integration of
-            various component releases.</para>
-    </note>
-
+        description for the <varname>dfs.support.append</varname> configuration.
+     </para>
+     </section>
        <section xml:id="hadoop.security">
           <title>Apache HBase on Secure Hadoop</title>
           <para>Apache HBase will run on any Hadoop 0.20.x that incorporates Hadoop
-          security features -- e.g. Y! 0.20S or CDH3B3 -- as long as you do as
+          security features as long as you do as
           suggested above and replace the Hadoop jar that ships with HBase
           with the secure version.  If you want to read more about how to setup
           Secure HBase, see <xref linkend="hbase.secure.configuration" />.</para>