/
TabletServerLogger.java
563 lines (504 loc) · 19.9 KB
/
TabletServerLogger.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
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
/*
* 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.accumulo.tserver.log;
import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.accumulo.core.client.Durability;
import org.apache.accumulo.core.data.Mutation;
import org.apache.accumulo.core.data.impl.KeyExtent;
import org.apache.accumulo.core.protobuf.ProtobufUtil;
import org.apache.accumulo.core.replication.ReplicationConfigurationUtil;
import org.apache.accumulo.core.util.SimpleThreadPool;
import org.apache.accumulo.fate.util.LoggingRunnable;
import org.apache.accumulo.fate.zookeeper.Retry;
import org.apache.accumulo.fate.zookeeper.RetryFactory;
import org.apache.accumulo.server.conf.TableConfiguration;
import org.apache.accumulo.server.fs.VolumeManager;
import org.apache.accumulo.server.replication.StatusUtil;
import org.apache.accumulo.server.replication.proto.Replication.Status;
import org.apache.accumulo.server.util.Halt;
import org.apache.accumulo.server.util.ReplicationTableUtil;
import org.apache.accumulo.tserver.Mutations;
import org.apache.accumulo.tserver.TabletMutations;
import org.apache.accumulo.tserver.TabletServer;
import org.apache.accumulo.tserver.log.DfsLogger.LoggerOperation;
import org.apache.accumulo.tserver.log.DfsLogger.ServerResources;
import org.apache.accumulo.tserver.tablet.CommitSession;
import org.apache.hadoop.fs.Path;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Central logging facility for the TServerInfo.
*
* Forwards in-memory updates to remote logs, carefully writing the same data to every log, while maintaining the maximum thread parallelism for greater
* performance. As new logs are used and minor compactions are performed, the metadata table is kept up-to-date.
*
*/
public class TabletServerLogger {
private static final Logger log = LoggerFactory.getLogger(TabletServerLogger.class);
private final AtomicLong logSizeEstimate = new AtomicLong();
private final long maxSize;
private final long maxAge;
private final TabletServer tserver;
// The current logger
private DfsLogger currentLog = null;
private final SynchronousQueue<Object> nextLog = new SynchronousQueue<>();
private ThreadPoolExecutor nextLogMaker;
// The current generation of logs.
// Because multiple threads can be using a log at one time, a log
// failure is likely to affect multiple threads, who will all attempt to
// create a new log. This will cause many unnecessary updates to the
// metadata table.
// We'll use this generational counter to determine if another thread has
// already fetched a new log.
private final AtomicInteger logId = new AtomicInteger();
// Use a ReadWriteLock to allow multiple threads to use the log set, but obtain a write lock to change them
private final ReentrantReadWriteLock logIdLock = new ReentrantReadWriteLock();
private final AtomicInteger seqGen = new AtomicInteger();
private final AtomicLong syncCounter;
private final AtomicLong flushCounter;
private long createTime = 0;
private final RetryFactory retryFactory;
private Retry retry = null;
static private abstract class TestCallWithWriteLock {
abstract boolean test();
abstract void withWriteLock() throws IOException;
}
/**
* Pattern taken from the documentation for ReentrantReadWriteLock
*
* @param rwlock
* lock to use
* @param code
* a test/work pair
*/
private static void testLockAndRun(final ReadWriteLock rwlock, TestCallWithWriteLock code) throws IOException {
// Get a read lock
rwlock.readLock().lock();
try {
// does some condition exist that needs the write lock?
if (code.test()) {
// Yes, let go of the readlock
rwlock.readLock().unlock();
// Grab the write lock
rwlock.writeLock().lock();
try {
// double-check the condition, since we let go of the lock
if (code.test()) {
// perform the work with with write lock held
code.withWriteLock();
}
} finally {
// regain the readlock
rwlock.readLock().lock();
// unlock the write lock
rwlock.writeLock().unlock();
}
}
} finally {
// always let go of the lock
rwlock.readLock().unlock();
}
}
public TabletServerLogger(TabletServer tserver, long maxSize, AtomicLong syncCounter, AtomicLong flushCounter, RetryFactory retryFactory, long maxAge) {
this.tserver = tserver;
this.maxSize = maxSize;
this.syncCounter = syncCounter;
this.flushCounter = flushCounter;
this.retryFactory = retryFactory;
this.retry = null;
this.maxAge = maxAge;
}
private DfsLogger initializeLoggers(final AtomicInteger logIdOut) throws IOException {
final AtomicReference<DfsLogger> result = new AtomicReference<>();
testLockAndRun(logIdLock, new TestCallWithWriteLock() {
@Override
boolean test() {
result.set(currentLog);
if (currentLog != null)
logIdOut.set(logId.get());
return currentLog == null;
}
@Override
void withWriteLock() throws IOException {
try {
createLogger();
result.set(currentLog);
if (currentLog != null)
logIdOut.set(logId.get());
else
logIdOut.set(-1);
} catch (IOException e) {
log.error("Unable to create loggers", e);
}
}
});
return result.get();
}
/**
* Get the current WAL file
*
* @return The name of the current log, or null if there is no current log.
*/
public String getLogFile() {
logIdLock.readLock().lock();
try {
if (null == currentLog) {
return null;
}
return currentLog.getFileName();
} finally {
logIdLock.readLock().unlock();
}
}
synchronized private void createLogger() throws IOException {
if (!logIdLock.isWriteLockedByCurrentThread()) {
throw new IllegalStateException("createLoggers should be called with write lock held!");
}
if (currentLog != null) {
throw new IllegalStateException("createLoggers should not be called when current log is set");
}
try {
startLogMaker();
Object next = nextLog.take();
if (next instanceof Exception) {
throw (Exception) next;
}
if (next instanceof DfsLogger) {
currentLog = (DfsLogger) next;
logId.incrementAndGet();
log.info("Using next log " + currentLog.getFileName());
// When we successfully create a WAL, make sure to reset the Retry.
if (null != retry) {
retry = null;
}
this.createTime = System.currentTimeMillis();
return;
} else {
throw new RuntimeException("Error: unexpected type seen: " + next);
}
} catch (Exception t) {
if (null == retry) {
retry = retryFactory.create();
}
// We have more retries or we exceeded the maximum number of accepted failures
if (retry.canRetry()) {
// Use the retry and record the time in which we did so
retry.useRetry();
try {
// Backoff
retry.waitForNextAttempt();
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new RuntimeException(e);
}
} else {
log.error("Repeatedly failed to create WAL. Going to exit tabletserver.", t);
// We didn't have retries or we failed too many times.
Halt.halt("Experienced too many errors creating WALs, giving up", 1);
}
// The exception will trigger the log creation to be re-attempted.
throw new RuntimeException(t);
}
}
private synchronized void startLogMaker() {
if (nextLogMaker != null) {
return;
}
nextLogMaker = new SimpleThreadPool(1, "WALog creator");
nextLogMaker.submit(new LoggingRunnable(log, new Runnable() {
@Override
public void run() {
final ServerResources conf = tserver.getServerConfig();
final VolumeManager fs = conf.getFileSystem();
while (!nextLogMaker.isShutdown()) {
DfsLogger alog = null;
try {
log.debug("Creating next WAL");
alog = new DfsLogger(conf, syncCounter, flushCounter);
alog.open(tserver.getClientAddressString());
String fileName = alog.getFileName();
log.debug("Created next WAL " + fileName);
tserver.addNewLogMarker(alog);
while (!nextLog.offer(alog, 12, TimeUnit.HOURS)) {
log.info("Our WAL was not used for 12 hours: " + fileName);
}
} catch (Exception t) {
log.error("Failed to open WAL", t);
if (null != alog) {
// It's possible that the sync of the header and OPEN record to the WAL failed
// We want to make sure that clean up the resources/thread inside the DfsLogger
// object before trying to create a new one.
try {
alog.close();
} catch (Exception e) {
log.error("Failed to close WAL after it failed to open", e);
}
// Try to avoid leaving a bunch of empty WALs lying around
try {
Path path = alog.getPath();
if (fs.exists(path)) {
fs.delete(path);
}
} catch (Exception e) {
log.warn("Failed to delete a WAL that failed to open", e);
}
}
try {
nextLog.offer(t, 12, TimeUnit.HOURS);
} catch (InterruptedException ex) {
// ignore
}
}
}
}
}));
}
public void resetLoggers() throws IOException {
logIdLock.writeLock().lock();
try {
close();
} finally {
logIdLock.writeLock().unlock();
}
}
synchronized private void close() throws IOException {
if (!logIdLock.isWriteLockedByCurrentThread()) {
throw new IllegalStateException("close should be called with write lock held!");
}
try {
if (null != currentLog) {
try {
currentLog.close();
} catch (DfsLogger.LogClosedException ex) {
// ignore
} catch (Throwable ex) {
log.error("Unable to cleanly close log " + currentLog.getFileName() + ": " + ex, ex);
} finally {
this.tserver.walogClosed(currentLog);
}
currentLog = null;
logSizeEstimate.set(0);
}
} catch (Throwable t) {
throw new IOException(t);
}
}
interface Writer {
LoggerOperation write(DfsLogger logger, int seq) throws Exception;
}
private int write(CommitSession commitSession, boolean mincFinish, Writer writer) throws IOException {
List<CommitSession> sessions = Collections.singletonList(commitSession);
return write(sessions, mincFinish, writer);
}
private int write(final Collection<CommitSession> sessions, boolean mincFinish, Writer writer) throws IOException {
// Work very hard not to lock this during calls to the outside world
int currentLogId = logId.get();
int seq = -1;
int attempt = 1;
boolean success = false;
while (!success) {
try {
// get a reference to the loggers that no other thread can touch
DfsLogger copy = null;
AtomicInteger currentId = new AtomicInteger(-1);
copy = initializeLoggers(currentId);
currentLogId = currentId.get();
// add the logger to the log set for the memory in the tablet,
// update the metadata table if we've never used this tablet
if (currentLogId == logId.get()) {
for (CommitSession commitSession : sessions) {
if (commitSession.beginUpdatingLogsUsed(copy, mincFinish)) {
try {
// Scribble out a tablet definition and then write to the metadata table
defineTablet(commitSession);
} finally {
commitSession.finishUpdatingLogsUsed();
}
// Need to release
KeyExtent extent = commitSession.getExtent();
if (ReplicationConfigurationUtil.isEnabled(extent, tserver.getTableConfiguration(extent))) {
Status status = StatusUtil.openWithUnknownLength(System.currentTimeMillis());
log.debug("Writing " + ProtobufUtil.toString(status) + " to metadata table for " + copy.getFileName());
// Got some new WALs, note this in the metadata table
ReplicationTableUtil.updateFiles(tserver, commitSession.getExtent(), copy.getFileName(), status);
}
}
}
}
// Make sure that the logs haven't changed out from underneath our copy
if (currentLogId == logId.get()) {
// write the mutation to the logs
seq = seqGen.incrementAndGet();
if (seq < 0)
throw new RuntimeException("Logger sequence generator wrapped! Onos!!!11!eleven");
LoggerOperation lop = writer.write(copy, seq);
lop.await();
// double-check: did the log set change?
success = (currentLogId == logId.get());
}
} catch (DfsLogger.LogClosedException ex) {
log.debug("Logs closed while writing, retrying " + attempt);
} catch (Exception t) {
if (attempt != 1) {
log.error("Unexpected error writing to log, retrying attempt " + attempt, t);
}
sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
} finally {
attempt++;
}
// Some sort of write failure occurred. Grab the write lock and reset the logs.
// But since multiple threads will attempt it, only attempt the reset when
// the logs haven't changed.
final int finalCurrent = currentLogId;
if (!success) {
testLockAndRun(logIdLock, new TestCallWithWriteLock() {
@Override
boolean test() {
return finalCurrent == logId.get();
}
@Override
void withWriteLock() throws IOException {
close();
closeForReplication(sessions);
}
});
}
}
// if the log gets too big or too old, reset it .. grab the write lock first
logSizeEstimate.addAndGet(4 * 3); // event, tid, seq overhead
testLockAndRun(logIdLock, new TestCallWithWriteLock() {
@Override
boolean test() {
return (logSizeEstimate.get() > maxSize) || ((System.currentTimeMillis() - createTime) > maxAge);
}
@Override
void withWriteLock() throws IOException {
close();
closeForReplication(sessions);
}
});
return seq;
}
protected void closeForReplication(Collection<CommitSession> sessions) {
// TODO We can close the WAL here for replication purposes
}
public int defineTablet(final CommitSession commitSession) throws IOException {
// scribble this into the metadata tablet, too.
return write(commitSession, false, new Writer() {
@Override
public LoggerOperation write(DfsLogger logger, int ignored) throws Exception {
logger.defineTablet(commitSession.getWALogSeq(), commitSession.getLogId(), commitSession.getExtent());
return DfsLogger.NO_WAIT_LOGGER_OP;
}
});
}
public int log(final CommitSession commitSession, final int tabletSeq, final Mutation m, final Durability durability) throws IOException {
if (durability == Durability.NONE) {
return -1;
}
if (durability == Durability.DEFAULT) {
throw new IllegalArgumentException("Unexpected durability " + durability);
}
int seq = write(commitSession, false, new Writer() {
@Override
public LoggerOperation write(DfsLogger logger, int ignored) throws Exception {
return logger.log(tabletSeq, commitSession.getLogId(), m, durability);
}
});
logSizeEstimate.addAndGet(m.numBytes());
return seq;
}
public int logManyTablets(Map<CommitSession,Mutations> mutations) throws IOException {
final Map<CommitSession,Mutations> loggables = new HashMap<>(mutations);
for (Entry<CommitSession,Mutations> entry : mutations.entrySet()) {
if (entry.getValue().getDurability() == Durability.NONE) {
loggables.remove(entry.getKey());
}
}
if (loggables.size() == 0)
return -1;
int seq = write(loggables.keySet(), false, new Writer() {
@Override
public LoggerOperation write(DfsLogger logger, int ignored) throws Exception {
List<TabletMutations> copy = new ArrayList<>(loggables.size());
for (Entry<CommitSession,Mutations> entry : loggables.entrySet()) {
CommitSession cs = entry.getKey();
Durability durability = entry.getValue().getDurability();
copy.add(new TabletMutations(cs.getLogId(), cs.getWALogSeq(), entry.getValue().getMutations(), durability));
}
return logger.logManyTablets(copy);
}
});
for (Mutations entry : loggables.values()) {
if (entry.getMutations().size() < 1) {
throw new IllegalArgumentException("logManyTablets: logging empty mutation list");
}
for (Mutation m : entry.getMutations()) {
logSizeEstimate.addAndGet(m.numBytes());
}
}
return seq;
}
public void minorCompactionFinished(final CommitSession commitSession, final String fullyQualifiedFileName, final int walogSeq, final Durability durability)
throws IOException {
long t1 = System.currentTimeMillis();
int seq = write(commitSession, true, new Writer() {
@Override
public LoggerOperation write(DfsLogger logger, int ignored) throws Exception {
return logger.minorCompactionFinished(walogSeq, commitSession.getLogId(), fullyQualifiedFileName, durability);
}
});
long t2 = System.currentTimeMillis();
log.debug(" wrote MinC finish {}: writeTime:{}ms durability:{}", seq, (t2 - t1), durability);
}
public int minorCompactionStarted(final CommitSession commitSession, final int seq, final String fullyQualifiedFileName, final Durability durability)
throws IOException {
write(commitSession, false, new Writer() {
@Override
public LoggerOperation write(DfsLogger logger, int ignored) throws Exception {
return logger.minorCompactionStarted(seq, commitSession.getLogId(), fullyQualifiedFileName, durability);
}
});
return seq;
}
public void recover(VolumeManager fs, KeyExtent extent, TableConfiguration tconf, List<Path> logs, Set<String> tabletFiles, MutationReceiver mr)
throws IOException {
try {
SortedLogRecovery recovery = new SortedLogRecovery(fs);
recovery.recover(extent, logs, tabletFiles, mr);
} catch (Exception e) {
throw new IOException(e);
}
}
}