001/** 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.replication.master; 019 020import static org.apache.hadoop.hbase.replication.ZKReplicationPeerStorage.PEERS_ZNODE; 021import static org.apache.hadoop.hbase.replication.ZKReplicationPeerStorage.PEERS_ZNODE_DEFAULT; 022import static org.apache.hadoop.hbase.replication.ZKReplicationStorageBase.REPLICATION_ZNODE; 023import static org.apache.hadoop.hbase.replication.ZKReplicationStorageBase.REPLICATION_ZNODE_DEFAULT; 024 025import java.io.IOException; 026import org.apache.hadoop.conf.Configuration; 027import org.apache.hadoop.hbase.HBaseConfiguration; 028import org.apache.hadoop.hbase.client.Admin; 029import org.apache.hadoop.hbase.client.Connection; 030import org.apache.hadoop.hbase.client.ConnectionFactory; 031import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil; 032import org.apache.hadoop.hbase.replication.ReplicationException; 033import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; 034import org.apache.hadoop.hbase.replication.ReplicationPeerStorage; 035import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; 036import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; 037import org.apache.hadoop.hbase.zookeeper.ZKUtil; 038import org.apache.hadoop.hbase.zookeeper.ZKWatcher; 039import org.apache.hadoop.hbase.zookeeper.ZNodePaths; 040import org.apache.yetus.audience.InterfaceAudience; 041import org.apache.yetus.audience.InterfaceStability; 042import org.apache.zookeeper.KeeperException; 043import org.slf4j.Logger; 044import org.slf4j.LoggerFactory; 045 046/** 047 * 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 048 * be removed in HBase 3.x. See HBASE-11393 049 */ 050@InterfaceAudience.Private 051@InterfaceStability.Unstable 052public class ReplicationPeerConfigUpgrader{ 053 054 private static final String TABLE_CFS_ZNODE = "zookeeper.znode.replication.peers.tableCFs"; 055 private static final String TABLE_CFS_ZNODE_DEFAULT = "tableCFs"; 056 057 private static final Logger LOG = LoggerFactory.getLogger(ReplicationPeerConfigUpgrader.class); 058 private final Configuration conf; 059 private final ZKWatcher zookeeper; 060 private final ReplicationPeerStorage peerStorage; 061 062 public ReplicationPeerConfigUpgrader(ZKWatcher zookeeper, Configuration conf) { 063 this.zookeeper = zookeeper; 064 this.conf = conf; 065 this.peerStorage = ReplicationStorageFactory.getReplicationPeerStorage(zookeeper, conf); 066 } 067 068 public void upgrade() throws Exception { 069 try (Connection conn = ConnectionFactory.createConnection(conf)) { 070 Admin admin = conn.getAdmin(); 071 admin.listReplicationPeers().forEach((peerDesc) -> { 072 String peerId = peerDesc.getPeerId(); 073 ReplicationPeerConfig peerConfig = peerDesc.getPeerConfig(); 074 if ((peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty()) 075 || (peerConfig.getTableCFsMap() != null && !peerConfig.getTableCFsMap().isEmpty())) { 076 peerConfig.setReplicateAllUserTables(false); 077 try { 078 admin.updateReplicationPeerConfig(peerId, peerConfig); 079 } catch (Exception e) { 080 LOG.error("Failed to upgrade replication peer config for peerId=" + peerId, e); 081 } 082 } 083 }); 084 } 085 } 086 087 public void copyTableCFs() throws ReplicationException { 088 for (String peerId : peerStorage.listPeerIds()) { 089 if (!copyTableCFs(peerId)) { 090 LOG.error("upgrade tableCFs failed for peerId=" + peerId); 091 } 092 } 093 } 094 095 protected String getTableCFsNode(String peerId) { 096 String replicationZNode = ZNodePaths.joinZNode(zookeeper.getZNodePaths().baseZNode, 097 conf.get(REPLICATION_ZNODE, REPLICATION_ZNODE_DEFAULT)); 098 String peersZNode = 099 ZNodePaths.joinZNode(replicationZNode, conf.get(PEERS_ZNODE, PEERS_ZNODE_DEFAULT)); 100 return ZNodePaths.joinZNode(peersZNode, 101 ZNodePaths.joinZNode(peerId, conf.get(TABLE_CFS_ZNODE, TABLE_CFS_ZNODE_DEFAULT))); 102 } 103 104 public boolean copyTableCFs(String peerId) throws ReplicationException { 105 String tableCFsNode = getTableCFsNode(peerId); 106 try { 107 if (ZKUtil.checkExists(zookeeper, tableCFsNode) != -1) { 108 ReplicationPeerConfig rpc = peerStorage.getPeerConfig(peerId); 109 // We only need to copy data from tableCFs node to rpc Node the first time hmaster start. 110 if (rpc.getTableCFsMap() == null || rpc.getTableCFsMap().isEmpty()) { 111 // we copy TableCFs node into PeerNode 112 LOG.info("Copy table ColumnFamilies into peer=" + peerId); 113 ReplicationProtos.TableCF[] tableCFs = 114 ReplicationPeerConfigUtil.parseTableCFs(ZKUtil.getData(this.zookeeper, tableCFsNode)); 115 if (tableCFs != null && tableCFs.length > 0) { 116 rpc.setTableCFsMap(ReplicationPeerConfigUtil.convert2Map(tableCFs)); 117 peerStorage.updatePeerConfig(peerId, rpc); 118 } 119 } else { 120 LOG.info("No tableCFs in peerNode:" + peerId); 121 } 122 } 123 } catch (KeeperException e) { 124 LOG.warn("NOTICE!! Update peerId failed, peerId=" + peerId, e); 125 return false; 126 } catch (InterruptedException e) { 127 LOG.warn("NOTICE!! Update peerId failed, peerId=" + peerId, e); 128 return false; 129 } catch (IOException e) { 130 LOG.warn("NOTICE!! Update peerId failed, peerId=" + peerId, e); 131 return false; 132 } 133 return true; 134 } 135 136 private static void printUsageAndExit() { 137 System.err.printf( 138 "Usage: hbase org.apache.hadoop.hbase.replication.master.ReplicationPeerConfigUpgrader" 139 + " [options]"); 140 System.err.println(" where [options] are:"); 141 System.err.println(" -h|-help Show this help and exit."); 142 System.err.println(" copyTableCFs Copy table-cfs to replication peer config"); 143 System.err.println(" upgrade Upgrade replication peer config to new format"); 144 System.err.println(); 145 System.exit(1); 146 } 147 148 public static void main(String[] args) throws Exception { 149 if (args.length != 1) { 150 printUsageAndExit(); 151 } 152 if (args[0].equals("-help") || args[0].equals("-h")) { 153 printUsageAndExit(); 154 } else if (args[0].equals("copyTableCFs")) { 155 Configuration conf = HBaseConfiguration.create(); 156 try (ZKWatcher zkw = new ZKWatcher(conf, "ReplicationPeerConfigUpgrader", null)) { 157 ReplicationPeerConfigUpgrader tableCFsUpdater = 158 new ReplicationPeerConfigUpgrader(zkw, conf); 159 tableCFsUpdater.copyTableCFs(); 160 } 161 } else if (args[0].equals("upgrade")) { 162 Configuration conf = HBaseConfiguration.create(); 163 try (ZKWatcher zkw = new ZKWatcher(conf, "ReplicationPeerConfigUpgrader", null)) { 164 ReplicationPeerConfigUpgrader upgrader = new ReplicationPeerConfigUpgrader(zkw, conf); 165 upgrader.upgrade(); 166 } 167 } else { 168 printUsageAndExit(); 169 } 170 } 171}