forked from voldemort/voldemort
/
BdbStorageEngine.java
661 lines (560 loc) · 24.1 KB
/
BdbStorageEngine.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
/*
* Copyright 2008-2009 LinkedIn, Inc
*
* 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.
*/
package voldemort.store.bdb;
import java.io.File;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.codec.binary.Hex;
import org.apache.log4j.Logger;
import voldemort.VoldemortException;
import voldemort.annotations.jmx.JmxOperation;
import voldemort.serialization.IdentitySerializer;
import voldemort.serialization.Serializer;
import voldemort.serialization.VersionedSerializer;
import voldemort.server.protocol.admin.AsyncOperationStatus;
import voldemort.store.NoSuchCapabilityException;
import voldemort.store.PersistenceFailureException;
import voldemort.store.StorageEngine;
import voldemort.store.StorageInitializationException;
import voldemort.store.Store;
import voldemort.store.StoreCapabilityType;
import voldemort.store.StoreUtils;
import voldemort.store.backup.NativeBackupable;
import voldemort.store.bdb.stats.BdbEnvironmentStats;
import voldemort.utils.ByteArray;
import voldemort.utils.ByteUtils;
import voldemort.utils.ClosableIterator;
import voldemort.utils.Pair;
import voldemort.utils.Utils;
import voldemort.versioning.ObsoleteVersionException;
import voldemort.versioning.Occurred;
import voldemort.versioning.VectorClock;
import voldemort.versioning.Version;
import voldemort.versioning.Versioned;
import com.google.common.collect.Lists;
import com.sleepycat.je.Cursor;
import com.sleepycat.je.Database;
import com.sleepycat.je.DatabaseEntry;
import com.sleepycat.je.DatabaseException;
import com.sleepycat.je.DatabaseStats;
import com.sleepycat.je.Environment;
import com.sleepycat.je.LockMode;
import com.sleepycat.je.OperationStatus;
import com.sleepycat.je.StatsConfig;
import com.sleepycat.je.Transaction;
/**
* A store that uses BDB for persistence
*
*
*/
public class BdbStorageEngine implements StorageEngine<ByteArray, byte[], byte[]>, NativeBackupable {
private static final Logger logger = Logger.getLogger(BdbStorageEngine.class);
private static final Hex hexCodec = new Hex();
private final String name;
private Database bdbDatabase;
private final Environment environment;
private final VersionedSerializer<byte[]> versionedSerializer;
private final AtomicBoolean isOpen;
private final LockMode readLockMode;
private final Serializer<Version> versionSerializer;
private final BdbEnvironmentStats bdbEnvironmentStats;
private final AtomicBoolean isTruncating = new AtomicBoolean(false);
public BdbStorageEngine(String name,
Environment environment,
Database database,
BdbRuntimeConfig config) {
this.name = Utils.notNull(name);
this.bdbDatabase = Utils.notNull(database);
this.environment = Utils.notNull(environment);
this.versionedSerializer = new VersionedSerializer<byte[]>(new IdentitySerializer());
this.versionSerializer = new Serializer<Version>() {
public byte[] toBytes(Version object) {
return ((VectorClock) object).toBytes();
}
public Version toObject(byte[] bytes) {
return versionedSerializer.getVersion(bytes);
}
};
this.isOpen = new AtomicBoolean(true);
this.readLockMode = config.getLockMode();
this.bdbEnvironmentStats = new BdbEnvironmentStats(environment,
config.getStatsCacheTtlMs(),
config.getExposeSpaceUtil());
}
public String getName() {
return name;
}
public ClosableIterator<Pair<ByteArray, Versioned<byte[]>>> entries() {
try {
Cursor cursor = getBdbDatabase().openCursor(null, null);
return new BdbEntriesIterator(cursor);
} catch(DatabaseException e) {
logger.error(e);
throw new PersistenceFailureException(e);
}
}
public ClosableIterator<ByteArray> keys() {
try {
Cursor cursor = getBdbDatabase().openCursor(null, null);
return new BdbKeysIterator(cursor);
} catch(DatabaseException e) {
logger.error(e);
throw new PersistenceFailureException(e);
}
}
public void truncate() {
if(isTruncating.compareAndSet(false, true)) {
Transaction transaction = null;
boolean succeeded = false;
try {
transaction = this.environment.beginTransaction(null, null);
// close current bdbDatabase first
bdbDatabase.close();
// truncate the database
environment.truncateDatabase(transaction, this.getName(), false);
succeeded = true;
} catch(DatabaseException e) {
logger.error(e);
throw new VoldemortException("Failed to truncate Bdb store " + getName(), e);
} finally {
commitOrAbort(succeeded, transaction);
// reopen the bdb database for future queries.
if(reopenBdbDatabase()) {
isTruncating.compareAndSet(true, false);
} else {
throw new VoldemortException("Failed to reopen Bdb Database after truncation, All request will fail on store "
+ getName());
}
}
} else {
throw new VoldemortException("Store " + getName()
+ " is already truncating, cannot start another one.");
}
}
private void commitOrAbort(boolean succeeded, Transaction transaction) {
try {
if(succeeded) {
attemptCommit(transaction);
} else {
attemptAbort(transaction);
}
} catch(Exception e) {
logger.error(e);
}
}
/**
* Reopens the bdb Database after a successful truncate operation.
*/
private boolean reopenBdbDatabase() {
try {
bdbDatabase = environment.openDatabase(null,
this.getName(),
this.bdbDatabase.getConfig());
return true;
} catch(DatabaseException e) {
throw new StorageInitializationException("Failed to reinitialize BdbStorageEngine for store:"
+ getName() + " after truncation.",
e);
}
}
public List<Version> getVersions(ByteArray key) {
return get(key, null, readLockMode, versionSerializer);
}
public List<Versioned<byte[]>> get(ByteArray key, byte[] transforms)
throws PersistenceFailureException {
return get(key, transforms, readLockMode, versionedSerializer);
}
private <T> List<T> get(ByteArray key,
@SuppressWarnings("unused") byte[] transforms,
LockMode lockMode,
Serializer<T> serializer) throws PersistenceFailureException {
StoreUtils.assertValidKey(key);
long startTimeNs = -1;
if(logger.isTraceEnabled())
startTimeNs = System.nanoTime();
Cursor cursor = null;
try {
cursor = getBdbDatabase().openCursor(null, null);
List<T> result = get(cursor, key, lockMode, serializer);
// If null, try again in different locking mode to
// avoid null result due to gap between delete and new write
if(result.size() == 0 && lockMode != LockMode.DEFAULT) {
return get(cursor, key, LockMode.DEFAULT, serializer);
} else {
return result;
}
} catch(DatabaseException e) {
logger.error(e);
throw new PersistenceFailureException(e);
} finally {
if(logger.isTraceEnabled()) {
logger.trace("Completed GET from key " + key + " (keyRef: "
+ System.identityHashCode(key) + ") in "
+ (System.nanoTime() - startTimeNs) + " ns at "
+ System.currentTimeMillis());
}
attemptClose(cursor);
}
}
/**
* truncate() operation mandates that all opened Database be closed before
* attempting truncation.
* <p>
* This method throws an exception while truncation is happening to any
* request attempting in parallel with store truncation.
*
* @return
*/
private Database getBdbDatabase() {
if(isTruncating.get()) {
throw new VoldemortException("Bdb Store " + getName()
+ " is currently truncating cannot serve any request.");
}
return bdbDatabase;
}
public Map<ByteArray, List<Versioned<byte[]>>> getAll(Iterable<ByteArray> keys,
Map<ByteArray, byte[]> transforms)
throws VoldemortException {
long startTimeNs = -1;
if(logger.isTraceEnabled())
startTimeNs = System.nanoTime();
StoreUtils.assertValidKeys(keys);
Map<ByteArray, List<Versioned<byte[]>>> result = StoreUtils.newEmptyHashMap(keys);
Cursor cursor = null;
String keyStr = "";
try {
cursor = getBdbDatabase().openCursor(null, null);
for(ByteArray key: keys) {
if(logger.isTraceEnabled())
keyStr += key + " ";
List<Versioned<byte[]>> values = get(cursor, key, readLockMode, versionedSerializer);
if(!values.isEmpty())
result.put(key, values);
}
} catch(DatabaseException e) {
logger.error(e);
throw new PersistenceFailureException(e);
} finally {
attemptClose(cursor);
}
if(logger.isTraceEnabled())
logger.trace("Completed GETALL from keys " + keyStr + " in "
+ (System.nanoTime() - startTimeNs) + " ns at "
+ System.currentTimeMillis());
return result;
}
private static <T> List<T> get(Cursor cursor,
ByteArray key,
LockMode lockMode,
Serializer<T> serializer) throws DatabaseException {
StoreUtils.assertValidKey(key);
long startTimeNs = -1;
if(logger.isTraceEnabled())
startTimeNs = System.nanoTime();
DatabaseEntry keyEntry = new DatabaseEntry(key.get());
DatabaseEntry valueEntry = new DatabaseEntry();
List<T> results = Lists.newArrayList();
for(OperationStatus status = cursor.getSearchKey(keyEntry, valueEntry, lockMode); status == OperationStatus.SUCCESS; status = cursor.getNextDup(keyEntry,
valueEntry,
lockMode)) {
results.add(serializer.toObject(valueEntry.getData()));
}
if(logger.isTraceEnabled()) {
logger.trace("Completed GET from key " + key + " in "
+ (System.nanoTime() - startTimeNs) + " ns at "
+ System.currentTimeMillis());
}
return results;
}
public void put(ByteArray key, Versioned<byte[]> value, byte[] transforms)
throws PersistenceFailureException {
StoreUtils.assertValidKey(key);
long startTimeNs = -1;
if(logger.isTraceEnabled())
startTimeNs = System.nanoTime();
DatabaseEntry keyEntry = new DatabaseEntry(key.get());
boolean succeeded = false;
Transaction transaction = null;
Cursor cursor = null;
try {
transaction = this.environment.beginTransaction(null, null);
// Check existing values
// if there is a version obsoleted by this value delete it
// if there is a version later than this one, throw an exception
DatabaseEntry valueEntry = new DatabaseEntry();
cursor = getBdbDatabase().openCursor(transaction, null);
for(OperationStatus status = cursor.getSearchKey(keyEntry, valueEntry, LockMode.RMW); status == OperationStatus.SUCCESS; status = cursor.getNextDup(keyEntry,
valueEntry,
LockMode.RMW)) {
VectorClock clock = new VectorClock(valueEntry.getData());
Occurred occurred = value.getVersion().compare(clock);
if(occurred == Occurred.BEFORE)
throw new ObsoleteVersionException("Key "
+ new String(hexCodec.encode(key.get()))
+ " "
+ value.getVersion().toString()
+ " is obsolete, it is no greater than the current version of "
+ clock + ".");
else if(occurred == Occurred.AFTER)
// best effort delete of obsolete previous value!
cursor.delete();
}
// Okay so we cleaned up all the prior stuff, so now we are good to
// insert the new thing
valueEntry = new DatabaseEntry(versionedSerializer.toBytes(value));
OperationStatus status = cursor.put(keyEntry, valueEntry);
if(status != OperationStatus.SUCCESS)
throw new PersistenceFailureException("Put operation failed with status: " + status);
succeeded = true;
} catch(DatabaseException e) {
logger.error(e);
throw new PersistenceFailureException(e);
} finally {
attemptClose(cursor);
if(succeeded)
attemptCommit(transaction);
else
attemptAbort(transaction);
}
if(logger.isTraceEnabled()) {
logger.trace("Completed PUT to key " + key + " (keyRef: "
+ System.identityHashCode(key) + " value " + value + " in "
+ (System.nanoTime() - startTimeNs) + " ns at "
+ System.currentTimeMillis());
}
}
public boolean delete(ByteArray key, Version version) throws PersistenceFailureException {
StoreUtils.assertValidKey(key);
long startTimeNs = -1;
if(logger.isTraceEnabled())
startTimeNs = System.nanoTime();
boolean deletedSomething = false;
Cursor cursor = null;
Transaction transaction = null;
try {
transaction = this.environment.beginTransaction(null, null);
DatabaseEntry keyEntry = new DatabaseEntry(key.get());
DatabaseEntry valueEntry = new DatabaseEntry();
cursor = getBdbDatabase().openCursor(transaction, null);
OperationStatus status = cursor.getSearchKey(keyEntry,
valueEntry,
LockMode.READ_UNCOMMITTED);
while(status == OperationStatus.SUCCESS) {
// if version is null no comparison is necessary
if(new VectorClock(valueEntry.getData()).compare(version) == Occurred.BEFORE) {
cursor.delete();
deletedSomething = true;
}
status = cursor.getNextDup(keyEntry, valueEntry, LockMode.READ_UNCOMMITTED);
}
return deletedSomething;
} catch(DatabaseException e) {
logger.error(e);
throw new PersistenceFailureException(e);
} finally {
if(logger.isTraceEnabled()) {
logger.trace("Completed DELETE of key " + key + " (keyRef: "
+ System.identityHashCode(key) + ") in "
+ (System.nanoTime() - startTimeNs) + " ns at "
+ System.currentTimeMillis());
}
try {
attemptClose(cursor);
} finally {
attemptCommit(transaction);
}
}
}
public Object getCapability(StoreCapabilityType capability) {
throw new NoSuchCapabilityException(capability, getName());
}
@Override
public int hashCode() {
return name.hashCode();
}
@Override
public boolean equals(Object o) {
if(o == null || !Store.class.isAssignableFrom(o.getClass()))
return false;
Store<?, ?, ?> s = (Store<?, ?, ?>) o;
return s.getName().equals(s.getName());
}
public void close() throws PersistenceFailureException {
try {
if(this.isOpen.compareAndSet(true, false))
this.getBdbDatabase().close();
} catch(DatabaseException e) {
logger.error(e);
throw new PersistenceFailureException("Shutdown failed.", e);
}
}
private void attemptAbort(Transaction transaction) {
try {
if(transaction != null)
transaction.abort();
} catch(Exception e) {
logger.error("Abort failed!", e);
}
}
private void attemptCommit(Transaction transaction) {
try {
transaction.commit();
} catch(DatabaseException e) {
logger.error("Transaction commit failed!", e);
attemptAbort(transaction);
throw new PersistenceFailureException(e);
}
}
private static void attemptClose(Cursor cursor) {
try {
if(cursor != null)
cursor.close();
} catch(DatabaseException e) {
logger.error("Error closing cursor.", e);
throw new PersistenceFailureException(e.getMessage(), e);
}
}
public DatabaseStats getStats(boolean setFast) {
try {
StatsConfig config = new StatsConfig();
config.setFast(setFast);
return this.getBdbDatabase().getStats(config);
} catch(DatabaseException e) {
logger.error(e);
throw new VoldemortException(e);
}
}
@JmxOperation(description = "A variety of quickly computable stats about the BDB for this store.")
public String getBdbStats() {
return getBdbStats(true);
}
@JmxOperation(description = "A variety of stats about the BDB for this store.")
public String getBdbStats(boolean fast) {
String dbStats = getStats(fast).toString();
logger.debug(dbStats);
return dbStats;
}
public BdbEnvironmentStats getBdbEnvironmentStats() {
return bdbEnvironmentStats;
}
private static abstract class BdbIterator<T> implements ClosableIterator<T> {
private final boolean noValues;
final Cursor cursor;
private T current;
private volatile boolean isOpen;
public BdbIterator(Cursor cursor, boolean noValues) {
this.cursor = cursor;
isOpen = true;
this.noValues = noValues;
DatabaseEntry keyEntry = new DatabaseEntry();
DatabaseEntry valueEntry = new DatabaseEntry();
if(noValues)
valueEntry.setPartial(true);
try {
cursor.getFirst(keyEntry, valueEntry, LockMode.READ_UNCOMMITTED);
} catch(DatabaseException e) {
logger.error(e);
throw new PersistenceFailureException(e);
}
if(keyEntry.getData() != null)
current = get(keyEntry, valueEntry);
}
protected abstract T get(DatabaseEntry key, DatabaseEntry value);
protected abstract void moveCursor(DatabaseEntry key, DatabaseEntry value)
throws DatabaseException;
public final boolean hasNext() {
return current != null;
}
public final T next() {
if(!isOpen)
throw new PersistenceFailureException("Call to next() on a closed iterator.");
DatabaseEntry keyEntry = new DatabaseEntry();
DatabaseEntry valueEntry = new DatabaseEntry();
if(noValues)
valueEntry.setPartial(true);
try {
moveCursor(keyEntry, valueEntry);
} catch(DatabaseException e) {
logger.error(e);
throw new PersistenceFailureException(e);
}
T previous = current;
if(keyEntry.getData() == null)
current = null;
else
current = get(keyEntry, valueEntry);
return previous;
}
public final void remove() {
throw new UnsupportedOperationException("No removal y'all.");
}
public final void close() {
try {
cursor.close();
isOpen = false;
} catch(DatabaseException e) {
logger.error(e);
}
}
@Override
protected final void finalize() {
if(isOpen) {
logger.error("Failure to close cursor, will be forcably closed.");
close();
}
}
}
private static class BdbKeysIterator extends BdbIterator<ByteArray> {
public BdbKeysIterator(Cursor cursor) {
super(cursor, true);
}
@Override
protected ByteArray get(DatabaseEntry key, DatabaseEntry value) {
return new ByteArray(key.getData());
}
@Override
protected void moveCursor(DatabaseEntry key, DatabaseEntry value) throws DatabaseException {
cursor.getNextNoDup(key, value, LockMode.READ_UNCOMMITTED);
}
}
private static class BdbEntriesIterator extends BdbIterator<Pair<ByteArray, Versioned<byte[]>>> {
public BdbEntriesIterator(Cursor cursor) {
super(cursor, false);
}
@Override
protected Pair<ByteArray, Versioned<byte[]>> get(DatabaseEntry key, DatabaseEntry value) {
VectorClock clock = new VectorClock(value.getData());
byte[] bytes = ByteUtils.copy(value.getData(),
clock.sizeInBytes(),
value.getData().length);
return Pair.create(new ByteArray(key.getData()), new Versioned<byte[]>(bytes, clock));
}
@Override
protected void moveCursor(DatabaseEntry key, DatabaseEntry value) throws DatabaseException {
cursor.getNext(key, value, LockMode.READ_UNCOMMITTED);
}
}
public boolean isPartitionAware() {
return false;
}
public void nativeBackup(File toDir,
boolean verifyFiles,
boolean isIncremental,
AsyncOperationStatus status) {
new BdbNativeBackup(environment, verifyFiles, isIncremental).performBackup(toDir, status);
}
}