/
LockResolver.java
345 lines (289 loc) · 12.8 KB
/
LockResolver.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
/*
* Licensed to the Apache Software Foundation (ASF) under one or more contributor license
* agreements. See the NOTICE file distributed with this work for additional information regarding
* copyright ownership. The ASF licenses this file to you 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 org.apache.fluo.core.impl;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import org.apache.accumulo.core.client.AccumuloException;
import org.apache.accumulo.core.client.AccumuloSecurityException;
import org.apache.accumulo.core.client.BatchScanner;
import org.apache.accumulo.core.client.ConditionalWriter;
import org.apache.accumulo.core.client.ConditionalWriter.Status;
import org.apache.accumulo.core.client.IteratorSetting;
import org.apache.accumulo.core.data.ByteSequence;
import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.Mutation;
import org.apache.accumulo.core.data.Range;
import org.apache.accumulo.core.data.Value;
import org.apache.fluo.accumulo.iterators.OpenReadLockIterator;
import org.apache.fluo.accumulo.iterators.PrewriteIterator;
import org.apache.fluo.accumulo.util.ColumnConstants;
import org.apache.fluo.accumulo.util.ColumnType;
import org.apache.fluo.accumulo.util.ReadLockUtil;
import org.apache.fluo.accumulo.values.DelLockValue;
import org.apache.fluo.accumulo.values.DelReadLockValue;
import org.apache.fluo.accumulo.values.LockValue;
import org.apache.fluo.accumulo.values.ReadLockValue;
import org.apache.fluo.api.data.Bytes;
import org.apache.fluo.api.data.Column;
import org.apache.fluo.api.data.RowColumn;
import org.apache.fluo.core.util.ByteUtil;
import org.apache.fluo.core.util.ColumnUtil;
import org.apache.fluo.core.util.ConditionalFlutation;
import org.apache.fluo.core.util.FluoCondition;
import org.apache.fluo.core.util.SpanUtil;
import static org.apache.fluo.api.observer.Observer.NotificationType.STRONG;
/**
* This is utility code for either rolling forward or back failed transactions. A transaction is
* deemed to have failed if the reading transaction waited too long or the transactor id does not
* exist in zookeeper.
*/
public class LockResolver {
private static Map<PrimaryRowColumn, List<LockInfo>> groupLocksByPrimary(List<LockInfo> locks) {
Map<PrimaryRowColumn, List<LockInfo>> groupedLocks = new HashMap<>();
Map<PrimaryRowColumn, Long> transactorIds = new HashMap<>();
for (LockInfo lockInfo : locks) {
PrimaryRowColumn prc = new PrimaryRowColumn(lockInfo.prow, lockInfo.pcol, lockInfo.lockTs);
List<LockInfo> lockList = groupedLocks.computeIfAbsent(prc, k -> new ArrayList<>());
Long trid = transactorIds.get(prc);
if (trid == null) {
transactorIds.put(prc, lockInfo.transactorId);
} else if (!trid.equals(lockInfo.transactorId)) {
// sanity check.. its assumed that all locks w/ the same PrimaryRowColumn should have the
// same transactor id as well
throw new IllegalStateException("transactor ids not equals " + prc + " "
+ lockInfo.entry.getKey() + " " + trid + " " + lockInfo.transactorId);
}
lockList.add(lockInfo);
}
return groupedLocks;
}
private static class LockInfo {
final Bytes prow;
final Column pcol;
final Long transactorId;
final long lockTs;
final boolean isReadLock;
final Entry<Key, Value> entry;
public LockInfo(Entry<Key, Value> kve) {
long rawTs = kve.getKey().getTimestamp();
this.entry = kve;
if (ColumnType.from(rawTs) == ColumnType.RLOCK) {
this.lockTs = ReadLockUtil.decodeTs(rawTs);
ReadLockValue rlv = new ReadLockValue(kve.getValue().get());
this.prow = rlv.getPrimaryRow();
this.pcol = rlv.getPrimaryColumn();
this.transactorId = rlv.getTransactor();
this.isReadLock = true;
} else {
this.lockTs = kve.getKey().getTimestamp() & ColumnConstants.TIMESTAMP_MASK;
LockValue lv = new LockValue(kve.getValue().get());
this.prow = lv.getPrimaryRow();
this.pcol = lv.getPrimaryColumn();
this.transactorId = lv.getTransactor();
this.isReadLock = false;
}
}
}
/**
* Attempts to roll forward or roll back a set of locks encountered by a transaction reading data.
*
* @param env environment
* @param startTs The logical start time from the oracle of the transaction that encountered the
* lock
* @param stats Stats object for the transaction that encountered the lock
* @param locksKVs List of locks
* @param startTime The wall time that the transaction that encountered the lock first saw the
* lock
* @return true if all locks passed in were resolved (rolled forward or back)
*/
static boolean resolveLocks(Environment env, long startTs, TxStats stats,
List<Entry<Key, Value>> locksKVs, long startTime) {
// check if transactor is still alive
int numResolved = 0;
Map<ByteSequence, Mutation> mutations = new HashMap<>();
boolean timedOut = false;
TransactorCache transactorCache = env.getSharedResources().getTransactorCache();
List<LockInfo> locks = new ArrayList<>();
locksKVs.forEach(e -> locks.add(new LockInfo(e)));
List<LockInfo> locksToRecover;
if (System.currentTimeMillis() - startTime > env.getConfiguration()
.getTransactionRollbackTime()) {
locksToRecover = locks;
stats.incrementTimedOutLocks(locksToRecover.size());
timedOut = true;
} else {
locksToRecover = new ArrayList<>(locks.size());
for (LockInfo lockInfo : locks) {
if (transactorCache.checkTimedout(lockInfo.transactorId, lockInfo.lockTs)) {
locksToRecover.add(lockInfo);
stats.incrementTimedOutLocks();
} else if (!transactorCache.checkExists(lockInfo.transactorId)) {
locksToRecover.add(lockInfo);
stats.incrementDeadLocks();
}
}
}
Map<PrimaryRowColumn, List<LockInfo>> groupedLocks = groupLocksByPrimary(locksToRecover);
if (timedOut) {
Set<Entry<PrimaryRowColumn, List<LockInfo>>> es = groupedLocks.entrySet();
for (Entry<PrimaryRowColumn, List<LockInfo>> entry : es) {
long lockTs = entry.getKey().startTs;
Long transactorId = entry.getValue().get(0).transactorId;
transactorCache.addTimedoutTransactor(transactorId, lockTs, startTime);
}
}
TxInfoCache txiCache = env.getSharedResources().getTxInfoCache();
Set<Entry<PrimaryRowColumn, List<LockInfo>>> es = groupedLocks.entrySet();
for (Entry<PrimaryRowColumn, List<LockInfo>> group : es) {
TxInfo txInfo = txiCache.getTransactionInfo(group.getKey());
switch (txInfo.status) {
case COMMITTED:
commitColumns(env, group.getKey(), group.getValue(), txInfo.commitTs, mutations);
numResolved += group.getValue().size();
break;
case LOCKED:
if (rollbackPrimary(env, startTs, group.getKey(), txInfo.lockValue)) {
rollback(env, startTs, group.getKey(), group.getValue(), mutations);
numResolved += group.getValue().size();
}
break;
case ROLLED_BACK:
// TODO ensure this if ok if there concurrent rollback
rollback(env, startTs, group.getKey(), group.getValue(), mutations);
numResolved += group.getValue().size();
break;
case UNKNOWN:
default:
throw new IllegalStateException(
"can not abort : " + group.getKey() + " (" + txInfo.status + ")");
}
}
if (mutations.size() > 0) {
env.getSharedResources().getBatchWriter().writeMutations(new ArrayList<>(mutations.values()));
}
return numResolved == locks.size();
}
private static void rollback(Environment env, long startTs, PrimaryRowColumn prc,
List<LockInfo> value, Map<ByteSequence, Mutation> mutations) {
for (LockInfo lockInfo : value) {
if (isPrimary(prc, lockInfo.entry.getKey())) {
continue;
}
Mutation mut = getMutation(lockInfo.entry.getKey().getRowData(), mutations);
Key k = lockInfo.entry.getKey();
if (lockInfo.isReadLock) {
mut.put(k.getColumnFamilyData().toArray(), k.getColumnQualifierData().toArray(),
k.getColumnVisibilityParsed(),
ColumnType.RLOCK.encode(ReadLockUtil.encodeTs(lockInfo.lockTs, true)),
DelReadLockValue.encodeRollback());
} else {
mut.put(k.getColumnFamilyData().toArray(), k.getColumnQualifierData().toArray(),
k.getColumnVisibilityParsed(), ColumnType.DEL_LOCK.encode(lockInfo.lockTs),
DelLockValue.encodeRollback(false, true));
}
}
}
private static boolean rollbackPrimary(Environment env, long startTs, PrimaryRowColumn prc,
byte[] lockValue) {
// TODO review use of PrewriteIter here
IteratorSetting iterConf = new IteratorSetting(10, PrewriteIterator.class);
PrewriteIterator.setSnaptime(iterConf, startTs);
ConditionalFlutation delLockMutation = new ConditionalFlutation(env, prc.prow,
new FluoCondition(env, prc.pcol).setIterators(iterConf).setValue(lockValue));
delLockMutation.put(prc.pcol, ColumnType.DEL_LOCK.encode(prc.startTs),
DelLockValue.encodeRollback(true, true));
ConditionalWriter cw = null;
cw = env.getSharedResources().getConditionalWriter();
// TODO handle other conditional writer cases
try {
return cw.write(delLockMutation).getStatus() == Status.ACCEPTED;
} catch (AccumuloException e) {
throw new RuntimeException(e);
} catch (AccumuloSecurityException e) {
throw new RuntimeException(e);
}
}
private static void commitColumns(Environment env, PrimaryRowColumn prc, List<LockInfo> value,
long commitTs, Map<ByteSequence, Mutation> mutations) {
for (LockInfo lockInfo : value) {
if (isPrimary(prc, lockInfo.entry.getKey())) {
continue;
}
long lockTs = lockInfo.lockTs;
// TODO may be that a stronger sanity check that could be done here
if (commitTs < lockTs) {
throw new IllegalStateException(
"bad commitTs : " + lockInfo.entry.getKey() + " (" + commitTs + "<" + lockTs + ")");
}
Mutation mut = getMutation(lockInfo.entry.getKey().getRowData(), mutations);
Column col = ColumnUtil.convert(lockInfo.entry.getKey());
if (lockInfo.isReadLock) {
ColumnUtil.commitColumn(env, false, false, col, false, false, true, lockTs, commitTs,
env.getConfiguredObservers().getObservedColumns(STRONG), mut);
} else {
LockValue lv = new LockValue(lockInfo.entry.getValue().get());
ColumnUtil.commitColumn(env, lv.isTrigger(), false, col, lv.isWrite(), lv.isDelete(), false,
lockTs, commitTs, env.getConfiguredObservers().getObservedColumns(STRONG), mut);
}
}
}
private static Mutation getMutation(ByteSequence row, Map<ByteSequence, Mutation> mutations) {
Mutation mut = mutations.get(row);
if (mut == null) {
mut = new Mutation(row.toArray());
mutations.put(row, mut);
}
return mut;
}
private static boolean isPrimary(PrimaryRowColumn prc, Key k) {
return prc.prow.equals(ByteUtil.toBytes(k.getRowData()))
&& prc.pcol.equals(SpanUtil.toRowColumn(k).getColumn());
}
static List<Entry<Key, Value>> getOpenReadLocks(Environment env,
Map<Bytes, Set<Column>> rowColsToCheck) throws Exception {
List<Range> ranges = new ArrayList<>();
for (Entry<Bytes, Set<Column>> e1 : rowColsToCheck.entrySet()) {
for (Column col : e1.getValue()) {
Key start = SpanUtil.toKey(new RowColumn(e1.getKey(), col));
Key end = new Key(start);
end.setTimestamp(ColumnType.LOCK.first());
ranges.add(new Range(start, true, end, false));
}
}
BatchScanner bscanner = null;
try {
bscanner =
env.getAccumuloClient().createBatchScanner(env.getTable(), env.getAuthorizations(), 1);
bscanner.setRanges(ranges);
IteratorSetting iterCfg = new IteratorSetting(10, OpenReadLockIterator.class);
bscanner.addScanIterator(iterCfg);
List<Entry<Key, Value>> ret = new ArrayList<>();
for (Entry<Key, Value> entry : bscanner) {
if (ColumnType.from(entry.getKey()) == ColumnType.RLOCK) {
ret.add(entry);
}
}
return ret;
} finally {
if (bscanner != null) {
bscanner.close();
}
}
}
}