1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.coprocessor.example;
20
21 import java.io.IOException;
22 import java.util.Collections;
23 import java.util.List;
24 import java.util.NavigableSet;
25
26 import org.apache.commons.logging.Log;
27 import org.apache.commons.logging.LogFactory;
28 import org.apache.hadoop.hbase.CoprocessorEnvironment;
29 import org.apache.hadoop.hbase.HConstants;
30 import org.apache.hadoop.hbase.client.Scan;
31 import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
32 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
33 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
34 import org.apache.hadoop.hbase.coprocessor.RegionObserver;
35 import org.apache.hadoop.hbase.regionserver.Store;
36 import org.apache.hadoop.hbase.regionserver.ScanInfo;
37 import org.apache.hadoop.hbase.regionserver.InternalScanner;
38 import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
39 import org.apache.hadoop.hbase.regionserver.ScanType;
40 import org.apache.hadoop.hbase.regionserver.Store;
41 import org.apache.hadoop.hbase.regionserver.StoreScanner;
42 import org.apache.hadoop.hbase.util.Bytes;
43 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
44 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
45 import org.apache.zookeeper.KeeperException;
46 import org.apache.zookeeper.WatchedEvent;
47 import org.apache.zookeeper.Watcher;
48 import org.apache.zookeeper.ZooKeeper;
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65 public class ZooKeeperScanPolicyObserver extends BaseRegionObserver {
66 public static final String node = "/backup/example/lastbackup";
67 public static final String zkkey = "ZK";
68 private static final Log LOG = LogFactory.getLog(ZooKeeperScanPolicyObserver.class);
69
70
71
72
73 private static class ZKWatcher implements Watcher {
74 private byte[] data = null;
75 private ZooKeeper zk;
76 private volatile boolean needSetup = true;
77 private volatile long lastSetupTry = 0;
78
79 public ZKWatcher(ZooKeeper zk) {
80 this.zk = zk;
81
82 getData();
83 }
84
85
86
87
88
89
90
91
92
93
94
95
96 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="REC_CATCH_EXCEPTION")
97 public byte[] getData() {
98
99 if (needSetup && EnvironmentEdgeManager.currentTimeMillis() > lastSetupTry + 30000) {
100 synchronized (this) {
101
102 if (needSetup) {
103 needSetup = false;
104 } else {
105 return data;
106 }
107 }
108
109
110 try {
111 LOG.debug("Connecting to ZK");
112
113 lastSetupTry = EnvironmentEdgeManager.currentTimeMillis();
114 if (zk.exists(node, false) != null) {
115 data = zk.getData(node, this, null);
116 LOG.debug("Read synchronously: "+(data == null ? "null" : Bytes.toLong(data)));
117 } else {
118 zk.exists(node, this);
119 }
120 } catch (Exception x) {
121
122 needSetup = true;
123 }
124 }
125 return data;
126 }
127
128 @Override
129 public void process(WatchedEvent event) {
130 switch(event.getType()) {
131 case NodeDataChanged:
132 case NodeCreated:
133 try {
134
135 data = zk.getData(node, this, null);
136 LOG.debug("Read asynchronously: "+(data == null ? "null" : Bytes.toLong(data)));
137 } catch (InterruptedException ix) {
138 } catch (KeeperException kx) {
139 needSetup = true;
140 }
141 break;
142
143 case NodeDeleted:
144 try {
145
146 zk.exists(node, this);
147 data = null;
148 } catch (InterruptedException ix) {
149 } catch (KeeperException kx) {
150 needSetup = true;
151 }
152 break;
153
154 default:
155
156 }
157 }
158 }
159
160 @Override
161 public void start(CoprocessorEnvironment e) throws IOException {
162 RegionCoprocessorEnvironment re = (RegionCoprocessorEnvironment) e;
163 if (!re.getSharedData().containsKey(zkkey)) {
164
165
166 re.getSharedData().putIfAbsent(
167 zkkey,
168 new ZKWatcher(re.getRegionServerServices().getZooKeeper()
169 .getRecoverableZooKeeper().getZooKeeper()));
170 }
171 }
172
173 @Override
174 public void stop(CoprocessorEnvironment e) throws IOException {
175
176 }
177
178 protected ScanInfo getScanInfo(Store store, RegionCoprocessorEnvironment e) {
179 byte[] data = ((ZKWatcher)e.getSharedData().get(zkkey)).getData();
180 if (data == null) {
181 return null;
182 }
183 ScanInfo oldSI = store.getScanInfo();
184 if (oldSI.getTtl() == Long.MAX_VALUE) {
185 return null;
186 }
187 long ttl = Math.max(EnvironmentEdgeManager.currentTimeMillis() -
188 Bytes.toLong(data), oldSI.getTtl());
189 return new ScanInfo(store.getFamily(), ttl,
190 oldSI.getTimeToPurgeDeletes(), oldSI.getComparator());
191 }
192
193 @Override
194 public InternalScanner preFlushScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
195 Store store, KeyValueScanner memstoreScanner, InternalScanner s) throws IOException {
196 ScanInfo scanInfo = getScanInfo(store, c.getEnvironment());
197 if (scanInfo == null) {
198
199 return null;
200 }
201 Scan scan = new Scan();
202 scan.setMaxVersions(scanInfo.getMaxVersions());
203 return new StoreScanner(store, scanInfo, scan, Collections.singletonList(memstoreScanner),
204 ScanType.COMPACT_RETAIN_DELETES, store.getSmallestReadPoint(), HConstants.OLDEST_TIMESTAMP);
205 }
206
207 @Override
208 public InternalScanner preCompactScannerOpen(
209 final ObserverContext<RegionCoprocessorEnvironment> c,
210 Store store, List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs,
211 InternalScanner s) throws IOException {
212 ScanInfo scanInfo = getScanInfo(store, c.getEnvironment());
213 if (scanInfo == null) {
214
215 return null;
216 }
217 Scan scan = new Scan();
218 scan.setMaxVersions(scanInfo.getMaxVersions());
219 return new StoreScanner(store, scanInfo, scan, scanners, scanType,
220 store.getSmallestReadPoint(), earliestPutTs);
221 }
222
223 @Override
224 public KeyValueScanner preStoreScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
225 final Store store, final Scan scan, final NavigableSet<byte[]> targetCols,
226 final KeyValueScanner s) throws IOException {
227 ScanInfo scanInfo = getScanInfo(store, c.getEnvironment());
228 if (scanInfo == null) {
229
230 return null;
231 }
232 return new StoreScanner(store, scanInfo, scan, targetCols);
233 }
234 }