/
DistributedTransactionService.java
2219 lines (1594 loc) · 70.2 KB
/
DistributedTransactionService.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
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
Copyright (C) SYSTAP, LLC DBA Blazegraph 2006-2016. All rights reserved.
Contact:
SYSTAP, LLC DBA Blazegraph
2501 Calvert ST NW #106
Washington, DC 20008
licenses@blazegraph.com
This program is free software; you can redistribute it and/or modify
it under the terms of the GNU General Public License as published by
the Free Software Foundation; version 2 of the License.
This program is distributed in the hope that it will be useful,
but WITHOUT ANY WARRANTY; without even the implied warranty of
MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
GNU General Public License for more details.
You should have received a copy of the GNU General Public License
along with this program; if not, write to the Free Software
Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA
*/
/*
* Created on Dec 18, 2008
*/
package com.bigdata.service;
import java.io.BufferedInputStream;
import java.io.BufferedOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.LinkedList;
import java.util.List;
import java.util.Properties;
import java.util.UUID;
import java.util.concurrent.BrokenBarrierException;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CyclicBarrier;
import java.util.concurrent.Future;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.Condition;
import java.util.zip.Adler32;
import com.bigdata.btree.BTree;
import com.bigdata.btree.IRangeQuery;
import com.bigdata.btree.ITuple;
import com.bigdata.btree.ITupleIterator;
import com.bigdata.concurrent.LockManager;
import com.bigdata.concurrent.LockManagerTask;
import com.bigdata.config.LongValidator;
import com.bigdata.counters.CounterSet;
import com.bigdata.counters.Instrument;
import com.bigdata.journal.IDistributedTransactionService;
import com.bigdata.journal.ITransactionService;
import com.bigdata.journal.ITx;
import com.bigdata.journal.Name2Addr;
import com.bigdata.journal.RunState;
import com.bigdata.util.concurrent.ExecutionExceptions;
/**
* Implementation for an {@link IBigdataFederation} supporting both single-phase
* commits (for transactions that execute on a single {@link IDataService}) and
* distributed commits.
*
* @author <a href="mailto:thompsonbry@users.sourceforge.net">Bryan Thompson</a>
* @version $Id$
*/
public abstract class DistributedTransactionService extends
AbstractTransactionService implements IDistributedTransactionService {
/**
* Options understood by this service.
*
* @author <a href="mailto:thompsonbry@users.sourceforge.net">Bryan Thompson</a>
*/
public interface Options extends AbstractTransactionService.Options {
/**
* The directory in which the persistent state of this service will be
* stored.
*/
String DATA_DIR = DistributedTransactionService.class.getName()
+ ".dataDir";
/**
* The interval in milliseconds between writing a snapshot of the index
* of accessible commit points into the {@link #DATA_DIR} ({@value #DEFAULT_SHAPSHOT_INTERVAL}).
* <p>
* Two snapshots are retained of the commit time index so that those
* historical commit times required for reading on committed states of
* the database GT the <i>releaseTime</i> may be on hand after a
* service restart. Two snapshots are maintained, with the older
* snapshot being overwritten each time. A snapshot is written every N
* milliseconds, where N is configured using this property, and also
* when the service is shutdown.
* <p>
* This MAY be ZERO (0L) to disable snapshots - a feature that is used
* by the {@link EmbeddedFederation} when run in a diskless mode.
*/
String SHAPSHOT_INTERVAL = DistributedTransactionService.class
.getName()
+ ".snapshotInterval";
/** 5 minutes (in millseconds). */
String DEFAULT_SHAPSHOT_INTERVAL = ""
+ (5 * 60 * 1000);
}
/**
* A map of the distributed transactions that are currently committing.
*
* @todo config for initial capacity and concurrency?
*/
private final ConcurrentHashMap<Long/* tx */, DistributedTxCommitTask/* state */> commitList = new ConcurrentHashMap<Long, DistributedTxCommitTask>();
/**
* The {@link LockManager} used to impose a partial ordering on the prepare
* phase of distributed transaction commits using index partition names as
* the named resources for which the tasks must contend.
*/
private final LockManager<String> indexLockManager = new LockManager<String>(
0/* maxConcurrencyIsIgnored */, true/* predeclareLocks */);
/**
* The {@link LockManager} used to impose a partial ordering on the commit
* phase of distributed transaction commits using {@link IDataService}
* {@link UUID}s as the named resources for which the tasks must contend.
*/
private final LockManager<UUID> dataServiceLockManager = new LockManager<UUID>(
0/* maxConcurrencyIsIgnored */, true/* predeclareLocks */);
/**
* A {@link BTree} containing a log of the historical commit points.
* <p>
* The main things that it gives us are (a) the half-open ranges within
* which we can allocate read-historical transactions; and (b) the last
* commit time on record. It seems that creating an image of the log every N
* seconds should be sufficient.
* <p>
* Note: Read and write operations on this index MUST be synchronized on the
* index object.
*/
protected final CommitTimeIndex commitTimeIndex;
/**
* True iff the service does not write any state on the disk.
*/
private final boolean isTransient;
/**
* The data directory -or- <code>null</code> iff the service is transient.
*/
protected final File dataDir;
/**
* The interval in milliseconds between logging an image of the
* {@link #commitTimeIndex}.
*
* @see Options#COMMIT_TIME_INDEX_SHAPSHOT_INTERVAL
*/
private final long snapshotInterval;
/**
* The last (known) commit time.
*/
private volatile long lastCommitTime = 0L;
/**
* @param properties
*/
public DistributedTransactionService(final Properties properties) {
super(properties);
if (properties.getProperty(Options.DATA_DIR) == null) {
throw new RuntimeException("Required property: " + Options.DATA_DIR);
}
snapshotInterval = LongValidator.GTE_ZERO.parse(
Options.SHAPSHOT_INTERVAL, properties.getProperty(
Options.SHAPSHOT_INTERVAL,
Options.DEFAULT_SHAPSHOT_INTERVAL));
if (log.isInfoEnabled())
log.info(Options.SHAPSHOT_INTERVAL + "=" + snapshotInterval);
isTransient = snapshotInterval == 0;
if (isTransient) {
dataDir = null;
} else {
dataDir = new File(properties.getProperty(Options.DATA_DIR));
if (log.isInfoEnabled())
log.info(Options.DATA_DIR + "=" + dataDir);
}
// Create transient BTree for the commit time log.
commitTimeIndex = CommitTimeIndex.createTransient();
setup();
if (log.isInfoEnabled())
log.info("lastCommitTime=" + lastCommitTime + ", #commitTimes="
+ commitTimeIndex.getEntryCount());
}
/**
* Either creates the data directory or reads the {@link #commitTimeIndex}
* from files in an existing data directory.
*/
private void setup() {
if(isTransient) {
// nothing committed yet.
lastCommitTime = 0L;
return;
}
if (!dataDir.exists()) {
/*
* New service if its data directory does not exist.
*/
if (!dataDir.mkdirs() && !dataDir.mkdirs()) {
throw new RuntimeException("Could not create: " + dataDir);
}
// nothing committed yet.
lastCommitTime = 0L;
return;
}
{
// the files on which the images should have been written.
final File file0 = new File(dataDir, BASENAME + "0" + EXT);
final File file1 = new File(dataDir, BASENAME + "1" + EXT);
if (!file0.exists() && !file1.exists()) {
log.warn("No commit time logs - assuming new service: dataDir="
+ dataDir);
// nothing committed yet.
lastCommitTime = 0L;
return;
}
// timestamps on those files (zero if the file does not exist)
final long time0 = file0.lastModified();
final long time1 = file1.lastModified();
// true iff file0 is more recent.
final boolean isFile0 = (time0 != 0L && time1 != 0L) //
? (time0 > time1 ? true: false)// Note: both files exist.
: (time0 != 0L ? true: false)// Note: only one file exists
;
final File file = isFile0 ? file0 : file1;
// System.err.println("file0: "+file0.lastModified());
// System.err.println("file1: "+file1.lastModified());
// System.err.println("isFile0="+isFile0);
/*
* Note: On restart the value of this counter is set to either
* ONE(1) or TWO(1) depending on which snapshot file is more
* current.
*
* It is ONE(1) if we read file0 since the counter would be ONE(1)
* after we write file0 for the first time.
*
* It is TWO(2) if we read file1 since the counter would be TWO(2)
* after we write file1 for the first time.
*/
snapshotCount = isFile0 ? 1 : 2;
try {
// read most recent image.
final long entryCount = SnapshotHelper.read(commitTimeIndex,
file);
log.warn("Read snapshot: entryCount=" + entryCount + ", file="
+ file);
} catch (IOException ex) {
throw new RuntimeException("Could not read file: " + file, ex);
}
}
if (commitTimeIndex.getEntryCount() == 0) {
// nothing in the commit time log.
lastCommitTime = 0;
} else {
// the last commit time in the log. @todo write unit test to
// verify on restart.
lastCommitTime = commitTimeIndex.decodeKey(commitTimeIndex
.keyAt(commitTimeIndex.getEntryCount() - 1));
}
}
/**
* Basename for the files written in the {@link #dataDir} containing images
* of the {@link #commitTimeIndex}.
*/
static protected final String BASENAME = "commitTime";
/**
* Extension for the files written in the {@link #dataDir} containing
* snapshots of the {@link #commitTimeIndex}.
*/
static protected final String EXT = ".snapshot";
/**
* #of times we have written a snapshot of the {@link #commitTimeIndex}.
*/
private long snapshotCount = 0L;
/**
* Runs the {@link SnapshotTask} once.
*/
public void snapshot() {
new SnapshotTask().run();
}
/**
* A task that writes a snapshot of the commit time index onto a pair of
* alternating files. This is in the spirit of the Challis algorithm, but
* the approach is less rigorous.
*
* @author <a href="mailto:thompsonbry@users.sourceforge.net">Bryan Thompson</a>
*/
private class SnapshotTask implements Runnable {
/**
* Note: Anything thrown out of this method will cause the task to no
* longer be scheduled!
*/
public void run() {
if(isTransient) {
// snapshot not supported for transient service.
throw new RuntimeException("Service is transient");
}
lock.lock();
try {
final long begin = System.currentTimeMillis();
// either 0 or 1.
final int i = (int) snapshotCount % 2;
final File file = new File(dataDir, BASENAME + i + EXT);
if (!dataDir.exists()) {
if (!dataDir.mkdirs() && !dataDir.mkdirs()) {
throw new RuntimeException("Could not create: " + dataDir);
}
}
final long entryCount;
synchronized (commitTimeIndex) {
entryCount = SnapshotHelper.write(commitTimeIndex, file);
}
// increment counter iff successful.
snapshotCount++;
final long elapsed = System.currentTimeMillis() - begin;
log.warn("snapshot: snapshotCount=" + snapshotCount
+ ", entryCount=" + entryCount + ", file=" + file
+ ", elapsed=" + elapsed);
} catch (Throwable t) {
log.error(t.getMessage(), t);
return;
} finally {
lock.unlock();
}
}
};
/**
* A helper class for reading and writing snapshots of the commit time
* index. The image contains the commit timestamps in order.
* <p>
* Note: The caller must prevent concurrent changes to the index.
*
* @todo write counters into the files since the system clock could be
* messed with on before a restart but the counters will always be
* valid. we would then either read both and choose one, or have a
* method to report the header with the earlier counter.
*
* @todo Checksum the commit time log file? this is easily done either using
* a {@link ByteBuffer} or using {@link Adler32}.
*
* @author <a href="mailto:thompsonbry@users.sourceforge.net">Bryan Thompson</a>
*/
public static class SnapshotHelper {
static public long read(CommitTimeIndex ndx, File file)
throws IOException {
final FileInputStream is = new FileInputStream(file);
try {
final BufferedInputStream bis = new BufferedInputStream(is);
final DataInputStream dis = new DataInputStream(bis);
return SnapshotHelper.read(ndx, dis);
} finally {
is.close();
}
}
static public long read(CommitTimeIndex ndx, DataInputStream is)
throws IOException {
final long n = is.readLong();
for (int i = 0; i < n; i++) {
ndx.add(is.readLong());
}
return n;
}
static public long write(final CommitTimeIndex ndx, final File file)
throws IOException {
final FileOutputStream os = new FileOutputStream(file);
try {
final BufferedOutputStream bos = new BufferedOutputStream(os);
final DataOutputStream dos = new DataOutputStream(bos);
// write the image on the file.
final long entryCount = SnapshotHelper.write(ndx, dos);
dos.flush();
bos.flush();
return entryCount;
} finally {
os.close();
}
}
static public long write(final CommitTimeIndex ndx,
final DataOutputStream os) throws IOException {
final long entryCount = ndx.getEntryCount();
os.writeLong(entryCount);
final ITupleIterator itr = ndx.rangeIterator();
int n = 0;
while (itr.hasNext()) {
final ITuple tuple = itr.next();
final long commitTime = ndx.decodeKey(tuple.getKey());
os.writeLong(commitTime);
n++;
}
if (n != entryCount) {
/*
* Note: probable error is the caller not preventing concurrent
* modification.
*/
throw new AssertionError();
}
return entryCount;
}
}
public DistributedTransactionService start() {
/*
* Note: lock makes operation _mostly_ atomic even though the base class
* changes the runState. For example, new transactions can not start
* without this lock.
*/
lock.lock();
try {
super.start();
addScheduledTasks();
return this;
} finally {
lock.unlock();
}
}
/**
* Adds the scheduled tasks.
*/
protected void addScheduledTasks() {
if (!lock.isHeldByCurrentThread())
throw new IllegalMonitorStateException();
final AbstractFederation fed = (AbstractFederation) getFederation();
// @todo config options (verify units).
notifyFuture = fed.addScheduledTask(new NotifyReleaseTimeTask(),
60/* initialDelay */, 60/* delay */, TimeUnit.SECONDS);
if (snapshotInterval != 0L) {
// start the snapshot task.
writeFuture = fed.addScheduledTask(
new SnapshotTask(),
snapshotInterval/* initialDelay */,
snapshotInterval/* delay */,
TimeUnit.MILLISECONDS);
}
}
private ScheduledFuture notifyFuture = null;
private ScheduledFuture writeFuture = null;
public void shutdown() {
lock.lock();
try {
switch (getRunState()) {
case Shutdown:
case ShutdownNow:
case Halted:
return;
}
/*
* First make sure that all tx are terminated - this is important
* otherwise we will write the commit time index image before we
* have the last commit times on hand.
*/
super.shutdown();
/*
* No need to interrupt this task. It will complete soon enough.
* However, we do want to cancel it so it will stop running.
*/
if (notifyFuture != null)
notifyFuture.cancel(false/* mayInterruptIfRunning */);
/*
* Cancel this task, but DO NOT interrupt it to avoid a partial
* write if there is a write in progress. If there is a write in
* progress, then we will wind up writing it again immediately since
* we do that below. This is Ok. We will just have a current image
* and a nearly current image.
*/
if (writeFuture != null)
writeFuture.cancel(false/* mayInterruptIfRunning */);
if (snapshotInterval != 0L) {
// write a final image during shutdown.
new SnapshotTask().run();
}
} finally {
lock.unlock();
}
}
public void shutdownNow() {
lock.lock();
try {
switch (getRunState()) {
case ShutdownNow:
case Halted:
return;
}
/*
* First make sure that all tx are terminated - this is important
* otherwise we will write the commit time index image before we
* have the last commit times on hand.
*/
super.shutdownNow();
/*
* Cancel and interrupt if running.
*/
if (notifyFuture != null)
notifyFuture.cancel(true/* mayInterruptIfRunning */);
/*
* Cancel this task and interrupt if running. Interrupting this will
* leave a partial snapshot on the disk, but we do not advance the
* counter unless the snapshot is successful so we will overwrite
* that partial snapshot below when we write a final snapshot.
*/
if (writeFuture != null)
writeFuture.cancel(true/* mayInterruptIfRunning */);
if (snapshotInterval != 0L) {
// write a final snapshot during shutdown.
snapshot();
}
} finally {
lock.unlock();
}
}
public void destroy() {
lock.lock();
try {
super.destroy();
if (!isTransient) {
// delete the commit time index log files.
new File(dataDir, BASENAME + "0" + EXT).delete();
new File(dataDir, BASENAME + "1" + EXT).delete();
// delete the data directory (works iff it is empty).
dataDir.delete();
}
} finally {
lock.unlock();
}
}
/**
* Extended to truncate the head of the {@link #commitTimeIndex} such only
* the commit times requires for reading on timestamps GTE to the new
* releaseTime are retained.
*/
protected void setReleaseTime(long releaseTime) {
super.setReleaseTime(releaseTime);
/*
* Truncate the head of the commit time index since we will no longer
* grant transactions whose start time is LTE the new releaseTime.
*/
// Note: Use the current value.
releaseTime = getReleaseTime();
if (releaseTime > 0) {
synchronized (commitTimeIndex) {
/*
* The exclusive upper bound is the timestamp of the earliest
* commit point on which we can read with this [releaseTime].
*/
final long toKey = commitTimeIndex.find(releaseTime + 1);
final ITupleIterator itr = commitTimeIndex.rangeIterator(0L,
toKey, 0/* capacity */, IRangeQuery.KEYS
| IRangeQuery.CURSOR, null/* filter */);
while (itr.hasNext()) {
itr.next();
// remove the tuple from the index.
itr.remove();
}
}
}
}
/**
* Return the proxies for the services participating in a distributed
* transaction commit or abort.
* <p>
* Note: This method is here so that it may be readily overriden for unit
* tests.
*
* @param uuids
* The {@link UUID}s of the participating services.
*
* @return The corresponding service proxies.
*/
protected ITxCommitProtocol[] getDataServices(UUID[] uuids) {
return getFederation().getDataServices(uuids);
}
/**
* Task runs {@link ITxCommitProtocol#abort(long)}.
*
* @author <a href="mailto:thompsonbry@users.sourceforge.net">Bryan Thompson</a>
*/
private static class AbortTask implements Callable<Void> {
private final ITxCommitProtocol service;
private final TxState state;
public AbortTask(final ITxCommitProtocol service, final TxState state) {
if (service == null)
throw new IllegalArgumentException();
if (state == null)
throw new IllegalArgumentException();
this.service = service;
this.state = state;
}
public Void call() throws Exception {
service.abort(state.tx);
return null;
}
}
@Override
protected void abortImpl(final TxState state) throws Exception {
if(!state.lock.isHeldByCurrentThread())
throw new IllegalMonitorStateException();
if (!state.isActive())
throw new IllegalStateException();
if(state.isReadOnly()) {
/*
* Note: There is no local state for read-only tx so we do not need
* to message the data services.
*/
state.setRunState(RunState.Aborted);
return;
}
final UUID[] uuids = state.getDataServiceUUIDs();
final ITxCommitProtocol[] services = getDataServices(uuids);
final List<Callable<Void>> tasks = new ArrayList<Callable<Void>>(
uuids.length);
for (ITxCommitProtocol dataService : services) {
tasks.add(new AbortTask(dataService, state));
}
final List<Future<Void>> futures = getFederation().getExecutorService()
.invokeAll(tasks);
List<Throwable> causes = null;
for (Future<Void> f : futures) {
try {
// verify no errors.
f.get();
} catch (Throwable t) {
/*
* Collect all causes and always log an error if any data
* service abort fails.
*
* Note: If an exception is thrown here the transaction will be
* aborted regardless. Howwever, the data service which threw
* the exception may still have local state on hand for the tx.
*/
log.error(t, t);
if (causes == null) {
causes = new LinkedList<Throwable>();
}
causes.add(t);
}
}
state.setRunState(RunState.Aborted);
if (causes != null) {
throw new ExecutionExceptions(state.toString(), causes);
}
}
/**
* There are two distinct commit protocols depending on whether the
* transaction write set is distributed across more than one
* {@link IDataService}. When write set of the transaction lies entirely on
* a single {@link IDataService}, an optimized commit protocol is used.
* When the write set of the transaction is distributed, a 3-phase commit is
* used with most of the work occurring during the "prepare" phase and a
* very rapid "commit" phase. If a distributed commit fails, even during the
* "commit", then the transaction will be rolled back on all participating
* {@link IDataService}s.
*
* <h3>Single phase commits</h3>
*
* A simple commit protocol is used when the write set of the transaction
* resides entirely on a single {@link IDataService}. Such commits DO NOT
* contend for named resource locks (either on the index names or on the
* {@link IDataService} {@link UUID}s). Since such transactions DO NOT have
* dependencies outside of the specific {@link IDataService}, a necessary
* and sufficient partial order will be imposed on the executing tasks
* locally by the {@link IDataService} on which they are executing based
* solely on the named resources which they declare. Without dependencies on
* distributed resources, this can not deadlock.
*
* <h3>Distributed commits</h3>
*
* Transaction commits for a distributed database MUST be prepared in a
* partial order so that they do not deadlock when acquiring the necessary
* locks on the named indices on the local data services. That partial order
* is imposed using the {@link #indexLockManager}. The named index locks
* are pre-declared at the start of the distributed commit protocol and are
* held through both the prepare and commit phases until the end of the
* commit protocol. The distributed commit must obtain a lock on all of the
* necessary named index resources before proceeding. If there is an
* existing commit using some of those resources, then any concurrent commit
* requiring any of those resources will block. The {@link LockManager} is
* configured to require pre-declaration of locks. Deadlocks are NOT
* possible when the locks are pre-declared.
* <p>
* A secondary partial ordering is established based on the
* {@link IDataService} {@link UUID}s during the commit phase. This partial
* order is necessary to avoid deadlocks for concurrently executing commit
* phases of distributed transactions that DO NOT share named index locks.
* Without a partial order over the participating {@link IDataService}s,
* deadlocks could arise because each transaction will grab an exclusive
* lock on the write service for each participating {@link IDataService}.
* By ordering those lock requests, we again ensure that deadlocks can not
* occur.
* <p>
* Note: The prepare phase for distributed commits allows the maximum
* possible concurrency. This is especially important as validation and
* merging down onto the unisolated indices can have significant length for
* large transactions.
* <p>
* The commit phase should be very fast, with syncing the disk providing the
* primary source of latency. All participating indices on the participating
* data services have already been checkpointed. Once the commitTime is
* assigned by the {@link DistributedTransactionService}, the group commit
* need only update the root block on the live journal and sync to disk.
*
* @todo Place timeout on the commit phase where the tx will abort unless
* all participants join at the "committed" barrier within ~ 250ms.
* That should be a generous timeout, but track aborts for this reason
* specifically since they may indicate interesting problems (heavy
* swapping, network issues, etc).
*
* @todo make sure that we checkpoint the commit record index and
* {@link Name2Addr} before requesting the commitTime to remove even
* more latency.
*/
@Override
protected long commitImpl(final TxState state) throws Exception {
if (state.isReadOnly() || state.getDataServiceCount() == 0) {
/*