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 java.util.Map;
21 import java.util.concurrent.atomic.AtomicInteger;
22
23 import org.apache.commons.logging.Log;
24 import org.apache.commons.logging.LogFactory;
25 import org.apache.hadoop.hbase.classification.InterfaceAudience;
26 import org.apache.hadoop.hbase.HRegionInfo;
27 import org.apache.hadoop.hbase.ServerName;
28 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
29 import org.apache.zookeeper.AsyncCallback.StringCallback;
30 import org.apache.zookeeper.KeeperException;
31 import org.apache.zookeeper.ZooKeeper;
32 import org.apache.zookeeper.data.Stat;
33
34
35
36
37
38 @InterfaceAudience.Private
39 public class OfflineCallback implements StringCallback {
40 private static final Log LOG = LogFactory.getLog(OfflineCallback.class);
41 private final ExistCallback callBack;
42 private final ZooKeeperWatcher zkw;
43 private final ServerName destination;
44 private final AtomicInteger counter;
45
46 OfflineCallback(final ZooKeeperWatcher zkw,
47 final ServerName destination, final AtomicInteger counter,
48 final Map<String, Integer> offlineNodesVersions) {
49 this.callBack = new ExistCallback(
50 destination, counter, offlineNodesVersions);
51 this.destination = destination;
52 this.counter = counter;
53 this.zkw = zkw;
54 }
55
56 @Override
57 public void processResult(int rc, String path, Object ctx, String name) {
58 if (rc == KeeperException.Code.NODEEXISTS.intValue()) {
59 LOG.warn("Node for " + path + " already exists");
60 } else if (rc != 0) {
61
62 LOG.warn("rc != 0 for " + path + " -- retryable connectionloss -- " +
63 "FIX see http://wiki.apache.org/hadoop/ZooKeeper/FAQ#A2");
64 this.counter.addAndGet(1);
65 return;
66 }
67
68 if (LOG.isDebugEnabled()) {
69 LOG.debug("rs=" + ctx + ", server=" + destination);
70 }
71
72
73 ZooKeeper zk = this.zkw.getRecoverableZooKeeper().getZooKeeper();
74 zk.exists(path, this.zkw, callBack, ctx);
75 }
76
77
78
79
80
81 static class ExistCallback implements StatCallback {
82 private static final Log LOG = LogFactory.getLog(ExistCallback.class);
83 private final Map<String, Integer> offlineNodesVersions;
84 private final AtomicInteger counter;
85 private ServerName destination;
86
87 ExistCallback(final ServerName destination,
88 final AtomicInteger counter,
89 final Map<String, Integer> offlineNodesVersions) {
90 this.offlineNodesVersions = offlineNodesVersions;
91 this.destination = destination;
92 this.counter = counter;
93 }
94
95 @Override
96 public void processResult(int rc, String path, Object ctx, Stat stat) {
97 if (rc != 0) {
98
99 LOG.warn("rc != 0 for " + path + " -- retryable connectionloss -- " +
100 "FIX see http://wiki.apache.org/hadoop/ZooKeeper/FAQ#A2");
101 this.counter.addAndGet(1);
102 return;
103 }
104
105 if (LOG.isDebugEnabled()) {
106 LOG.debug("rs=" + ctx + ", server=" + destination);
107 }
108 HRegionInfo region = ((RegionState)ctx).getRegion();
109 offlineNodesVersions.put(
110 region.getEncodedName(), Integer.valueOf(stat.getVersion()));
111 this.counter.addAndGet(1);
112 }
113 }
114 }