-
Notifications
You must be signed in to change notification settings - Fork 3.3k
/
TableReplicationPeerStorage.java
171 lines (158 loc) · 6.68 KB
/
TableReplicationPeerStorage.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.replication;
import static org.apache.hadoop.hbase.replication.ReplicationUtils.PEER_STATE_DISABLED_BYTES;
import static org.apache.hadoop.hbase.replication.ReplicationUtils.PEER_STATE_ENABLED_BYTES;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Table based replication peer storage.
*/
@InterfaceAudience.Private
public class TableReplicationPeerStorage extends TableReplicationStorageBase
implements ReplicationPeerStorage {
public TableReplicationPeerStorage(ZKWatcher zookeeper, Configuration conf) throws IOException {
super(zookeeper, conf);
}
@Override
public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
throws ReplicationException {
try (Table table = getReplicationMetaTable()) {
Put put = new Put(Bytes.toBytes(peerId));
put.addColumn(FAMILY_PEER, QUALIFIER_PEER_CONFIG,
ReplicationPeerConfigUtil.toByteArray(peerConfig));
put.addColumn(FAMILY_PEER, QUALIFIER_PEER_STATE,
enabled ? PEER_STATE_ENABLED_BYTES : PEER_STATE_DISABLED_BYTES);
table.put(put);
} catch (IOException e) {
throw new ReplicationException("Failed to add peer " + peerId, e);
}
}
@Override
public void removePeer(String peerId) throws ReplicationException {
try (Table table = getReplicationMetaTable()) {
Delete delete = new Delete(Bytes.toBytes(peerId));
table.delete(delete);
} catch (IOException e) {
throw new ReplicationException("Failed to remove peer " + peerId, e);
}
}
// TODO make it to be a checkExistAndMutate operation.
private boolean peerExist(String peerId, Table table) throws IOException {
Get get = new Get(Bytes.toBytes(peerId));
get.addColumn(FAMILY_PEER, QUALIFIER_PEER_STATE);
return table.exists(get);
}
@Override
public void setPeerState(String peerId, boolean enabled) throws ReplicationException {
try (Table table = getReplicationMetaTable()) {
if (!peerExist(peerId, table)) {
throw new ReplicationException("Peer " + peerId + " does not exist.");
}
Put put = new Put(Bytes.toBytes(peerId));
put.addColumn(FAMILY_PEER, QUALIFIER_PEER_STATE,
enabled ? PEER_STATE_ENABLED_BYTES : PEER_STATE_DISABLED_BYTES);
table.put(put);
} catch (IOException e) {
throw new ReplicationException(
"Failed to set peer state, peerId=" + peerId + ", state=" + enabled, e);
}
}
@Override
public void updatePeerConfig(String peerId, ReplicationPeerConfig peerConfig)
throws ReplicationException {
try (Table table = getReplicationMetaTable()) {
if (!peerExist(peerId, table)) {
throw new ReplicationException("Peer " + peerId + " does not exist.");
}
Put put = new Put(Bytes.toBytes(peerId));
put.addColumn(FAMILY_PEER, QUALIFIER_PEER_CONFIG,
ReplicationPeerConfigUtil.toByteArray(peerConfig));
table.put(put);
} catch (IOException e) {
throw new ReplicationException("Failed to update peer configuration, peerId=" + peerId, e);
}
}
@Override
public List<String> listPeerIds() throws ReplicationException {
try (Table table = getReplicationMetaTable()) {
Scan scan = new Scan().addColumn(FAMILY_PEER, QUALIFIER_PEER_STATE);
try (ResultScanner scanner = table.getScanner(scan)) {
List<String> peerIds = new ArrayList<>();
for (Result r : scanner) {
peerIds.add(Bytes.toString(r.getRow()));
}
return peerIds;
}
} catch (IOException e) {
throw new ReplicationException("Failed to list peers", e);
}
}
@Override
public boolean isPeerEnabled(String peerId) throws ReplicationException {
try (Table table = getReplicationMetaTable()) {
Get get = new Get(Bytes.toBytes(peerId)).addColumn(FAMILY_PEER, QUALIFIER_PEER_STATE);
Result r = table.get(get);
if (r == null) {
throw new ReplicationException("Peer " + peerId + " does not found");
}
return Arrays.equals(PEER_STATE_ENABLED_BYTES, r.getValue(FAMILY_PEER, QUALIFIER_PEER_STATE));
} catch (IOException e) {
throw new ReplicationException("Failed to read the peer state, peerId=" + peerId, e);
}
}
@Override
public ReplicationPeerConfig getPeerConfig(String peerId) throws ReplicationException {
try (Table table = getReplicationMetaTable()) {
Get get = new Get(Bytes.toBytes(peerId)).addColumn(FAMILY_PEER, QUALIFIER_PEER_CONFIG);
Result r = table.get(get);
if (r == null) {
throw new ReplicationException("Peer " + peerId + " does not found");
}
byte[] data = r.getValue(FAMILY_PEER, QUALIFIER_PEER_CONFIG);
if (data == null || data.length == 0) {
throw new ReplicationException(
"Replication peer config data shouldn't be empty, peerId=" + peerId);
}
try {
return ReplicationPeerConfigUtil.parsePeerFrom(data);
} catch (DeserializationException e) {
throw new ReplicationException(
"Failed to parse replication peer config for peer with id=" + peerId, e);
}
} catch (IOException e) {
throw new ReplicationException(
"Failed to read the peer configuration in hbase:replication, peerId=" + peerId, e);
}
}
}