forked from voldemort/voldemort
/
ConsistencyFixWorker.java
331 lines (294 loc) · 13.5 KB
/
ConsistencyFixWorker.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
/*
* Copyright 2013 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.utils;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.BlockingQueue;
import org.apache.log4j.Logger;
import voldemort.VoldemortException;
import voldemort.client.protocol.admin.QueryKeyResult;
import voldemort.store.routed.NodeValue;
import voldemort.store.routed.ReadRepairer;
import voldemort.utils.ConsistencyFix.BadKeyResult;
import voldemort.utils.ConsistencyFix.Status;
import voldemort.versioning.ObsoleteVersionException;
import voldemort.versioning.VectorClock;
import voldemort.versioning.Versioned;
import com.google.common.collect.Lists;
class ConsistencyFixWorker implements Runnable {
private static final Logger logger = Logger.getLogger(ConsistencyFixWorker.class);
private static final int fakeNodeID = Integer.MIN_VALUE;
private final String keyInHexFormat;
private final ConsistencyFix consistencyFix;
private final BlockingQueue<BadKeyResult> badKeyQOut;
private final QueryKeyResult orphanedValues;
/**
* Normal use case constructor.
*
* @param keyInHexFormat
* @param consistencyFix
* @param badKeyQOut
*/
ConsistencyFixWorker(String keyInHexFormat,
ConsistencyFix consistencyFix,
BlockingQueue<BadKeyResult> badKeyQOut) {
this(keyInHexFormat, consistencyFix, badKeyQOut, null);
}
/**
* Constructor for "orphaned values" use case. I.e., there are values for
* the specific key that exist somewhere and may need to be written to the
* nodes which actually host the key.
*
* @param keyInHexFormat
* @param consistencyFix
* @param badKeyQOut
* @param orphanedValues Set to null if no orphaned values to be included.
*/
ConsistencyFixWorker(String keyInHexFormat,
ConsistencyFix consistencyFix,
BlockingQueue<BadKeyResult> badKeyQOut,
QueryKeyResult orphanedValues) {
this.keyInHexFormat = keyInHexFormat;
this.consistencyFix = consistencyFix;
this.badKeyQOut = badKeyQOut;
this.orphanedValues = orphanedValues;
}
private String myName() {
return Thread.currentThread().getName() + "-" + ConsistencyFixWorker.class.getName();
}
@Override
public void run() {
logger.trace("About to process key " + keyInHexFormat + " (" + myName() + ")");
Status status = doConsistencyFix(keyInHexFormat);
logger.trace("Finished processing key " + keyInHexFormat + " (" + myName() + ")");
consistencyFix.getStats().incrementCount();
if(status != Status.SUCCESS) {
try {
badKeyQOut.put(consistencyFix.new BadKeyResult(keyInHexFormat, status));
} catch(InterruptedException ie) {
logger.warn("Worker thread " + myName() + " interrupted.");
}
consistencyFix.getStats().incrementFailures();
}
}
public Status doConsistencyFix(String keyInHexFormat) {
// Initialization.
byte[] keyInBytes;
List<Integer> nodeIdList = null;
int masterPartitionId = -1;
try {
keyInBytes = ByteUtils.fromHexString(keyInHexFormat);
masterPartitionId = consistencyFix.getStoreInstance().getMasterPartitionId(keyInBytes);
nodeIdList = consistencyFix.getStoreInstance()
.getReplicationNodeList(masterPartitionId);
} catch(Exception exception) {
logger.info("Aborting fixKey due to bad init.");
if(logger.isDebugEnabled()) {
exception.printStackTrace();
}
return Status.BAD_INIT;
}
ByteArray keyAsByteArray = new ByteArray(keyInBytes);
// Do the reads
Map<Integer, QueryKeyResult> nodeIdToKeyValues = doReads(nodeIdList,
keyInBytes,
keyInHexFormat);
// Process read replies (i.e., nodeIdToKeyValues)
ProcessReadRepliesResult result = processReadReplies(nodeIdList,
keyAsByteArray,
keyInHexFormat,
nodeIdToKeyValues);
if(result.status != Status.SUCCESS) {
return result.status;
}
// Resolve conflicts indicated in nodeValues
List<NodeValue<ByteArray, byte[]>> toReadRepair = resolveReadConflicts(result.nodeValues);
// Do the repairs
Status status = doRepairPut(toReadRepair);
// return status of last operation (success or otherwise)
return status;
}
/**
*
* @param nodeIdList
* @param keyInBytes
* @param keyInHexFormat
* @return
*/
private Map<Integer, QueryKeyResult> doReads(final List<Integer> nodeIdList,
final byte[] keyInBytes,
final String keyInHexFormat) {
Map<Integer, QueryKeyResult> nodeIdToKeyValues = new HashMap<Integer, QueryKeyResult>();
ByteArray key = new ByteArray(keyInBytes);
for(int nodeId: nodeIdList) {
List<Versioned<byte[]>> values = null;
try {
values = consistencyFix.getAdminClient().storeOps.getNodeKey(consistencyFix.getStoreName(),
nodeId,
key);
nodeIdToKeyValues.put(nodeId, new QueryKeyResult(key, values));
} catch(VoldemortException ve) {
nodeIdToKeyValues.put(nodeId, new QueryKeyResult(key, ve));
}
}
return nodeIdToKeyValues;
}
/**
* Result of an invocation of processReadReplies
*/
private class ProcessReadRepliesResult {
public final Status status;
public final List<NodeValue<ByteArray, byte[]>> nodeValues;
/**
* Constructor for error status
*/
ProcessReadRepliesResult(Status status) {
this.status = status;
this.nodeValues = null;
}
/**
* Constructor for success
*/
ProcessReadRepliesResult(List<NodeValue<ByteArray, byte[]>> nodeValues) {
this.status = Status.SUCCESS;
this.nodeValues = nodeValues;
}
}
/**
*
* @param nodeIdList
* @param keyAsByteArray
* @param keyInHexFormat
* @param nodeIdToKeyValues
* @param nodeValues Effectively the output of this method. Must pass in a
* non-null object to be populated by this method.
* @return
*/
private ProcessReadRepliesResult processReadReplies(final List<Integer> nodeIdList,
final ByteArray keyAsByteArray,
final String keyInHexFormat,
final Map<Integer, QueryKeyResult> nodeIdToKeyValues) {
List<NodeValue<ByteArray, byte[]>> nodeValues = new ArrayList<NodeValue<ByteArray, byte[]>>();
boolean exceptionsEncountered = false;
for(int nodeId: nodeIdList) {
QueryKeyResult keyValue;
if(nodeIdToKeyValues.containsKey(nodeId)) {
keyValue = nodeIdToKeyValues.get(nodeId);
if(keyValue.hasException()) {
logger.debug("Exception encountered while fetching key " + keyInHexFormat
+ " from node with nodeId " + nodeId + " : "
+ keyValue.getException().getMessage());
exceptionsEncountered = true;
} else {
if(keyValue.getValues().isEmpty()) {
Versioned<byte[]> versioned = new Versioned<byte[]>(null);
nodeValues.add(new NodeValue<ByteArray, byte[]>(nodeId,
keyValue.getKey(),
versioned));
} else {
for(Versioned<byte[]> value: keyValue.getValues()) {
nodeValues.add(new NodeValue<ByteArray, byte[]>(nodeId,
keyValue.getKey(),
value));
}
}
}
} else {
logger.debug("No key-value returned from node with id:" + nodeId);
Versioned<byte[]> versioned = new Versioned<byte[]>(null);
nodeValues.add(new NodeValue<ByteArray, byte[]>(nodeId, keyAsByteArray, versioned));
}
}
if(exceptionsEncountered) {
logger.info("Aborting fixKey because exceptions were encountered when fetching key-values.");
return new ProcessReadRepliesResult(Status.FETCH_EXCEPTION);
}
return new ProcessReadRepliesResult(nodeValues);
}
/**
* Decide on the specific key-value to write everywhere.
*
* @param nodeValues
* @return The subset of entries from nodeValues that need to be repaired.
*/
private List<NodeValue<ByteArray, byte[]>> resolveReadConflicts(final List<NodeValue<ByteArray, byte[]>> nodeValues) {
// If orphaned values exist, add them to fake nodes to be processed by
// "getRepairs"
int currentFakeNodeId = fakeNodeID;
if(this.orphanedValues != null) {
for(Versioned<byte[]> value: this.orphanedValues.getValues()) {
nodeValues.add(new NodeValue<ByteArray, byte[]>(currentFakeNodeId,
this.orphanedValues.getKey(),
value));
currentFakeNodeId++;
}
}
// Some cut-paste-and-modify (CPAM) coding from
// store/routed/action/AbstractReadRepair.java and
// store/routed/ThreadPoolRoutedStore.java
ReadRepairer<ByteArray, byte[]> readRepairer = new ReadRepairer<ByteArray, byte[]>();
List<NodeValue<ByteArray, byte[]>> toReadRepair = Lists.newArrayList();
for(NodeValue<ByteArray, byte[]> v: readRepairer.getRepairs(nodeValues)) {
if(v.getNodeId() > currentFakeNodeId) {
// Only copy repairs intended for real nodes.
Versioned<byte[]> versioned = Versioned.value(v.getVersioned().getValue(),
((VectorClock) v.getVersion()).clone());
toReadRepair.add(new NodeValue<ByteArray, byte[]>(v.getNodeId(),
v.getKey(),
versioned));
}
}
if(logger.isDebugEnabled()) {
for(NodeValue<ByteArray, byte[]> nodeKeyValue: toReadRepair) {
logger.debug("\tRepair key " + nodeKeyValue.getKey() + "on node with id "
+ nodeKeyValue.getNodeId() + " for version "
+ nodeKeyValue.getVersion());
}
}
return toReadRepair;
}
/**
*
* @param toReadRepair Effectively the output of this method. Must pass in a
* non-null object to be populated by this method.
* @return
*/
public Status doRepairPut(final List<NodeValue<ByteArray, byte[]>> toReadRepair) {
boolean allRepairsSuccessful = true;
for(NodeValue<ByteArray, byte[]> nodeKeyValue: toReadRepair) {
try {
consistencyFix.maybePutThrottle(nodeKeyValue.getNodeId());
consistencyFix.getAdminClient().storeOps.putNodeKeyValue(consistencyFix.getStoreName(),
nodeKeyValue);
} catch(ObsoleteVersionException ove) {
// NOOP. Treat OVE as success.
} catch(VoldemortException ve) {
allRepairsSuccessful = false;
logger.debug("Repair of key " + nodeKeyValue.getKey() + "on node with id "
+ nodeKeyValue.getNodeId() + " for version "
+ nodeKeyValue.getVersion() + " failed because of exception : "
+ ve.getMessage());
}
}
if(!allRepairsSuccessful) {
logger.info("Aborting fixKey because exceptions were encountered when reparing key-values.");
return Status.REPAIR_EXCEPTION;
}
return Status.SUCCESS;
}
}