/
BookieWatcher.java
314 lines (267 loc) · 11.7 KB
/
BookieWatcher.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
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
package org.apache.bookkeeper.client;
/**
* 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.
*/
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException;
import org.apache.bookkeeper.conf.ClientConfiguration;
import org.apache.bookkeeper.util.BookKeeperConstants;
import org.apache.bookkeeper.util.SafeRunnable;
import org.apache.bookkeeper.util.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.AsyncCallback.ChildrenCallback;
import org.apache.zookeeper.KeeperException.Code;
import org.apache.zookeeper.KeeperException.NodeExistsException;
import org.apache.zookeeper.ZooDefs.Ids;
/**
* This class is responsible for maintaining a consistent view of what bookies
* are available by reading Zookeeper (and setting watches on the bookie nodes).
* When a bookie fails, the other parts of the code turn to this class to find a
* replacement
*
*/
class BookieWatcher implements Watcher, ChildrenCallback {
static final Logger logger = LoggerFactory.getLogger(BookieWatcher.class);
// Bookie registration path in ZK
private final String bookieRegistrationPath;
static final Set<InetSocketAddress> EMPTY_SET = new HashSet<InetSocketAddress>();
public static int ZK_CONNECT_BACKOFF_SEC = 1;
final BookKeeper bk;
HashSet<InetSocketAddress> knownBookies = new HashSet<InetSocketAddress>();
final ScheduledExecutorService scheduler;
SafeRunnable reReadTask = new SafeRunnable() {
@Override
public void safeRun() {
readBookies();
}
};
private ReadOnlyBookieWatcher readOnlyBookieWatcher;
public BookieWatcher(ClientConfiguration conf,
ScheduledExecutorService scheduler,
BookKeeper bk) throws KeeperException, InterruptedException {
this.bk = bk;
// ZK bookie registration path
this.bookieRegistrationPath = conf.getZkAvailableBookiesPath();
this.scheduler = scheduler;
readOnlyBookieWatcher = new ReadOnlyBookieWatcher(conf, bk);
}
public void readBookies() {
readBookies(this);
}
public void readBookies(ChildrenCallback callback) {
bk.getZkHandle().getChildren(this.bookieRegistrationPath, this, callback, null);
}
@Override
public void process(WatchedEvent event) {
readBookies();
}
@Override
public void processResult(int rc, String path, Object ctx, List<String> children) {
if (rc != KeeperException.Code.OK.intValue()) {
//logger.error("Error while reading bookies", KeeperException.create(Code.get(rc), path));
// try the read after a second again
scheduler.schedule(reReadTask, ZK_CONNECT_BACKOFF_SEC, TimeUnit.SECONDS);
return;
}
// Just exclude the 'readonly' znode to exclude r-o bookies from
// available nodes list.
children.remove(BookKeeperConstants.READONLY);
HashSet<InetSocketAddress> newBookieAddrs = convertToBookieAddresses(children);
final HashSet<InetSocketAddress> deadBookies;
synchronized (this) {
deadBookies = (HashSet<InetSocketAddress>)knownBookies.clone();
deadBookies.removeAll(newBookieAddrs);
// No need to close readonly bookie clients.
deadBookies.removeAll(readOnlyBookieWatcher.getReadOnlyBookies());
knownBookies = newBookieAddrs;
}
if (bk.getBookieClient() != null) {
bk.getBookieClient().closeClients(deadBookies);
}
}
private static HashSet<InetSocketAddress> convertToBookieAddresses(List<String> children) {
// Read the bookie addresses into a set for efficient lookup
HashSet<InetSocketAddress> newBookieAddrs = new HashSet<InetSocketAddress>();
for (String bookieAddrString : children) {
InetSocketAddress bookieAddr;
try {
bookieAddr = StringUtils.parseAddr(bookieAddrString);
} catch (IOException e) {
logger.error("Could not parse bookie address: " + bookieAddrString + ", ignoring this bookie");
continue;
}
newBookieAddrs.add(bookieAddr);
}
return newBookieAddrs;
}
/**
* Blocks until bookies are read from zookeeper, used in the {@link BookKeeper} constructor.
* @throws InterruptedException
* @throws KeeperException
*/
public void readBookiesBlocking() throws InterruptedException, KeeperException {
// Read readonly bookies first
readOnlyBookieWatcher.readROBookiesBlocking();
final LinkedBlockingQueue<Integer> queue = new LinkedBlockingQueue<Integer>();
readBookies(new ChildrenCallback() {
public void processResult(int rc, String path, Object ctx, List<String> children) {
try {
BookieWatcher.this.processResult(rc, path, ctx, children);
queue.put(rc);
} catch (InterruptedException e) {
logger.error("Interruped when trying to read bookies in a blocking fashion");
throw new RuntimeException(e);
}
}
});
int rc = queue.take();
if (rc != KeeperException.Code.OK.intValue()) {
throw KeeperException.create(Code.get(rc));
}
}
/**
* Wrapper over the {@link #getAdditionalBookies(Set, int)} method when there is no exclusion list (or exisiting bookies)
* @param numBookiesNeeded
* @return
* @throws BKNotEnoughBookiesException
*/
public ArrayList<InetSocketAddress> getNewBookies(int numBookiesNeeded) throws BKNotEnoughBookiesException {
return getAdditionalBookies(EMPTY_SET, numBookiesNeeded);
}
/**
* Wrapper over the {@link #getAdditionalBookies(Set, int)} method when you just need 1 extra bookie
* @param existingBookies
* @return
* @throws BKNotEnoughBookiesException
*/
public InetSocketAddress getAdditionalBookie(List<InetSocketAddress> existingBookies)
throws BKNotEnoughBookiesException {
return getAdditionalBookies(new HashSet<InetSocketAddress>(existingBookies), 1).get(0);
}
/**
* Returns additional bookies given an exclusion list and how many are needed
* @param existingBookies
* @param numAdditionalBookiesNeeded
* @return
* @throws BKNotEnoughBookiesException
*/
public ArrayList<InetSocketAddress> getAdditionalBookies(Set<InetSocketAddress> existingBookies,
int numAdditionalBookiesNeeded) throws BKNotEnoughBookiesException {
ArrayList<InetSocketAddress> newBookies = new ArrayList<InetSocketAddress>();
if (numAdditionalBookiesNeeded <= 0) {
return newBookies;
}
List<InetSocketAddress> allBookies;
synchronized (this) {
allBookies = new ArrayList<InetSocketAddress>(knownBookies);
}
Collections.shuffle(allBookies);
for (InetSocketAddress bookie : allBookies) {
if (existingBookies.contains(bookie)) {
continue;
}
newBookies.add(bookie);
numAdditionalBookiesNeeded--;
if (numAdditionalBookiesNeeded == 0) {
return newBookies;
}
}
throw new BKNotEnoughBookiesException();
}
/**
* Watcher implementation to watch the readonly bookies under
* <available>/readonly
*/
private static class ReadOnlyBookieWatcher implements Watcher, ChildrenCallback {
private final static Logger LOG = LoggerFactory.getLogger(ReadOnlyBookieWatcher.class);
private HashSet<InetSocketAddress> readOnlyBookies = new HashSet<InetSocketAddress>();
private BookKeeper bk;
private String readOnlyBookieRegPath;
public ReadOnlyBookieWatcher(ClientConfiguration conf, BookKeeper bk) throws KeeperException,
InterruptedException {
this.bk = bk;
readOnlyBookieRegPath = conf.getZkAvailableBookiesPath() + "/"
+ BookKeeperConstants.READONLY;
if (null == bk.getZkHandle().exists(readOnlyBookieRegPath, false)) {
try {
bk.getZkHandle().create(readOnlyBookieRegPath, new byte[0], Ids.OPEN_ACL_UNSAFE,
CreateMode.PERSISTENT);
} catch (NodeExistsException e) {
// this node is just now created by someone.
}
}
}
@Override
public void process(WatchedEvent event) {
readROBookies();
}
// read the readonly bookies in blocking fashion. Used only for first
// time.
void readROBookiesBlocking() throws InterruptedException, KeeperException {
final LinkedBlockingQueue<Integer> queue = new LinkedBlockingQueue<Integer>();
readROBookies(new ChildrenCallback() {
public void processResult(int rc, String path, Object ctx, List<String> children) {
try {
ReadOnlyBookieWatcher.this.processResult(rc, path, ctx, children);
queue.put(rc);
} catch (InterruptedException e) {
logger.error("Interruped when trying to read readonly bookies in a blocking fashion");
throw new RuntimeException(e);
}
}
});
int rc = queue.take();
if (rc != KeeperException.Code.OK.intValue()) {
throw KeeperException.create(Code.get(rc));
}
}
// Read children and register watcher for readonly bookies path
void readROBookies(ChildrenCallback callback) {
bk.getZkHandle().getChildren(this.readOnlyBookieRegPath, this, callback, null);
}
void readROBookies() {
readROBookies(this);
}
@Override
public void processResult(int rc, String path, Object ctx, List<String> children) {
if (rc != Code.OK.intValue()) {
LOG.error("Not able to read readonly bookies : ", KeeperException.create(Code.get(rc)));
return;
}
HashSet<InetSocketAddress> newReadOnlyBookies = convertToBookieAddresses(children);
readOnlyBookies = newReadOnlyBookies;
}
// returns the readonly bookies
public HashSet<InetSocketAddress> getReadOnlyBookies() {
return readOnlyBookies;
}
}
}