/
PendingReadOp.java
351 lines (299 loc) · 12.9 KB
/
PendingReadOp.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
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
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.net.InetSocketAddress;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.ScheduledFuture;
import java.util.ArrayList;
import java.util.Enumeration;
import java.util.NoSuchElementException;
import java.util.Queue;
import java.util.BitSet;
import java.util.Set;
import java.util.HashSet;
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.bookkeeper.client.AsyncCallback.ReadCallback;
import org.apache.bookkeeper.client.BKException.BKDigestMatchException;
import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.jboss.netty.buffer.ChannelBuffer;
import org.jboss.netty.buffer.ChannelBufferInputStream;
/**
* Sequence of entries of a ledger that represents a pending read operation.
* When all the data read has come back, the application callback is called.
* This class could be improved because we could start pushing data to the
* application as soon as it arrives rather than waiting for the whole thing.
*
*/
class PendingReadOp implements Enumeration<LedgerEntry>, ReadEntryCallback {
Logger LOG = LoggerFactory.getLogger(PendingReadOp.class);
final int speculativeReadTimeout;
final private ScheduledExecutorService scheduler;
private ScheduledFuture<?> speculativeTask = null;
Queue<LedgerEntryRequest> seq;
Set<InetSocketAddress> heardFromHosts;
ReadCallback cb;
Object ctx;
LedgerHandle lh;
long numPendingEntries;
long startEntryId;
long endEntryId;
final int maxMissedReadsAllowed;
class LedgerEntryRequest extends LedgerEntry {
final static int NOT_FOUND = -1;
int nextReplicaIndexToReadFrom = 0;
AtomicBoolean complete = new AtomicBoolean(false);
int firstError = BKException.Code.OK;
int numMissedEntryReads = 0;
final ArrayList<InetSocketAddress> ensemble;
final List<Integer> writeSet;
final BitSet sentReplicas;
final BitSet erroredReplicas;
LedgerEntryRequest(ArrayList<InetSocketAddress> ensemble, long lId, long eId) {
super(lId, eId);
this.ensemble = ensemble;
this.writeSet = lh.distributionSchedule.getWriteSet(entryId);
this.sentReplicas = new BitSet(lh.getLedgerMetadata().getWriteQuorumSize());
this.erroredReplicas = new BitSet(lh.getLedgerMetadata().getWriteQuorumSize());
}
private int getReplicaIndex(InetSocketAddress host) {
int bookieIndex = ensemble.indexOf(host);
if (bookieIndex == -1) {
return NOT_FOUND;
}
return writeSet.indexOf(bookieIndex);
}
private BitSet getSentToBitSet() {
BitSet b = new BitSet(ensemble.size());
for (int i = 0; i < sentReplicas.length(); i++) {
if (sentReplicas.get(i)) {
b.set(writeSet.get(i));
}
}
return b;
}
private BitSet getHeardFromBitSet(Set<InetSocketAddress> heardFromHosts) {
BitSet b = new BitSet(ensemble.size());
for (InetSocketAddress i : heardFromHosts) {
int index = ensemble.indexOf(i);
if (index != -1) {
b.set(index);
}
}
return b;
}
private boolean readsOutstanding() {
return (sentReplicas.cardinality() - erroredReplicas.cardinality()) > 0;
}
/**
* Send to next replica speculatively, if required and possible.
* This returns the host we may have sent to for unit testing.
* @return host we sent to if we sent. null otherwise.
*/
synchronized InetSocketAddress maybeSendSpeculativeRead(Set<InetSocketAddress> heardFromHosts) {
if (nextReplicaIndexToReadFrom >= lh.getLedgerMetadata().getWriteQuorumSize()) {
return null;
}
BitSet sentTo = getSentToBitSet();
BitSet heardFrom = getHeardFromBitSet(heardFromHosts);
sentTo.and(heardFrom);
// only send another read, if we have had no response at all (even for other entries)
// from any of the other bookies we have sent the request to
if (sentTo.cardinality() == 0) {
return sendNextRead();
} else {
return null;
}
}
synchronized InetSocketAddress sendNextRead() {
if (nextReplicaIndexToReadFrom >= lh.metadata.getWriteQuorumSize()) {
// we are done, the read has failed from all replicas, just fail the
// read
// Do it a bit perssimistically, only when finished trying all replicas
// to check whether we received more missed reads than maxMissedReadsAllowed
if (BKException.Code.BookieHandleNotAvailableException == firstError &&
numMissedEntryReads > maxMissedReadsAllowed) {
firstError = BKException.Code.NoSuchEntryException;
}
submitCallback(firstError);
return null;
}
int replica = nextReplicaIndexToReadFrom;
int bookieIndex = lh.distributionSchedule.getWriteSet(entryId).get(nextReplicaIndexToReadFrom);
nextReplicaIndexToReadFrom++;
try {
InetSocketAddress to = ensemble.get(bookieIndex);
sendReadTo(to, this);
sentReplicas.set(replica);
return to;
} catch (InterruptedException ie) {
LOG.error("Interrupted reading entry " + this, ie);
Thread.currentThread().interrupt();
submitCallback(BKException.Code.ReadException);
return null;
}
}
synchronized void logErrorAndReattemptRead(InetSocketAddress host, String errMsg, int rc) {
if (BKException.Code.OK == firstError ||
BKException.Code.NoSuchEntryException == firstError) {
firstError = rc;
} else if (BKException.Code.BookieHandleNotAvailableException == firstError &&
BKException.Code.NoSuchEntryException != rc) {
// if other exception rather than NoSuchEntryException is returned
// we need to update firstError to indicate that it might be a valid read but just failed.
firstError = rc;
}
if (BKException.Code.NoSuchEntryException == rc) {
++numMissedEntryReads;
}
int bookieIndex = lh.distributionSchedule.getWriteSet(entryId).get(nextReplicaIndexToReadFrom - 1);
LOG.error(errMsg + " while reading entry: " + entryId + " ledgerId: " + lh.ledgerId + " from bookie: "
+ host);
int replica = getReplicaIndex(host);
if (replica == NOT_FOUND) {
LOG.error("Received error from a host which is not in the ensemble {} {}.", host, ensemble);
return;
}
erroredReplicas.set(replica);
if (!readsOutstanding()) {
sendNextRead();
}
}
// return true if we managed to complete the entry
boolean complete(InetSocketAddress host, final ChannelBuffer buffer) {
ChannelBufferInputStream is;
try {
is = lh.macManager.verifyDigestAndReturnData(entryId, buffer);
} catch (BKDigestMatchException e) {
logErrorAndReattemptRead(host, "Mac mismatch", BKException.Code.DigestMatchException);
return false;
}
if (!complete.getAndSet(true)) {
entryDataStream = is;
/*
* The length is a long and it is the last field of the metadata of an entry.
* Consequently, we have to subtract 8 from METADATA_LENGTH to get the length.
*/
length = buffer.getLong(DigestManager.METADATA_LENGTH - 8);
return true;
} else {
return false;
}
}
boolean isComplete() {
return complete.get();
}
public String toString() {
return String.format("L%d-E%d", ledgerId, entryId);
}
}
PendingReadOp(LedgerHandle lh, ScheduledExecutorService scheduler,
long startEntryId, long endEntryId, ReadCallback cb, Object ctx) {
seq = new ArrayBlockingQueue<LedgerEntryRequest>((int) ((endEntryId + 1) - startEntryId));
this.cb = cb;
this.ctx = ctx;
this.lh = lh;
this.startEntryId = startEntryId;
this.endEntryId = endEntryId;
this.scheduler = scheduler;
numPendingEntries = endEntryId - startEntryId + 1;
maxMissedReadsAllowed = lh.metadata.getWriteQuorumSize() - lh.metadata.getAckQuorumSize();
speculativeReadTimeout = lh.bk.getConf().getSpeculativeReadTimeout();
heardFromHosts = new HashSet<InetSocketAddress>();
}
public void initiate() throws InterruptedException {
long nextEnsembleChange = startEntryId, i = startEntryId;
ArrayList<InetSocketAddress> ensemble = null;
if (speculativeReadTimeout > 0) {
speculativeTask = scheduler.scheduleWithFixedDelay(new Runnable() {
public void run() {
for (LedgerEntryRequest r : seq) {
if (!r.isComplete()) {
r.maybeSendSpeculativeRead(heardFromHosts);
}
}
}
}, speculativeReadTimeout, speculativeReadTimeout, TimeUnit.MILLISECONDS);
}
do {
LOG.debug("Acquiring lock: {}", i);
if (i == nextEnsembleChange) {
ensemble = lh.metadata.getEnsemble(i);
nextEnsembleChange = lh.metadata.getNextEnsembleChange(i);
}
LedgerEntryRequest entry = new LedgerEntryRequest(ensemble, lh.ledgerId, i);
seq.add(entry);
i++;
entry.sendNextRead();
} while (i <= endEntryId);
}
private static class ReadContext {
final InetSocketAddress to;
final LedgerEntryRequest entry;
ReadContext(InetSocketAddress to, LedgerEntryRequest entry) {
this.to = to;
this.entry = entry;
}
}
void sendReadTo(InetSocketAddress to, LedgerEntryRequest entry) throws InterruptedException {
lh.opCounterSem.acquire();
lh.bk.bookieClient.readEntry(to, lh.ledgerId, entry.entryId,
this, new ReadContext(to, entry));
}
@Override
public void readEntryComplete(int rc, long ledgerId, final long entryId, final ChannelBuffer buffer, Object ctx) {
final ReadContext rctx = (ReadContext)ctx;
final LedgerEntryRequest entry = rctx.entry;
lh.opCounterSem.release();
if (rc != BKException.Code.OK) {
entry.logErrorAndReattemptRead(rctx.to, "Error: " + BKException.getMessage(rc), rc);
return;
}
heardFromHosts.add(rctx.to);
if (entry.complete(rctx.to, buffer)) {
numPendingEntries--;
}
if (numPendingEntries == 0) {
submitCallback(BKException.Code.OK);
}
if(numPendingEntries < 0)
LOG.error("Read too many values");
}
private void submitCallback(int code) {
if (speculativeTask != null) {
speculativeTask.cancel(true);
}
cb.readComplete(code, lh, PendingReadOp.this, PendingReadOp.this.ctx);
}
public boolean hasMoreElements() {
return !seq.isEmpty();
}
public LedgerEntry nextElement() throws NoSuchElementException {
return seq.remove();
}
public int size() {
return seq.size();
}
}