HBASE-12490 Replace uses of setAutoFlush(boolean, boolean) (Solomon Duskis)

This commit is contained in:
stack 2014-12-02 09:33:51 -08:00
parent 091142f47a
commit 2b976c6bd1
18 changed files with 62 additions and 62 deletions

View File

@ -662,7 +662,7 @@ public class TestAsyncProcess {
HTable ht = new HTable(); HTable ht = new HTable();
MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), conf, true); MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), conf, true);
ht.ap = ap; ht.ap = ap;
ht.setAutoFlush(true, true); ht.setAutoFlushTo(true);
if (bufferOn) { if (bufferOn) {
ht.setWriteBufferSize(1024L * 1024L); ht.setWriteBufferSize(1024L * 1024L);
} else { } else {
@ -710,7 +710,7 @@ public class TestAsyncProcess {
HTable ht = new HTable(); HTable ht = new HTable();
MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), conf, true); MyAsyncProcess ap = new MyAsyncProcess(createHConnection(), conf, true);
ht.ap = ap; ht.ap = ap;
ht.setAutoFlush(false, true); ht.setAutoFlushTo(false);
ht.setWriteBufferSize(0); ht.setWriteBufferSize(0);
Put p = createPut(1, false); Put p = createPut(1, false);
@ -738,7 +738,7 @@ public class TestAsyncProcess {
public void testWithNoClearOnFail() throws IOException { public void testWithNoClearOnFail() throws IOException {
HTable ht = new HTable(); HTable ht = new HTable();
ht.ap = new MyAsyncProcess(createHConnection(), conf, true); ht.ap = new MyAsyncProcess(createHConnection(), conf, true);
ht.setAutoFlush(false, false); ht.setAutoFlush(false);
Put p = createPut(1, false); Put p = createPut(1, false);
ht.put(p); ht.put(p);
@ -805,7 +805,7 @@ public class TestAsyncProcess {
ht.ap.serverTrackerTimeout = 1; ht.ap.serverTrackerTimeout = 1;
Put p = createPut(1, false); Put p = createPut(1, false);
ht.setAutoFlush(false, false); ht.setAutoFlush(false);
ht.put(p); ht.put(p);
try { try {
@ -827,7 +827,7 @@ public class TestAsyncProcess {
Assert.assertNotNull(ht.ap.createServerErrorTracker()); Assert.assertNotNull(ht.ap.createServerErrorTracker());
Put p = createPut(1, true); Put p = createPut(1, true);
ht.setAutoFlush(false, false); ht.setAutoFlush(false);
ht.put(p); ht.put(p);
try { try {

View File

@ -340,7 +340,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
byte[] id; byte[] id;
long count = 0; long count = 0;
int i; int i;
HTable table; Table table;
long numNodes; long numNodes;
long wrap; long wrap;
int width; int width;
@ -363,7 +363,7 @@ public class IntegrationTestBigLinkedList extends IntegrationTestBase {
protected void instantiateHTable(Configuration conf) throws IOException { protected void instantiateHTable(Configuration conf) throws IOException {
table = new HTable(conf, getTableName(conf)); table = new HTable(conf, getTableName(conf));
table.setAutoFlush(false, true); table.setAutoFlushTo(false);
table.setWriteBufferSize(4 * 1024 * 1024); table.setWriteBufferSize(4 * 1024 * 1024);
} }

View File

@ -184,8 +184,8 @@ public class IntegrationTestBigLinkedListWithVisibility extends IntegrationTestB
@Override @Override
protected void instantiateHTable(Configuration conf) throws IOException { protected void instantiateHTable(Configuration conf) throws IOException {
for (int i = 0; i < DEFAULT_TABLES_COUNT; i++) { for (int i = 0; i < DEFAULT_TABLES_COUNT; i++) {
HTable table = new HTable(conf, getTableName(i)); Table table = new HTable(conf, getTableName(i));
table.setAutoFlush(true, true); table.setAutoFlushTo(true);
//table.setWriteBufferSize(4 * 1024 * 1024); //table.setWriteBufferSize(4 * 1024 * 1024);
this.tables[i] = table; this.tables[i] = table;
} }

View File

@ -20,12 +20,7 @@ package org.apache.hadoop.hbase.test;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import java.io.IOException; import com.google.common.collect.Sets;
import java.util.Random;
import java.util.Set;
import java.util.UUID;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.CommandLine;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
@ -49,6 +44,7 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.ScannerCallable; import org.apache.hadoop.hbase.client.ScannerCallable;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.mapreduce.NMapInputFormat; import org.apache.hadoop.hbase.mapreduce.NMapInputFormat;
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil; import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
@ -68,7 +64,12 @@ import org.apache.hadoop.util.ToolRunner;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
import com.google.common.collect.Sets; import java.io.IOException;
import java.util.Random;
import java.util.Set;
import java.util.UUID;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
/** /**
* A large test which loads a lot of data that has internal references, and * A large test which loads a lot of data that has internal references, and
@ -164,7 +165,7 @@ public void cleanUpCluster() throws Exception {
extends Mapper<NullWritable, NullWritable, NullWritable, NullWritable> extends Mapper<NullWritable, NullWritable, NullWritable, NullWritable>
{ {
protected long recordsToWrite; protected long recordsToWrite;
protected HTable table; protected Table table;
protected Configuration conf; protected Configuration conf;
protected int numBackReferencesPerRow; protected int numBackReferencesPerRow;
protected String shortTaskId; protected String shortTaskId;
@ -181,7 +182,7 @@ public void cleanUpCluster() throws Exception {
numBackReferencesPerRow = conf.getInt(NUM_BACKREFS_KEY, NUM_BACKREFS_DEFAULT); numBackReferencesPerRow = conf.getInt(NUM_BACKREFS_KEY, NUM_BACKREFS_DEFAULT);
table = new HTable(conf, TableName.valueOf(tableName)); table = new HTable(conf, TableName.valueOf(tableName));
table.setWriteBufferSize(4*1024*1024); table.setWriteBufferSize(4*1024*1024);
table.setAutoFlush(false, true); table.setAutoFlushTo(false);
String taskId = conf.get("mapreduce.task.attempt.id"); String taskId = conf.get("mapreduce.task.attempt.id");
Matcher matcher = Pattern.compile(".+_m_(\\d+_\\d+)").matcher(taskId); Matcher matcher = Pattern.compile(".+_m_(\\d+_\\d+)").matcher(taskId);

View File

@ -234,12 +234,12 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool {
private LinkedBlockingQueue<Long> insertData() throws IOException, InterruptedException { private LinkedBlockingQueue<Long> insertData() throws IOException, InterruptedException {
LinkedBlockingQueue<Long> rowKeys = new LinkedBlockingQueue<Long>(25000); LinkedBlockingQueue<Long> rowKeys = new LinkedBlockingQueue<Long>(25000);
HTable ht = new HTable(util.getConfiguration(), this.tableName); Table ht = new HTable(util.getConfiguration(), this.tableName);
byte[] value = new byte[300]; byte[] value = new byte[300];
for (int x = 0; x < 5000; x++) { for (int x = 0; x < 5000; x++) {
TraceScope traceScope = Trace.startSpan("insertData", Sampler.ALWAYS); TraceScope traceScope = Trace.startSpan("insertData", Sampler.ALWAYS);
try { try {
ht.setAutoFlush(false, true); ht.setAutoFlushTo(false);
for (int i = 0; i < 5; i++) { for (int i = 0; i < 5; i++) {
long rk = random.nextLong(); long rk = random.nextLong();
rowKeys.add(rk); rowKeys.add(rk);

View File

@ -18,22 +18,6 @@
*/ */
package org.apache.hadoop.hbase.rest; package org.apache.hadoop.hbase.rest;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.io.PrintStream;
import java.lang.reflect.Constructor;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Date;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.TreeMap;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -53,11 +37,11 @@ import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager; import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.filter.BinaryComparator; import org.apache.hadoop.hbase.filter.BinaryComparator;
import org.apache.hadoop.hbase.filter.CompareFilter; import org.apache.hadoop.hbase.filter.CompareFilter;
import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.Filter;
@ -91,6 +75,22 @@ import org.apache.hadoop.util.LineReader;
import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner; import org.apache.hadoop.util.ToolRunner;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.io.PrintStream;
import java.lang.reflect.Constructor;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Date;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.TreeMap;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
/** /**
* Script used evaluating Stargate performance and scalability. Runs a SG * Script used evaluating Stargate performance and scalability. Runs a SG
* client that steps through one of a set of hardcoded tests or 'experiments' * client that steps through one of a set of hardcoded tests or 'experiments'
@ -870,7 +870,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
protected final int totalRows; protected final int totalRows;
private final Status status; private final Status status;
protected TableName tableName; protected TableName tableName;
protected HTableInterface table; protected Table table;
protected volatile Configuration conf; protected volatile Configuration conf;
protected boolean flushCommits; protected boolean flushCommits;
protected boolean writeToWAL; protected boolean writeToWAL;
@ -909,7 +909,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
void testSetup() throws IOException { void testSetup() throws IOException {
this.table = connection.getTable(tableName); this.table = connection.getTable(tableName);
this.table.setAutoFlush(false, true); this.table.setAutoFlushTo(false);
} }
void testTakedown() throws IOException { void testTakedown() throws IOException {

View File

@ -104,7 +104,7 @@ public class MultiTableOutputFormat extends OutputFormat<ImmutableBytesWritable,
if (!tables.containsKey(tableName)) { if (!tables.containsKey(tableName)) {
LOG.debug("Opening HTable \"" + Bytes.toString(tableName.get())+ "\" for writing"); LOG.debug("Opening HTable \"" + Bytes.toString(tableName.get())+ "\" for writing");
HTable table = new HTable(conf, TableName.valueOf(tableName.get())); HTable table = new HTable(conf, TableName.valueOf(tableName.get()));
table.setAutoFlush(false, true); table.setAutoFlushTo(false);
tables.put(tableName, table); tables.put(tableName, table);
} }
return tables.get(tableName); return tables.get(tableName);

View File

@ -195,7 +195,7 @@ implements Configurable {
} }
this.connection = ConnectionFactory.createConnection(this.conf); this.connection = ConnectionFactory.createConnection(this.conf);
this.table = connection.getTable(TableName.valueOf(tableName)); this.table = connection.getTable(TableName.valueOf(tableName));
((HTable) this.table).setAutoFlush(false, true); this.table.setAutoFlushTo(false);
LOG.info("Created table instance for " + tableName); LOG.info("Created table instance for " + tableName);
} catch(IOException e) { } catch(IOException e) {
LOG.error(e); LOG.error(e);

View File

@ -3906,7 +3906,7 @@ public class TestFromClientSide {
final int NB_BATCH_ROWS = 10; final int NB_BATCH_ROWS = 10;
HTable table = TEST_UTIL.createTable(Bytes.toBytes("testRowsPutBufferedOneFlush"), HTable table = TEST_UTIL.createTable(Bytes.toBytes("testRowsPutBufferedOneFlush"),
new byte [][] {CONTENTS_FAMILY, SMALL_FAMILY}); new byte [][] {CONTENTS_FAMILY, SMALL_FAMILY});
table.setAutoFlush(false, true); table.setAutoFlushTo(false);
ArrayList<Put> rowsUpdate = new ArrayList<Put>(); ArrayList<Put> rowsUpdate = new ArrayList<Put>();
for (int i = 0; i < NB_BATCH_ROWS * 10; i++) { for (int i = 0; i < NB_BATCH_ROWS * 10; i++) {
byte[] row = Bytes.toBytes("row" + i); byte[] row = Bytes.toBytes("row" + i);
@ -3947,7 +3947,7 @@ public class TestFromClientSide {
final int NB_BATCH_ROWS = 10; final int NB_BATCH_ROWS = 10;
HTable table = TEST_UTIL.createTable(Bytes.toBytes("testRowsPutBufferedManyManyFlushes"), HTable table = TEST_UTIL.createTable(Bytes.toBytes("testRowsPutBufferedManyManyFlushes"),
new byte[][] {CONTENTS_FAMILY, SMALL_FAMILY }); new byte[][] {CONTENTS_FAMILY, SMALL_FAMILY });
table.setAutoFlush(false, true); table.setAutoFlushTo(false);
table.setWriteBufferSize(10); table.setWriteBufferSize(10);
ArrayList<Put> rowsUpdate = new ArrayList<Put>(); ArrayList<Put> rowsUpdate = new ArrayList<Put>();
for (int i = 0; i < NB_BATCH_ROWS * 10; i++) { for (int i = 0; i < NB_BATCH_ROWS * 10; i++) {

View File

@ -69,7 +69,7 @@ public class TestHTableUtil {
public void testBucketPut() throws Exception { public void testBucketPut() throws Exception {
byte [] TABLE = Bytes.toBytes("testBucketPut"); byte [] TABLE = Bytes.toBytes("testBucketPut");
HTable ht = TEST_UTIL.createTable(TABLE, FAMILY); HTable ht = TEST_UTIL.createTable(TABLE, FAMILY);
ht.setAutoFlush(false, true); ht.setAutoFlushTo(false);
List<Put> puts = new ArrayList<Put>(); List<Put> puts = new ArrayList<Put>();
puts.add( createPut("row1") ); puts.add( createPut("row1") );

View File

@ -263,8 +263,8 @@ public class TestMultiParallel {
private void doTestFlushCommits(boolean doAbort) throws Exception { private void doTestFlushCommits(boolean doAbort) throws Exception {
// Load the data // Load the data
LOG.info("get new table"); LOG.info("get new table");
HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE); Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
table.setAutoFlush(false, true); table.setAutoFlushTo(false);
table.setWriteBufferSize(10 * 1024 * 1024); table.setWriteBufferSize(10 * 1024 * 1024);
LOG.info("constructPutRequests"); LOG.info("constructPutRequests");

View File

@ -177,7 +177,7 @@ public class TestHTableWrapper {
boolean initialAutoFlush = hTableInterface.isAutoFlush(); boolean initialAutoFlush = hTableInterface.isAutoFlush();
hTableInterface.setAutoFlushTo(false); hTableInterface.setAutoFlushTo(false);
assertFalse(hTableInterface.isAutoFlush()); assertFalse(hTableInterface.isAutoFlush());
hTableInterface.setAutoFlush(true, true); hTableInterface.setAutoFlushTo(true);
assertTrue(hTableInterface.isAutoFlush()); assertTrue(hTableInterface.isAutoFlush());
hTableInterface.setAutoFlushTo(initialAutoFlush); hTableInterface.setAutoFlushTo(initialAutoFlush);
} }

View File

@ -883,7 +883,7 @@ public class TestDistributedLogSplitting {
List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads(); List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null); final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE); Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
final SplitLogManager slm = master.getMasterFileSystem().splitLogManager; final SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
Set<HRegionInfo> regionSet = new HashSet<HRegionInfo>(); Set<HRegionInfo> regionSet = new HashSet<HRegionInfo>();
@ -923,7 +923,7 @@ public class TestDistributedLogSplitting {
if (key == null || key.length == 0) { if (key == null || key.length == 0) {
key = new byte[] { 0, 0, 0, 0, 1 }; key = new byte[] { 0, 0, 0, 0, 1 };
} }
ht.setAutoFlush(true, true); ht.setAutoFlushTo(true);
Put put = new Put(key); Put put = new Put(key);
put.add(Bytes.toBytes("family"), Bytes.toBytes("c1"), new byte[]{'b'}); put.add(Bytes.toBytes("family"), Bytes.toBytes("c1"), new byte[]{'b'});
ht.put(put); ht.put(put);
@ -1595,7 +1595,7 @@ public class TestDistributedLogSplitting {
* Load table with puts and deletes with expected values so that we can verify later * 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 { private void prepareData(final HTable t, final byte[] f, final byte[] column) throws IOException {
t.setAutoFlush(false, true); t.setAutoFlushTo(false);
byte[] k = new byte[3]; byte[] k = new byte[3];
// add puts // add puts

View File

@ -351,7 +351,7 @@ public class TestRegionServerMetrics {
TEST_UTIL.createTable(tableName, cf); TEST_UTIL.createTable(tableName, cf);
HTable t = new HTable(conf, tableName); HTable t = new HTable(conf, tableName);
t.setAutoFlush(false, true); t.setAutoFlushTo(false);
for (int insertCount =0; insertCount < 100; insertCount++) { for (int insertCount =0; insertCount < 100; insertCount++) {
Put p = new Put(Bytes.toBytes("" + insertCount + "row")); Put p = new Put(Bytes.toBytes("" + insertCount + "row"));
p.add(cf, qualifier, val); p.add(cf, qualifier, val);

View File

@ -428,7 +428,7 @@ public class TestLogRolling {
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY)); desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(desc); admin.createTable(desc);
HTable table = new HTable(TEST_UTIL.getConfiguration(), desc.getTableName()); Table table = new HTable(TEST_UTIL.getConfiguration(), desc.getTableName());
server = TEST_UTIL.getRSForFirstRegionInTable(desc.getTableName()); server = TEST_UTIL.getRSForFirstRegionInTable(desc.getTableName());
final WAL log = server.getWAL(null); final WAL log = server.getWAL(null);
@ -455,7 +455,7 @@ public class TestLogRolling {
writeData(table, 1002); writeData(table, 1002);
table.setAutoFlush(true, true); table.setAutoFlushTo(true);
long curTime = System.currentTimeMillis(); long curTime = System.currentTimeMillis();
LOG.info("log.getCurrentFileName()): " + DefaultWALProvider.getCurrentFileName(log)); LOG.info("log.getCurrentFileName()): " + DefaultWALProvider.getCurrentFileName(log));

View File

@ -22,8 +22,6 @@ import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail; import static org.junit.Assert.fail;
import java.io.IOException;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.LargeTests; import org.apache.hadoop.hbase.LargeTests;
@ -36,11 +34,12 @@ import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
import java.io.IOException;
/** /**
* Test handling of changes to the number of a peer's regionservers. * Test handling of changes to the number of a peer's regionservers.
*/ */
@ -54,7 +53,7 @@ public class TestReplicationChangingPeerRegionservers extends TestReplicationBas
*/ */
@Before @Before
public void setUp() throws Exception { public void setUp() throws Exception {
((HTable)htable1).setAutoFlush(false, true); htable1.setAutoFlushTo(false);
// Starting and stopping replication can make us miss new logs, // Starting and stopping replication can make us miss new logs,
// rolling like this makes sure the most recent one gets added to the queue // rolling like this makes sure the most recent one gets added to the queue
for (JVMClusterUtil.RegionServerThread r : for (JVMClusterUtil.RegionServerThread r :

View File

@ -68,7 +68,7 @@ public class TestReplicationSmallTests extends TestReplicationBase {
*/ */
@Before @Before
public void setUp() throws Exception { public void setUp() throws Exception {
((HTable)htable1).setAutoFlush(true, true); htable1.setAutoFlushTo(true);
// Starting and stopping replication can make us miss new logs, // Starting and stopping replication can make us miss new logs,
// rolling like this makes sure the most recent one gets added to the queue // rolling like this makes sure the most recent one gets added to the queue
for ( JVMClusterUtil.RegionServerThread r : for ( JVMClusterUtil.RegionServerThread r :
@ -246,7 +246,7 @@ public class TestReplicationSmallTests extends TestReplicationBase {
LOG.info("testSmallBatch"); LOG.info("testSmallBatch");
Put put; Put put;
// normal Batch tests // normal Batch tests
((HTable)htable1).setAutoFlush(false, true); htable1.setAutoFlushTo(false);
for (int i = 0; i < NB_ROWS_IN_BATCH; i++) { for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
put = new Put(Bytes.toBytes(i)); put = new Put(Bytes.toBytes(i));
put.add(famName, row, row); put.add(famName, row, row);
@ -386,7 +386,7 @@ public class TestReplicationSmallTests extends TestReplicationBase {
public void testLoading() throws Exception { public void testLoading() throws Exception {
LOG.info("Writing out rows to table1 in testLoading"); LOG.info("Writing out rows to table1 in testLoading");
htable1.setWriteBufferSize(1024); htable1.setWriteBufferSize(1024);
((HTable)htable1).setAutoFlush(false, true); htable1.setAutoFlushTo(false);
for (int i = 0; i < NB_ROWS_IN_BIG_BATCH; i++) { for (int i = 0; i < NB_ROWS_IN_BIG_BATCH; i++) {
Put put = new Put(Bytes.toBytes(i)); Put put = new Put(Bytes.toBytes(i));
put.add(famName, row, row); put.add(famName, row, row);

View File

@ -673,9 +673,9 @@ public class SnapshotTestingUtils {
loadData(util, new HTable(util.getConfiguration(), tableName), rows, families); loadData(util, new HTable(util.getConfiguration(), tableName), rows, families);
} }
public static void loadData(final HBaseTestingUtility util, final HTable table, int rows, public static void loadData(final HBaseTestingUtility util, final Table table, int rows,
byte[]... families) throws IOException, InterruptedException { byte[]... families) throws IOException, InterruptedException {
table.setAutoFlush(false, true); table.setAutoFlushTo(false);
// Ensure one row per region // Ensure one row per region
assertTrue(rows >= KEYS.length); assertTrue(rows >= KEYS.length);