forked from voldemort/voldemort
/
KratiStorageEngine.java
342 lines (289 loc) · 12.2 KB
/
KratiStorageEngine.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
package voldemort.store.krati;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.File;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import krati.array.DataArray;
import krati.core.segment.SegmentFactory;
import krati.store.DynamicDataStore;
import krati.util.FnvHashFunction;
import org.apache.log4j.Logger;
import voldemort.VoldemortException;
import voldemort.store.AbstractStorageEngine;
import voldemort.store.StoreUtils;
import voldemort.utils.ByteArray;
import voldemort.utils.ClosableIterator;
import voldemort.utils.Pair;
import voldemort.utils.StripedLock;
import voldemort.versioning.ObsoleteVersionException;
import voldemort.versioning.Occurred;
import voldemort.versioning.VectorClock;
import voldemort.versioning.Version;
import voldemort.versioning.Versioned;
public class KratiStorageEngine extends AbstractStorageEngine<ByteArray, byte[], byte[]> {
private static final Logger logger = Logger.getLogger(KratiStorageEngine.class);
private final DynamicDataStore datastore;
private final StripedLock locks;
public KratiStorageEngine(String name,
SegmentFactory segmentFactory,
int segmentFileSizeMB,
int lockStripes,
double hashLoadFactor,
int initLevel,
File dataDirectory) {
super(name);
try {
this.datastore = new DynamicDataStore(dataDirectory,
initLevel,
segmentFileSizeMB,
segmentFactory,
hashLoadFactor,
new FnvHashFunction());
this.locks = new StripedLock(lockStripes);
} catch(Exception e) {
throw new VoldemortException("Failure initializing store.", e);
}
}
@Override
public Map<ByteArray, List<Versioned<byte[]>>> getAll(Iterable<ByteArray> keys,
Map<ByteArray, byte[]> transforms)
throws VoldemortException {
StoreUtils.assertValidKeys(keys);
return StoreUtils.getAll(this, keys, null);
}
@Override
public List<Version> getVersions(ByteArray key) {
return StoreUtils.getVersions(get(key, null));
}
@Override
public void truncate() {
try {
datastore.clear();
} catch(Exception e) {
logger.error("Failed to truncate store '" + getName() + "': ", e);
throw new VoldemortException("Failed to truncate store '" + getName() + "'.");
}
}
@Override
public List<Versioned<byte[]>> get(ByteArray key, byte[] transforms) throws VoldemortException {
StoreUtils.assertValidKey(key);
try {
return disassembleValues(datastore.get(key.get()));
} catch(Exception e) {
logger.error("Error reading value: ", e);
throw new VoldemortException("Error reading value: ", e);
}
}
@Override
public ClosableIterator<Pair<ByteArray, Versioned<byte[]>>> entries() {
List<Pair<ByteArray, Versioned<byte[]>>> returnedList = new ArrayList<Pair<ByteArray, Versioned<byte[]>>>();
DataArray array = datastore.getDataArray();
for(int index = 0; index < array.length(); index++) {
byte[] returnedBytes = array.get(index);
if(returnedBytes != null) {
// Extract the key value pair from this
// TODO: Move to DynamicDataStore code
ByteBuffer bb = ByteBuffer.wrap(returnedBytes);
int cnt = bb.getInt();
if(cnt > 0) {
int keyLen = bb.getInt();
byte[] key = new byte[keyLen];
bb.get(key);
int valueLen = bb.getInt();
byte[] value = new byte[valueLen];
bb.get(value);
List<Versioned<byte[]>> versions;
try {
versions = disassembleValues(value);
} catch(IOException e) {
versions = null;
}
if(versions != null) {
Iterator<Versioned<byte[]>> iterVersions = versions.iterator();
while(iterVersions.hasNext()) {
Versioned<byte[]> currentVersion = iterVersions.next();
returnedList.add(Pair.create(new ByteArray(key), currentVersion));
}
}
}
}
}
return new KratiClosableIterator(returnedList);
}
@Override
public ClosableIterator<ByteArray> keys() {
return StoreUtils.keys(entries());
}
@Override
public ClosableIterator<Pair<ByteArray, Versioned<byte[]>>> entries(int partition) {
throw new UnsupportedOperationException("Partition based entries scan not supported for this storage type");
}
@Override
public ClosableIterator<ByteArray> keys(int partition) {
throw new UnsupportedOperationException("Partition based key scan not supported for this storage type");
}
@Override
public boolean delete(ByteArray key, Version maxVersion) throws VoldemortException {
StoreUtils.assertValidKey(key);
synchronized(this.locks.lockFor(key.get())) {
if(maxVersion == null) {
try {
return datastore.delete(key.get());
} catch(Exception e) {
logger.error("Failed to delete key: ", e);
throw new VoldemortException("Failed to delete key: " + key, e);
}
}
List<Versioned<byte[]>> returnedValuesList = this.get(key, null);
// Case if there is nothing to delete
if(returnedValuesList.size() == 0) {
return false;
}
Iterator<Versioned<byte[]>> iter = returnedValuesList.iterator();
while(iter.hasNext()) {
Versioned<byte[]> currentValue = iter.next();
Version currentVersion = currentValue.getVersion();
if(currentVersion.compare(maxVersion) == Occurred.BEFORE) {
iter.remove();
}
}
try {
if(returnedValuesList.size() == 0)
return datastore.delete(key.get());
else
return datastore.put(key.get(), assembleValues(returnedValuesList));
} catch(Exception e) {
String message = "Failed to delete key " + key;
logger.error(message, e);
throw new VoldemortException(message, e);
}
}
}
@Override
public void put(ByteArray key, Versioned<byte[]> value, byte[] transforms)
throws VoldemortException {
StoreUtils.assertValidKey(key);
synchronized(this.locks.lockFor(key.get())) {
// First get the value
List<Versioned<byte[]>> existingValuesList = this.get(key, null);
// If no value, add one
if(existingValuesList.size() == 0) {
existingValuesList = new ArrayList<Versioned<byte[]>>();
existingValuesList.add(new Versioned<byte[]>(value.getValue(), value.getVersion()));
} else {
// Update the value
List<Versioned<byte[]>> removedValueList = new ArrayList<Versioned<byte[]>>();
for(Versioned<byte[]> versioned: existingValuesList) {
Occurred occurred = value.getVersion().compare(versioned.getVersion());
if(occurred == Occurred.BEFORE)
throw new ObsoleteVersionException("Obsolete version for key '" + key
+ "': " + value.getVersion());
else if(occurred == Occurred.AFTER)
removedValueList.add(versioned);
}
existingValuesList.removeAll(removedValueList);
existingValuesList.add(value);
}
try {
datastore.put(key.get(), assembleValues(existingValuesList));
} catch(Exception e) {
String message = "Failed to put key " + key;
logger.error(message, e);
throw new VoldemortException(message, e);
}
}
}
@Override
public List<Versioned<byte[]>> multiVersionPut(ByteArray key,
final List<Versioned<byte[]>> values)
throws VoldemortException {
StoreUtils.assertValidKey(key);
List<Versioned<byte[]>> valuesInStorage = null;
List<Versioned<byte[]>> obsoleteVals = null;
synchronized(this.locks.lockFor(key.get())) {
valuesInStorage = this.get(key, null);
obsoleteVals = computeVersionsToStore(valuesInStorage, values);
try {
datastore.put(key.get(), assembleValues(valuesInStorage));
} catch(Exception e) {
String message = "Failed to put key " + key;
logger.error(message, e);
throw new VoldemortException(message, e);
}
}
return obsoleteVals;
}
/**
* Store the versioned values
*
* @param values list of versioned bytes
* @return the list of versioned values rolled into an array of bytes
*/
private byte[] assembleValues(List<Versioned<byte[]>> values) throws IOException {
ByteArrayOutputStream stream = new ByteArrayOutputStream();
DataOutputStream dataStream = new DataOutputStream(stream);
for(Versioned<byte[]> value: values) {
byte[] object = value.getValue();
dataStream.writeInt(object.length);
dataStream.write(object);
VectorClock clock = (VectorClock) value.getVersion();
dataStream.writeInt(clock.sizeInBytes());
dataStream.write(clock.toBytes());
}
return stream.toByteArray();
}
/**
* Splits up value into multiple versioned values
*
* @param value
* @return
* @throws IOException
*/
private List<Versioned<byte[]>> disassembleValues(byte[] values) throws IOException {
if(values == null)
return new ArrayList<Versioned<byte[]>>(0);
List<Versioned<byte[]>> returnList = new ArrayList<Versioned<byte[]>>();
ByteArrayInputStream stream = new ByteArrayInputStream(values);
DataInputStream dataStream = new DataInputStream(stream);
while(dataStream.available() > 0) {
byte[] object = new byte[dataStream.readInt()];
dataStream.read(object);
byte[] clockBytes = new byte[dataStream.readInt()];
dataStream.read(clockBytes);
VectorClock clock = new VectorClock(clockBytes);
returnList.add(new Versioned<byte[]>(object, clock));
}
return returnList;
}
private class KratiClosableIterator implements
ClosableIterator<Pair<ByteArray, Versioned<byte[]>>> {
private Iterator<Pair<ByteArray, Versioned<byte[]>>> iter;
public KratiClosableIterator(List<Pair<ByteArray, Versioned<byte[]>>> list) {
iter = list.iterator();
}
@Override
public void close() {
// Nothing to close here
}
@Override
public boolean hasNext() {
return iter.hasNext();
}
@Override
public Pair<ByteArray, Versioned<byte[]>> next() {
return iter.next();
}
@Override
public void remove() {
Pair<ByteArray, Versioned<byte[]>> currentPair = next();
delete(currentPair.getFirst(), currentPair.getSecond().getVersion());
}
}
}