/
TSOClient.java
641 lines (565 loc) · 21.7 KB
/
TSOClient.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
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
/**
* Copyright (c) 2011 Yahoo! Inc. All rights reserved.
*
* Licensed 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. See accompanying LICENSE file.
*/
package com.yahoo.omid.client;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Queue;
import java.util.Set;
import java.util.Timer;
import java.util.TimerTask;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.Executors;
import org.apache.hadoop.conf.Configuration;
import org.jboss.netty.bootstrap.ClientBootstrap;
import org.jboss.netty.channel.Channel;
import org.jboss.netty.channel.ChannelFactory;
import org.jboss.netty.channel.ChannelFuture;
import org.jboss.netty.channel.ChannelFutureListener;
import org.jboss.netty.channel.ChannelHandlerContext;
import org.jboss.netty.channel.ChannelStateEvent;
import org.jboss.netty.channel.ExceptionEvent;
import org.jboss.netty.channel.MessageEvent;
import org.jboss.netty.channel.SimpleChannelHandler;
import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory;
import org.jboss.netty.handler.execution.ExecutionHandler;
import org.jboss.netty.handler.execution.OrderedMemoryAwareThreadPoolExecutor;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.yahoo.omid.tso.Committed;
import com.yahoo.omid.tso.RowKey;
import com.yahoo.omid.tso.TSOMessage;
import com.yahoo.omid.tso.messages.AbortRequest;
import com.yahoo.omid.tso.messages.AbortedTransactionReport;
import com.yahoo.omid.tso.messages.CommitQueryRequest;
import com.yahoo.omid.tso.messages.CommitQueryResponse;
import com.yahoo.omid.tso.messages.CommitRequest;
import com.yahoo.omid.tso.messages.CommitResponse;
import com.yahoo.omid.tso.messages.CommittedTransactionReport;
import com.yahoo.omid.tso.messages.FullAbortReport;
import com.yahoo.omid.tso.messages.LargestDeletedTimestampReport;
import com.yahoo.omid.tso.messages.TimestampRequest;
import com.yahoo.omid.tso.messages.TimestampResponse;
import com.yahoo.omid.tso.serialization.TSODecoder;
import com.yahoo.omid.tso.serialization.TSOEncoder;
public class TSOClient extends SimpleChannelHandler {
private static final Logger LOG = LoggerFactory.getLogger(TSOClient.class);
public static long askedTSO = 0;
public enum Result {
OK, ABORTED
};
private Queue<CreateCallback> createCallbacks;
private Map<Long, CommitCallback> commitCallbacks;
private Map<Long, List<CommitQueryCallback>> isCommittedCallbacks;
private Committed committed = new Committed();
private Set<Long> aborted = Collections.synchronizedSet(new HashSet<Long>(1000));
private long largestDeletedTimestamp;
private long connectionTimestamp = 0;
private boolean hasConnectionTimestamp = false;
private ChannelFactory factory;
private ClientBootstrap bootstrap;
private Channel channel;
private InetSocketAddress addr;
private int max_retries;
private int retries;
private int retry_delay_ms;
private Timer retryTimer;
private enum State {
DISCONNECTED, CONNECTING, CONNECTED, RETRY_CONNECT_WAIT
};
private interface Op {
public void execute(Channel channel);
public void error(Exception e);
}
private class AbortOp implements Op {
long transactionId;
AbortOp(long transactionid) throws IOException {
this.transactionId = transactionid;
}
public void execute(Channel channel) {
try {
synchronized (commitCallbacks) {
if (commitCallbacks.containsKey(transactionId)) {
throw new IOException("Already committing transaction " + transactionId);
}
}
AbortRequest ar = new AbortRequest();
ar.startTimestamp = transactionId;
ChannelFuture f = channel.write(ar);
f.addListener(new ChannelFutureListener() {
public void operationComplete(ChannelFuture future) {
if (!future.isSuccess()) {
error(new IOException("Error writing to socket"));
}
}
});
} catch (Exception e) {
error(e);
}
}
public void error(Exception e) {
}
}
private class NewTimestampOp implements Op {
private CreateCallback cb;
NewTimestampOp(CreateCallback cb) {
this.cb = cb;
}
public void execute(Channel channel) {
try {
synchronized(createCallbacks) {
createCallbacks.add(cb);
}
TimestampRequest tr = new TimestampRequest();
ChannelFuture f = channel.write(tr);
f.addListener(new ChannelFutureListener() {
public void operationComplete(ChannelFuture future) {
if (!future.isSuccess()) {
error(new IOException("Error writing to socket"));
}
}
});
} catch (Exception e) {
error(e);
}
}
public void error(Exception e) {
synchronized(createCallbacks) {
createCallbacks.remove();
}
cb.error(e);
}
}
private class CommitQueryOp implements Op {
long startTimestamp;
long pendingWriteTimestamp;
CommitQueryCallback cb;
CommitQueryOp(long startTimestamp, long pendingWriteTimestamp, CommitQueryCallback cb) {
this.startTimestamp = startTimestamp;
this.pendingWriteTimestamp = pendingWriteTimestamp;
this.cb = cb;
}
public void execute(Channel channel) {
try {
synchronized(isCommittedCallbacks) {
List<CommitQueryCallback> callbacks = isCommittedCallbacks.get(startTimestamp);
if (callbacks == null) {
callbacks = new ArrayList<CommitQueryCallback>(1);
}
callbacks.add(cb);
isCommittedCallbacks.put(startTimestamp, callbacks);
}
CommitQueryRequest qr = new CommitQueryRequest(startTimestamp,
pendingWriteTimestamp);
ChannelFuture f = channel.write(qr);
f.addListener(new ChannelFutureListener() {
public void operationComplete(ChannelFuture future) {
if (!future.isSuccess()) {
error(new IOException("Error writing to socket"));
}
}
});
} catch (Exception e) {
error(e);
}
}
public void error(Exception e) {
synchronized(isCommittedCallbacks) {
isCommittedCallbacks.remove(startTimestamp);
}
cb.error(e);
}
}
private class CommitOp implements Op {
long transactionId;
RowKey[] rows;
CommitCallback cb;
CommitOp(long transactionid, RowKey[] rows, CommitCallback cb) throws IOException {
this.transactionId = transactionid;
this.rows = rows;
this.cb = cb;
}
public void execute(Channel channel) {
try {
synchronized(commitCallbacks) {
if (commitCallbacks.containsKey(transactionId)) {
throw new IOException("Already committing transaction " + transactionId);
}
commitCallbacks.put(transactionId, cb);
}
CommitRequest cr = new CommitRequest();
cr.startTimestamp = transactionId;
cr.rows = rows;
ChannelFuture f = channel.write(cr);
f.addListener(new ChannelFutureListener() {
public void operationComplete(ChannelFuture future) {
if (!future.isSuccess()) {
error(new IOException("Error writing to socket"));
}
}
});
} catch (Exception e) {
error(e);
}
}
public void error(Exception e) {
synchronized(commitCallbacks) {
commitCallbacks.remove(transactionId);
}
cb.error(e);
}
}
private class AbortCompleteOp implements Op {
long transactionId;
AbortCompleteCallback cb;
AbortCompleteOp(long transactionId, AbortCompleteCallback cb) throws IOException {
this.transactionId = transactionId;
this.cb = cb;
}
public void execute(Channel channel) {
try {
FullAbortReport far = new FullAbortReport();
far.startTimestamp = transactionId;
ChannelFuture f = channel.write(far);
f.addListener(new ChannelFutureListener() {
public void operationComplete(ChannelFuture future) {
if (!future.isSuccess()) {
error(new IOException("Error writing to socket"));
} else {
cb.complete();
}
}
});
} catch (Exception e) {
error(e);
}
}
public void error(Exception e) {
cb.error(e);
}
}
private ArrayBlockingQueue<Op> queuedOps;
private State state;
public TSOClient(Configuration conf) throws IOException {
state = State.DISCONNECTED;
queuedOps = new ArrayBlockingQueue<Op>(200);
retryTimer = new Timer(true);
commitCallbacks = Collections.synchronizedMap(new HashMap<Long, CommitCallback>());
isCommittedCallbacks = Collections.synchronizedMap(new HashMap<Long, List<CommitQueryCallback>>());
createCallbacks = new ConcurrentLinkedQueue<CreateCallback>();
channel = null;
System.out.println("Starting TSOClient");
// Start client with Nb of active threads = 3 as maximum.
factory = new NioClientSocketChannelFactory(Executors
.newCachedThreadPool(), Executors.newCachedThreadPool(), 3);
// Create the bootstrap
bootstrap = new ClientBootstrap(factory);
int executorThreads = conf.getInt("tso.executor.threads", 3);
bootstrap.getPipeline().addLast("executor", new ExecutionHandler(
new OrderedMemoryAwareThreadPoolExecutor(executorThreads, 1024*1024, 4*1024*1024)));
bootstrap.getPipeline().addLast("handler", this);
bootstrap.setOption("tcpNoDelay", false);
bootstrap.setOption("keepAlive", true);
bootstrap.setOption("reuseAddress", true);
bootstrap.setOption("connectTimeoutMillis", 100);
String host = conf.get("tso.host");
int port = conf.getInt("tso.port", 1234);
max_retries = conf.getInt("tso.max_retries", 100);
retry_delay_ms = conf.getInt("tso.retry_delay_ms", 1000);
if (host == null) {
throw new IOException("tso.host missing from configuration");
}
addr = new InetSocketAddress(host, port);
connectIfNeeded();
}
private State connectIfNeeded() throws IOException {
synchronized (state) {
if (state == State.CONNECTED || state == State.CONNECTING) {
return state;
}
if (state == State.RETRY_CONNECT_WAIT) {
return State.CONNECTING;
}
if (retries > max_retries) {
IOException e = new IOException("Max connection retries exceeded");
bailout(e);
throw e;
}
retries++;
bootstrap.connect(addr).addListener(new ChannelFutureListener() {
@Override
public void operationComplete(ChannelFuture future) throws Exception {
LOG.debug("Connection completed. Success: " + future.isSuccess());
}
});
state = State.CONNECTING;
return state;
}
}
private void withConnection(Op op) throws IOException {
State state = connectIfNeeded();
if (state == State.CONNECTING) {
try {
queuedOps.put(op);
} catch (InterruptedException e) {
throw new IOException("Couldn't add new operation", e);
}
} else if (state == State.CONNECTED) {
op.execute(channel);
} else {
throw new IOException("Invalid connection state " + state);
}
}
public void getNewTimestamp(CreateCallback cb) throws IOException {
withConnection(new NewTimestampOp(cb));
}
public void isCommitted(long startTimestamp, long pendingWriteTimestamp, CommitQueryCallback cb)
throws IOException {
withConnection(new CommitQueryOp(startTimestamp, pendingWriteTimestamp, cb));
}
public void abort(long transactionId) throws IOException {
withConnection(new AbortOp(transactionId));
}
public void commit(long transactionId, RowKey[] rows, CommitCallback cb) throws IOException {
withConnection(new CommitOp(transactionId, rows, cb));
}
public void completeAbort(long transactionId, AbortCompleteCallback cb) throws IOException {
withConnection(new AbortCompleteOp(transactionId, cb));
}
@Override
synchronized
public void channelOpen(ChannelHandlerContext ctx, ChannelStateEvent e) {
e.getChannel().getPipeline().addFirst("decoder", new TSODecoder());
e.getChannel().getPipeline().addAfter("decoder", "encoder",
new TSOEncoder());
}
/**
* Starts the traffic
*/
@Override
synchronized
public void channelConnected(ChannelHandlerContext ctx, ChannelStateEvent e) {
synchronized (state) {
channel = e.getChannel();
state = State.CONNECTED;
retries = 0;
}
clearState();
LOG.debug("Channel connected");
Op o = queuedOps.poll();;
while (o != null && state == State.CONNECTED) {
o.execute(channel);
o = queuedOps.poll();
}
}
private void clearState() {
committed = new Committed();
aborted.clear();
largestDeletedTimestamp = 0;
connectionTimestamp = 0;
hasConnectionTimestamp = false;
}
@Override
synchronized
public void channelDisconnected(ChannelHandlerContext ctx, ChannelStateEvent e)
throws Exception {
synchronized(state) {
LOG.debug("Channel disconnected");
channel = null;
state = State.DISCONNECTED;
for (CreateCallback cb : createCallbacks) {
cb.error(new IOException("Channel Disconnected"));
}
for (CommitCallback cb : commitCallbacks.values()) {
cb.error(new IOException("Channel Disconnected"));
}
for (List<CommitQueryCallback> lcqb : isCommittedCallbacks.values()) {
for (CommitQueryCallback cqb : lcqb) {
cqb.error(new IOException("Channel Disconnected"));
}
}
createCallbacks.clear();
commitCallbacks.clear();
isCommittedCallbacks.clear();
connectIfNeeded();
}
}
public boolean validRead(long transaction, long startTimestamp) throws IOException {
if (transaction == startTimestamp)
return true;
if (aborted.contains(transaction))
return false;
long commitTimestamp = committed.getCommit(transaction);
if (commitTimestamp != -1)
return commitTimestamp <= startTimestamp;
if (hasConnectionTimestamp && transaction > connectionTimestamp)
return transaction <= largestDeletedTimestamp;
if (transaction <= largestDeletedTimestamp)
return true;
// System.out.format("Asking TSO... hasConnectionTimestamp: %s connectionTimestamp: %d transaction: %d startTimestamp: %d\n",
// Boolean.valueOf(hasConnectionTimestamp).toString(), connectionTimestamp, transaction, startTimestamp);
askedTSO++;
SyncCommitQueryCallback cb = new SyncCommitQueryCallback();
isCommitted(startTimestamp, transaction, cb);
try {
cb.await();
} catch (InterruptedException e) {
throw new IOException("Commit query didn't complete", e);
}
return cb.isCommitted();
}
/**
* When a message is received, handle it based on its type
*/
@Override
public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) {
if (LOG.isTraceEnabled()) {
LOG.trace("messageReceived " + e.getMessage());
}
Object msg = e.getMessage();
if (msg instanceof CommitResponse) {
CommitResponse r = (CommitResponse)msg;
CommitCallback cb = null;
synchronized (commitCallbacks) {
cb = commitCallbacks.remove(r.startTimestamp);
}
if (cb == null) {
LOG.error("Received a commit response for a nonexisting commit");
return;
}
cb.complete(r.committed ? Result.OK : Result.ABORTED, r.commitTimestamp);
} else if (msg instanceof TimestampResponse) {
CreateCallback cb = createCallbacks.poll();
long timestamp = ((TimestampResponse)msg).timestamp;
if (!hasConnectionTimestamp || timestamp < connectionTimestamp) {
hasConnectionTimestamp = true;
connectionTimestamp = timestamp;
}
if (cb == null) {
LOG.error("Receiving a timestamp response, but none requested: " + timestamp);
return;
}
cb.complete(timestamp);
} else if (msg instanceof CommitQueryResponse) {
CommitQueryResponse r = (CommitQueryResponse)msg;
if (r.commitTimestamp != 0) {
committed.commit(r.queryTimestamp, r.commitTimestamp);
} else if (r.committed) {
committed.commit(r.queryTimestamp, largestDeletedTimestamp);
}
List<CommitQueryCallback> cbs = null;
synchronized (isCommittedCallbacks) {
cbs = isCommittedCallbacks.remove(r.startTimestamp);
}
if (cbs == null) {
LOG.error("Received a commit query response for a nonexisting request");
return;
}
for (CommitQueryCallback cb : cbs) {
cb.complete(r.committed);
}
} else if (msg instanceof CommittedTransactionReport) {
CommittedTransactionReport ctr = (CommittedTransactionReport) msg;
committed.commit(ctr.startTimestamp, ctr.commitTimestamp);
} else if (msg instanceof FullAbortReport) {
FullAbortReport r = (FullAbortReport) msg;
aborted.remove(r.startTimestamp);
} else if (msg instanceof AbortedTransactionReport) {
AbortedTransactionReport r = (AbortedTransactionReport) msg;
aborted.add(r.startTimestamp);
} else if (msg instanceof LargestDeletedTimestampReport) {
LargestDeletedTimestampReport r = (LargestDeletedTimestampReport) msg;
largestDeletedTimestamp = r.largestDeletedTimestamp;
committed.raiseLargestDeletedTransaction(r.largestDeletedTimestamp);
} else {
LOG.error("Unknown message received " + msg);
}
processMessage((TSOMessage) msg);
}
@Override
public void exceptionCaught(ChannelHandlerContext ctx,
ExceptionEvent e)
throws Exception {
LOG.error("Unexpected exception", e.getCause());
synchronized(state) {
if (state == State.CONNECTING) {
state = State.RETRY_CONNECT_WAIT;
if (LOG.isDebugEnabled()) {
LOG.debug("Retrying connect in " + retry_delay_ms + "ms " + retries);
}
try {
retryTimer.schedule(new TimerTask() {
public void run() {
synchronized (state) {
state = State.DISCONNECTED;
try {
connectIfNeeded();
} catch (IOException e) {
bailout(e);
}
}
}
}, retry_delay_ms);
} catch (Exception cause) {
bailout(cause);
}
} else {
LOG.error("Exception on channel", e.getCause());
}
}
}
public void bailout(Exception cause) {
synchronized (state) {
state = State.DISCONNECTED;
}
LOG.error("Unrecoverable error in client, bailing out", cause);
Exception e = new IOException("Unrecoverable error", cause);
Op o = queuedOps.poll();;
while (o != null) {
o.error(e);
o = queuedOps.poll();
}
synchronized (createCallbacks) {
for (CreateCallback cb : createCallbacks) {
cb.error(e);
}
createCallbacks.clear();
}
synchronized(commitCallbacks) {
for (CommitCallback cb : commitCallbacks.values()) {
cb.error(e);
}
commitCallbacks.clear();
}
synchronized(isCommittedCallbacks) {
for (List<CommitQueryCallback> cbs : isCommittedCallbacks.values()) {
for (CommitQueryCallback cb : cbs) {
cb.error(e);
}
}
isCommittedCallbacks.clear();
}
}
protected void processMessage(TSOMessage msg) {
// TODO Auto-generated method stub
}
}