HBASE-20083 Fix findbugs error for ReplicationSyncUp

This commit is contained in:
zhangduo 2018-02-26 16:37:58 +08:00
parent 8c74d177f6
commit 2beda62a10
2 changed files with 18 additions and 45 deletions

View File

@ -49,35 +49,18 @@ import org.apache.hadoop.util.ToolRunner;
*/
public class ReplicationSyncUp extends Configured implements Tool {
private static Configuration conf;
private static final long SLEEP_TIME = 10000;
// although the tool is designed to be run on command line
// this api is provided for executing the tool through another app
public static void setConfigure(Configuration config) {
conf = config;
}
/**
* Main program
* @param args
* @throws Exception
*/
public static void main(String[] args) throws Exception {
if (conf == null) conf = HBaseConfiguration.create();
int ret = ToolRunner.run(conf, new ReplicationSyncUp(), args);
int ret = ToolRunner.run(HBaseConfiguration.create(), new ReplicationSyncUp(), args);
System.exit(ret);
}
@Override
public int run(String[] args) throws Exception {
Replication replication;
ReplicationSourceManager manager;
FileSystem fs;
Path oldLogDir, logDir, walRootDir;
ZKWatcher zkw;
Abortable abortable = new Abortable() {
@Override
public void abort(String why, Throwable e) {
@ -88,23 +71,19 @@ public class ReplicationSyncUp extends Configured implements Tool {
return false;
}
};
Configuration conf = getConf();
try (ZKWatcher zkw =
new ZKWatcher(conf, "syncupReplication" + System.currentTimeMillis(), abortable, true)) {
Path walRootDir = FSUtils.getWALRootDir(conf);
FileSystem fs = FSUtils.getWALFileSystem(conf);
Path oldLogDir = new Path(walRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
Path logDir = new Path(walRootDir, HConstants.HREGION_LOGDIR_NAME);
zkw =
new ZKWatcher(conf, "syncupReplication" + System.currentTimeMillis(), abortable,
true);
walRootDir = FSUtils.getWALRootDir(conf);
fs = FSUtils.getWALFileSystem(conf);
oldLogDir = new Path(walRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
logDir = new Path(walRootDir, HConstants.HREGION_LOGDIR_NAME);
System.out.println("Start Replication Server start");
replication = new Replication();
replication.initialize(new DummyServer(zkw), fs, logDir, oldLogDir, null);
manager = replication.getReplicationManager();
manager.init().get();
try {
System.out.println("Start Replication Server start");
Replication replication = new Replication();
replication.initialize(new DummyServer(zkw), fs, logDir, oldLogDir, null);
ReplicationSourceManager manager = replication.getReplicationManager();
manager.init().get();
while (manager.activeFailoverTaskCount() > 0) {
Thread.sleep(SLEEP_TIME);
}
@ -114,15 +93,12 @@ public class ReplicationSyncUp extends Configured implements Tool {
manager.join();
} catch (InterruptedException e) {
System.err.println("didn't wait long enough:" + e);
return (-1);
} finally {
zkw.close();
return -1;
}
return 0;
}
static class DummyServer implements Server {
class DummyServer implements Server {
String hostname;
ZKWatcher zkw;
@ -138,7 +114,7 @@ public class ReplicationSyncUp extends Configured implements Tool {
@Override
public Configuration getConfiguration() {
return conf;
return getConf();
}
@Override
@ -191,7 +167,6 @@ public class ReplicationSyncUp extends Configured implements Tool {
@Override
public ClusterConnection getClusterConnection() {
// TODO Auto-generated method stub
return null;
}

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.replication.regionserver.ReplicationSyncUp;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.util.ToolRunner;
import org.junit.Before;
import org.junit.ClassRule;
import org.junit.Test;
@ -420,9 +421,6 @@ public class TestReplicationSyncUpTool extends TestReplicationBase {
}
protected void syncUp(HBaseTestingUtility ut) throws Exception {
ReplicationSyncUp.setConfigure(ut.getConfiguration());
String[] arguments = new String[] { null };
new ReplicationSyncUp().run(arguments);
ToolRunner.run(ut.getConfiguration(), new ReplicationSyncUp(), new String[0]);
}
}