1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.master;
19
20 import static org.apache.hadoop.hbase.zookeeper.ZKUtil.joinZNode;
21 import com.google.common.util.concurrent.ThreadFactoryBuilder;
22 import java.util.ArrayList;
23 import java.util.List;
24 import java.util.concurrent.ConcurrentNavigableMap;
25 import java.util.concurrent.ThreadFactory;
26 import org.apache.hadoop.hbase.HRegionLocation;
27 import org.apache.hadoop.hbase.classification.InterfaceAudience;
28 import org.apache.hadoop.hbase.exceptions.DeserializationException;
29 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
30 import org.apache.hadoop.hbase.types.CopyOnWriteArrayMap;
31 import org.apache.hadoop.hbase.util.RetryCounter;
32 import org.apache.hadoop.hbase.util.RetryCounterFactory;
33 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
34 import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
35 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
36 import org.apache.zookeeper.KeeperException;
37 import org.slf4j.Logger;
38 import org.slf4j.LoggerFactory;
39
40
41
42
43
44
45
46 @InterfaceAudience.Private
47 public class MetaRegionLocationCache extends ZooKeeperListener {
48
49 private static final Logger LOG = LoggerFactory.getLogger(MetaRegionLocationCache.class);
50
51
52
53
54 private static final int MAX_ZK_META_FETCH_RETRIES = 10;
55
56
57
58 private static final int SLEEP_INTERVAL_MS_BETWEEN_RETRIES = 1000;
59 private static final int SLEEP_INTERVAL_MS_MAX = 10000;
60 private final RetryCounterFactory retryCounterFactory =
61 new RetryCounterFactory(MAX_ZK_META_FETCH_RETRIES, SLEEP_INTERVAL_MS_BETWEEN_RETRIES);
62
63
64
65
66
67
68
69
70 private final CopyOnWriteArrayMap<Integer, HRegionLocation> cachedMetaLocations;
71
72 private enum ZNodeOpType {
73 INIT,
74 CREATED,
75 CHANGED,
76 DELETED
77 }
78
79 public MetaRegionLocationCache(ZooKeeperWatcher zkWatcher) {
80 super(zkWatcher);
81 cachedMetaLocations = new CopyOnWriteArrayMap<>();
82 watcher.registerListener(this);
83
84
85
86
87
88 ThreadFactory threadFactory = new ThreadFactoryBuilder().setDaemon(true).build();
89 final RetryCounterFactory retryFactory = new RetryCounterFactory(
90 Integer.MAX_VALUE, SLEEP_INTERVAL_MS_BETWEEN_RETRIES, SLEEP_INTERVAL_MS_MAX);
91 threadFactory.newThread(
92 new Runnable() {
93 @Override
94 public void run() {
95 MetaRegionLocationCache.this.loadMetaLocationsFromZk(
96 retryFactory.create(), ZNodeOpType.INIT);
97 }
98 }).start();
99 }
100
101
102
103
104
105
106 private void loadMetaLocationsFromZk(RetryCounter retryCounter, ZNodeOpType opType) {
107 List<String> znodes = null;
108 while (retryCounter.shouldRetry()) {
109 try {
110 znodes = watcher.getMetaReplicaNodesAndWatchChildren();
111 break;
112 } catch (KeeperException ke) {
113 LOG.debug("Error populating initial meta locations", ke);
114 if (!retryCounter.shouldRetry()) {
115
116
117 watcher.abort("Error populating meta locations", ke);
118 return;
119 }
120 try {
121 retryCounter.sleepUntilNextRetry();
122 } catch (InterruptedException ie) {
123 LOG.error("Interrupted while loading meta locations from ZK", ie);
124 Thread.currentThread().interrupt();
125 return;
126 }
127 }
128 }
129 if (znodes == null || znodes.isEmpty()) {
130
131
132 return;
133 }
134 if (znodes.size() == cachedMetaLocations.size()) {
135
136 return;
137 }
138 for (String znode: znodes) {
139 String path = joinZNode(watcher.baseZNode, znode);
140 updateMetaLocation(path, opType);
141 }
142 }
143
144
145
146
147
148
149
150
151 private HRegionLocation getMetaRegionLocation(int replicaId)
152 throws KeeperException {
153 RegionState metaRegionState;
154 try {
155 byte[] data = ZKUtil.getDataAndWatch(watcher,
156 watcher.getZNodeForReplica(replicaId));
157 metaRegionState = ProtobufUtil.parseMetaRegionStateFrom(data, replicaId);
158 } catch (DeserializationException e) {
159 throw ZKUtil.convert(e);
160 }
161 return new HRegionLocation(metaRegionState.getRegion(), metaRegionState.getServerName());
162 }
163
164 private void updateMetaLocation(String path, ZNodeOpType opType) {
165 if (!isValidMetaZNode(path)) {
166 return;
167 }
168 LOG.debug("Updating meta znode for path {}: {}", path, opType.name());
169 int replicaId = watcher.getMetaReplicaIdFromPath(path);
170 RetryCounter retryCounter = retryCounterFactory.create();
171 HRegionLocation location = null;
172 while (retryCounter.shouldRetry()) {
173 try {
174 if (opType == ZNodeOpType.DELETED) {
175 if (!ZKUtil.watchAndCheckExists(watcher, path)) {
176
177 break;
178 }
179
180
181 }
182 location = getMetaRegionLocation(replicaId);
183 break;
184 } catch (KeeperException e) {
185 LOG.debug("Error getting meta location for path {}", path, e);
186 if (!retryCounter.shouldRetry()) {
187 LOG.warn("Error getting meta location for path {}. Retries exhausted.", path, e);
188 break;
189 }
190 try {
191 retryCounter.sleepUntilNextRetry();
192 } catch (InterruptedException ie) {
193 Thread.currentThread().interrupt();
194 return;
195 }
196 }
197 }
198 if (location == null) {
199 cachedMetaLocations.remove(replicaId);
200 return;
201 }
202 cachedMetaLocations.put(replicaId, location);
203 }
204
205
206
207
208
209 public List<HRegionLocation> getMetaRegionLocations() {
210 ConcurrentNavigableMap<Integer, HRegionLocation> snapshot =
211 cachedMetaLocations.tailMap(cachedMetaLocations.firstKey());
212 List<HRegionLocation> result = new ArrayList<>();
213 if (snapshot.isEmpty()) {
214
215
216 return result;
217 }
218
219
220 for (HRegionLocation location: snapshot.values()) {
221 result.add(location);
222 }
223 return result;
224 }
225
226
227
228
229
230 private boolean isValidMetaZNode(String path) {
231 return watcher.isAnyMetaReplicaZNode(path);
232 }
233
234 @Override
235 public void nodeCreated(String path) {
236 updateMetaLocation(path, ZNodeOpType.CREATED);
237 }
238
239 @Override
240 public void nodeDeleted(String path) {
241 updateMetaLocation(path, ZNodeOpType.DELETED);
242 }
243
244 @Override
245 public void nodeDataChanged(String path) {
246 updateMetaLocation(path, ZNodeOpType.CHANGED);
247 }
248
249 @Override
250 public void nodeChildrenChanged(String path) {
251 if (!path.equals(watcher.baseZNode)) {
252 return;
253 }
254 loadMetaLocationsFromZk(retryCounterFactory.create(), ZNodeOpType.CHANGED);
255 }
256 }