From 1a9b556474d01480762bf8df431c499443d24737 Mon Sep 17 00:00:00 2001 From: stack Date: Tue, 2 Dec 2014 09:51:19 -0800 Subject: [PATCH] Revert "HBASE-12558 TestHCM.testClusterStatus Unexpected exception, expected but was -- ADDED DEBUG" This reverts commit 8b8f2026bda6a329d6d82376dbb884c8f183a967. Overcommit. Added "HBASE-12490 Replace uses of setAutoFlush(boolean, boolean)" by mistake Conflicts: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java --- .../hbase/client/ClusterStatusListener.java | 43 ++----------------- .../apache/hadoop/hbase/client/HTable.java | 12 +++--- .../hadoop/hbase/client/TestAsyncProcess.java | 10 ++--- .../test/IntegrationTestBigLinkedList.java | 2 +- ...rationTestBigLinkedListWithVisibility.java | 2 +- .../test/IntegrationTestLoadAndVerify.java | 2 +- .../IntegrationTestSendTraceRequests.java | 2 +- .../hbase/rest/PerformanceEvaluation.java | 2 +- .../mapreduce/MultiTableOutputFormat.java | 2 +- .../hbase/mapreduce/TableOutputFormat.java | 2 +- .../hbase/master/ClusterStatusPublisher.java | 5 +-- .../hbase/client/TestFromClientSide.java | 6 +-- .../hbase/client/TestMultiParallel.java | 2 +- .../hbase/coprocessor/TestHTableWrapper.java | 2 +- .../master/TestDistributedLogSplitting.java | 4 +- .../regionserver/TestRegionServerMetrics.java | 2 +- .../regionserver/wal/TestLogRolling.java | 2 +- ...tReplicationChangingPeerRegionservers.java | 2 +- .../TestReplicationSmallTests.java | 6 +-- .../hbase/snapshot/SnapshotTestingUtils.java | 2 +- 20 files changed, 36 insertions(+), 76 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java index 57861df276e..2e2ea65f90f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java @@ -21,10 +21,7 @@ package org.apache.hadoop.hbase.client; import io.netty.bootstrap.Bootstrap; -import io.netty.bootstrap.ChannelFactory; import io.netty.buffer.ByteBufInputStream; -import io.netty.channel.Channel; -import io.netty.channel.ChannelException; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelOption; import io.netty.channel.EventLoopGroup; @@ -32,9 +29,7 @@ import io.netty.channel.SimpleChannelInboundHandler; import io.netty.channel.nio.NioEventLoopGroup; import io.netty.channel.socket.DatagramChannel; import io.netty.channel.socket.DatagramPacket; -import io.netty.channel.socket.InternetProtocolFamily; import io.netty.channel.socket.nio.NioDatagramChannel; -import io.netty.util.internal.StringUtil; import java.io.Closeable; import java.io.IOException; @@ -59,7 +54,6 @@ import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos; import org.apache.hadoop.hbase.util.Addressing; import org.apache.hadoop.hbase.util.ExceptionUtil; import org.apache.hadoop.hbase.util.Threads; -import org.apache.hadoop.hbase.util.ReflectionUtils; /** @@ -212,17 +206,12 @@ class ClusterStatusListener implements Closeable { throw new IOException("Can't connect to " + mcAddress, e); } - InternetProtocolFamily family = InternetProtocolFamily.IPv4; - if (ina instanceof Inet6Address) { - family = InternetProtocolFamily.IPv6; - } - try { Bootstrap b = new Bootstrap(); b.group(group) - .channelFactory(new HBaseDatagramChannelFactory(NioDatagramChannel.class, family)) - .option(ChannelOption.SO_REUSEADDR, true) - .handler(new ClusterStatusHandler()); + .channel(NioDatagramChannel.class) + .option(ChannelOption.SO_REUSEADDR, true) + .handler(new ClusterStatusHandler()); channel = (DatagramChannel)b.bind(bindAddress, port).sync().channel(); } catch (InterruptedException e) { @@ -234,32 +223,6 @@ class ClusterStatusListener implements Closeable { channel.joinGroup(ina, ni, null, channel.newPromise()); } - private class HBaseDatagramChannelFactory implements ChannelFactory { - private final Class clazz; - private InternetProtocolFamily family; - - HBaseDatagramChannelFactory(Class clazz, InternetProtocolFamily family) { - this.clazz = clazz; - this.family = family; - } - - @Override - public T newChannel() { - try { - return ReflectionUtils.instantiateWithCustomCtor(clazz.getName(), - new Class[] { InternetProtocolFamily.class }, new Object[] { family }); - - } catch (Throwable t) { - throw new ChannelException("Unable to create Channel from class " + clazz, t); - } - } - - @Override - public String toString() { - return StringUtil.simpleClassName(clazz) + ".class"; - } - } - @Override public void close() { if (channel != null) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java index 1e473f582de..61cb8b8c07c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java @@ -116,17 +116,18 @@ public class HTable implements HTableInterface, RegionLocator { private TableConfiguration tableConfiguration; protected List writeAsyncBuffer = new LinkedList(); private long writeBufferSize; - private boolean clearBufferOnFail = true; - private boolean autoFlush = true; - protected long currentWriteBufferSize = 0 ; - private boolean closed = false; + private boolean clearBufferOnFail; + private boolean autoFlush; + protected long currentWriteBufferSize; protected int scannerCaching; private ExecutorService pool; // For Multi & Scan + private boolean closed; private int operationTimeout; private final boolean cleanupPoolOnClose; // shutdown the pool in close() private final boolean cleanupConnectionOnClose; // close the connection in close() private Consistency defaultConsistency = Consistency.STRONG; + /** The Async process for puts with autoflush set to false or multiputs */ protected AsyncProcess ap; /** The Async process for batch */ @@ -325,10 +326,9 @@ public class HTable implements HTableInterface, RegionLocator { /** * For internal testing. - * @throws IOException */ @VisibleForTesting - protected HTable() throws IOException { + protected HTable() { tableName = null; tableConfiguration = new TableConfiguration(); cleanupPoolOnClose = false; diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java index 8a3aafc6509..8d77d7aca58 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java @@ -663,7 +663,7 @@ public class TestAsyncProcess { HTable ht = new HTable(); MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), conf, true); ht.ap = ap; - ht.setAutoFlushTo(true); + ht.setAutoFlush(true, true); if (bufferOn) { ht.setWriteBufferSize(1024L * 1024L); } else { @@ -711,7 +711,7 @@ public class TestAsyncProcess { HTable ht = new HTable(); MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), conf, true); ht.ap = ap; - ht.setAutoFlushTo(false); + ht.setAutoFlush(false, true); ht.setWriteBufferSize(0); Put p = createPut(1, false); @@ -739,7 +739,7 @@ public class TestAsyncProcess { public void testWithNoClearOnFail() throws IOException { HTable ht = new HTable(); ht.ap = new MyAsyncProcess(createHConnection(), conf, true); - ht.setAutoFlush(false); + ht.setAutoFlush(false, false); Put p = createPut(1, false); ht.put(p); @@ -806,7 +806,7 @@ public class TestAsyncProcess { ht.ap.serverTrackerTimeout = 1; Put p = createPut(1, false); - ht.setAutoFlush(false); + ht.setAutoFlush(false, false); ht.put(p); try { @@ -828,7 +828,7 @@ public class TestAsyncProcess { Assert.assertNotNull(ht.ap.createServerErrorTracker()); Put p = createPut(1, true); - ht.setAutoFlush(false); + ht.setAutoFlush(false, false); ht.put(p); try { diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java index 37e4f8b300f..177341f4721 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedList.java @@ -363,7 +363,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase { protected void instantiateHTable(Configuration conf) throws IOException { table = new HTable(conf, getTableName(conf)); - table.setAutoFlushTo(false); + table.setAutoFlush(false, true); table.setWriteBufferSize(4 * 1024 * 1024); } diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java index dc517a5c558..603c3df4f21 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestBigLinkedListWithVisibility.java @@ -185,7 +185,7 @@ public class IntegrationTestBigLinkedListWithVisibility extends IntegrationTestB protected void instantiateHTable(Configuration conf) throws IOException { for (int i = 0; i < DEFAULT_TABLES_COUNT; i++) { HTable table = new HTable(conf, getTableName(i)); - table.setAutoFlushTo(true); + table.setAutoFlush(true, true); //table.setWriteBufferSize(4 * 1024 * 1024); this.tables[i] = table; } diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java index 60f20a5b0a1..5c9a9ad222b 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestLoadAndVerify.java @@ -181,7 +181,7 @@ public void cleanUpCluster() throws Exception { numBackReferencesPerRow = conf.getInt(NUM_BACKREFS_KEY, NUM_BACKREFS_DEFAULT); table = new HTable(conf, TableName.valueOf(tableName)); table.setWriteBufferSize(4*1024*1024); - table.setAutoFlushTo(false); + table.setAutoFlush(false, true); String taskId = conf.get("mapreduce.task.attempt.id"); Matcher matcher = Pattern.compile(".+_m_(\\d+_\\d+)").matcher(taskId); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/trace/IntegrationTestSendTraceRequests.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/trace/IntegrationTestSendTraceRequests.java index b1cf57e3df4..c96a6ac1522 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/trace/IntegrationTestSendTraceRequests.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/trace/IntegrationTestSendTraceRequests.java @@ -239,7 +239,7 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool { for (int x = 0; x < 5000; x++) { TraceScope traceScope = Trace.startSpan("insertData", Sampler.ALWAYS); try { - ht.setAutoFlushTo(false); + ht.setAutoFlush(false, true); for (int i = 0; i < 5; i++) { long rk = random.nextLong(); rowKeys.add(rk); diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java index b02f069561e..7e17c014be5 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/PerformanceEvaluation.java @@ -909,7 +909,7 @@ public class PerformanceEvaluation extends Configured implements Tool { void testSetup() throws IOException { this.table = connection.getTable(tableName); - this.table.setAutoFlushTo(false); + this.table.setAutoFlush(false, true); } void testTakedown() throws IOException { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableOutputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableOutputFormat.java index 20cf50a7893..62a9626c926 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableOutputFormat.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableOutputFormat.java @@ -104,7 +104,7 @@ public class MultiTableOutputFormat extends OutputFormat DEFAULT_STATUS_PUBLISHER_CLASS = @@ -173,6 +170,7 @@ public class ClusterStatusPublisher extends Chore { null, null); + publisher.publish(cs); } @@ -216,7 +214,6 @@ public class ClusterStatusPublisher extends Chore { } res.add(toSend.getKey()); - LOG.debug("###add dead server " + toSend.getKey()); } return res; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java index 9554d97c27c..31ed81766f9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java @@ -3907,7 +3907,7 @@ public class TestFromClientSide { final int NB_BATCH_ROWS = 10; HTable table = TEST_UTIL.createTable(Bytes.toBytes("testRowsPutBufferedOneFlush"), new byte [][] {CONTENTS_FAMILY, SMALL_FAMILY}); - table.setAutoFlushTo(false); + table.setAutoFlush(false, true); ArrayList rowsUpdate = new ArrayList(); for (int i = 0; i < NB_BATCH_ROWS * 10; i++) { byte[] row = Bytes.toBytes("row" + i); @@ -3948,7 +3948,7 @@ public class TestFromClientSide { final int NB_BATCH_ROWS = 10; HTable table = TEST_UTIL.createTable(Bytes.toBytes("testRowsPutBufferedManyManyFlushes"), new byte[][] {CONTENTS_FAMILY, SMALL_FAMILY }); - table.setAutoFlushTo(false); + table.setAutoFlush(false, true); table.setWriteBufferSize(10); ArrayList rowsUpdate = new ArrayList(); for (int i = 0; i < NB_BATCH_ROWS * 10; i++) { @@ -4277,7 +4277,7 @@ public class TestFromClientSide { new byte[][] { HConstants.CATALOG_FAMILY, Bytes.toBytes("info2") }, 1, 1024); // set block size to 64 to making 2 kvs into one block, bypassing the walkForwardInSingleRow // in Store.rowAtOrBeforeFromStoreFile - table.setAutoFlushTo(true); + table.setAutoFlush(true); String regionName = table.getRegionLocations().firstKey().getEncodedName(); HRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableAname).getFromOnlineRegions(regionName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java index 47bb569dfe5..69267ec77fa 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java @@ -263,7 +263,7 @@ public class TestMultiParallel { // Load the data LOG.info("get new table"); HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE); - table.setAutoFlushTo(false); + table.setAutoFlush(false, true); table.setWriteBufferSize(10 * 1024 * 1024); LOG.info("constructPutRequests"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java index 4649961b176..de0057cb899 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestHTableWrapper.java @@ -178,7 +178,7 @@ public class TestHTableWrapper { boolean initialAutoFlush = hTableInterface.isAutoFlush(); hTableInterface.setAutoFlushTo(false); assertFalse(hTableInterface.isAutoFlush()); - hTableInterface.setAutoFlushTo(true); + hTableInterface.setAutoFlush(true, true); assertTrue(hTableInterface.isAutoFlush()); hTableInterface.setAutoFlushTo(initialAutoFlush); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java index f37c1ebca7c..5e867adef7d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java @@ -921,7 +921,7 @@ public class TestDistributedLogSplitting { if (key == null || key.length == 0) { key = new byte[] { 0, 0, 0, 0, 1 }; } - ht.setAutoFlushTo(true); + ht.setAutoFlush(true, true); Put put = new Put(key); put.add(Bytes.toBytes("family"), Bytes.toBytes("c1"), new byte[]{'b'}); ht.put(put); @@ -1612,7 +1612,7 @@ public class TestDistributedLogSplitting { * Load table with puts and deletes with expected values so that we can verify later */ private void prepareData(final HTable t, final byte[] f, final byte[] column) throws IOException { - t.setAutoFlushTo(false); + t.setAutoFlush(false, true); byte[] k = new byte[3]; // add puts diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java index d3285a3291e..3ae82ee3b62 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java @@ -348,7 +348,7 @@ public class TestRegionServerMetrics { TEST_UTIL.createTable(tableName, cf); HTable t = new HTable(conf, tableName); - t.setAutoFlushTo(false); + t.setAutoFlush(false, true); for (int insertCount =0; insertCount < 100; insertCount++) { Put p = new Put(Bytes.toBytes("" + insertCount + "row")); p.add(cf, qualifier, val); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java index 86e77ad2b51..9c6584e42b6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java @@ -456,7 +456,7 @@ public class TestLogRolling { writeData(table, 1002); - table.setAutoFlushTo(true); + table.setAutoFlush(true, true); long curTime = System.currentTimeMillis(); LOG.info("log.getCurrentFileName()): " + DefaultWALProvider.getCurrentFileName(log)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java index 67f2031f28a..031960776e8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java @@ -54,7 +54,7 @@ public class TestReplicationChangingPeerRegionservers extends TestReplicationBas */ @Before public void setUp() throws Exception { - htable1.setAutoFlushTo(false); + ((HTable)htable1).setAutoFlush(false, true); // Starting and stopping replication can make us miss new logs, // rolling like this makes sure the most recent one gets added to the queue for (JVMClusterUtil.RegionServerThread r : diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java index 43770828184..c12089a7921 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTests.java @@ -69,7 +69,7 @@ public class TestReplicationSmallTests extends TestReplicationBase { */ @Before public void setUp() throws Exception { - htable1.setAutoFlushTo(true); + ((HTable)htable1).setAutoFlush(true, true); // Starting and stopping replication can make us miss new logs, // rolling like this makes sure the most recent one gets added to the queue for ( JVMClusterUtil.RegionServerThread r : @@ -247,7 +247,7 @@ public class TestReplicationSmallTests extends TestReplicationBase { LOG.info("testSmallBatch"); Put put; // normal Batch tests - htable1.setAutoFlushTo(false); + ((HTable)htable1).setAutoFlush(false, true); for (int i = 0; i < NB_ROWS_IN_BATCH; i++) { put = new Put(Bytes.toBytes(i)); put.add(famName, row, row); @@ -387,7 +387,7 @@ public class TestReplicationSmallTests extends TestReplicationBase { public void testLoading() throws Exception { LOG.info("Writing out rows to table1 in testLoading"); htable1.setWriteBufferSize(1024); - ((HTable)htable1).setAutoFlushTo(false); + ((HTable)htable1).setAutoFlush(false, true); for (int i = 0; i < NB_ROWS_IN_BIG_BATCH; i++) { Put put = new Put(Bytes.toBytes(i)); put.add(famName, row, row); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java index cebb3c4395e..a3d1aac7147 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java @@ -678,7 +678,7 @@ public class SnapshotTestingUtils { public static void loadData(final HBaseTestingUtility util, final HTable table, int rows, byte[]... families) throws IOException, InterruptedException { - table.setAutoFlushTo(false); + table.setAutoFlush(false, true); // Ensure one row per region assertTrue(rows >= KEYS.length);