neuyilan commented on code in PR #9667: URL: https://github.com/apache/iotdb/pull/9667#discussion_r1197631144
########## node-commons/src/assembly/resources/conf/iotdb-common.properties: ########## @@ -541,6 +541,26 @@ cluster_name=defaultCluster # Datatype: int # device_path_cache_size=500000 +# When flushing a MemTable, the range of thread number that will be available for each pipeline state. +# Set to 1 when less than or equal to 0. +# Datatype: int +# flush_min_sub_thread_num=1 +# flush_max_sub_thread_num=16 + +# If the idle ratio of a DynamicThread is below this value, it will try to add a new thread in +# its group if there are fewer threads than flushMemTableMaxSubThread. Review Comment: May be flush_max_sub_thread_num ? ########## node-commons/src/assembly/resources/conf/iotdb-common.properties: ########## @@ -541,6 +541,26 @@ cluster_name=defaultCluster # Datatype: int # device_path_cache_size=500000 +# When flushing a MemTable, the range of thread number that will be available for each pipeline state. +# Set to 1 when less than or equal to 0. +# Datatype: int +# flush_min_sub_thread_num=1 +# flush_max_sub_thread_num=16 + +# If the idle ratio of a DynamicThread is below this value, it will try to add a new thread in +# its group if there are fewer threads than flushMemTableMaxSubThread. +# Datatype: double +#dynamic_min_idle_ratio=0.1 + +# If the idle ratio of a DynamicThread is over this value, it will try to exit if there are more +# threads than flushMemTableMinSubThread. Review Comment: the same ########## node-commons/src/main/java/org/apache/iotdb/commons/concurrent/dynamic/DynamicThread.java: ########## @@ -0,0 +1,117 @@ +/* + * 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.commons.concurrent.dynamic; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * DynamicThread record the idle time and running time of the thread and trigger addThread() or + * onThreadExit() in DynamicThreadGroup to change the number of threads in a thread group + * dynamically. IMPORTANT: the implementation must call idleToRunning(), runningToIdle(), and + * shouldExit() properly in runInternal(). + */ +public abstract class DynamicThread implements Runnable { + + private static final Logger logger = LoggerFactory.getLogger(DynamicThread.class); + private DynamicThreadGroup threadGroup; + private long idleStart; + private long runningStart; + private long idleTimeSum; + private long runningTimeSum; + // TODO: add configuration for the values + private double maximumIdleRatio = 0.5; + private double minimumIdleRatio = 0.1; + private long minimumRunningTime = 10_000_000_000L; Review Comment: ```suggestion private long minimumRunningTimeNs = 10_000_000_000L; ``` ########## server/src/main/java/org/apache/iotdb/db/engine/flush/MemTableFlushTask.java: ########## @@ -218,159 +265,59 @@ public void syncFlushMemTable() throws ExecutionException, InterruptedException System.currentTimeMillis() - start); } - /** encoding task (second task of pipeline) */ - private Runnable encodingTask = - new Runnable() { - - @SuppressWarnings("squid:S135") - @Override - public void run() { - LOGGER.debug( - "Database {} memtable flushing to file {} starts to encoding data.", - storageGroup, - writer.getFile().getName()); - while (true) { - - Object task; - try { - task = encodingTaskQueue.take(); - } catch (InterruptedException e1) { - LOGGER.error("Take task into ioTaskQueue Interrupted"); - Thread.currentThread().interrupt(); - break; - } - if (task instanceof StartFlushGroupIOTask || task instanceof EndChunkGroupIoTask) { - try { - ioTaskQueue.put(task); - } catch ( - @SuppressWarnings("squid:S2142") - InterruptedException e) { - LOGGER.error( - "Database {} memtable flushing to file {}, encoding task is interrupted.", - storageGroup, - writer.getFile().getName(), - e); - // generally it is because the thread pool is shutdown so the task should be aborted - break; - } - } else if (task instanceof TaskEnd) { - break; - } else { - long starTime = System.currentTimeMillis(); - IWritableMemChunk writableMemChunk = (IWritableMemChunk) task; - IChunkWriter seriesWriter = writableMemChunk.createIChunkWriter(); - writableMemChunk.encode(seriesWriter); - seriesWriter.sealCurrentPage(); - seriesWriter.clearPageWriter(); - try { - ioTaskQueue.put(seriesWriter); - } catch (InterruptedException e) { - LOGGER.error("Put task into ioTaskQueue Interrupted"); - Thread.currentThread().interrupt(); - } - long subTaskTime = System.currentTimeMillis() - starTime; - WRITING_METRICS.recordFlushSubTaskCost(WritingMetrics.ENCODING_TASK, subTaskTime); - memSerializeTime += subTaskTime; - } - } - try { - ioTaskQueue.put(new TaskEnd()); - } catch (InterruptedException e) { - LOGGER.error("Put task into ioTaskQueue Interrupted"); - Thread.currentThread().interrupt(); - } - - if (!storageGroup.startsWith(IoTDBMetricsUtils.DATABASE)) { - int lastIndex = storageGroup.lastIndexOf("-"); - if (lastIndex == -1) { - lastIndex = storageGroup.length(); - } - MetricService.getInstance() - .gaugeWithInternalReportAsync( - memTable.getTotalPointsNum(), - Metric.POINTS.toString(), - MetricLevel.CORE, - Tag.DATABASE.toString(), - storageGroup.substring(0, lastIndex), - Tag.TYPE.toString(), - "flush", - Tag.REGION.toString(), - dataRegionId); - } - - LOGGER.info( - "Database {}, flushing memtable {} into disk: Encoding data cost " + "{} ms.", - storageGroup, - writer.getFile().getName(), - memSerializeTime); - WRITING_METRICS.recordFlushCost(WritingMetrics.FLUSH_STAGE_ENCODING, memSerializeTime); - } - }; - - /** io task (third task of pipeline) */ - @SuppressWarnings("squid:S135") - private Runnable ioTask = - () -> { - LOGGER.debug( - "Database {} memtable flushing to file {} start io.", - storageGroup, - writer.getFile().getName()); - while (true) { - Object ioMessage = null; - try { - ioMessage = ioTaskQueue.take(); - } catch (InterruptedException e1) { - LOGGER.error("take task from ioTaskQueue Interrupted"); - Thread.currentThread().interrupt(); - break; - } - long starTime = System.currentTimeMillis(); - try { - if (ioMessage instanceof StartFlushGroupIOTask) { - this.writer.startChunkGroup(((StartFlushGroupIOTask) ioMessage).deviceId); - } else if (ioMessage instanceof TaskEnd) { - break; - } else if (ioMessage instanceof EndChunkGroupIoTask) { - this.writer.setMinPlanIndex(memTable.getMinPlanIndex()); - this.writer.setMaxPlanIndex(memTable.getMaxPlanIndex()); - this.writer.endChunkGroup(); - } else { - ((IChunkWriter) ioMessage).writeToFileWriter(this.writer); - } - } catch (IOException e) { - LOGGER.error( - "Database {} memtable {}, io task meets error.", storageGroup, memTable, e); - throw new FlushRunTimeException(e); - } - long subTaskTime = System.currentTimeMillis() - starTime; - ioTime += subTaskTime; - WRITING_METRICS.recordFlushSubTaskCost(WritingMetrics.IO_TASK, subTaskTime); - } - LOGGER.debug( - "flushing a memtable to file {} in database {}, io cost {}ms", - writer.getFile().getName(), - storageGroup, - ioTime); - WRITING_METRICS.recordFlushTsFileSize(storageGroup, writer.getFile().length()); - WRITING_METRICS.recordFlushCost(WritingMetrics.FLUSH_STAGE_IO, ioTime); - }; - - static class TaskEnd { + protected void metricFlush() { + if (!storageGroup.startsWith(IoTDBMetricsUtils.DATABASE)) { + int lastIndex = storageGroup.lastIndexOf("-"); + if (lastIndex == -1) { + lastIndex = storageGroup.length(); + } + MetricService.getInstance() + .gaugeWithInternalReportAsync( + memTable.getTotalPointsNum(), + Metric.POINTS.toString(), + MetricLevel.CORE, + Tag.DATABASE.toString(), + storageGroup.substring(0, lastIndex), + Tag.TYPE.toString(), + "flush", + Tag.REGION.toString(), + dataRegionId); + } + } - TaskEnd() {} + private DynamicThread newSortThread() { + return new TaskRunner( + sortTasks, this::cleanSortThread, sortTaskQueue, encodingTaskQueue, taskName + "-sort"); } - static class EndChunkGroupIoTask { + private DynamicThread newEncodingThread() { + return new TaskRunner( + encodingTasks, + this::cleanEncodingThread, + encodingTaskQueue, + ioTaskQueue, + taskName + "-encode"); + } - EndChunkGroupIoTask() {} + private DynamicThread newIOThread() { + return new TaskRunner(null, this::cleanIOThread, ioTaskQueue, ioTaskQueue, taskName + "-io"); } - static class StartFlushGroupIOTask { + private void cleanSortThread() { + metricFlush(); + WRITING_METRICS.recordFlushCost( + WritingMetrics.FLUSH_STAGE_SORT, allContext.getSortTime().get()); + } - private final String deviceId; + private void cleanEncodingThread() { + metricFlush(); + WRITING_METRICS.recordFlushCost( + WritingMetrics.FLUSH_STAGE_ENCODING, allContext.getEncodingTime().get()); + } - StartFlushGroupIOTask(String deviceId) { - this.deviceId = deviceId; - } + private void cleanIOThread() { + metricFlush(); + WRITING_METRICS.recordFlushCost(WritingMetrics.FLUSH_STAGE_IO, allContext.getIoTime().get()); + WRITING_METRICS.recordFlushTsFileSize(storageGroup, allContext.getWriter().getFile().length()); } Review Comment: why call` metricFlush() `after each task pipeline? ########## node-commons/src/main/java/org/apache/iotdb/commons/concurrent/pipeline/TaskRunner.java: ########## @@ -0,0 +1,82 @@ +/* + * 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.commons.concurrent.pipeline; + +import org.apache.iotdb.commons.concurrent.dynamic.DynamicThread; +import org.apache.iotdb.commons.concurrent.dynamic.DynamicThreadGroup; + +import java.util.concurrent.BlockingQueue; + +/** + * A thread that will continuously take tasks from an input queue, run the task, and insert the next + * task to the output queue. By connecting multiple TaskRunners with input queues and output queues, + * a pipeline is formed. + */ +public class TaskRunner extends DynamicThread { + + private Runnable cleanUp; + private BlockingQueue<Task> input; + private BlockingQueue<Task> output; + private String taskName; + + public TaskRunner( + DynamicThreadGroup threadGroup, + Runnable cleanUp, + BlockingQueue<Task> input, + BlockingQueue<Task> output, + String taskName) { + super(threadGroup); + this.cleanUp = cleanUp; + this.input = input; + this.output = output; + this.taskName = taskName; + } + + @Override + public void runInternal() { + String baseName = Thread.currentThread().getName(); + Thread.currentThread().setName(baseName + "-" + taskName); + try { + while (!Thread.interrupted()) { + Task task; + try { + task = input.take(); + idleToRunning(); + task.run(); + Task nextTask = task.nextTask(); + if (nextTask != null) { + output.put(nextTask); + } + runningToIdle(); + + if (shouldExit()) { + break; + } + } catch (InterruptedException e1) { + Thread.currentThread().interrupt(); + break; + } + } + + cleanUp.run(); + } finally { + Thread.currentThread().setName(baseName); + } + } +} Review Comment: If the task in the input queue is completely fetched, it becomes blocked, and the thread is blocked in `input.take()`, so it cannot calculate the idle ratio and cannot exit. If there are many tasks, then the thread should not be idle. So in actual operation, the scenario where threads automatically exit mainly occurs when the production speed of producers cannot keep up with the consumption speed of consumers? For example, the speed of sort task generation cannot keep up with the speed of sort task processing; The speed of encoding task production cannot keep up with the speed of encoding task consumption? -- 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]
