forked from voldemort/voldemort
/
RedirectingStore.java
416 lines (376 loc) · 18.2 KB
/
RedirectingStore.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
/*
* Copyright 2008-2010 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.rebalancing;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.log4j.Logger;
import voldemort.VoldemortException;
import voldemort.annotations.jmx.JmxGetter;
import voldemort.annotations.jmx.JmxSetter;
import voldemort.client.protocol.RequestFormatType;
import voldemort.client.rebalance.RebalancePartitionsInfo;
import voldemort.cluster.Node;
import voldemort.cluster.failuredetector.FailureDetector;
import voldemort.server.RequestRoutingType;
import voldemort.server.StoreRepository;
import voldemort.store.DelegatingStore;
import voldemort.store.Store;
import voldemort.store.StoreUtils;
import voldemort.store.UnreachableStoreException;
import voldemort.store.metadata.MetadataStore;
import voldemort.store.metadata.MetadataStore.VoldemortState;
import voldemort.store.socket.SocketStoreFactory;
import voldemort.utils.ByteArray;
import voldemort.utils.Time;
import voldemort.versioning.ObsoleteVersionException;
import voldemort.versioning.Version;
import voldemort.versioning.Versioned;
import com.google.common.collect.HashMultimap;
import com.google.common.collect.Iterables;
import com.google.common.collect.Maps;
import com.google.common.collect.Multimap;
/**
* The RedirectingStore extends {@link DelegatingStore}
* <p>
* If current server_state is {@link VoldemortState#REBALANCING_MASTER_SERVER} <br>
* then before serving any client request do a remote get() call, put it locally
* ignoring any {@link ObsoleteVersionException} and then serve the client
* requests. This piece of code is run on the stealer nodes.
*/
public class RedirectingStore extends DelegatingStore<ByteArray, byte[], byte[]> {
private final static Logger logger = Logger.getLogger(RedirectingStore.class);
private final MetadataStore metadata;
private final StoreRepository storeRepository;
private final SocketStoreFactory storeFactory;
private FailureDetector failureDetector;
private AtomicBoolean isRedirectingStoreEnabled;
public RedirectingStore(Store<ByteArray, byte[], byte[]> innerStore,
MetadataStore metadata,
StoreRepository storeRepository,
FailureDetector detector,
SocketStoreFactory storeFactory) {
super(innerStore);
this.metadata = metadata;
this.storeRepository = storeRepository;
this.storeFactory = storeFactory;
this.failureDetector = detector;
this.isRedirectingStoreEnabled = new AtomicBoolean(true);
}
@JmxSetter(name = "setRedirectingStoreEnabled", description = "Enable the redirecting store for this store")
public void setIsRedirectingStoreEnabled(boolean isRedirectingStoreEnabled) {
logger.info("Setting redirecting store flag for " + getName() + " to "
+ isRedirectingStoreEnabled);
this.isRedirectingStoreEnabled.set(isRedirectingStoreEnabled);
}
@JmxGetter(name = "isRedirectingStoreEnabled", description = "Get the redirecting store state for this store")
public boolean getIsRedirectingStoreEnabled() {
return this.isRedirectingStoreEnabled.get();
}
private RebalancePartitionsInfo redirectingKey(ByteArray key) {
if(VoldemortState.REBALANCING_MASTER_SERVER.equals(metadata.getServerState())
&& isRedirectingStoreEnabled.get()) {
return metadata.getRebalancerState().find(getName(),
metadata.getRoutingStrategy(getName())
.getPartitionList(key.get()),
metadata.getCluster()
.getNodeById(metadata.getNodeId())
.getPartitionIds());
}
return null;
}
@Override
public List<Versioned<byte[]>> get(ByteArray key, byte[] transforms) throws VoldemortException {
RebalancePartitionsInfo stealInfo = redirectingKey(key);
/**
* If I am rebalancing for this key, try to do remote get(), put it
* locally first to get the correct version ignoring any
* {@link ObsoleteVersionException}
*/
// TODO this is correct per se. But, had a heavy performance hit for
// cross zone moves. Check locally first, if you cannot find, then go
// remote and store it locally.
if(stealInfo != null) {
proxyGetAndLocalPut(key, stealInfo.getDonorId(), transforms);
}
return getInnerStore().get(key, transforms);
}
@Override
public List<Version> getVersions(ByteArray key) {
RebalancePartitionsInfo stealInfo = redirectingKey(key);
/**
* If I am rebalancing for this key, try to do remote get(), put it
* locally first to get the correct version ignoring any
* {@link ObsoleteVersionException}.
*/
// TODO this is correct per se. But, had a heavy performance hit for
// cross zone moves. Check locally first, if you cannot find, then go
// remote and store it locally.
if(stealInfo != null) {
proxyGetAndLocalPut(key, stealInfo.getDonorId(), null);
}
return getInnerStore().getVersions(key);
}
@Override
public Map<ByteArray, List<Versioned<byte[]>>> getAll(Iterable<ByteArray> keys,
Map<ByteArray, byte[]> transforms)
throws VoldemortException {
Map<ByteArray, RebalancePartitionsInfo> rebalancePartitionsInfoPerKey = Maps.newHashMapWithExpectedSize(Iterables.size(keys));
for(ByteArray key: keys) {
RebalancePartitionsInfo info = redirectingKey(key);
if(info != null) {
rebalancePartitionsInfoPerKey.put(key, info);
}
}
// TODO this is correct per se. But, had a heavy performance hit for
// cross zone moves. Check locally first, if you cannot find, then go
// remote and store it locally.
if(!rebalancePartitionsInfoPerKey.isEmpty()) {
proxyGetAllAndLocalPut(rebalancePartitionsInfoPerKey, transforms);
}
return getInnerStore().getAll(keys, transforms);
}
@Override
public void put(ByteArray key, Versioned<byte[]> value, byte[] transforms)
throws VoldemortException {
RebalancePartitionsInfo stealInfo = redirectingKey(key);
/**
* If I am rebalancing for this key, try to do remote get() , put it
* locally first to get the correct version ignoring any
* {@link ObsoleteVersionException}
*/
if(stealInfo != null)
proxyGetAndLocalPut(key, stealInfo.getDonorId(), transforms);
// Just sychronous replication,if remote fails, I fail.
if(stealInfo != null)
proxyPut(key, value, transforms, stealInfo.getDonorId());
// TODO if I fail though for some reason, the aborting rebalance can
// surface phantom writes
getInnerStore().put(key, value, transforms);
}
/**
* TODO : Handle delete correctly.
* <p>
* The options are:
* <ol>
* <li>
* Delete locally and on remote node as well. The issue is cursor is open in
* READ_UNCOMMITED mode while rebalancing and can push the value back.</li>
* <li>
* Keep the operation in separate slop store and apply all deletes after
* rebalancing.</li>
* <li>
* Do not worry about deletes for now, Voldemort in general has an issue
* that if node goes down during a delete, we will still keep the old
* version.</li>
* </ol>
*/
@Override
public boolean delete(ByteArray key, Version version) throws VoldemortException {
StoreUtils.assertValidKey(key);
return getInnerStore().delete(key, version);
}
/**
* Performs a back-door proxy get to
* {@link voldemort.client.rebalance.RebalancePartitionsInfo#getDonorId()
* getDonorId}
*
* @param key Key
* @param donorNodeId donor node id
* @throws ProxyUnreachableException if donor node can't be reached
*/
private List<Versioned<byte[]>> proxyGet(ByteArray key, int donorNodeId, byte[] transform) {
Node donorNode = metadata.getCluster().getNodeById(donorNodeId);
checkNodeAvailable(donorNode);
long startNs = System.nanoTime();
try {
Store<ByteArray, byte[], byte[]> redirectingStore = getRedirectingSocketStore(getName(),
donorNodeId);
List<Versioned<byte[]>> values = redirectingStore.get(key, transform);
recordSuccess(donorNode, startNs);
return values;
} catch(UnreachableStoreException e) {
recordException(donorNode, startNs, e);
throw new ProxyUnreachableException("Failed to reach proxy node " + donorNode, e);
}
}
/**
* Replay the put to a remote proxy node so we will have the data available
* at the proxy host, in case the rebalancing fails
*
* @param key
* @param value
* @param transforms
* @param donorNodeId
* @throws ProxyUnreachableException if donor node can't be reached
*/
private void proxyPut(ByteArray key, Versioned<byte[]> value, byte[] transforms, int donorNodeId) {
Node donorNode = metadata.getCluster().getNodeById(donorNodeId);
checkNodeAvailable(donorNode);
long startNs = System.nanoTime();
try {
Store<ByteArray, byte[], byte[]> redirectingStore = getRedirectingSocketStore(getName(),
donorNodeId);
redirectingStore.put(key, value, transforms);
recordSuccess(donorNode, startNs);
} catch(UnreachableStoreException e) {
recordException(donorNode, startNs, e);
throw new ProxyUnreachableException("Failed to reach proxy node " + donorNode, e);
}
}
private void checkNodeAvailable(Node donorNode) {
if(!failureDetector.isAvailable(donorNode))
throw new ProxyUnreachableException("Failed to reach proxy node " + donorNode
+ " is marked down by failure detector.");
}
/**
* Performs a back-door proxy get to
* {@link voldemort.client.rebalance.RebalancePartitionsInfo#getDonorId()
* getDonorId}
*
* @param rebalancePartitionsInfoPerKey Map of keys to corresponding
* partition info
* @param transforms Map of keys to their corresponding transforms
* @throws ProxyUnreachableException if donor node can't be reached
*/
private Map<ByteArray, List<Versioned<byte[]>>> proxyGetAll(Map<ByteArray, RebalancePartitionsInfo> rebalancePartitionsInfoPerKey,
Map<ByteArray, byte[]> transforms)
throws VoldemortException {
Multimap<Integer, ByteArray> donorNodeToKeys = HashMultimap.create();
int numKeys = 0;
// Transform the map of key to plan to a map of donor node id to keys
for(Map.Entry<ByteArray, RebalancePartitionsInfo> entry: rebalancePartitionsInfoPerKey.entrySet()) {
numKeys++;
donorNodeToKeys.put(entry.getValue().getDonorId(), entry.getKey());
}
Map<ByteArray, List<Versioned<byte[]>>> gatherMap = Maps.newHashMapWithExpectedSize(numKeys);
for(int donorNodeId: donorNodeToKeys.keySet()) {
Node donorNode = metadata.getCluster().getNodeById(donorNodeId);
checkNodeAvailable(donorNode);
long startNs = System.nanoTime();
try {
Map<ByteArray, List<Versioned<byte[]>>> resultsForNode = getRedirectingSocketStore(getName(),
donorNodeId).getAll(donorNodeToKeys.get(donorNodeId),
transforms);
recordSuccess(donorNode, startNs);
for(Map.Entry<ByteArray, List<Versioned<byte[]>>> entry: resultsForNode.entrySet()) {
gatherMap.put(entry.getKey(), entry.getValue());
}
} catch(UnreachableStoreException e) {
recordException(donorNode, startNs, e);
throw new ProxyUnreachableException("Failed to reach proxy node " + donorNode, e);
}
}
return gatherMap;
}
/**
* In <code>REBALANCING_MASTER_SERVER</code> state put should be committed
* on stealer node. To follow Voldemort version guarantees, stealer node
* should query donor node and put that value (proxyValue) before committing
* the value from client.
* <p>
* Stealer node should ignore {@link ObsoleteVersionException} while
* commiting proxyValue to local storage.
*
* @param key Key
* @param donorId donorId
* @return Returns the proxy value
* @throws VoldemortException if {@link #proxyGet(ByteArray, int)} fails
*/
private List<Versioned<byte[]>> proxyGetAndLocalPut(ByteArray key,
int donorId,
byte[] transforms)
throws VoldemortException {
List<Versioned<byte[]>> proxyValues = proxyGet(key, donorId, transforms);
for(Versioned<byte[]> proxyValue: proxyValues) {
try {
getInnerStore().put(key, proxyValue, null);
} catch(ObsoleteVersionException e) {
// ignore these
}
}
return proxyValues;
}
/**
* Similar to {@link #proxyGetAndLocalPut(ByteArray, int)} but meant for
* {@link #getAll(Iterable)}
*
* @param rebalancePartitionsInfoPerKey Map of keys which are being routed
* to their corresponding plan
* @param transforms Map of key to their corresponding transforms
* @return Returns a map of key to its corresponding list of values
* @throws VoldemortException if {@link #proxyGetAll(List, List)} fails
*/
private Map<ByteArray, List<Versioned<byte[]>>> proxyGetAllAndLocalPut(Map<ByteArray, RebalancePartitionsInfo> rebalancePartitionsInfoPerKey,
Map<ByteArray, byte[]> transforms)
throws VoldemortException {
Map<ByteArray, List<Versioned<byte[]>>> proxyKeyValues = proxyGetAll(rebalancePartitionsInfoPerKey,
transforms);
for(Map.Entry<ByteArray, List<Versioned<byte[]>>> keyValuePair: proxyKeyValues.entrySet()) {
for(Versioned<byte[]> proxyValue: keyValuePair.getValue()) {
try {
getInnerStore().put(keyValuePair.getKey(), proxyValue, null);
} catch(ObsoleteVersionException e) {
// ignore these
}
}
}
return proxyKeyValues;
}
/**
* Get the {@link voldemort.store.socket.SocketStore} to redirect to for the
* donor, creating one if needed.
*
* @param storeName Name of the store
* @param donorNodeId Donor node id
* @return <code>SocketStore</code> object for <code>storeName</code> and
* <code>donorNodeId</code>
*/
private Store<ByteArray, byte[], byte[]> getRedirectingSocketStore(String storeName,
int donorNodeId) {
if(!storeRepository.hasRedirectingSocketStore(storeName, donorNodeId)) {
synchronized(storeRepository) {
if(!storeRepository.hasRedirectingSocketStore(storeName, donorNodeId)) {
Node donorNode = getNodeIfPresent(donorNodeId);
logger.info("Creating new redirecting store for donor node "
+ donorNode.getId() + " and store " + storeName);
storeRepository.addRedirectingSocketStore(donorNode.getId(),
storeFactory.create(storeName,
donorNode.getHost(),
donorNode.getSocketPort(),
RequestFormatType.PROTOCOL_BUFFERS,
RequestRoutingType.IGNORE_CHECKS));
}
}
}
return storeRepository.getRedirectingSocketStore(storeName, donorNodeId);
}
private Node getNodeIfPresent(int donorId) {
try {
return metadata.getCluster().getNodeById(donorId);
} catch(Exception e) {
throw new VoldemortException("Failed to get donorNode " + donorId
+ " from current cluster " + metadata.getCluster()
+ " at node " + metadata.getNodeId(), e);
}
}
private void recordException(Node node, long startNs, UnreachableStoreException e) {
failureDetector.recordException(node, (System.nanoTime() - startNs) / Time.NS_PER_MS, e);
}
private void recordSuccess(Node node, long startNs) {
failureDetector.recordSuccess(node, (System.nanoTime() - startNs) / Time.NS_PER_MS);
}
}