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; 019 020import java.util.Arrays; 021import java.util.Collections; 022import java.util.List; 023import org.apache.hadoop.conf.Configuration; 024import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil; 025import org.apache.hadoop.hbase.exceptions.DeserializationException; 026import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; 027import org.apache.hadoop.hbase.zookeeper.ZKUtil; 028import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp; 029import org.apache.hadoop.hbase.zookeeper.ZKWatcher; 030import org.apache.hadoop.hbase.zookeeper.ZNodePaths; 031import org.apache.yetus.audience.InterfaceAudience; 032import org.apache.zookeeper.KeeperException; 033 034/** 035 * ZK based replication peer storage. 036 */ 037@InterfaceAudience.Private 038public class ZKReplicationPeerStorage extends ZKReplicationStorageBase 039 implements ReplicationPeerStorage { 040 041 public static final String PEERS_ZNODE = "zookeeper.znode.replication.peers"; 042 public static final String PEERS_ZNODE_DEFAULT = "peers"; 043 044 public static final String PEERS_STATE_ZNODE = "zookeeper.znode.replication.peers.state"; 045 public static final String PEERS_STATE_ZNODE_DEFAULT = "peer-state"; 046 047 public static final byte[] ENABLED_ZNODE_BYTES = 048 toByteArray(ReplicationProtos.ReplicationState.State.ENABLED); 049 public static final byte[] DISABLED_ZNODE_BYTES = 050 toByteArray(ReplicationProtos.ReplicationState.State.DISABLED); 051 052 /** 053 * The name of the znode that contains the replication status of a remote slave (i.e. peer) 054 * cluster. 055 */ 056 private final String peerStateNodeName; 057 058 /** 059 * The name of the znode that contains a list of all remote slave (i.e. peer) clusters. 060 */ 061 private final String peersZNode; 062 063 public ZKReplicationPeerStorage(ZKWatcher zookeeper, Configuration conf) { 064 super(zookeeper, conf); 065 this.peerStateNodeName = conf.get(PEERS_STATE_ZNODE, PEERS_STATE_ZNODE_DEFAULT); 066 String peersZNodeName = conf.get(PEERS_ZNODE, PEERS_ZNODE_DEFAULT); 067 this.peersZNode = ZNodePaths.joinZNode(replicationZNode, peersZNodeName); 068 } 069 070 public String getPeerStateNode(String peerId) { 071 return ZNodePaths.joinZNode(getPeerNode(peerId), peerStateNodeName); 072 } 073 074 public String getPeerNode(String peerId) { 075 return ZNodePaths.joinZNode(peersZNode, peerId); 076 } 077 078 @Override 079 public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) 080 throws ReplicationException { 081 try { 082 ZKUtil.createWithParents(zookeeper, peersZNode); 083 ZKUtil.multiOrSequential(zookeeper, 084 Arrays.asList( 085 ZKUtilOp.createAndFailSilent(getPeerNode(peerId), 086 ReplicationPeerConfigUtil.toByteArray(peerConfig)), 087 ZKUtilOp.createAndFailSilent(getPeerStateNode(peerId), 088 enabled ? ENABLED_ZNODE_BYTES : DISABLED_ZNODE_BYTES)), 089 false); 090 } catch (KeeperException e) { 091 throw new ReplicationException("Could not add peer with id=" + peerId + ", peerConfif=>" 092 + peerConfig + ", state=" + (enabled ? "ENABLED" : "DISABLED"), e); 093 } 094 } 095 096 @Override 097 public void removePeer(String peerId) throws ReplicationException { 098 try { 099 ZKUtil.deleteNodeRecursively(zookeeper, getPeerNode(peerId)); 100 } catch (KeeperException e) { 101 throw new ReplicationException("Could not remove peer with id=" + peerId, e); 102 } 103 } 104 105 @Override 106 public void setPeerState(String peerId, boolean enabled) throws ReplicationException { 107 byte[] stateBytes = enabled ? ENABLED_ZNODE_BYTES : DISABLED_ZNODE_BYTES; 108 try { 109 ZKUtil.setData(zookeeper, getPeerStateNode(peerId), stateBytes); 110 } catch (KeeperException e) { 111 throw new ReplicationException("Unable to change state of the peer with id=" + peerId, e); 112 } 113 } 114 115 @Override 116 public void updatePeerConfig(String peerId, ReplicationPeerConfig peerConfig) 117 throws ReplicationException { 118 try { 119 ZKUtil.setData(this.zookeeper, getPeerNode(peerId), 120 ReplicationPeerConfigUtil.toByteArray(peerConfig)); 121 } catch (KeeperException e) { 122 throw new ReplicationException( 123 "There was a problem trying to save changes to the " + "replication peer " + peerId, e); 124 } 125 } 126 127 @Override 128 public List<String> listPeerIds() throws ReplicationException { 129 try { 130 List<String> children = ZKUtil.listChildrenNoWatch(zookeeper, peersZNode); 131 return children != null ? children : Collections.emptyList(); 132 } catch (KeeperException e) { 133 throw new ReplicationException("Cannot get the list of peers", e); 134 } 135 } 136 137 @Override 138 public boolean isPeerEnabled(String peerId) throws ReplicationException { 139 try { 140 return Arrays.equals(ENABLED_ZNODE_BYTES, 141 ZKUtil.getData(zookeeper, getPeerStateNode(peerId))); 142 } catch (KeeperException | InterruptedException e) { 143 throw new ReplicationException("Unable to get status of the peer with id=" + peerId, e); 144 } 145 } 146 147 @Override 148 public ReplicationPeerConfig getPeerConfig(String peerId) throws ReplicationException { 149 byte[] data; 150 try { 151 data = ZKUtil.getData(zookeeper, getPeerNode(peerId)); 152 } catch (KeeperException | InterruptedException e) { 153 throw new ReplicationException("Error getting configuration for peer with id=" + peerId, e); 154 } 155 if (data == null || data.length == 0) { 156 throw new ReplicationException( 157 "Replication peer config data shouldn't be empty, peerId=" + peerId); 158 } 159 try { 160 return ReplicationPeerConfigUtil.parsePeerFrom(data); 161 } catch (DeserializationException e) { 162 throw new ReplicationException( 163 "Failed to parse replication peer config for peer with id=" + peerId, e); 164 } 165 } 166}