forked from apache/iotdb
-
Notifications
You must be signed in to change notification settings - Fork 0
/
WALBuffer.java
529 lines (482 loc) · 17.3 KB
/
WALBuffer.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
/*
* 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.iotdb.db.wal.buffer;
import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
import org.apache.iotdb.commons.concurrent.ThreadName;
import org.apache.iotdb.db.conf.IoTDBConfig;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertNode;
import org.apache.iotdb.db.utils.MmapUtil;
import org.apache.iotdb.db.wal.exception.WALNodeClosedException;
import org.apache.iotdb.db.wal.utils.listener.WALFlushListener;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.MappedByteBuffer;
import java.util.Collections;
import java.util.LinkedList;
import java.util.List;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
/**
* This buffer guarantees the concurrent safety and uses double buffers mechanism to accelerate
* writes and avoid waiting for buffer syncing to disk.
*/
public class WALBuffer extends AbstractWALBuffer {
private static final Logger logger = LoggerFactory.getLogger(WALBuffer.class);
private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
private static final int WAL_BUFFER_SIZE = config.getWalBufferSize();
private static final int QUEUE_CAPACITY = config.getWalBufferQueueCapacity();
/** whether close method is called */
private volatile boolean isClosed = false;
/** WALEntries */
private final BlockingQueue<WALEntry> walEntries = new ArrayBlockingQueue<>(QUEUE_CAPACITY);
/** lock to provide synchronization for double buffers mechanism, protecting buffers status */
private final Lock buffersLock = new ReentrantLock();
/** condition to guarantee correctness of switching buffers */
private final Condition idleBufferReadyCondition = buffersLock.newCondition();
// region these variables should be protected by buffersLock
/** two buffers switch between three statuses (there is always 1 buffer working) */
// buffer in working status, only updated by serializeThread
private volatile ByteBuffer workingBuffer;
// buffer in idle status
private volatile ByteBuffer idleBuffer;
// buffer in syncing status, serializeThread makes sure no more writes to syncingBuffer
private volatile ByteBuffer syncingBuffer;
// endregion
/** single thread to serialize WALEntry to workingBuffer */
private final ExecutorService serializeThread;
/** single thread to sync syncingBuffer to disk */
private final ExecutorService syncBufferThread;
public WALBuffer(String identifier, String logDirectory) throws FileNotFoundException {
this(identifier, logDirectory, 0, 0L);
}
public WALBuffer(
String identifier, String logDirectory, int startFileVersion, long startSearchIndex)
throws FileNotFoundException {
super(identifier, logDirectory, startFileVersion, startSearchIndex);
allocateBuffers();
serializeThread =
IoTDBThreadPoolFactory.newSingleThreadExecutor(
ThreadName.WAL_SERIALIZE.getName() + "(node-" + identifier + ")");
syncBufferThread =
IoTDBThreadPoolFactory.newSingleThreadExecutor(
ThreadName.WAL_SYNC.getName() + "(node-" + identifier + ")");
// start receiving serialize tasks
serializeThread.submit(new SerializeTask());
}
private void allocateBuffers() {
try {
workingBuffer = ByteBuffer.allocateDirect(WAL_BUFFER_SIZE / 2);
idleBuffer = ByteBuffer.allocateDirect(WAL_BUFFER_SIZE / 2);
} catch (OutOfMemoryError e) {
logger.error("Fail to allocate wal node-{}'s buffer because out of memory.", identifier, e);
close();
throw e;
}
}
@Override
public void write(WALEntry walEntry) {
if (isClosed) {
logger.error(
"Fail to write WALEntry into wal node-{} because this node is closed.", identifier);
walEntry.getWalFlushListener().fail(new WALNodeClosedException(identifier));
return;
}
// just add this WALEntry to queue
try {
walEntries.put(walEntry);
} catch (InterruptedException e) {
logger.warn("Interrupted when waiting for adding WALEntry to buffer.");
Thread.currentThread().interrupt();
}
}
// region Task of serializeThread
/** This task serializes WALEntry to workingBuffer and will call fsync at last. */
private class SerializeTask implements Runnable {
private final IWALByteBufferView byteBufferVew = new ByteBufferView();
private final List<WALFlushListener> fsyncListeners = new LinkedList<>();
private int batchSize = 0;
private WALFlushListener rollWALFileWriterListener = null;
@Override
public void run() {
try {
serialize();
} finally {
serializeThread.submit(new SerializeTask());
}
}
/** In order to control memory usage of blocking queue, get 1 and then serialize 1 */
private void serialize() {
// try to get first WALEntry with blocking interface
try {
WALEntry firstWALEntry = walEntries.take();
boolean returnFlag = handleWALEntry(firstWALEntry);
if (returnFlag) {
return;
}
} catch (InterruptedException e) {
logger.warn(
"Interrupted when waiting for taking WALEntry from blocking queue to serialize.");
Thread.currentThread().interrupt();
}
// for better fsync performance, sleep a while to enlarge write batch
long fsyncDelay = config.getFsyncWalDelayInMs();
if (fsyncDelay > 0) {
try {
Thread.sleep(fsyncDelay);
} catch (InterruptedException e) {
logger.warn("Interrupted when sleeping a while to enlarge wal write batch.");
Thread.currentThread().interrupt();
}
}
// try to get more WALEntries with non-blocking interface to enlarge write batch
while (walEntries.peek() != null && batchSize < QUEUE_CAPACITY) {
WALEntry walEntry = walEntries.poll();
boolean returnFlag = handleWALEntry(walEntry);
if (returnFlag) {
return;
}
}
// call fsync at last and set fsyncListeners
if (batchSize > 0) {
fsyncWorkingBuffer(currentSearchIndex, fsyncListeners, rollWALFileWriterListener);
}
}
/**
* @return true if fsyncWorkingBuffer has been called, which means this serialization task
* should be ended.
*/
private boolean handleWALEntry(WALEntry walEntry) {
if (walEntry.isSignal()) {
return handleSignalEntry((SignalWALEntry) walEntry);
}
boolean success = handleInfoEntry(walEntry);
if (success) {
++batchSize;
fsyncListeners.add(walEntry.getWalFlushListener());
}
return false;
}
/**
* Handle a normal WALEntry.
*
* @return true if serialization is successful.
*/
private boolean handleInfoEntry(WALEntry walEntry) {
try {
walEntry.serialize(byteBufferVew);
} catch (Exception e) {
logger.error(
"Fail to serialize WALEntry to wal node-{}'s buffer, discard it.", identifier, e);
walEntry.getWalFlushListener().fail(e);
return false;
}
// update search index
if (walEntry.getType() == WALEntryType.INSERT_TABLET_NODE
|| walEntry.getType() == WALEntryType.INSERT_ROW_NODE) {
InsertNode insertNode = (InsertNode) walEntry.getValue();
if (insertNode.getSearchIndex() != InsertNode.NO_CONSENSUS_INDEX) {
currentSearchIndex = insertNode.getSearchIndex();
}
}
return true;
}
/**
* @return true if fsyncWorkingBuffer has been called, which means this serialization task
* should be ended.
*/
private boolean handleSignalEntry(SignalWALEntry signalWALEntry) {
switch (signalWALEntry.getSignalType()) {
case ROLL_WAL_LOG_WRITER_SIGNAL:
rollWALFileWriterListener = signalWALEntry.getWalFlushListener();
fsyncWorkingBuffer(currentSearchIndex, fsyncListeners, rollWALFileWriterListener);
return true;
case CLOSE_SIGNAL:
boolean dataExists = batchSize > 0;
if (dataExists) {
fsyncWorkingBuffer(currentSearchIndex, fsyncListeners, rollWALFileWriterListener);
}
return dataExists;
default:
return false;
}
}
}
/**
* This view uses workingBuffer lock-freely because workingBuffer is only updated by
* serializeThread and this class is only used by serializeThread.
*/
private class ByteBufferView implements IWALByteBufferView {
private void ensureEnoughSpace(int bytesNum) {
if (workingBuffer.remaining() < bytesNum) {
rollBuffer();
}
}
private void rollBuffer() {
syncWorkingBuffer(currentSearchIndex);
}
@Override
public void put(byte b) {
ensureEnoughSpace(Byte.BYTES);
workingBuffer.put(b);
}
@Override
public void put(byte[] src) {
int offset = 0;
while (true) {
int leftCapacity = workingBuffer.remaining();
int needCapacity = src.length - offset;
if (leftCapacity >= needCapacity) {
workingBuffer.put(src, offset, needCapacity);
break;
} else {
workingBuffer.put(src, offset, leftCapacity);
offset += leftCapacity;
rollBuffer();
}
}
}
@Override
public void putChar(char value) {
ensureEnoughSpace(Character.BYTES);
workingBuffer.putChar(value);
}
@Override
public void putShort(short value) {
ensureEnoughSpace(Short.BYTES);
workingBuffer.putShort(value);
}
@Override
public void putInt(int value) {
ensureEnoughSpace(Integer.BYTES);
workingBuffer.putInt(value);
}
@Override
public void putLong(long value) {
ensureEnoughSpace(Long.BYTES);
workingBuffer.putLong(value);
}
@Override
public void putFloat(float value) {
ensureEnoughSpace(Float.BYTES);
workingBuffer.putFloat(value);
}
@Override
public void putDouble(double value) {
ensureEnoughSpace(Double.BYTES);
workingBuffer.putDouble(value);
}
}
/** Notice: this method only called when buffer is exhausted by SerializeTask. */
private void syncWorkingBuffer(long searchIndex) {
switchWorkingBufferToFlushing();
syncBufferThread.submit(new SyncBufferTask(searchIndex, false));
}
/** Notice: this method only called at the last of SerializeTask. */
private void fsyncWorkingBuffer(
long searchIndex,
List<WALFlushListener> fsyncListeners,
WALFlushListener rollWALFileWriterListener) {
switchWorkingBufferToFlushing();
syncBufferThread.submit(
new SyncBufferTask(searchIndex, true, fsyncListeners, rollWALFileWriterListener));
}
// only called by serializeThread
private void switchWorkingBufferToFlushing() {
buffersLock.lock();
try {
while (idleBuffer == null) {
idleBufferReadyCondition.await();
}
syncingBuffer = workingBuffer;
workingBuffer = idleBuffer;
workingBuffer.clear();
idleBuffer = null;
} catch (InterruptedException e) {
logger.warn("Interrupted When waiting for available working buffer.");
Thread.currentThread().interrupt();
} finally {
buffersLock.unlock();
}
}
// endregion
// region Task of syncBufferThread
/**
* This task syncs syncingBuffer to disk. The precondition is that syncingBuffer cannot be null.
*/
private class SyncBufferTask implements Runnable {
private final long searchIndex;
private final boolean forceFlag;
private final List<WALFlushListener> fsyncListeners;
private final WALFlushListener rollWALFileWriterListener;
public SyncBufferTask(long searchIndex, boolean forceFlag) {
this(searchIndex, forceFlag, null, null);
}
public SyncBufferTask(
long searchIndex,
boolean forceFlag,
List<WALFlushListener> fsyncListeners,
WALFlushListener rollWALFileWriterListener) {
this.searchIndex = searchIndex;
this.forceFlag = forceFlag;
this.fsyncListeners = fsyncListeners == null ? Collections.emptyList() : fsyncListeners;
this.rollWALFileWriterListener = rollWALFileWriterListener;
}
@Override
public void run() {
// flush buffer to os
try {
currentWALFileWriter.write(syncingBuffer);
} catch (Throwable e) {
logger.error(
"Fail to sync wal node-{}'s buffer, change system mode to read-only.", identifier, e);
config.setReadOnly(true);
} finally {
switchSyncingBufferToIdle();
}
// force os cache to the storage device
if (forceFlag) {
try {
currentWALFileWriter.force();
} catch (IOException e) {
logger.error(
"Fail to fsync wal node-{}'s log writer, change system mode to read-only.",
identifier,
e);
for (WALFlushListener fsyncListener : fsyncListeners) {
fsyncListener.fail(e);
}
config.setReadOnly(true);
}
// notify all waiting listeners
for (WALFlushListener fsyncListener : fsyncListeners) {
fsyncListener.succeed();
}
}
// try to roll log writer
try {
if (rollWALFileWriterListener != null
|| (forceFlag
&& currentWALFileWriter.size() >= config.getWalFileSizeThresholdInByte())) {
rollLogWriter(searchIndex);
if (rollWALFileWriterListener != null) {
rollWALFileWriterListener.succeed();
}
}
} catch (IOException e) {
logger.error(
"Fail to roll wal node-{}'s log writer, change system mode to read-only.",
identifier,
e);
if (rollWALFileWriterListener != null) {
rollWALFileWriterListener.fail(e);
}
config.setReadOnly(true);
}
}
}
// only called by syncBufferThread
private void switchSyncingBufferToIdle() {
buffersLock.lock();
try {
// No need to judge whether idleBuffer is null because syncingBuffer is not null
// and there is only one buffer can be null between syncingBuffer and idleBuffer
idleBuffer = syncingBuffer;
syncingBuffer = null;
idleBufferReadyCondition.signalAll();
} finally {
buffersLock.unlock();
}
}
@Override
public void waitForFlush() throws InterruptedException {
buffersLock.lock();
try {
idleBufferReadyCondition.await();
} finally {
buffersLock.unlock();
}
}
@Override
public boolean waitForFlush(long time, TimeUnit unit) throws InterruptedException {
buffersLock.lock();
try {
return idleBufferReadyCondition.await(time, unit);
} finally {
buffersLock.unlock();
}
}
// endregion
@Override
public void close() {
isClosed = true;
// first waiting serialize and sync tasks finished, then release all resources
if (serializeThread != null) {
// add close signal WALEntry to notify serializeThread
try {
walEntries.put(new SignalWALEntry(SignalWALEntry.SignalType.CLOSE_SIGNAL));
} catch (InterruptedException e) {
logger.error("Fail to put CLOSE_SIGNAL to walEntries.", e);
}
shutdownThread(serializeThread, ThreadName.WAL_SERIALIZE);
}
if (syncBufferThread != null) {
shutdownThread(syncBufferThread, ThreadName.WAL_SYNC);
}
if (currentWALFileWriter != null) {
try {
currentWALFileWriter.close();
} catch (IOException e) {
logger.error("Fail to close wal node-{}'s log writer.", identifier, e);
}
}
if (workingBuffer != null) {
MmapUtil.clean((MappedByteBuffer) workingBuffer);
}
if (idleBuffer != null) {
MmapUtil.clean((MappedByteBuffer) workingBuffer);
}
if (syncingBuffer != null) {
MmapUtil.clean((MappedByteBuffer) syncingBuffer);
}
}
private void shutdownThread(ExecutorService thread, ThreadName threadName) {
thread.shutdown();
try {
if (!thread.awaitTermination(30, TimeUnit.SECONDS)) {
logger.warn("Waiting thread {} to be terminated is timeout", threadName.getName());
}
} catch (InterruptedException e) {
logger.warn("Thread {} still doesn't exit after 30s", threadName.getName());
Thread.currentThread().interrupt();
}
}
@Override
public boolean isAllWALEntriesConsumed() {
return walEntries.isEmpty();
}
}