HBASE-16938 TableCFsUpdater maybe failed due to no write permission on peerNode

Signed-off-by: Enis Soztutar <enis@apache.org>
This commit is contained in:
Guanghao Zhang 2016-10-25 09:50:47 +08:00 committed by Enis Soztutar
parent 07757501d7
commit 12eec5bc5d
1 changed files with 32 additions and 3 deletions

View File

@ -22,6 +22,7 @@ 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;
import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper; import org.apache.hadoop.hbase.client.replication.ReplicationSerDeHelper;
@ -37,7 +38,7 @@ import java.io.IOException;
import java.util.List; import java.util.List;
/** /**
* This class is used to upgrade TableCFs from HBase 1.x to HBase 2.x. * This class is used to upgrade TableCFs from HBase 1.0, 1.1, 1.2, 1.3 to HBase 1.4 or 2.x.
* It will be removed in HBase 3.x. See HBASE-11393 * It will be removed in HBase 3.x. See HBASE-11393
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
@ -56,7 +57,7 @@ public class TableCFsUpdater extends ReplicationStateZKBase {
try { try {
znodes = ZKUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode); znodes = ZKUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode);
} catch (KeeperException e) { } catch (KeeperException e) {
LOG.warn("", e); LOG.error("Failed to get peers znode", e);
} }
if (znodes != null) { if (znodes != null) {
for (String peerId : znodes) { for (String peerId : znodes) {
@ -119,4 +120,32 @@ public class TableCFsUpdater extends ReplicationStateZKBase {
} }
} }
private static void printUsageAndExit() {
System.err.printf("Usage: bin/hbase org.apache.hadoop.hbase.replication.master.TableCFsUpdater [options]");
System.err.println(" where [options] are:");
System.err.println(" -h|-help Show this help and exit.");
System.err.println(" update Copy table-cfs to replication peer config");
System.err.println();
System.exit(1);
}
public static void main(String[] args) throws Exception {
if (args.length != 1) {
printUsageAndExit();
}
if (args[0].equals("-help") || args[0].equals("-h")) {
printUsageAndExit();
} else if (args[0].equals("update")) {
Configuration conf = HBaseConfiguration.create();
ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "TableCFsUpdater", null);
try {
TableCFsUpdater tableCFsUpdater = new TableCFsUpdater(zkw, conf, null);
tableCFsUpdater.update();
} finally {
zkw.close();
}
} else {
printUsageAndExit();
}
}
} }