HBASE-8895 Misc: replace hbase.client.retries.number w/ define and remove unused imports

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1500988 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2013-07-08 22:18:03 +00:00
parent ba683b9386
commit 5747b899bd
29 changed files with 91 additions and 97 deletions

View File

@ -1977,7 +1977,7 @@ public class HBaseAdmin implements Abortable, Closeable {
Configuration copyOfConf = HBaseConfiguration.create(conf);
// We set it to make it fail as soon as possible if HBase is not available
copyOfConf.setInt("hbase.client.retries.number", 1);
copyOfConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
copyOfConf.setInt("zookeeper.recovery.retry", 0);
HConnectionManager.HConnectionImplementation connection

View File

@ -23,8 +23,8 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.RetryCounter;
import org.apache.hadoop.hbase.util.RetryCounterFactory;
import org.apache.hadoop.hbase.util.BasicRetryAccountant;
import org.apache.hadoop.hbase.util.RetryAccountant;
import org.apache.zookeeper.AsyncCallback;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
@ -76,7 +76,7 @@ public class RecoverableZooKeeper {
private static final Log LOG = LogFactory.getLog(RecoverableZooKeeper.class);
// the actual ZooKeeper client instance
volatile private ZooKeeper zk;
private final RetryCounterFactory retryCounterFactory;
private final BasicRetryAccountantFactory retryCounterFactory;
// An identifier of this process in the cluster
private final String identifier;
private final byte[] id;
@ -111,7 +111,7 @@ public class RecoverableZooKeeper {
// TODO: Add support for zk 'chroot'; we don't add it to the quorumServers String as we should.
this.zk = new ZooKeeper(quorumServers, sessionTimeout, watcher);
this.retryCounterFactory =
new RetryCounterFactory(maxRetries, retryIntervalMillis);
new BasicRetryAccountantFactory(maxRetries, retryIntervalMillis);
if (identifier == null || identifier.length() == 0) {
// the identifier = processID@hostName
@ -146,7 +146,7 @@ public class RecoverableZooKeeper {
*/
public void delete(String path, int version)
throws InterruptedException, KeeperException {
RetryCounter retryCounter = retryCounterFactory.create();
RetryAccountant retryCounter = retryCounterFactory.create();
boolean isRetry = false; // False for first attempt, true for all retries.
while (true) {
try {
@ -173,8 +173,7 @@ public class RecoverableZooKeeper {
throw e;
}
}
retryCounter.sleepUntilNextRetry();
retryCounter.useRetry();
retryCounter.sleep();
isRetry = true;
}
}
@ -185,7 +184,7 @@ public class RecoverableZooKeeper {
*/
public Stat exists(String path, Watcher watcher)
throws KeeperException, InterruptedException {
RetryCounter retryCounter = retryCounterFactory.create();
RetryAccountant retryCounter = retryCounterFactory.create();
while (true) {
try {
return zk.exists(path, watcher);
@ -201,8 +200,7 @@ public class RecoverableZooKeeper {
throw e;
}
}
retryCounter.sleepUntilNextRetry();
retryCounter.useRetry();
retryCounter.sleep();
}
}
@ -212,7 +210,7 @@ public class RecoverableZooKeeper {
*/
public Stat exists(String path, boolean watch)
throws KeeperException, InterruptedException {
RetryCounter retryCounter = retryCounterFactory.create();
RetryAccountant retryCounter = retryCounterFactory.create();
while (true) {
try {
return zk.exists(path, watch);
@ -228,17 +226,15 @@ public class RecoverableZooKeeper {
throw e;
}
}
retryCounter.sleepUntilNextRetry();
retryCounter.useRetry();
retryCounter.sleep();
}
}
private void retryOrThrow(RetryCounter retryCounter, KeeperException e,
private void retryOrThrow(RetryAccountant retryCounter, KeeperException e,
String opName) throws KeeperException {
LOG.warn("Possibly transient ZooKeeper, quorum=" + quorumServers + ", exception=" + e);
if (!retryCounter.shouldRetry()) {
LOG.error("ZooKeeper " + opName + " failed after "
+ retryCounter.getMaxRetries() + " retries");
if (!retryCounter.retry()) {
LOG.error("ZooKeeper " + opName + " failed " + retryCounter);
throw e;
}
}
@ -249,7 +245,7 @@ public class RecoverableZooKeeper {
*/
public List<String> getChildren(String path, Watcher watcher)
throws KeeperException, InterruptedException {
RetryCounter retryCounter = retryCounterFactory.create();
RetryAccountant retryCounter = retryCounterFactory.create();
while (true) {
try {
return zk.getChildren(path, watcher);
@ -265,8 +261,7 @@ public class RecoverableZooKeeper {
throw e;
}
}
retryCounter.sleepUntilNextRetry();
retryCounter.useRetry();
retryCounter.sleep();
}
}
@ -276,7 +271,7 @@ public class RecoverableZooKeeper {
*/
public List<String> getChildren(String path, boolean watch)
throws KeeperException, InterruptedException {
RetryCounter retryCounter = retryCounterFactory.create();
RetryAccountant retryCounter = retryCounterFactory.create();
while (true) {
try {
return zk.getChildren(path, watch);
@ -292,8 +287,7 @@ public class RecoverableZooKeeper {
throw e;
}
}
retryCounter.sleepUntilNextRetry();
retryCounter.useRetry();
retryCounter.sleep();
}
}
@ -303,7 +297,7 @@ public class RecoverableZooKeeper {
*/
public byte[] getData(String path, Watcher watcher, Stat stat)
throws KeeperException, InterruptedException {
RetryCounter retryCounter = retryCounterFactory.create();
RetryAccountant retryCounter = retryCounterFactory.create();
while (true) {
try {
byte[] revData = zk.getData(path, watcher, stat);
@ -320,8 +314,7 @@ public class RecoverableZooKeeper {
throw e;
}
}
retryCounter.sleepUntilNextRetry();
retryCounter.useRetry();
retryCounter.sleep();
}
}
@ -331,7 +324,7 @@ public class RecoverableZooKeeper {
*/
public byte[] getData(String path, boolean watch, Stat stat)
throws KeeperException, InterruptedException {
RetryCounter retryCounter = retryCounterFactory.create();
RetryAccountant retryCounter = retryCounterFactory.create();
while (true) {
try {
byte[] revData = zk.getData(path, watch, stat);
@ -348,8 +341,7 @@ public class RecoverableZooKeeper {
throw e;
}
}
retryCounter.sleepUntilNextRetry();
retryCounter.useRetry();
retryCounter.sleep();
}
}
@ -361,7 +353,7 @@ public class RecoverableZooKeeper {
*/
public Stat setData(String path, byte[] data, int version)
throws KeeperException, InterruptedException {
RetryCounter retryCounter = retryCounterFactory.create();
RetryAccountant retryCounter = retryCounterFactory.create();
byte[] newData = appendMetaData(data);
boolean isRetry = false;
while (true) {
@ -394,8 +386,7 @@ public class RecoverableZooKeeper {
throw e;
}
}
retryCounter.sleepUntilNextRetry();
retryCounter.useRetry();
retryCounter.sleep();
isRetry = true;
}
}
@ -436,7 +427,7 @@ public class RecoverableZooKeeper {
private String createNonSequential(String path, byte[] data, List<ACL> acl,
CreateMode createMode) throws KeeperException, InterruptedException {
RetryCounter retryCounter = retryCounterFactory.create();
RetryAccountant retryCounter = retryCounterFactory.create();
boolean isRetry = false; // False for first attempt, true for all retries.
while (true) {
try {
@ -473,8 +464,7 @@ public class RecoverableZooKeeper {
throw e;
}
}
retryCounter.sleepUntilNextRetry();
retryCounter.useRetry();
retryCounter.sleep();
isRetry = true;
}
}
@ -482,7 +472,7 @@ public class RecoverableZooKeeper {
private String createSequential(String path, byte[] data,
List<ACL> acl, CreateMode createMode)
throws KeeperException, InterruptedException {
RetryCounter retryCounter = retryCounterFactory.create();
RetryAccountant retryCounter = retryCounterFactory.create();
boolean first = true;
String newPath = path+this.identifier;
while (true) {
@ -508,8 +498,7 @@ public class RecoverableZooKeeper {
throw e;
}
}
retryCounter.sleepUntilNextRetry();
retryCounter.useRetry();
retryCounter.sleep();
}
}
/**
@ -545,7 +534,7 @@ public class RecoverableZooKeeper {
*/
public List<OpResult> multi(Iterable<Op> ops)
throws KeeperException, InterruptedException {
RetryCounter retryCounter = retryCounterFactory.create();
RetryAccountant retryCounter = retryCounterFactory.create();
Iterable<Op> multiOps = prepareZKMulti(ops);
while (true) {
try {
@ -562,8 +551,7 @@ public class RecoverableZooKeeper {
throw e;
}
}
retryCounter.sleepUntilNextRetry();
retryCounter.useRetry();
retryCounter.sleep();
}
}
@ -671,4 +659,22 @@ public class RecoverableZooKeeper {
public String getIdentifier() {
return identifier;
}
}
/**
* Factory to create {@link BasicRetryAccountant}s. Saves having to have maxRetries and
* retryIntervalMillis available everywhere.
*/
static class BasicRetryAccountantFactory {
private final int maxRetries;
private final int retryIntervalMillis;
BasicRetryAccountantFactory(int maxRetries, int retryIntervalMillis) {
this.maxRetries = maxRetries;
this.retryIntervalMillis = retryIntervalMillis;
}
RetryAccountant create() {
return new BasicRetryAccountant(maxRetries, retryIntervalMillis);
}
}
}

View File

@ -73,7 +73,7 @@ public class TestSnapshotFromAdmin {
.mock(HConnectionManager.HConnectionImplementation.class);
Configuration conf = HBaseConfiguration.create();
// setup the conf to match the expected properties
conf.setInt("hbase.client.retries.number", numRetries);
conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, numRetries);
conf.setLong("hbase.client.pause", pauseTime);
// mock the master admin to our mock
MasterAdminKeepAliveConnection mockMaster = Mockito.mock(MasterAdminKeepAliveConnection.class);

View File

@ -202,7 +202,7 @@ public class HMasterCommandLine extends ServerCommandLine {
try {
Configuration conf = getConf();
// Don't try more than once
conf.setInt("hbase.client.retries.number", 1);
conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
adm = new HBaseAdmin(getConf());
} catch (MasterNotRunningException e) {
LOG.error("Master not running");

View File

@ -20,9 +20,7 @@
package org.apache.hadoop.hbase.util;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
@ -35,16 +33,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
@ -78,7 +67,7 @@ public class MetaUtils {
*/
public MetaUtils(Configuration conf) throws IOException {
this.conf = conf;
conf.setInt("hbase.client.retries.number", 1);
conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
this.metaRegion = null;
initialize();
}

View File

@ -34,7 +34,7 @@ module Hbase
else
self.configuration = org.apache.hadoop.hbase.HBaseConfiguration.create
# Turn off retries in hbase and ipc. Human doesn't want to wait on N retries.
configuration.setInt("hbase.client.retries.number", 7)
configuration.setInt(org.apache.hadoop.hbase.HConstants::HBASE_CLIENT_RETRIES_NUMBER, 7)
configuration.setInt("ipc.client.connect.max.retries", 3)
end
end

View File

@ -92,7 +92,7 @@ public class TestZooKeeperTableArchiveClient {
// only compact with 3 files
conf.setInt("hbase.hstore.compaction.min", 3);
// drop the number of attempts for the hbase admin
conf.setInt("hbase.client.retries.number", 1);
conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
}
@After

View File

@ -77,7 +77,7 @@ public class TestMetaReaderEditor {
// Tests to 4 retries every 5 seconds. Make it try every 1 second so more
// responsive. 1 second is default as is ten retries.
c.setLong("hbase.client.pause", 1000);
c.setInt("hbase.client.retries.number", 10);
c.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 10);
zkw = new ZooKeeperWatcher(c, "TestMetaReaderEditor", ABORTABLE);
CT = new CatalogTracker(zkw, c, ABORTABLE);
CT.start();

View File

@ -68,7 +68,7 @@ public class TestCloneSnapshotFromClient {
TEST_UTIL.getConfiguration().setInt("hbase.hstore.compactionThreshold", 10);
TEST_UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100);
TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 250);
TEST_UTIL.getConfiguration().setInt("hbase.client.retries.number", 6);
TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
TEST_UTIL.getConfiguration().setBoolean(
"hbase.master.enabletable.roundrobin", true);
TEST_UTIL.startMiniCluster(3);

View File

@ -72,7 +72,7 @@ public class TestRestoreSnapshotFromClient {
TEST_UTIL.getConfiguration().setInt("hbase.hstore.compactionThreshold", 10);
TEST_UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100);
TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 250);
TEST_UTIL.getConfiguration().setInt("hbase.client.retries.number", 6);
TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
TEST_UTIL.getConfiguration().setBoolean(
"hbase.master.enabletable.roundrobin", true);
TEST_UTIL.startMiniCluster(3);

View File

@ -26,6 +26,7 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.LargeTests;
import org.junit.AfterClass;
import org.junit.BeforeClass;
@ -47,7 +48,7 @@ public class TestShell {
TEST_UTIL.getConfiguration().setBoolean("hbase.online.schema.update.enable", true);
TEST_UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100);
TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 250);
TEST_UTIL.getConfiguration().setInt("hbase.client.retries.number", 6);
TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
TEST_UTIL.startMiniCluster();
// Configure jruby runtime

View File

@ -85,7 +85,7 @@ public class TestSnapshotCloneIndependence {
// drop the number of attempts for the hbase admin
conf.setInt("hbase.regionserver.msginterval", 100);
conf.setInt("hbase.client.pause", 250);
conf.setInt("hbase.client.retries.number", 6);
conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
conf.setBoolean("hbase.master.enabletable.roundrobin", true);
// Avoid potentially aggressive splitting which would cause snapshot to fail
conf.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,

View File

@ -86,7 +86,7 @@ public class TestSnapshotFromClient {
// block writes if we get to 12 store files
conf.setInt("hbase.hstore.blockingStoreFiles", 12);
// drop the number of attempts for the hbase admin
conf.setInt("hbase.client.retries.number", 1);
conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
// Enable snapshot
conf.setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
conf.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,

View File

@ -107,7 +107,7 @@ public class TestSnapshotMetadata {
// drop the number of attempts for the hbase admin
conf.setInt("hbase.regionserver.msginterval", 100);
conf.setInt("hbase.client.pause", 250);
conf.setInt("hbase.client.retries.number", 6);
conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
conf.setBoolean("hbase.master.enabletable.roundrobin", true);
// Avoid potentially aggressive splitting which would cause snapshot to fail
conf.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,

View File

@ -35,6 +35,7 @@ import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
@ -111,7 +112,7 @@ public class TestRowProcessorEndpoint {
Configuration conf = util.getConfiguration();
conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
RowProcessorEndpoint.class.getName());
conf.setInt("hbase.client.retries.number", 1);
conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
conf.setLong("hbase.hregion.row.processor.timeout", 1000L);
util.startMiniCluster();
}

View File

@ -30,6 +30,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.exceptions.MasterNotRunningException;
@ -161,7 +162,7 @@ public class TestFilterWithScanLimits {
private static void initialize(Configuration conf) {
TestFilterWithScanLimits.conf = HBaseConfiguration.create(conf);
TestFilterWithScanLimits.conf.setInt("hbase.client.retries.number", 1);
TestFilterWithScanLimits.conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
try {
admin = new HBaseAdmin(conf);
} catch (MasterNotRunningException e) {

View File

@ -29,6 +29,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.exceptions.MasterNotRunningException;
@ -166,7 +167,7 @@ public class TestFilterWrapper {
private static void initialize(Configuration conf) {
TestFilterWrapper.conf = HBaseConfiguration.create(conf);
TestFilterWrapper.conf.setInt("hbase.client.retries.number", 1);
TestFilterWrapper.conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
try {
admin = new HBaseAdmin(conf);
} catch (MasterNotRunningException e) {

View File

@ -167,7 +167,7 @@ public class TestTableMapReduce {
HTable table = new HTable(UTIL.getConfiguration(), tableName);
boolean verified = false;
long pause = UTIL.getConfiguration().getLong("hbase.client.pause", 5 * 1000);
int numRetries = UTIL.getConfiguration().getInt("hbase.client.retries.number", 5);
int numRetries = UTIL.getConfiguration().getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5);
for (int i = 0; i < numRetries; i++) {
try {
LOG.info("Verification attempt #" + i);

View File

@ -39,6 +39,7 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.client.HTable;
@ -258,7 +259,7 @@ public class TestImportTsv implements Configurable {
HTable table = new HTable(conf, tableName);
boolean verified = false;
long pause = conf.getLong("hbase.client.pause", 5 * 1000);
int numRetries = conf.getInt("hbase.client.retries.number", 5);
int numRetries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5);
for (int i = 0; i < numRetries; i++) {
try {
Scan scan = new Scan();

View File

@ -163,7 +163,7 @@ public class TestMultithreadedTableMapper {
HTable table = new HTable(new Configuration(UTIL.getConfiguration()), tableName);
boolean verified = false;
long pause = UTIL.getConfiguration().getLong("hbase.client.pause", 5 * 1000);
int numRetries = UTIL.getConfiguration().getInt("hbase.client.retries.number", 5);
int numRetries = UTIL.getConfiguration().getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5);
for (int i = 0; i < numRetries; i++) {
try {
LOG.info("Verification attempt #" + i);

View File

@ -30,6 +30,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.client.HTable;
@ -175,7 +176,7 @@ public class TestTableMapReduce {
HTable table = new HTable(new Configuration(UTIL.getConfiguration()), tableName);
boolean verified = false;
long pause = UTIL.getConfiguration().getLong("hbase.client.pause", 5 * 1000);
int numRetries = UTIL.getConfiguration().getInt("hbase.client.retries.number", 5);
int numRetries = UTIL.getConfiguration().getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 5);
for (int i = 0; i < numRetries; i++) {
try {
LOG.info("Verification attempt #" + i);

View File

@ -115,7 +115,7 @@ public class TestSnapshotFromMaster {
// block writes if we get to 12 store files
conf.setInt("hbase.hstore.blockingStoreFiles", 12);
// drop the number of attempts for the hbase admin
conf.setInt("hbase.client.retries.number", 1);
conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
// Ensure no extra cleaners on by default (e.g. TimeToLiveHFileCleaner)
conf.set(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS, "");
conf.set(HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS, "");

View File

@ -173,7 +173,7 @@ public class TestFSErrorsExposed {
util.getConfiguration().setInt(
"hbase.regionserver.optionallogflushinterval", Integer.MAX_VALUE);
util.getConfiguration().setInt("hbase.client.retries.number", 3);
util.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 3);
util.startMiniCluster(1);
byte[] tableName = Bytes.toBytes("table");
@ -187,7 +187,7 @@ public class TestFSErrorsExposed {
);
admin.createTable(desc);
// Make it fail faster.
util.getConfiguration().setInt("hbase.client.retries.number", 1);
util.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
// Make a new Configuration so it makes a new connection that has the
// above configuration on it; else we use the old one w/ 10 as default.
HTable table = new HTable(new Configuration(util.getConfiguration()), tableName);

View File

@ -102,7 +102,7 @@ public class TestReplicationBase {
// Base conf2 on conf1 so it gets the right zk cluster.
conf2 = HBaseConfiguration.create(conf1);
conf2.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/2");
conf2.setInt("hbase.client.retries.number", 6);
conf2.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
conf2.setBoolean(HConstants.REPLICATION_ENABLE_KEY, true);
conf2.setBoolean("dfs.support.append", true);

View File

@ -19,45 +19,38 @@
package org.apache.hadoop.hbase.snapshot;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.net.URI;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
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.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.snapshot.ExportSnapshot;
import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil;
import org.apache.hadoop.mapreduce.Job;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
@ -86,7 +79,7 @@ public class TestExportSnapshot {
TEST_UTIL.getConfiguration().setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
TEST_UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100);
TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 250);
TEST_UTIL.getConfiguration().setInt("hbase.client.retries.number", 6);
TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
TEST_UTIL.getConfiguration().setBoolean("hbase.master.enabletable.roundrobin", true);
TEST_UTIL.startMiniCluster(3);
}

View File

@ -106,7 +106,7 @@ public class TestFlushSnapshotFromClient {
// block writes if we get to 12 store files
conf.setInt("hbase.hstore.blockingStoreFiles", 12);
// drop the number of attempts for the hbase admin
conf.setInt("hbase.client.retries.number", 1);
conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
// Enable snapshot
conf.setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
conf.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,

View File

@ -75,7 +75,7 @@ public class TestRestoreFlushSnapshotFromClient {
TEST_UTIL.getConfiguration().setBoolean("hbase.online.schema.update.enable", true);
TEST_UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100);
TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 250);
TEST_UTIL.getConfiguration().setInt("hbase.client.retries.number", 6);
TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6);
TEST_UTIL.getConfiguration().setBoolean(
"hbase.master.enabletable.roundrobin", true);

View File

@ -35,7 +35,7 @@ unless defined?($TEST_CLUSTER)
$TEST_CLUSTER = HBaseTestingUtility.new
$TEST_CLUSTER.configuration.setInt("hbase.regionserver.msginterval", 100)
$TEST_CLUSTER.configuration.setInt("hbase.client.pause", 250)
$TEST_CLUSTER.configuration.setInt("hbase.client.retries.number", 6)
$TEST_CLUSTER.configuration.setInt(org.apache.hadoop.hbase.HConstants::HBASE_CLIENT_RETRIES_NUMBER, 6)
$TEST_CLUSTER.startMiniCluster
@own_cluster = true
end

View File

@ -531,7 +531,7 @@
using method parallelization class ! -->
<testFailureIgnore>false</testFailureIgnore>
<forkedProcessTimeoutInSeconds>${surefire.timeout}</forkedProcessTimeoutInSeconds>
<argLine>-enableassertions -Xmx1900m
<argLine>-enableassertions -Xmx3g -XX:+CMSClassUnloadingEnabled -XX:+CMSPermGenSweepingEnabled -XX:MaxPermSize=1g
-Djava.security.egd=file:/dev/./urandom -Djava.net.preferIPv4Stack=true</argLine>
<redirectTestOutputToFile>${test.output.tofile}</redirectTestOutputToFile>
</configuration>
@ -1387,7 +1387,7 @@
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<configuration>
<argLine>-enableassertions -Xmx1900m -Djava.security.egd=file:/dev/./urandom -Djava.net.preferIPv4Stack=true</argLine>
<argLine>-enableassertions -Xmx3gm -XX:+CMSClassUnloadingEnabled -XX:+CMSPermGenSweepingEnabled -XX:MaxPermSize=1g -Djava.security.egd=file:/dev/./urandom -Djava.net.preferIPv4Stack=true</argLine>
<systemProperties>
<property>
<name>java.net.preferIPv4Stack</name>