wangchao316 commented on a change in pull request #5320:
URL: https://github.com/apache/iotdb/pull/5320#discussion_r834060850
##########
File path: server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
##########
@@ -139,46 +142,39 @@
/** When inserting rejected exceeds this, throw an exception. Unit:
millisecond */
private int maxWaitingTimeWhenInsertBlockedInMs = 10000;
- /** Is the write ahead log enable. */
- private boolean enableWal = true;
- private volatile boolean readOnly = false;
+ /** this variable set timestamp precision as millisecond, microsecond or
nanosecond */
+ private String timestampPrecision = "ms";
- private boolean enableDiscardOutOfOrderData = false;
+ // region Write Ahead Log Configuration
Review comment:
this context?
##########
File path: server/src/assembly/resources/conf/iotdb-engine.properties
##########
@@ -50,25 +50,45 @@ rpc_port=6667
### Write Ahead Log Configuration
####################
-# Is insert ahead log enable
-# Datatype: boolean
-# enable_wal=true
+# Write mode of wal
+# The details of these three modes are as follows:
+# 1. DISABLE: the system will disable wal.
+# 2. SYNC: the system will submit wal synchronously, write request will not
return until its wal is fsynced to the disk successfully.
+# 3. ASYNC: the system will submit wal asynchronously, write request will
return immediately no matter its wal is fsynced to the disk successfully.
+# The write performance order is DISABLE > ASYNC > SYNC, but only SYNC mode
can ensure data durability.
+# wal_mode=SYNC
+
+# Duration a wal flush operation will wait before calling fsync
+# A duration greater than 0 batches multiple wal fsync calls into one. This is
useful when disks are slow or WAL write contention exists.
+# Datatype: long
+# sync_wal_delay_in_ms=0
-# Add a switch to drop ouf-of-order data
-# Out-of-order data will impact the aggregation query a lot. Users may not
care about discarding some out-of-order data.
-# Datatype: boolean
-# enable_discard_out_of_order_data=false
+# Max number of wal nodes, each node corresponds to one wal directory
+# The default value 0 means the concurrent wal number will be 2 * 'number of
wal dirs'.
+# Datatype: int
+# max_wal_num=0
-# When a certain amount of insert ahead log is reached, it will be flushed to
disk
-# It is possible to lose at most flush_wal_threshold operations
+# Buffer size of each wal node
+# If it sets a value smaller than 0, use the default value 16777216 bytes
(16MB).
# Datatype: int
-# flush_wal_threshold=10000
+# wal_buffer_size_in_byte=16777216
-# The cycle when insert ahead log is periodically forced to be written to
disk(in milliseconds)
-# If force_wal_period_in_ms = 0 it means force insert ahead log to be written
to disk after each refreshment
-# Set this parameter to 0 may slow down the ingestion on slow disk.
-# Datatype: long
-# force_wal_period_in_ms=100
+# Buffer entry size of each wal buffer
+# If it sets a value smaller than 0, use the default value 16384 bytes (16KB).
+# Datatype: int
+# wal_buffer_entry_size_in_byte=16384
+
+# Max storage space for wal
+# The default value 0 means the storage space will not be controlled.
+# Notice: If this parameter is too small, the write performance may decline.
+# Datatype: int
+# wal_storage_space_in_mb=0
Review comment:
this default parameter is 0 ?
##########
File path:
server/src/main/java/org/apache/iotdb/db/wal/buffer/AbstractWALBuffer.java
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.db.engine.fileSystem.SystemFileFactory;
+import org.apache.iotdb.db.wal.io.ILogWriter;
+import org.apache.iotdb.db.wal.io.WALWriter;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public abstract class AbstractWALBuffer implements IWALBuffer {
+ private static final Logger logger =
LoggerFactory.getLogger(AbstractWALBuffer.class);
+ /** use size limit to control WALEdit number in each file */
+ protected static final long LOG_SIZE_LIMIT = 10 * 1024 * 1024;
+
+ /** WALNode identifier of this buffer */
+ protected final String identifier;
+ /** directory to store .wal files */
+ protected final String logDirectory;
+ /** current wal file version id */
+ protected final AtomicInteger currentLogVersion = new AtomicInteger();
+ /** current wal file log writer */
+ protected volatile ILogWriter currentLogWriter;
+
+ public AbstractWALBuffer(String identifier, String logDirectory) throws
FileNotFoundException {
+ this.identifier = identifier;
+ this.logDirectory = logDirectory;
+ File logDirFile = SystemFileFactory.INSTANCE.getFile(logDirectory);
+ if (!logDirFile.exists() && logDirFile.mkdirs()) {
+ logger.info("create folder {} for wal buffer-{}.", logDirectory,
identifier);
+ }
+ currentLogWriter =
+ new WALWriter(
Review comment:
super class of WALWriter is AbstractWALBuffer, why new WALWriter in
AbstractWALBuffer ?
##########
File path: server/src/main/java/org/apache/iotdb/db/wal/buffer/WALBuffer.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.qp.physical.crud.DeletePlan;
+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 {
+ /** Maximum number of WALEdits in one serialize task */
+ public static final int BATCH_SIZE_LIMIT = 100;
+
+ private static final Logger logger =
LoggerFactory.getLogger(WALBuffer.class);
+ private static final IoTDBConfig config =
IoTDBDescriptor.getInstance().getConfig();
+ private static final long SYNC_WAL_DELAY_IN_MS =
config.getSyncWalDelayInMs();
+ private static final int WAL_BUFFER_SIZE = config.getWalBufferSize();
+ /** notify serializeThread to stop */
+ private static final WALEdit CLOSE_SIGNAL = new WALEdit(-1, new
DeletePlan());
+
+ /** whether close method is called */
+ private volatile boolean isClosed = false;
+ /** WALEdits */
+ private final BlockingQueue<WALEdit> walEdits = new
ArrayBlockingQueue<>(BATCH_SIZE_LIMIT * 10);
+ /** 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;
+ /** 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();
+ /** single thread to serialize WALEdit 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 {
+ super(identifier, logDirectory);
+ 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);
Review comment:
Can the number of ByteBuffer.allocateDirect be controlled? If the number
of ByteBuffer.allocateDirect is infinite, the memory will be used up?
##########
File path:
server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
##########
@@ -496,6 +501,11 @@ void updatePlanIndexes(long index) {
minPlanIndex = Math.min(index, minPlanIndex);
}
+ @Override
+ public int getMemTableId() {
+ return memTableId;
Review comment:
memTableId is a fixed value, memTableIdCounter.getAndIncrement() == 0.
##########
File path: server/src/main/java/org/apache/iotdb/db/wal/WALManager.java
##########
@@ -0,0 +1,238 @@
+/*
+ * 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;
+
+import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
+import org.apache.iotdb.commons.concurrent.ThreadName;
+import org.apache.iotdb.commons.exception.StartupException;
+import org.apache.iotdb.commons.service.IService;
+import org.apache.iotdb.commons.service.ServiceType;
+import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.db.conf.IoTDBConfig;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.conf.directories.FolderManager;
+import org.apache.iotdb.db.conf.directories.strategy.DirectoryStrategyType;
+import org.apache.iotdb.db.exception.DiskSpaceInsufficientException;
+import org.apache.iotdb.db.wal.node.IWALNode;
+import org.apache.iotdb.db.wal.node.WALFakeNode;
+import org.apache.iotdb.db.wal.node.WALNode;
+import org.apache.iotdb.db.wal.utils.WALMode;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+/** This class is used to manage all wal nodes */
+public class WALManager implements IService {
+ public static final long FSYNC_CHECKPOINT_FILE_DELAY_IN_MS = 200;
+ public static final long DELETE_WAL_FILES_DELAY_IN_MS = 10 * 60 * 1000;
+
+ private static final Logger logger =
LoggerFactory.getLogger(WALManager.class);
+ private static final IoTDBConfig config =
IoTDBDescriptor.getInstance().getConfig();
+ private static final int MAX_WAL_NUM =
+ config.getMaxWalNum() > 0 ? config.getMaxWalNum() :
config.getWalDirs().length * 2;
+
+ /** manage wal folders */
+ private FolderManager folderManager;
+ /** protect concurrent safety of walNodes, nodeCursor and nodeIdCounter */
+ private final Lock nodesLock = new ReentrantLock();
+ /** wal nodes, the max number of wal nodes is MAX_WAL_NUM */
+ private final List<WALNode> walNodes = new ArrayList<>(MAX_WAL_NUM);
+ /** help allocate node for users */
+ private int nodeCursor = -1;
+ /** each wal node has a unique long value identifier */
+ private long nodeIdCounter = -1;
Review comment:
nodeIdCounter should use AtomicLong.
##########
File path: server/src/main/java/org/apache/iotdb/db/wal/buffer/WALBuffer.java
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.qp.physical.crud.DeletePlan;
+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 {
+ /** Maximum number of WALEdits in one serialize task */
+ public static final int BATCH_SIZE_LIMIT = 100;
+
+ private static final Logger logger =
LoggerFactory.getLogger(WALBuffer.class);
+ private static final IoTDBConfig config =
IoTDBDescriptor.getInstance().getConfig();
+ private static final long SYNC_WAL_DELAY_IN_MS =
config.getSyncWalDelayInMs();
+ private static final int WAL_BUFFER_SIZE = config.getWalBufferSize();
+ /** notify serializeThread to stop */
+ private static final WALEdit CLOSE_SIGNAL = new WALEdit(-1, new
DeletePlan());
+
+ /** whether close method is called */
+ private volatile boolean isClosed = false;
+ /** WALEdits */
+ private final BlockingQueue<WALEdit> walEdits = new
ArrayBlockingQueue<>(BATCH_SIZE_LIMIT * 10);
+ /** 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;
+ /** 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();
+ /** single thread to serialize WALEdit 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 {
+ super(identifier, logDirectory);
+ 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(WALEdit edit) {
+ if (isClosed) {
+ logger.error(
+ "Fail to write WALEdit into wal node-{} because this node is
closed.", identifier);
+ edit.getWalFlushListener().fail(new WALNodeClosedException(identifier));
+ return;
+ }
+ // only add this WALEdit to queue
+ try {
+ walEdits.put(edit);
+ } catch (InterruptedException e) {
+ logger.warn("Interrupted when waiting for adding WalEdit to buffer.");
+ Thread.currentThread().interrupt();
+ }
+ }
+
+ // region Task of serializeThread
+ /** This task serializes WALEdit 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<>();
+
+ @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() {
+ // for better fsync performance, sleep a while to enlarge write batch
+ if (SYNC_WAL_DELAY_IN_MS > 0) {
+ try {
+ Thread.sleep(SYNC_WAL_DELAY_IN_MS);
+ } catch (InterruptedException e) {
+ logger.warn("Interrupted when sleeping a while to enlarge wal write
batch.");
+ Thread.currentThread().interrupt();
+ }
+ }
+ // try to get first WALEdit with blocking interface
+ int batchSize = 0;
+ try {
+ WALEdit edit = walEdits.take();
+ try {
+ if (edit != CLOSE_SIGNAL) {
+ edit.serialize(byteBufferVew);
+ ++batchSize;
+ fsyncListeners.add(edit.getWalFlushListener());
+ }
+ } catch (Exception e) {
+ logger.error(
+ "Fail to serialize WALEdit to wal node-{}'s buffer, discard
it.", identifier, e);
+ edit.getWalFlushListener().fail(e);
+ }
+ } catch (InterruptedException e) {
+ logger.warn(
+ "Interrupted when waiting for taking WALEdit from blocking queue
to serialize.");
+ Thread.currentThread().interrupt();
+ }
+ // try to get more WALEdits with non-blocking interface to enlarge write
batch
+ while (walEdits.peek() != null && batchSize < BATCH_SIZE_LIMIT) {
+ WALEdit edit = walEdits.poll();
+ if (edit == null || edit == CLOSE_SIGNAL) {
+ break;
+ } else {
+ try {
+ edit.serialize(byteBufferVew);
+ } catch (Exception e) {
+ logger.error(
+ "Fail to serialize WALEdit to wal node-{}'s buffer, discard
it.", identifier, e);
+ edit.getWalFlushListener().fail(e);
+ continue;
+ }
+ ++batchSize;
+ fsyncListeners.add(edit.getWalFlushListener());
+ }
+ }
+ // call fsync at last and set fsyncListeners
+ if (batchSize > 0) {
+ fsyncWorkingBuffer(fsyncListeners);
+ }
+ }
+ }
+
+ /**
+ * 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();
+ }
+
+ @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() {
+ switchIdleBufferToWorking();
+ syncBufferThread.submit(new SyncBufferTask(false));
+ }
+
+ /** Notice: this method only called at the last of SerializeTask. */
+ private void fsyncWorkingBuffer(List<WALFlushListener> fsyncListeners) {
+ switchIdleBufferToWorking();
+ syncBufferThread.submit(new SyncBufferTask(true, fsyncListeners));
+ }
+
+ // only called by serializeThread
+ private void switchIdleBufferToWorking() {
+ 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
Review comment:
delete
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]