/
BdbStorageEngine.java
421 lines (369 loc) · 16.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
/*
* 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 static voldemort.utils.Utils.assertNotNull;
import java.util.ArrayList;
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.serialization.IdentitySerializer;
import voldemort.serialization.VersionedSerializer;
import voldemort.store.NoSuchCapabilityException;
import voldemort.store.PersistenceFailureException;
import voldemort.store.StorageEngine;
import voldemort.store.Store;
import voldemort.store.StoreCapabilityType;
import voldemort.store.StoreUtils;
import voldemort.utils.ByteArray;
import voldemort.utils.ByteUtils;
import voldemort.utils.ClosableIterator;
import voldemort.utils.Pair;
import voldemort.versioning.ObsoleteVersionException;
import voldemort.versioning.Occured;
import voldemort.versioning.VectorClock;
import voldemort.versioning.Version;
import voldemort.versioning.Versioned;
import com.sleepycat.je.Cursor;
import com.sleepycat.je.Database;
import com.sleepycat.je.DatabaseEntry;
import com.sleepycat.je.DatabaseException;
import com.sleepycat.je.Environment;
import com.sleepycat.je.LockMode;
import com.sleepycat.je.OperationStatus;
import com.sleepycat.je.Transaction;
import com.sleepycat.je.EnvironmentConfig;
/**
* A store that uses BDB for persistence
*
* @author jay
*
*/
public class BdbStorageEngine implements StorageEngine<ByteArray, byte[]> {
private static final Logger logger = Logger.getLogger(BdbStorageEngine.class);
private static final Hex hexCodec = new Hex();
private final String name;
private final Database bdbDatabase;
private final Environment environment;
private final VersionedSerializer<byte[]> serializer;
private final AtomicBoolean isOpen;
public BdbStorageEngine(String name, Environment environment, Database database) {
assertNotNull("The store name cannot be null.", name);
assertNotNull("The database cannot be null.", database);
assertNotNull("The environment cannot be null.", environment);
this.name = name;
this.bdbDatabase = database;
this.environment = environment;
this.serializer = new VersionedSerializer<byte[]>(new IdentitySerializer());
this.isOpen = new AtomicBoolean(true);
try {
logger.info(" BDB[" + name + "] : cache size = " + environment.getConfig().getCacheSize());
logger.info(" BDB[" + name + "] : " + EnvironmentConfig.CLEANER_THREADS + " = "
+ environment.getConfig().getConfigParam(EnvironmentConfig.CLEANER_THREADS));
logger.info(" BDB[" + name + "] : " + EnvironmentConfig.CLEANER_MIN_FILE_UTILIZATION + " = "
+ environment.getConfig().getConfigParam(EnvironmentConfig.CLEANER_MIN_FILE_UTILIZATION));
logger.info(" BDB[" + name + "] : " + EnvironmentConfig.LOG_FILE_MAX + " = "
+ environment.getConfig().getConfigParam(EnvironmentConfig.LOG_FILE_MAX));
} catch (DatabaseException e) {
logger.error("Error getting config inforation for BDB at startup", e);
}
}
public String getName() {
return name;
}
public ClosableIterator<Pair<ByteArray, Versioned<byte[]>>> entries() {
try {
Cursor cursor = bdbDatabase.openCursor(null, null);
return new BdbStoreIterator(cursor);
} catch(DatabaseException e) {
throw new PersistenceFailureException(e);
}
}
static byte[] makeKey(byte[] key, Version version) {
VectorClock clock = (VectorClock) version;
int clockSize = clock == null ? 0 : clock.sizeInBytes();
byte[] keyBytes = new byte[2 + key.length + clockSize];
ByteUtils.writeShort(keyBytes, (short) key.length, 0);
System.arraycopy(key, 0, keyBytes, 2, key.length);
if(clock != null)
System.arraycopy(clock.toBytes(), 0, keyBytes, key.length + 2, clockSize);
return keyBytes;
}
static byte[] getObjKey(byte[] bytes) {
short size = ByteUtils.readShort(bytes, 0);
byte[] dest = new byte[size];
if(size > 0)
System.arraycopy(bytes, 2, dest, 0, size);
return dest;
}
static VectorClock getVersion(byte[] bytes) {
short size = ByteUtils.readShort(bytes, 0);
if(size >= bytes.length - 2)
return null;
else
return new VectorClock(ByteUtils.copy(bytes, size + 2, bytes.length));
}
static boolean isPrefix(byte[] prefix, byte[] complete) {
if(prefix.length > complete.length)
return false;
for(int i = 0; i < prefix.length; i++)
if(complete[i] != prefix[i])
return false;
return true;
}
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 List<Versioned<byte[]>> get(ByteArray key) throws PersistenceFailureException {
return get(key, LockMode.READ_UNCOMMITTED);
}
private List<Versioned<byte[]>> get(ByteArray key, LockMode lockMode)
throws PersistenceFailureException {
StoreUtils.assertValidKey(key);
Cursor cursor = null;
try {
cursor = bdbDatabase.openCursor(null, null);
return get(cursor, key, lockMode);
} catch(DatabaseException e) {
throw new PersistenceFailureException(e);
} finally {
attemptClose(cursor);
}
}
public Map<ByteArray, List<Versioned<byte[]>>> getAll(Iterable<ByteArray> keys)
throws VoldemortException {
StoreUtils.assertValidKeys(keys);
Map<ByteArray, List<Versioned<byte[]>>> result = StoreUtils.newEmptyHashMap(keys);
Cursor cursor = null;
try {
cursor = bdbDatabase.openCursor(null, null);
for(ByteArray key: keys) {
List<Versioned<byte[]>> values = get(cursor, key, LockMode.READ_UNCOMMITTED);
if(!values.isEmpty())
result.put(key, values);
}
} catch(DatabaseException e) {
throw new PersistenceFailureException(e);
} finally {
attemptClose(cursor);
}
return result;
}
private List<Versioned<byte[]>> get(Cursor cursor, ByteArray key, LockMode lockMode)
throws DatabaseException {
StoreUtils.assertValidKey(key);
DatabaseEntry keyEntry = new DatabaseEntry(key.get());
DatabaseEntry valueEntry = new DatabaseEntry();
List<Versioned<byte[]>> results = new ArrayList<Versioned<byte[]>>();
for(OperationStatus status = cursor.getSearchKey(keyEntry, valueEntry, lockMode); status == OperationStatus.SUCCESS; status = cursor.getNextDup(keyEntry,
valueEntry,
LockMode.RMW)) {
results.add(serializer.toObject(valueEntry.getData()));
}
return results;
}
public void put(ByteArray key, Versioned<byte[]> value) throws PersistenceFailureException {
StoreUtils.assertValidKey(key);
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 = bdbDatabase.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());
Occured occured = value.getVersion().compare(clock);
if(occured == Occured.BEFORE)
throw new ObsoleteVersionException("Key '"
+ new String(hexCodec.encode(key.get()))
+ "' " + value.getVersion().toString()
+ " is obsolete," + " current version is "
+ clock + ".");
else if(occured == Occured.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(serializer.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) {
throw new PersistenceFailureException(e);
} finally {
attemptClose(cursor);
if(succeeded)
attemptCommit(transaction);
else
attemptAbort(transaction);
}
}
public boolean delete(ByteArray key, Version version) throws PersistenceFailureException {
StoreUtils.assertValidKey(key);
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 = bdbDatabase.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) == Occured.BEFORE) {
cursor.delete();
deletedSomething = true;
}
status = cursor.getNextDup(keyEntry, valueEntry, LockMode.READ_UNCOMMITTED);
}
return deletedSomething;
} catch(DatabaseException e) {
throw new PersistenceFailureException(e);
} finally {
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.bdbDatabase.close();
} catch(DatabaseException e) {
throw new PersistenceFailureException("Shutdown failed.", e);
}
}
private void attemptAbort(Transaction transaction) {
try {
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 class BdbStoreIterator implements
ClosableIterator<Pair<ByteArray, Versioned<byte[]>>> {
private volatile boolean isOpen;
private final Cursor cursor;
private Pair<ByteArray, Versioned<byte[]>> current;
public BdbStoreIterator(Cursor cursor) {
this.cursor = cursor;
isOpen = true;
DatabaseEntry keyEntry = new DatabaseEntry();
DatabaseEntry valueEntry = new DatabaseEntry();
try {
cursor.getFirst(keyEntry, valueEntry, LockMode.READ_UNCOMMITTED);
} catch(DatabaseException e) {
throw new PersistenceFailureException(e);
}
current = getPair(keyEntry, valueEntry);
}
private Pair<ByteArray, Versioned<byte[]>> getPair(DatabaseEntry key, DatabaseEntry value) {
if(key == null || key.getData() == null) {
return null;
} else {
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));
}
}
public boolean hasNext() {
return current != null;
}
public Pair<ByteArray, Versioned<byte[]>> next() {
if(!isOpen)
throw new PersistenceFailureException("Call to next() on a closed iterator.");
DatabaseEntry keyEntry = new DatabaseEntry();
DatabaseEntry valueEntry = new DatabaseEntry();
try {
cursor.getNext(keyEntry, valueEntry, LockMode.READ_UNCOMMITTED);
} catch(DatabaseException e) {
throw new PersistenceFailureException(e);
}
Pair<ByteArray, Versioned<byte[]>> previous = current;
if(keyEntry.getData() == null)
current = null;
else
current = getPair(keyEntry, valueEntry);
return previous;
}
public void remove() {
throw new UnsupportedOperationException("No removal y'all.");
}
public void close() {
try {
cursor.close();
isOpen = false;
} catch(DatabaseException e) {
throw new PersistenceFailureException(e);
}
}
@Override
protected void finalize() {
if(isOpen) {
logger.error("Failure to close cursor, will be forcably closed.");
close();
}
}
}
}