From b7e7dad750707f1b5105f35263f06627b7f33ea8 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 26 Jun 2025 09:41:46 +0800 Subject: [PATCH 001/185] partial Update PipeDataNodeRuntimeAgent.java Update PipeTerminateEvent.java Update PipeTerminateEvent.java partially Update AbstractOperatePipeProcedureV2.java partial Update PipeTaskAgent.java Update DataNodeInternalRPCServiceImpl.java schema pool PR Revert report-interval --- .../org/apache/iotdb/rpc/TSStatusCode.java | 1 + .../agent/task/PipeConfigNodeTaskAgent.java | 8 + .../pipe/AbstractOperatePipeProcedureV2.java | 18 +- .../org/apache/iotdb/db/conf/IoTDBConfig.java | 30 ++-- .../apache/iotdb/db/conf/IoTDBDescriptor.java | 14 +- .../runtime/PipeDataNodeRuntimeAgent.java | 5 - .../agent/task/PipeDataNodeTaskAgent.java | 47 ++++- .../common/terminate/PipeTerminateEvent.java | 44 ----- .../resource/memory/PipeMemoryWeightUtil.java | 12 +- .../impl/DataNodeInternalRPCServiceImpl.java | 95 ++++++++-- ...tementDataTypeConvertExecutionVisitor.java | 169 +++++------------- .../LoadTsFileDataTypeConverter.java | 42 ++--- .../iotdb/commons/concurrent/ThreadName.java | 4 +- .../iotdb/commons/conf/CommonConfig.java | 2 +- .../pipe/agent/task/PipeTaskAgent.java | 26 ++- 15 files changed, 260 insertions(+), 257 deletions(-) diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java index 28569ebd7f2b..22bdeb3114e7 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java @@ -259,6 +259,7 @@ public enum TSStatusCode { PIPE_RECEIVER_USER_CONFLICT_EXCEPTION(1810), PIPE_CONFIG_RECEIVER_HANDSHAKE_NEEDED(1811), PIPE_TRANSFER_SLICE_OUT_OF_ORDER(1812), + PIPE_PUSH_META_TIMEOUT(1813), // Subscription SUBSCRIPTION_VERSION_ERROR(1900), diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/agent/task/PipeConfigNodeTaskAgent.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/agent/task/PipeConfigNodeTaskAgent.java index 0e90174a7f2c..bf145085c134 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/agent/task/PipeConfigNodeTaskAgent.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/agent/task/PipeConfigNodeTaskAgent.java @@ -46,11 +46,13 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; import java.util.stream.Collectors; public class PipeConfigNodeTaskAgent extends PipeTaskAgent { @@ -250,4 +252,10 @@ protected void collectPipeMetaListInternal( resp.setPipeRemainingEventCountList(pipeRemainingEventCountList); resp.setPipeRemainingTimeList(pipeRemainingTimeList); } + + @Override + public void runPipeTasks( + final Collection pipeTasks, final Consumer runSingle) { + pipeTasks.forEach(runSingle); + } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/AbstractOperatePipeProcedureV2.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/AbstractOperatePipeProcedureV2.java index 3ab75270b0bb..eb4ffc48c791 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/AbstractOperatePipeProcedureV2.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/AbstractOperatePipeProcedureV2.java @@ -422,12 +422,20 @@ public static Map pushPipeMetaToDataNodes( * @return Error messages for the given pipe after pushing pipe meta */ public static String parsePushPipeMetaExceptionForPipe( - String pipeName, Map respMap) { + final String pipeName, final Map respMap) { final StringBuilder exceptionMessageBuilder = new StringBuilder(); - for (Map.Entry respEntry : respMap.entrySet()) { - int dataNodeId = respEntry.getKey(); - TPushPipeMetaResp resp = respEntry.getValue(); + for (final Map.Entry respEntry : respMap.entrySet()) { + final int dataNodeId = respEntry.getKey(); + final TPushPipeMetaResp resp = respEntry.getValue(); + + if (resp.getStatus().getCode() == TSStatusCode.PIPE_PUSH_META_TIMEOUT.getStatusCode()) { + exceptionMessageBuilder.append( + String.format( + "DataNodeId: %s, Message: Timeout to wait for lock while processing pushPipeMeta on dataNodes.", + dataNodeId)); + continue; + } if (resp.getStatus().getCode() == TSStatusCode.PIPE_PUSH_META_ERROR.getStatusCode()) { if (!resp.isSetExceptionMessages()) { @@ -438,7 +446,7 @@ public static String parsePushPipeMetaExceptionForPipe( continue; } - AtomicBoolean hasException = new AtomicBoolean(false); + final AtomicBoolean hasException = new AtomicBoolean(false); resp.getExceptionMessages() .forEach( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java index d429afe529d0..2555931c3695 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java @@ -314,6 +314,8 @@ public class IoTDBConfig { private String extPipeDir = IoTDBConstant.EXT_FOLDER_NAME + File.separator + IoTDBConstant.EXT_PIPE_FOLDER_NAME; + private int pipeTaskThreadCount = 5; + /** External lib directory for MQTT, stores user-uploaded JAR files */ private String mqttDir = IoTDBConstant.EXT_FOLDER_NAME + File.separator + IoTDBConstant.MQTT_FOLDER_NAME; @@ -1046,6 +1048,8 @@ public class IoTDBConfig { /** Policy of DataNodeSchemaCache eviction */ private String dataNodeSchemaCacheEvictionPolicy = "FIFO"; + private int schemaThreadCount = 5; + private String readConsistencyLevel = "strong"; /** Maximum execution time of a DriverTask */ @@ -1147,8 +1151,6 @@ public class IoTDBConfig { private long loadTsFileTabletConversionBatchMemorySizeInBytes = 4096 * 1024; - private int loadTsFileTabletConversionThreadCount = 5; - private long loadChunkMetadataMemorySizeInBytes = 33554432; // 32MB private long loadMemoryAllocateRetryIntervalMs = 1000L; @@ -3345,6 +3347,14 @@ public void setExtPipeDir(String extPipeDir) { this.extPipeDir = extPipeDir; } + public int getPipeTaskThreadCount() { + return pipeTaskThreadCount; + } + + public void setPipeTaskThreadCount(int pipeTaskThreadCount) { + this.pipeTaskThreadCount = pipeTaskThreadCount; + } + public void setPartitionCacheSize(int partitionCacheSize) { this.partitionCacheSize = partitionCacheSize; } @@ -3481,6 +3491,14 @@ public void setDataNodeSchemaCacheEvictionPolicy(String dataNodeSchemaCacheEvict this.dataNodeSchemaCacheEvictionPolicy = dataNodeSchemaCacheEvictionPolicy; } + public int getSchemaThreadCount() { + return schemaThreadCount; + } + + public void setSchemaThreadCount(int schemaThreadCount) { + this.schemaThreadCount = schemaThreadCount; + } + public String getReadConsistencyLevel() { return readConsistencyLevel; } @@ -4026,14 +4044,6 @@ public void setLoadTsFileTabletConversionBatchMemorySizeInBytes( loadTsFileTabletConversionBatchMemorySizeInBytes; } - public int getLoadTsFileTabletConversionThreadCount() { - return loadTsFileTabletConversionThreadCount; - } - - public void setLoadTsFileTabletConversionThreadCount(int loadTsFileTabletConversionThreadCount) { - this.loadTsFileTabletConversionThreadCount = loadTsFileTabletConversionThreadCount; - } - public long getLoadChunkMetadataMemorySizeInBytes() { return loadChunkMetadataMemorySizeInBytes; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java index 7daa0e726c39..2d5e5cbcc544 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java @@ -955,6 +955,10 @@ public void loadProperties(TrimProperties properties) throws BadNodeUrlException } conf.setExtPipeDir(properties.getProperty("ext_pipe_dir", conf.getExtPipeDir()).trim()); + conf.setPipeTaskThreadCount( + Integer.parseInt( + properties.getProperty( + "pipe_task_thread_count", Integer.toString(conf.getPipeTaskThreadCount()).trim()))); // At the same time, set TSFileConfig List fsTypes = new ArrayList<>(); @@ -1091,6 +1095,11 @@ public void loadProperties(TrimProperties properties) throws BadNodeUrlException properties.getProperty( "datanode_schema_cache_eviction_policy", conf.getDataNodeSchemaCacheEvictionPolicy())); + conf.setSchemaThreadCount( + Integer.parseInt( + properties.getProperty( + "schema_thread_count", Integer.toString(conf.getSchemaThreadCount())))); + loadIoTConsensusProps(properties); loadPipeConsensusProps(properties); @@ -2386,11 +2395,6 @@ private void loadLoadTsFileProps(TrimProperties properties) throws IOException { properties.getProperty( "load_tsfile_tablet_conversion_batch_memory_size_in_bytes", String.valueOf(conf.getLoadTsFileTabletConversionBatchMemorySizeInBytes())))); - conf.setLoadTsFileTabletConversionThreadCount( - Integer.parseInt( - properties.getProperty( - "load_tsfile_tablet_conversion_thread_count", - String.valueOf(conf.getLoadTsFileTabletConversionThreadCount())))); conf.setLoadChunkMetadataMemorySizeInBytes( Long.parseLong( Optional.ofNullable( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java index 5a1f9d8b8456..4fd2a68fe60d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java @@ -35,7 +35,6 @@ import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; -import org.apache.iotdb.db.pipe.event.common.terminate.PipeTerminateEvent; import org.apache.iotdb.db.pipe.extractor.schemaregion.SchemaRegionListeningQueue; import org.apache.iotdb.db.pipe.resource.PipeDataNodeHardlinkOrCopiedFileDirStartupCleaner; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode; @@ -90,10 +89,6 @@ public synchronized void start() throws StartupException { "PipeTaskAgent#restartAllStuckPipes", PipeDataNodeAgent.task()::restartAllStuckPipes, PipeConfig.getInstance().getPipeStuckRestartIntervalSeconds()); - registerPeriodicalJob( - "PipeTaskAgent#flushDataRegionIfNeeded", - PipeTerminateEvent::flushDataRegionIfNeeded, - PipeConfig.getInstance().getPipeFlushAfterLastTerminateSeconds()); pipePeriodicalJobExecutor.start(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java index ef0d4c6b4d97..d427da148490 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java @@ -19,6 +19,10 @@ package org.apache.iotdb.db.pipe.agent.task; +import org.apache.iotdb.commons.concurrent.IoTThreadFactory; +import org.apache.iotdb.commons.concurrent.ThreadName; +import org.apache.iotdb.commons.concurrent.threadpool.WrappedThreadPoolExecutor; +import org.apache.iotdb.commons.conf.CommonDescriptor; import org.apache.iotdb.commons.consensus.DataRegionId; import org.apache.iotdb.commons.consensus.SchemaRegionId; import org.apache.iotdb.commons.consensus.index.ProgressIndex; @@ -78,6 +82,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -86,8 +91,15 @@ import java.util.Objects; import java.util.Optional; import java.util.Set; +import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; import java.util.stream.Collectors; import java.util.stream.StreamSupport; @@ -102,6 +114,18 @@ public class PipeDataNodeTaskAgent extends PipeTaskAgent { private static final Map PIPE_NAME_TO_LAST_RESTART_TIME_MAP = new ConcurrentHashMap<>(); + private final ExecutorService pipeExecutor = + new WrappedThreadPoolExecutor( + 0, + IoTDBDescriptor.getInstance().getConfig().getPipeTaskThreadCount(), + 0L, + TimeUnit.SECONDS, + new ArrayBlockingQueue<>( + IoTDBDescriptor.getInstance().getConfig().getSchemaThreadCount()), + new IoTThreadFactory(ThreadName.PIPE_PARALLEL_EXECUTION_POOL.getName()), + ThreadName.PIPE_PARALLEL_EXECUTION_POOL.getName(), + new ThreadPoolExecutor.CallerRunsPolicy()); + ////////////////////////// Pipe Task Management Entry ////////////////////////// @Override @@ -379,9 +403,8 @@ public void stopAllPipesWithCriticalException() { ///////////////////////// Heartbeat ///////////////////////// public void collectPipeMetaList(final TDataNodeHeartbeatResp resp) throws TException { - // Try the lock instead of directly acquire it to prevent the block of the cluster heartbeat - // 10s is the half of the HEARTBEAT_TIMEOUT_TIME defined in class BaseNodeCache in ConfigNode - if (!tryReadLockWithTimeOut(10)) { + if (!tryReadLockWithTimeOut( + CommonDescriptor.getInstance().getConfig().getCnConnectionTimeoutInMS())) { return; } try { @@ -817,6 +840,24 @@ public boolean hasPipeReleaseRegionRelatedResource(final int consensusGroupId) { } } + @Override + public void runPipeTasks( + final Collection pipeTasks, final Consumer runSingle) { + final Set> pipeFuture = new HashSet<>(); + + pipeTasks.forEach( + pipeTask -> pipeFuture.add(pipeExecutor.submit(() -> runSingle.accept(pipeTask)))); + + for (final Future future : pipeFuture) { + try { + future.get(); + } catch (final ExecutionException | InterruptedException e) { + LOGGER.warn("Exception occurs when executing pipe task: ", e); + throw new PipeException(e.toString()); + } + } + } + ///////////////////////// Pipe Consensus ///////////////////////// public ProgressIndex getPipeTaskProgressIndex(final String pipeName, final int consensusGroupId) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/terminate/PipeTerminateEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/terminate/PipeTerminateEvent.java index ca78dd7d50ad..91d38cf3361d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/terminate/PipeTerminateEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/terminate/PipeTerminateEvent.java @@ -19,23 +19,16 @@ package org.apache.iotdb.db.pipe.event.common.terminate; -import org.apache.iotdb.common.rpc.thrift.TFlushReq; import org.apache.iotdb.commons.consensus.index.ProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; import org.apache.iotdb.commons.pipe.agent.task.meta.PipeTaskMeta; -import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.datastructure.pattern.PipePattern; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; import org.apache.iotdb.db.pipe.agent.task.PipeDataNodeTask; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; -import org.apache.iotdb.db.storageengine.StorageEngine; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.atomic.AtomicLong; /** * The {@link PipeTerminateEvent} is an {@link EnrichedEvent} that controls the termination of pipe, @@ -45,40 +38,6 @@ */ public class PipeTerminateEvent extends EnrichedEvent { - private static final Logger LOGGER = LoggerFactory.getLogger(PipeTerminateEvent.class); - - private static final AtomicLong PROGRESS_REPORT_COUNT = new AtomicLong(0); - private static final AtomicLong LAST_PROGRESS_REPORT_TIME = new AtomicLong(0); - - public static void flushDataRegionIfNeeded() { - if (PROGRESS_REPORT_COUNT.get() > 0 - && PROGRESS_REPORT_COUNT.get() - > PipeConfig.getInstance().getPipeFlushAfterTerminateCount()) { - flushDataRegion(); - return; - } - - if (LAST_PROGRESS_REPORT_TIME.get() > 0 - && System.currentTimeMillis() - LAST_PROGRESS_REPORT_TIME.get() - > PipeConfig.getInstance().getPipeFlushAfterLastTerminateSeconds() * 1000L) { - flushDataRegion(); - } - } - - private static void flushDataRegion() { - try { - StorageEngine.getInstance().operateFlush(new TFlushReq()); - PROGRESS_REPORT_COUNT.set(0); - LAST_PROGRESS_REPORT_TIME.set(0); - LOGGER.info("Force flush all data regions because of last progress report time."); - } catch (final Exception e) { - LOGGER.warn( - "Failed to flush all data regions, please check the error message: {}", - e.getMessage(), - e); - } - } - private final int dataRegionId; public PipeTerminateEvent( @@ -135,9 +94,6 @@ public boolean mayEventPathsOverlappedWithPattern() { @Override public void reportProgress() { - PROGRESS_REPORT_COUNT.incrementAndGet(); - LAST_PROGRESS_REPORT_TIME.set(System.currentTimeMillis()); - // To avoid deadlock CompletableFuture.runAsync( () -> PipeDataNodeAgent.task().markCompleted(pipeName, dataRegionId)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryWeightUtil.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryWeightUtil.java index cd9a0a79cae0..d82c26144d6c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryWeightUtil.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryWeightUtil.java @@ -247,8 +247,8 @@ public static long calculateTabletSizeInBytes(Tablet tablet) { return totalSizeInBytes; } - public static long calculateBatchDataRamBytesUsed(BatchData batchData) { - long totalSizeInBytes = 0; + public static int calculateBatchDataRamBytesUsed(BatchData batchData) { + int totalSizeInBytes = 0; // timestamp totalSizeInBytes += 8; @@ -263,16 +263,16 @@ public static long calculateBatchDataRamBytesUsed(BatchData batchData) { continue; } // consider variable references (plus 8) and memory alignment (round up to 8) - totalSizeInBytes += roundUpToMultiple(primitiveType.getSize() + 8L, 8); + totalSizeInBytes += roundUpToMultiple(primitiveType.getSize() + 8, 8); } } else { if (type.isBinary()) { final Binary binary = batchData.getBinary(); // refer to org.apache.tsfile.utils.TsPrimitiveType.TsBinary.getSize totalSizeInBytes += - roundUpToMultiple((binary == null ? 8 : binary.ramBytesUsed()) + 8L, 8); + roundUpToMultiple((binary == null ? 8 : binary.getLength() + 8) + 8, 8); } else { - totalSizeInBytes += roundUpToMultiple(TsPrimitiveType.getByType(type).getSize() + 8L, 8); + totalSizeInBytes += roundUpToMultiple(TsPrimitiveType.getByType(type).getSize() + 8, 8); } } } @@ -287,7 +287,7 @@ public static long calculateBatchDataRamBytesUsed(BatchData batchData) { * @param n The specified multiple. * @return The nearest multiple of n greater than or equal to num. */ - private static long roundUpToMultiple(long num, int n) { + private static int roundUpToMultiple(int num, int n) { if (n == 0) { throw new IllegalArgumentException("The multiple n must be greater than 0"); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java index 7732814c1093..2927980d0037 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java @@ -40,6 +40,9 @@ import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot; import org.apache.iotdb.commons.client.request.AsyncRequestContext; import org.apache.iotdb.commons.cluster.NodeStatus; +import org.apache.iotdb.commons.concurrent.IoTThreadFactory; +import org.apache.iotdb.commons.concurrent.ThreadName; +import org.apache.iotdb.commons.concurrent.threadpool.WrappedThreadPoolExecutor; import org.apache.iotdb.commons.conf.CommonConfig; import org.apache.iotdb.commons.conf.CommonDescriptor; import org.apache.iotdb.commons.conf.ConfigurationFileUtils; @@ -271,9 +274,17 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -315,6 +326,18 @@ public class DataNodeInternalRPCServiceImpl implements IDataNodeRPCService.Iface private final CommonConfig commonConfig = CommonDescriptor.getInstance().getConfig(); + private final ExecutorService schemaExecutor = + new WrappedThreadPoolExecutor( + 0, + IoTDBDescriptor.getInstance().getConfig().getSchemaThreadCount(), + 0L, + TimeUnit.SECONDS, + new ArrayBlockingQueue<>( + IoTDBDescriptor.getInstance().getConfig().getSchemaThreadCount()), + new IoTThreadFactory(ThreadName.SCHEMA_PARALLEL_POOL.getName()), + ThreadName.SCHEMA_PARALLEL_POOL.getName(), + new ThreadPoolExecutor.CallerRunsPolicy()); + private static final String SYSTEM = "system"; public DataNodeInternalRPCServiceImpl() { @@ -1070,6 +1093,10 @@ public TPushPipeMetaResp pushPipeMeta(TPushPipeMetaReq req) { .map(PipeMeta::deserialize4TaskAgent) .collect(Collectors.toList())); + if (Objects.isNull(exceptionMessages)) { + return new TPushPipeMetaResp() + .setStatus(new TSStatus(TSStatusCode.PIPE_PUSH_META_TIMEOUT.getStatusCode())); + } return exceptionMessages.isEmpty() ? new TPushPipeMetaResp() .setStatus(new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode())) @@ -1321,16 +1348,31 @@ private TSStatus executeSchemaBlackListTask( final List statusList = Collections.synchronizedList(new ArrayList<>()); final AtomicBoolean hasFailure = new AtomicBoolean(false); - consensusGroupIdList.parallelStream() - .forEach( - consensusGroupId -> { - final TSStatus status = executeOnOneRegion.apply(consensusGroupId); - if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode() - && status.getCode() != TSStatusCode.ONLY_LOGICAL_VIEW.getStatusCode()) { - hasFailure.set(true); - } - statusList.add(status); - }); + final Set> schemaFuture = new HashSet<>(); + + consensusGroupIdList.forEach( + consensusGroupId -> + schemaFuture.add( + schemaExecutor.submit( + () -> { + final TSStatus status = executeOnOneRegion.apply(consensusGroupId); + if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode() + && status.getCode() != TSStatusCode.ONLY_LOGICAL_VIEW.getStatusCode()) { + hasFailure.set(true); + } + statusList.add(status); + }))); + + for (final Future future : schemaFuture) { + try { + future.get(); + } catch (final ExecutionException | InterruptedException e) { + LOGGER.warn("Exception occurs when executing internal schema task: ", e); + statusList.add( + new TSStatus(TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()) + .setMessage(e.toString())); + } + } if (hasFailure.get()) { return RpcUtils.getStatus(statusList); @@ -1349,15 +1391,30 @@ private TSStatus executeInternalSchemaTask( final List statusList = Collections.synchronizedList(new ArrayList<>()); final AtomicBoolean hasFailure = new AtomicBoolean(false); - consensusGroupIdList.parallelStream() - .forEach( - consensusGroupId -> { - final TSStatus status = executeOnOneRegion.apply(consensusGroupId); - if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - hasFailure.set(true); - } - statusList.add(status); - }); + final Set> schemaFuture = new HashSet<>(); + + consensusGroupIdList.forEach( + consensusGroupId -> + schemaFuture.add( + schemaExecutor.submit( + () -> { + final TSStatus status = executeOnOneRegion.apply(consensusGroupId); + if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + hasFailure.set(true); + } + statusList.add(status); + }))); + + for (final Future future : schemaFuture) { + try { + future.get(); + } catch (final ExecutionException | InterruptedException e) { + LOGGER.warn("Exception occurs when executing internal schema task: ", e); + statusList.add( + new TSStatus(TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()) + .setMessage(e.toString())); + } + } if (hasFailure.get()) { return RpcUtils.getStatus(statusList); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/converter/LoadTreeStatementDataTypeConvertExecutionVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/converter/LoadTreeStatementDataTypeConvertExecutionVisitor.java index 7dfe91ad62e3..cb147f9c2861 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/converter/LoadTreeStatementDataTypeConvertExecutionVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/converter/LoadTreeStatementDataTypeConvertExecutionVisitor.java @@ -20,9 +20,6 @@ package org.apache.iotdb.db.storageengine.load.converter; import org.apache.iotdb.common.rpc.thrift.TSStatus; -import org.apache.iotdb.commons.concurrent.IoTThreadFactory; -import org.apache.iotdb.commons.concurrent.ThreadName; -import org.apache.iotdb.commons.concurrent.threadpool.WrappedThreadPoolExecutor; import org.apache.iotdb.commons.pipe.datastructure.pattern.IoTDBPipePattern; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferTabletRawReq; @@ -48,14 +45,6 @@ import java.util.ArrayList; import java.util.List; import java.util.Optional; -import java.util.concurrent.ArrayBlockingQueue; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; -import java.util.concurrent.RejectedExecutionException; -import java.util.concurrent.RejectedExecutionHandler; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; import static org.apache.iotdb.db.pipe.resource.memory.PipeMemoryWeightUtil.calculateTabletSizeInBytes; @@ -70,9 +59,6 @@ public class LoadTreeStatementDataTypeConvertExecutionVisitor .getConfig() .getLoadTsFileTabletConversionBatchMemorySizeInBytes(); - private static final AtomicReference executorPool = - new AtomicReference<>(); - private final StatementExecutor statementExecutor; @FunctionalInterface @@ -80,21 +66,6 @@ public interface StatementExecutor { TSStatus execute(final Statement statement); } - public static class CallerBlocksPolicy implements RejectedExecutionHandler { - public CallerBlocksPolicy() {} - - public void rejectedExecution(Runnable r, ThreadPoolExecutor e) { - if (!e.isShutdown()) { - try { - e.getQueue().put(r); - } catch (InterruptedException ie) { - Thread.currentThread().interrupt(); - throw new RejectedExecutionException("task " + r + " rejected from " + e, ie); - } - } - } - } - public LoadTreeStatementDataTypeConvertExecutionVisitor( final StatementExecutor statementExecutor) { this.statementExecutor = statementExecutor; @@ -118,7 +89,6 @@ public Optional visitLoadFile( final List tabletRawReqSizes = new ArrayList<>(); try { - final List> executionFutures = new ArrayList<>(); for (final File file : loadTsFileStatement.getTsFiles()) { try (final TsFileInsertionScanDataContainer container = new TsFileInsertionScanDataContainer( @@ -136,16 +106,9 @@ file, new IoTDBPipePattern(null), Long.MIN_VALUE, Long.MAX_VALUE, null, null)) { continue; } - final InsertMultiTabletsStatement batchStatement = new InsertMultiTabletsStatement(); - batchStatement.setInsertTabletStatementList( - tabletRawReqs.stream() - .map( - req -> - new LoadConvertedInsertTabletStatement( - req.constructStatement(), - loadTsFileStatement.isConvertOnTypeMismatch())) - .collect(Collectors.toList())); - executionFutures.add(executeInsertMultiTabletsWithRetry(batchStatement)); + final TSStatus result = + executeInsertMultiTabletsWithRetry( + tabletRawReqs, loadTsFileStatement.isConvertOnTypeMismatch()); for (final long memoryCost : tabletRawReqSizes) { block.reduceMemoryUsage(memoryCost); @@ -153,6 +116,10 @@ file, new IoTDBPipePattern(null), Long.MIN_VALUE, Long.MAX_VALUE, null, null)) { tabletRawReqs.clear(); tabletRawReqSizes.clear(); + if (!handleTSStatus(result, loadTsFileStatement)) { + return Optional.empty(); + } + tabletRawReqs.add(tabletRawReq); tabletRawReqSizes.add(curMemory); block.addMemoryUsage(curMemory); @@ -166,36 +133,22 @@ file, new IoTDBPipePattern(null), Long.MIN_VALUE, Long.MAX_VALUE, null, null)) { if (!tabletRawReqs.isEmpty()) { try { - final InsertMultiTabletsStatement batchStatement = new InsertMultiTabletsStatement(); - batchStatement.setInsertTabletStatementList( - tabletRawReqs.stream() - .map( - req -> - new LoadConvertedInsertTabletStatement( - req.constructStatement(), - loadTsFileStatement.isConvertOnTypeMismatch())) - .collect(Collectors.toList())); - executionFutures.add(executeInsertMultiTabletsWithRetry(batchStatement)); + final TSStatus result = + executeInsertMultiTabletsWithRetry( + tabletRawReqs, loadTsFileStatement.isConvertOnTypeMismatch()); for (final long memoryCost : tabletRawReqSizes) { block.reduceMemoryUsage(memoryCost); } tabletRawReqs.clear(); tabletRawReqSizes.clear(); - } catch (final Exception e) { - LOGGER.warn( - "Failed to convert data type for LoadTsFileStatement: {}.", loadTsFileStatement, e); - return Optional.empty(); - } - } - for (final Future future : executionFutures) { - try { - if (!handleTSStatus(future.get(), loadTsFileStatement)) { + if (!handleTSStatus(result, loadTsFileStatement)) { return Optional.empty(); } - } catch (ExecutionException | InterruptedException e) { - LOGGER.warn("Exception occurs when executing insertion during tablet conversion: ", e); + } catch (final Exception e) { + LOGGER.warn( + "Failed to convert data type for LoadTsFileStatement: {}.", loadTsFileStatement, e); return Optional.empty(); } } @@ -226,67 +179,43 @@ file, new IoTDBPipePattern(null), Long.MIN_VALUE, Long.MAX_VALUE, null, null)) { return Optional.of(new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode())); } - private Future executeInsertMultiTabletsWithRetry( - final InsertMultiTabletsStatement batchStatement) { - return getExecutorPool() - .submit( - () -> { - TSStatus result; - try { - result = - batchStatement.accept( - LoadTsFileDataTypeConverter.STATEMENT_STATUS_VISITOR, - statementExecutor.execute(batchStatement)); - - // Retry max 5 times if the write process is rejected - for (int i = 0; - i < 5 - && result.getCode() - == TSStatusCode.LOAD_TEMPORARY_UNAVAILABLE_EXCEPTION.getStatusCode(); - i++) { - Thread.sleep(100L * (i + 1)); - result = - batchStatement.accept( - LoadTsFileDataTypeConverter.STATEMENT_STATUS_VISITOR, - statementExecutor.execute(batchStatement)); - } - } catch (final Exception e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - result = - batchStatement.accept( - LoadTsFileDataTypeConverter.STATEMENT_EXCEPTION_VISITOR, e); - } - return result; - }); - } - - public static WrappedThreadPoolExecutor getExecutorPool() { - if (executorPool.get() == null) { - synchronized (executorPool) { - if (executorPool.get() == null) { - executorPool.set( - new WrappedThreadPoolExecutor( - IoTDBDescriptor.getInstance() - .getConfig() - .getLoadTsFileTabletConversionThreadCount(), - IoTDBDescriptor.getInstance() - .getConfig() - .getLoadTsFileTabletConversionThreadCount(), - 0L, - TimeUnit.SECONDS, - new ArrayBlockingQueue<>( - IoTDBDescriptor.getInstance() - .getConfig() - .getLoadTsFileTabletConversionThreadCount()), - new IoTThreadFactory(ThreadName.LOAD_DATATYPE_CONVERT_POOL.getName()), - ThreadName.LOAD_DATATYPE_CONVERT_POOL.getName(), - new CallerBlocksPolicy())); - } + private TSStatus executeInsertMultiTabletsWithRetry( + final List tabletRawReqs, boolean isConvertOnTypeMismatch) { + final InsertMultiTabletsStatement batchStatement = new InsertMultiTabletsStatement(); + batchStatement.setInsertTabletStatementList( + tabletRawReqs.stream() + .map( + req -> + new LoadConvertedInsertTabletStatement( + req.constructStatement(), isConvertOnTypeMismatch)) + .collect(Collectors.toList())); + + TSStatus result; + try { + result = + batchStatement.accept( + LoadTsFileDataTypeConverter.STATEMENT_STATUS_VISITOR, + statementExecutor.execute(batchStatement)); + + // Retry max 5 times if the write process is rejected + for (int i = 0; + i < 5 + && result.getCode() + == TSStatusCode.LOAD_TEMPORARY_UNAVAILABLE_EXCEPTION.getStatusCode(); + i++) { + Thread.sleep(100L * (i + 1)); + result = + batchStatement.accept( + LoadTsFileDataTypeConverter.STATEMENT_STATUS_VISITOR, + statementExecutor.execute(batchStatement)); + } + } catch (final Exception e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); } + result = batchStatement.accept(LoadTsFileDataTypeConverter.STATEMENT_EXCEPTION_VISITOR, e); } - return executorPool.get(); + return result; } private static boolean handleTSStatus( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/converter/LoadTsFileDataTypeConverter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/converter/LoadTsFileDataTypeConverter.java index a46546e3e361..ee2a8fe25475 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/converter/LoadTsFileDataTypeConverter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/converter/LoadTsFileDataTypeConverter.java @@ -20,11 +20,7 @@ package org.apache.iotdb.db.storageengine.load.converter; import org.apache.iotdb.common.rpc.thrift.TSStatus; -import org.apache.iotdb.commons.conf.IoTDBConstant; -import org.apache.iotdb.db.auth.AuthorityChecker; import org.apache.iotdb.db.conf.IoTDBDescriptor; -import org.apache.iotdb.db.protocol.session.IClientSession; -import org.apache.iotdb.db.protocol.session.InternalClientSession; import org.apache.iotdb.db.protocol.session.SessionManager; import org.apache.iotdb.db.queryengine.plan.Coordinator; import org.apache.iotdb.db.queryengine.plan.analyze.ClusterPartitionFetcher; @@ -37,7 +33,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.time.ZoneId; import java.util.Optional; public class LoadTsFileDataTypeConverter { @@ -73,32 +68,17 @@ public Optional convertForTreeModel(final LoadTsFileStatement loadTsFi } private TSStatus executeForTreeModel(final Statement statement) { - final IClientSession session = - new InternalClientSession( - String.format( - "%s_%s", - LoadTsFileDataTypeConverter.class.getSimpleName(), - Thread.currentThread().getName())); - session.setUsername(AuthorityChecker.SUPER_USER); - session.setClientVersion(IoTDBConstant.ClientVersion.V_1_0); - session.setZoneId(ZoneId.systemDefault()); - - SESSION_MANAGER.registerSession(session); - try { - return Coordinator.getInstance() - .executeForTreeModel( - isGeneratedByPipe ? new PipeEnrichedStatement(statement) : statement, - SESSION_MANAGER.requestQueryId(), - SESSION_MANAGER.getSessionInfo(session), - "", - ClusterPartitionFetcher.getInstance(), - ClusterSchemaFetcher.getInstance(), - IoTDBDescriptor.getInstance().getConfig().getQueryTimeoutThreshold(), - false) - .status; - } finally { - SESSION_MANAGER.removeCurrSession(); - } + return Coordinator.getInstance() + .executeForTreeModel( + isGeneratedByPipe ? new PipeEnrichedStatement(statement) : statement, + SESSION_MANAGER.requestQueryId(), + SESSION_MANAGER.getSessionInfo(SESSION_MANAGER.getCurrSession()), + "", + ClusterPartitionFetcher.getInstance(), + ClusterSchemaFetcher.getInstance(), + IoTDBDescriptor.getInstance().getConfig().getQueryTimeoutThreshold(), + false) + .status; } public boolean isSuccessful(final TSStatus status) { diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java index 4cdb24cd328d..f5b853e590c7 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java @@ -68,6 +68,8 @@ public enum ThreadName { PBTREE_RELEASE_MONITOR("PBTree-Release-Task-Monitor"), PBTREE_FLUSH_MONITOR("PBTree-Flush-Monitor"), PBTREE_WORKER_POOL("PBTree-Worker-Pool"), + SCHEMA_PARALLEL_POOL("Schema-Parallel-Pool"), + // -------------------------- ClientService -------------------------- CLIENT_RPC_SERVICE("ClientRPC-Service"), CLIENT_RPC_PROCESSOR("ClientRPC-Processor"), @@ -142,7 +144,7 @@ public enum ThreadName { PIPE_RECEIVER_AIR_GAP_AGENT("Pipe-Receiver-Air-Gap-Agent"), PIPE_AIR_GAP_RECEIVER("Pipe-Air-Gap-Receiver"), PIPE_PROGRESS_INDEX_BACKGROUND_SERVICE("Pipe-Progress-Index-Background-Service"), - LOAD_DATATYPE_CONVERT_POOL("Load-Datatype-Convert-Pool"), + PIPE_PARALLEL_EXECUTION_POOL("Pipe-Parallel-Execution-Pool"), SUBSCRIPTION_EXECUTOR_POOL("Subscription-Executor-Pool"), SUBSCRIPTION_RUNTIME_META_SYNCER("Subscription-Runtime-Meta-Syncer"), WINDOW_EVALUATION_SERVICE("WindowEvaluationTaskPoolManager"), diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java index 59d348abe5ca..10fe9d9ebb7f 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java @@ -261,7 +261,7 @@ public class CommonConfig { (int) (RpcUtils.THRIFT_FRAME_MAX_SIZE * 0.8); private boolean isSeperatedPipeHeartbeatEnabled = true; - private int pipeHeartbeatIntervalSecondsForCollectingPipeMeta = 30; + private int pipeHeartbeatIntervalSecondsForCollectingPipeMeta = 3; private long pipeMetaSyncerInitialSyncDelayMinutes = 3; private long pipeMetaSyncerSyncIntervalMinutes = 3; private long pipeMetaSyncerAutoRestartPipeCheckIntervalRound = 1; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java index aeafccc21961..031551753b2e 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java @@ -19,6 +19,7 @@ package org.apache.iotdb.commons.pipe.agent.task; +import org.apache.iotdb.commons.conf.CommonDescriptor; import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeConnectorCriticalException; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeCriticalException; @@ -43,6 +44,7 @@ import org.slf4j.LoggerFactory; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; @@ -54,6 +56,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; import java.util.stream.Collectors; /** @@ -335,7 +338,10 @@ protected TPushPipeMetaRespExceptionMessage handleDropPipeInternal(final String public List handlePipeMetaChanges( final List pipeMetaListFromCoordinator) { - acquireWriteLock(); + if (tryWriteLockWithTimeOut( + CommonDescriptor.getInstance().getConfig().getDnConnectionTimeoutInMS() * 2L / 3)) { + return null; + } try { return handlePipeMetaChangesInternal(pipeMetaListFromCoordinator); } finally { @@ -473,7 +479,7 @@ private boolean createPipe(final PipeMeta pipeMetaFromCoordinator) throws Illega // Trigger create() method for each pipe task by parallel stream final long startTime = System.currentTimeMillis(); - pipeTasks.values().parallelStream().forEach(PipeTask::create); + runPipeTasks(pipeTasks.values(), PipeTask::create); LOGGER.info( "Create all pipe tasks on Pipe {} successfully within {} ms", pipeName, @@ -528,7 +534,7 @@ protected boolean dropPipe(final String pipeName, final long creationTime) { // Trigger drop() method for each pipe task by parallel stream final long startTime = System.currentTimeMillis(); - pipeTasks.values().parallelStream().forEach(PipeTask::drop); + runPipeTasks(pipeTasks.values(), PipeTask::drop); LOGGER.info( "Drop all pipe tasks on Pipe {} successfully within {} ms", pipeName, @@ -567,7 +573,7 @@ protected boolean dropPipe(final String pipeName) { // Trigger drop() method for each pipe task by parallel stream final long startTime = System.currentTimeMillis(); - pipeTasks.values().parallelStream().forEach(PipeTask::drop); + runPipeTasks(pipeTasks.values(), PipeTask::drop); LOGGER.info( "Drop all pipe tasks on Pipe {} successfully within {} ms", pipeName, @@ -600,7 +606,7 @@ protected void startPipe(final String pipeName, final long creationTime) { // Trigger start() method for each pipe task by parallel stream final long startTime = System.currentTimeMillis(); - pipeTasks.values().parallelStream().forEach(PipeTask::start); + runPipeTasks(pipeTasks.values(), PipeTask::start); LOGGER.info( "Start all pipe tasks on Pipe {} successfully within {} ms", pipeName, @@ -639,7 +645,7 @@ private void stopPipe(final String pipeName, final long creationTime) { // Trigger stop() method for each pipe task by parallel stream final long startTime = System.currentTimeMillis(); - pipeTasks.values().parallelStream().forEach(PipeTask::stop); + runPipeTasks(pipeTasks.values(), PipeTask::stop); LOGGER.info( "Stop all pipe tasks on Pipe {} successfully within {} ms", pipeName, @@ -1039,7 +1045,10 @@ private void stopAllPipesWithCriticalExceptionInternal(final int currentNodeId) public void collectPipeMetaList(final TPipeHeartbeatReq req, final TPipeHeartbeatResp resp) throws TException { - acquireReadLock(); + if (!tryReadLockWithTimeOut( + CommonDescriptor.getInstance().getConfig().getDnConnectionTimeoutInMS() * 2L / 3)) { + return; + } try { collectPipeMetaListInternal(req, resp); } finally { @@ -1050,6 +1059,9 @@ public void collectPipeMetaList(final TPipeHeartbeatReq req, final TPipeHeartbea protected abstract void collectPipeMetaListInternal( final TPipeHeartbeatReq req, final TPipeHeartbeatResp resp) throws TException; + public abstract void runPipeTasks( + final Collection pipeTasks, final Consumer runSingle); + ///////////////////////// Maintain meta info ///////////////////////// public long getPipeCreationTime(final String pipeName) { From 71f7c88fdd21d014ea1068dbba58815d4073a6ef Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 26 Jun 2025 15:27:17 +0800 Subject: [PATCH 002/185] wal deletion --- .../pipe/agent/runtime/PipeAgentLauncher.java | 2 - .../batch/PipeTabletEventPlainBatch.java | 3 +- .../IoTDBDataRegionAirGapConnector.java | 3 +- .../PipeConsensusAsyncConnector.java | 3 +- .../PipeConsensusSyncConnector.java | 2 +- .../PipeConsensusTransferBatchReqBuilder.java | 3 +- .../async/IoTDBDataRegionAsyncConnector.java | 3 +- .../sync/IoTDBDataRegionSyncConnector.java | 3 +- .../writeback/WriteBackConnector.java | 3 +- .../PipeInsertNodeTabletInsertionEvent.java | 108 +--- .../realtime/PipeRealtimeEventFactory.java | 14 +- .../PipeInsertionDataNodeListener.java | 8 +- .../db/pipe/metric/PipeDataNodeMetrics.java | 3 - .../metric/overview/PipeResourceMetrics.java | 7 - .../PipeWALInsertNodeCacheMetrics.java | 86 --- .../resource/PipeDataNodeResourceManager.java | 22 - .../db/pipe/resource/wal/PipeWALResource.java | 1 - .../resource/wal/PipeWALResourceManager.java | 1 - .../wal/hardlink/PipeWALHardlinkResource.java | 47 -- .../PipeWALHardlinkResourceManager.java | 152 ----- .../wal/selfhost/PipeWALSelfHostResource.java | 1 - .../PipeWALSelfHostResourceManager.java | 38 -- .../dataregion/memtable/TsFileProcessor.java | 15 +- .../dataregion/wal/buffer/WALBuffer.java | 6 - .../dataregion/wal/buffer/WALEntry.java | 4 +- .../wal/checkpoint/MemTableInfo.java | 19 +- .../dataregion/wal/io/WALByteBufReader.java | 1 - .../dataregion/wal/node/WALFakeNode.java | 4 +- .../dataregion/wal/node/WALNode.java | 96 +-- .../dataregion/wal/utils/WALEntryHandler.java | 233 ------- .../wal/utils/WALEntryPosition.java | 229 ------- .../wal/utils/WALInsertNodeCache.java | 335 ---------- .../wal/utils/listener/WALFlushListener.java | 12 +- .../extractor/PipeRealtimeExtractTest.java | 5 - .../PipeWALHardlinkResourceManagerTest.java | 1 - .../wal/checkpoint/CheckpointManagerTest.java | 172 ----- .../wal/node/WALEntryHandlerTest.java | 586 ------------------ .../dataregion/wal/node/WALNodeTest.java | 17 - .../wal/node/WalDeleteOutdatedNewTest.java | 5 - .../wal/utils/WALInsertNodeCacheTest.java | 190 ------ .../commons/service/metric/enums/Metric.java | 1 - 41 files changed, 51 insertions(+), 2393 deletions(-) delete mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeWALInsertNodeCacheMetrics.java delete mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/hardlink/PipeWALHardlinkResource.java delete mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/hardlink/PipeWALHardlinkResourceManager.java delete mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/selfhost/PipeWALSelfHostResourceManager.java delete mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/WALEntryHandler.java delete mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/WALEntryPosition.java delete mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/WALInsertNodeCache.java delete mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/checkpoint/CheckpointManagerTest.java delete mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALEntryHandlerTest.java delete mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/WALInsertNodeCacheTest.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeAgentLauncher.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeAgentLauncher.java index 96e2e4e72994..089d4e2fda72 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeAgentLauncher.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeAgentLauncher.java @@ -36,7 +36,6 @@ import org.apache.iotdb.db.protocol.client.ConfigNodeClientManager; import org.apache.iotdb.db.protocol.client.ConfigNodeInfo; import org.apache.iotdb.db.service.ResourcesInformationHolder; -import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALInsertNodeCache; import org.apache.iotdb.pipe.api.exception.PipeException; import org.apache.iotdb.rpc.TSStatusCode; @@ -161,7 +160,6 @@ public static synchronized void launchPipeTaskAgent() { try (final ConfigNodeClient configNodeClient = ConfigNodeClientManager.getInstance().borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { final TGetAllPipeInfoResp getAllPipeInfoResp = configNodeClient.getAllPipeInfo(); - WALInsertNodeCache.init(); PipeTabletEventBatch.init(); if (getAllPipeInfoResp.getStatus().getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { throw new StartupException("Failed to get pipe task meta from config node."); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventPlainBatch.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventPlainBatch.java index 292fc018ed79..33f228f42fb8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventPlainBatch.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventPlainBatch.java @@ -102,8 +102,7 @@ private int buildTabletInsertionBuffer(final TabletInsertionEvent event) (PipeInsertNodeTabletInsertionEvent) event; // Read the bytebuffer from the wal file and transfer it directly without serializing or // deserializing if possible - final InsertNode insertNode = - pipeInsertNodeTabletInsertionEvent.getInsertNodeViaCacheIfPossible(); + final InsertNode insertNode = pipeInsertNodeTabletInsertionEvent.getInsertNode(); if (Objects.isNull(insertNode)) { buffer = pipeInsertNodeTabletInsertionEvent.getByteBuffer(); binaryBuffers.add(buffer); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataRegionAirGapConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataRegionAirGapConnector.java index ee35ab46306a..bc4805e8c8ef 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataRegionAirGapConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataRegionAirGapConnector.java @@ -168,8 +168,7 @@ private void doTransfer( final AirGapSocket socket, final PipeInsertNodeTabletInsertionEvent pipeInsertNodeTabletInsertionEvent) throws PipeException, WALPipeException, IOException { - final InsertNode insertNode = - pipeInsertNodeTabletInsertionEvent.getInsertNodeViaCacheIfPossible(); + final InsertNode insertNode = pipeInsertNodeTabletInsertionEvent.getInsertNode(); final byte[] bytes = Objects.isNull(insertNode) ? PipeTransferTabletBinaryReq.toTPipeTransferBytes( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java index a74057c75c7f..1756da36db7d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java @@ -277,8 +277,7 @@ public void transfer(TabletInsertionEvent tabletInsertionEvent) throws Exception return; } - final InsertNode insertNode = - pipeInsertNodeTabletInsertionEvent.getInsertNodeViaCacheIfPossible(); + final InsertNode insertNode = pipeInsertNodeTabletInsertionEvent.getInsertNode(); final ProgressIndex progressIndex = pipeInsertNodeTabletInsertionEvent.getProgressIndex(); final TPipeConsensusTransferReq pipeConsensusTransferReq = Objects.isNull(insertNode) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java index f4690783fe13..6186d9f671ac 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java @@ -252,7 +252,7 @@ private void doTransfer(PipeInsertNodeTabletInsertionEvent pipeInsertNodeTabletI try (final SyncPipeConsensusServiceClient syncPipeConsensusServiceClient = syncRetryClientManager.borrowClient(getFollowerUrl())) { - insertNode = pipeInsertNodeTabletInsertionEvent.getInsertNodeViaCacheIfPossible(); + insertNode = pipeInsertNodeTabletInsertionEvent.getInsertNode(); progressIndex = pipeInsertNodeTabletInsertionEvent.getProgressIndex(); if (insertNode != null) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/builder/PipeConsensusTransferBatchReqBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/builder/PipeConsensusTransferBatchReqBuilder.java index 870cea0ee87a..a3f8fe557b7f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/builder/PipeConsensusTransferBatchReqBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/builder/PipeConsensusTransferBatchReqBuilder.java @@ -196,8 +196,7 @@ protected int buildTabletInsertionBuffer(TabletInsertionEvent event) throws WALP // Read the bytebuffer from the wal file and transfer it directly without serializing or // deserializing if possible - final InsertNode insertNode = - pipeInsertNodeTabletInsertionEvent.getInsertNodeViaCacheIfPossible(); + final InsertNode insertNode = pipeInsertNodeTabletInsertionEvent.getInsertNode(); // PipeConsensus will transfer binary data to TPipeConsensusTransferReq final ProgressIndex progressIndex = pipeInsertNodeTabletInsertionEvent.getProgressIndex(); if (Objects.isNull(insertNode)) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java index fd80f47c45f3..62871f039fb9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java @@ -265,8 +265,7 @@ private boolean transferInEventWithoutCheck(final TabletInsertionEvent tabletIns return false; } - final InsertNode insertNode = - pipeInsertNodeTabletInsertionEvent.getInsertNodeViaCacheIfPossible(); + final InsertNode insertNode = pipeInsertNodeTabletInsertionEvent.getInsertNode(); final TPipeTransferReq pipeTransferReq = compressIfNeeded( Objects.isNull(insertNode) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java index 784ee14a55a5..d996cac28208 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java @@ -294,8 +294,7 @@ private void doTransfer( // getDeviceId() may return null for InsertRowsNode, will be equal to getClient(null) clientAndStatus = clientManager.getClient(pipeInsertNodeTabletInsertionEvent.getDeviceId()); - final InsertNode insertNode = - pipeInsertNodeTabletInsertionEvent.getInsertNodeViaCacheIfPossible(); + final InsertNode insertNode = pipeInsertNodeTabletInsertionEvent.getInsertNode(); final TPipeTransferReq req = compressIfNeeded( insertNode != null diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/writeback/WriteBackConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/writeback/WriteBackConnector.java index 7d28f8609ad4..365950c51091 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/writeback/WriteBackConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/writeback/WriteBackConnector.java @@ -129,8 +129,7 @@ private void doTransfer( throws PipeException, WALPipeException { final TSStatus status; - final InsertNode insertNode = - pipeInsertNodeTabletInsertionEvent.getInsertNodeViaCacheIfPossible(); + final InsertNode insertNode = pipeInsertNodeTabletInsertionEvent.getInsertNode(); if (Objects.isNull(insertNode)) { status = PipeDataNodeAgent.receiver() diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java index e7814916e5d7..12d5eb1f5118 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java @@ -37,8 +37,6 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowsNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertTabletNode; import org.apache.iotdb.db.storageengine.dataregion.wal.exception.WALPipeException; -import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALEntryHandler; -import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALEntryPosition; import org.apache.iotdb.pipe.api.access.Row; import org.apache.iotdb.pipe.api.collector.RowCollector; import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; @@ -69,52 +67,26 @@ public class PipeInsertNodeTabletInsertionEvent extends EnrichedEvent LoggerFactory.getLogger(PipeInsertNodeTabletInsertionEvent.class); private static final long INSTANCE_SIZE = RamUsageEstimator.shallowSizeOfInstance(PipeInsertNodeTabletInsertionEvent.class) - + RamUsageEstimator.shallowSizeOfInstance(WALEntryHandler.class) - + RamUsageEstimator.shallowSizeOfInstance(WALEntryPosition.class) + RamUsageEstimator.shallowSizeOfInstance(AtomicInteger.class) + RamUsageEstimator.shallowSizeOfInstance(AtomicBoolean.class); - private final WALEntryHandler walEntryHandler; - private final boolean isAligned; - private final boolean isGeneratedByPipe; - private final AtomicReference allocatedMemoryBlock; private volatile List tablets; private List dataContainers; - private final PartialPath devicePath; + private final InsertNode insertNode; private ProgressIndex progressIndex; private long extractTime = 0; - public PipeInsertNodeTabletInsertionEvent( - final WALEntryHandler walEntryHandler, - final PartialPath devicePath, - final ProgressIndex progressIndex, - final boolean isAligned, - final boolean isGeneratedByPipe) { - this( - walEntryHandler, - devicePath, - progressIndex, - isAligned, - isGeneratedByPipe, - null, - 0, - null, - null, - Long.MIN_VALUE, - Long.MAX_VALUE); + public PipeInsertNodeTabletInsertionEvent(final InsertNode insertNode) { + this(insertNode, null, 0, null, null, Long.MIN_VALUE, Long.MAX_VALUE); } private PipeInsertNodeTabletInsertionEvent( - final WALEntryHandler walEntryHandler, - final PartialPath devicePath, - final ProgressIndex progressIndex, - final boolean isAligned, - final boolean isGeneratedByPipe, + final InsertNode insertNode, final String pipeName, final long creationTime, final PipeTaskMeta pipeTaskMeta, @@ -122,34 +94,25 @@ private PipeInsertNodeTabletInsertionEvent( final long startTime, final long endTime) { super(pipeName, creationTime, pipeTaskMeta, pattern, startTime, endTime); - this.walEntryHandler = walEntryHandler; // Record device path here so there's no need to get it from InsertNode cache later. - this.devicePath = devicePath; - this.progressIndex = progressIndex; - this.isAligned = isAligned; - this.isGeneratedByPipe = isGeneratedByPipe; + this.progressIndex = insertNode.getProgressIndex(); + this.insertNode = insertNode; this.allocatedMemoryBlock = new AtomicReference<>(); } - public InsertNode getInsertNode() throws WALPipeException { - return walEntryHandler.getInsertNode(); + public InsertNode getInsertNode() { + return insertNode; } public ByteBuffer getByteBuffer() throws WALPipeException { - return walEntryHandler.getByteBuffer(); - } - - // This method is a pre-determination of whether to use binary transfers. - // If the insert node is null in cache, it means that we need to read the bytebuffer from the wal, - // and when the pattern is default, we can transfer the bytebuffer directly without serializing or - // deserializing - public InsertNode getInsertNodeViaCacheIfPossible() { - return walEntryHandler.getInsertNodeViaCacheIfPossible(); + return insertNode.serializeToByteBuffer(); } public String getDeviceId() { - return Objects.nonNull(devicePath) ? devicePath.getFullPath() : null; + return Objects.nonNull(insertNode.getDevicePath()) + ? insertNode.getDevicePath().getFullPath() + : null; } /////////////////////////// EnrichedEvent /////////////////////////// @@ -158,7 +121,6 @@ public String getDeviceId() { public boolean internallyIncreaseResourceReferenceCount(final String holderMessage) { extractTime = System.nanoTime(); try { - PipeDataNodeResourceManager.wal().pin(walEntryHandler); if (Objects.nonNull(pipeName)) { PipeDataNodeRemainingEventAndTimeMetrics.getInstance() .increaseInsertNodeEventCount(pipeName, creationTime); @@ -167,10 +129,7 @@ public boolean internallyIncreaseResourceReferenceCount(final String holderMessa return true; } catch (final Exception e) { LOGGER.warn( - String.format( - "Increase reference count for memTable %d error. Holder Message: %s", - walEntryHandler.getMemTableId(), holderMessage), - e); + String.format("Increase reference count error. Holder Message: %s", holderMessage), e); return false; } } @@ -178,7 +137,6 @@ public boolean internallyIncreaseResourceReferenceCount(final String holderMessa @Override public boolean internallyDecreaseResourceReferenceCount(final String holderMessage) { try { - PipeDataNodeResourceManager.wal().unpin(walEntryHandler); // release the containers' memory and close memory block if (dataContainers != null) { dataContainers.clear(); @@ -188,10 +146,7 @@ public boolean internallyDecreaseResourceReferenceCount(final String holderMessa return true; } catch (final Exception e) { LOGGER.warn( - String.format( - "Decrease reference count for memtable %d error. Holder Message: %s", - walEntryHandler.getMemTableId(), holderMessage), - e); + String.format("Decrease reference count error. Holder Message: %s", holderMessage), e); return false; } finally { if (Objects.nonNull(pipeName)) { @@ -221,28 +176,18 @@ public PipeInsertNodeTabletInsertionEvent shallowCopySelfAndBindPipeTaskMetaForP final long startTime, final long endTime) { return new PipeInsertNodeTabletInsertionEvent( - walEntryHandler, - devicePath, - progressIndex, - isAligned, - isGeneratedByPipe, - pipeName, - creationTime, - pipeTaskMeta, - pattern, - startTime, - endTime); + insertNode, pipeName, creationTime, pipeTaskMeta, pattern, startTime, endTime); } @Override public boolean isGeneratedByPipe() { - return isGeneratedByPipe; + return insertNode.isGeneratedByPipe(); } @Override public boolean mayEventTimeOverlappedWithTimeRange() { try { - final InsertNode insertNode = getInsertNodeViaCacheIfPossible(); + final InsertNode insertNode = getInsertNode(); if (Objects.isNull(insertNode)) { return true; } @@ -286,7 +231,7 @@ public boolean mayEventTimeOverlappedWithTimeRange() { @Override public boolean mayEventPathsOverlappedWithPattern() { try { - final InsertNode insertNode = getInsertNodeViaCacheIfPossible(); + final InsertNode insertNode = getInsertNode(); if (Objects.isNull(insertNode)) { return true; } @@ -439,8 +384,8 @@ public List toRawTabletInsertionEvents() { @Override public String toString() { return String.format( - "PipeInsertNodeTabletInsertionEvent{walEntryHandler=%s, progressIndex=%s, isAligned=%s, isGeneratedByPipe=%s, dataContainers=%s}", - walEntryHandler, progressIndex, isAligned, isGeneratedByPipe, dataContainers) + "PipeInsertNodeTabletInsertionEvent{progressIndex=%s, isAligned=%s, isGeneratedByPipe=%s, dataContainers=%s}", + progressIndex, insertNode.isAligned(), insertNode.isGeneratedByPipe(), dataContainers) + " - " + super.toString(); } @@ -448,8 +393,8 @@ public String toString() { @Override public String coreReportMessage() { return String.format( - "PipeInsertNodeTabletInsertionEvent{walEntryHandler=%s, progressIndex=%s, isAligned=%s, isGeneratedByPipe=%s}", - walEntryHandler, progressIndex, isAligned, isGeneratedByPipe) + "PipeInsertNodeTabletInsertionEvent{progressIndex=%s, isAligned=%s, isGeneratedByPipe=%s}", + progressIndex, insertNode.isAligned(), insertNode.isGeneratedByPipe()) + " - " + super.coreReportMessage(); } @@ -476,28 +421,24 @@ protected void trackResource() { @Override public PipeEventResource eventResourceBuilder() { return new PipeInsertNodeTabletInsertionEventResource( - this.isReleased, this.referenceCount, this.walEntryHandler, this.allocatedMemoryBlock); + this.isReleased, this.referenceCount, this.allocatedMemoryBlock); } private static class PipeInsertNodeTabletInsertionEventResource extends PipeEventResource { - private final WALEntryHandler walEntryHandler; private final AtomicReference allocatedMemoryBlock; private PipeInsertNodeTabletInsertionEventResource( final AtomicBoolean isReleased, final AtomicInteger referenceCount, - final WALEntryHandler walEntryHandler, final AtomicReference allocatedMemoryBlock) { super(isReleased, referenceCount); - this.walEntryHandler = walEntryHandler; this.allocatedMemoryBlock = allocatedMemoryBlock; } @Override protected void finalizeResource() { try { - PipeDataNodeResourceManager.wal().unpin(walEntryHandler); allocatedMemoryBlock.getAndUpdate( memoryBlock -> { if (Objects.nonNull(memoryBlock)) { @@ -506,8 +447,7 @@ protected void finalizeResource() { return null; }); } catch (final Exception e) { - LOGGER.warn( - "Decrease reference count for memTable {} error.", walEntryHandler.getMemTableId(), e); + LOGGER.warn("Decrease reference count error.", e); } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java index ef9db9a4cb02..ce4179f18a66 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java @@ -28,7 +28,6 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; -import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALEntryHandler; public class PipeRealtimeEventFactory { @@ -41,18 +40,9 @@ public static PipeRealtimeEvent createRealtimeEvent( } public static PipeRealtimeEvent createRealtimeEvent( - final WALEntryHandler walEntryHandler, - final InsertNode insertNode, - final TsFileResource resource) { + final InsertNode insertNode, final TsFileResource resource) { return TS_FILE_EPOCH_MANAGER.bindPipeInsertNodeTabletInsertionEvent( - new PipeInsertNodeTabletInsertionEvent( - walEntryHandler, - insertNode.getDevicePath(), - insertNode.getProgressIndex(), - insertNode.isAligned(), - insertNode.isGeneratedByPipe()), - insertNode, - resource); + new PipeInsertNodeTabletInsertionEvent(insertNode), insertNode, resource); } public static PipeRealtimeEvent createRealtimeEvent( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java index 452c7188dec3..0769a54bbbab 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java @@ -27,7 +27,6 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; -import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALEntryHandler; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -116,10 +115,7 @@ public void listenToTsFile( } public void listenToInsertNode( - String dataRegionId, - WALEntryHandler walEntryHandler, - InsertNode insertNode, - TsFileResource tsFileResource) { + String dataRegionId, InsertNode insertNode, TsFileResource tsFileResource) { if (listenToInsertNodeExtractorCount.get() == 0) { return; } @@ -132,7 +128,7 @@ public void listenToInsertNode( } assigner.publishToAssign( - PipeRealtimeEventFactory.createRealtimeEvent(walEntryHandler, insertNode, tsFileResource)); + PipeRealtimeEventFactory.createRealtimeEvent(insertNode, tsFileResource)); } public void listenToHeartbeat(boolean shouldPrintMessage) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeMetrics.java index 3f03ce580fd2..1358a0bc08f1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeMetrics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeMetrics.java @@ -24,7 +24,6 @@ import org.apache.iotdb.db.pipe.metric.overview.PipeHeartbeatEventMetrics; import org.apache.iotdb.db.pipe.metric.overview.PipeResourceMetrics; import org.apache.iotdb.db.pipe.metric.overview.PipeTsFileToTabletsMetrics; -import org.apache.iotdb.db.pipe.metric.overview.PipeWALInsertNodeCacheMetrics; import org.apache.iotdb.db.pipe.metric.processor.PipeProcessorMetrics; import org.apache.iotdb.db.pipe.metric.receiver.PipeDataNodeReceiverMetrics; import org.apache.iotdb.db.pipe.metric.schema.PipeSchemaRegionConnectorMetrics; @@ -47,7 +46,6 @@ public void bindTo(final AbstractMetricService metricService) { PipeProcessorMetrics.getInstance().bindTo(metricService); PipeDataRegionConnectorMetrics.getInstance().bindTo(metricService); PipeHeartbeatEventMetrics.getInstance().bindTo(metricService); - PipeWALInsertNodeCacheMetrics.getInstance().bindTo(metricService); PipeResourceMetrics.getInstance().bindTo(metricService); PipeEventCommitMetrics.getInstance().bindTo(metricService); PipeSchemaRegionListenerMetrics.getInstance().bindTo(metricService); @@ -65,7 +63,6 @@ public void unbindFrom(final AbstractMetricService metricService) { PipeProcessorMetrics.getInstance().unbindFrom(metricService); PipeDataRegionConnectorMetrics.getInstance().unbindFrom(metricService); PipeHeartbeatEventMetrics.getInstance().unbindFrom(metricService); - PipeWALInsertNodeCacheMetrics.getInstance().unbindFrom(metricService); PipeResourceMetrics.getInstance().unbindFrom(metricService); PipeEventCommitMetrics.getInstance().unbindFrom(metricService); PipeSchemaRegionListenerMetrics.getInstance().unbindFrom(metricService); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeResourceMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeResourceMetrics.java index c9115575000c..b8f47e29698d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeResourceMetrics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeResourceMetrics.java @@ -25,7 +25,6 @@ import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryManager; import org.apache.iotdb.db.pipe.resource.tsfile.PipeTsFileResourceManager; -import org.apache.iotdb.db.pipe.resource.wal.PipeWALResourceManager; import org.apache.iotdb.metrics.AbstractMetricService; import org.apache.iotdb.metrics.metricsets.IMetricSet; import org.apache.iotdb.metrics.utils.MetricLevel; @@ -75,11 +74,6 @@ public void bindTo(final AbstractMetricService metricService) { Tag.NAME.toString(), PIPE_TOTAL_MEMORY); // resource reference count - metricService.createAutoGauge( - Metric.PIPE_PINNED_MEMTABLE_COUNT.toString(), - MetricLevel.IMPORTANT, - PipeDataNodeResourceManager.wal(), - PipeWALResourceManager::getPinnedWalCount); metricService.createAutoGauge( Metric.PIPE_LINKED_TSFILE_COUNT.toString(), MetricLevel.IMPORTANT, @@ -116,7 +110,6 @@ public void unbindFrom(final AbstractMetricService metricService) { metricService.remove( MetricType.AUTO_GAUGE, Metric.PIPE_MEM.toString(), Tag.NAME.toString(), PIPE_TOTAL_MEMORY); // resource reference count - metricService.remove(MetricType.AUTO_GAUGE, Metric.PIPE_PINNED_MEMTABLE_COUNT.toString()); metricService.remove(MetricType.AUTO_GAUGE, Metric.PIPE_LINKED_TSFILE_COUNT.toString()); metricService.remove(MetricType.AUTO_GAUGE, Metric.PIPE_LINKED_TSFILE_SIZE.toString()); // phantom reference count diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeWALInsertNodeCacheMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeWALInsertNodeCacheMetrics.java deleted file mode 100644 index b2e605bf84c1..000000000000 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeWALInsertNodeCacheMetrics.java +++ /dev/null @@ -1,86 +0,0 @@ -/* - * 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.pipe.metric.overview; - -import org.apache.iotdb.commons.service.metric.enums.Metric; -import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALInsertNodeCache; -import org.apache.iotdb.metrics.AbstractMetricService; -import org.apache.iotdb.metrics.metricsets.IMetricSet; -import org.apache.iotdb.metrics.utils.MetricLevel; -import org.apache.iotdb.metrics.utils.MetricType; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class PipeWALInsertNodeCacheMetrics implements IMetricSet { - - private static final Logger LOGGER = LoggerFactory.getLogger(PipeWALInsertNodeCacheMetrics.class); - - //////////////////////////// bindTo & unbindFrom (metric framework) //////////////////////////// - - @Override - public void bindTo(AbstractMetricService metricService) { - metricService.createAutoGauge( - Metric.PIPE_WAL_INSERT_NODE_CACHE_HIT_RATE.toString(), - MetricLevel.IMPORTANT, - WALInsertNodeCache.getInstance(), - WALInsertNodeCache::getCacheHitRate); - metricService.createAutoGauge( - Metric.PIPE_WAL_INSERT_NODE_CACHE_HIT_COUNT.toString(), - MetricLevel.IMPORTANT, - WALInsertNodeCache.getInstance(), - WALInsertNodeCache::getCacheHitCount); - metricService.createAutoGauge( - Metric.PIPE_WAL_INSERT_NODE_CACHE_REQUEST_COUNT.toString(), - MetricLevel.IMPORTANT, - WALInsertNodeCache.getInstance(), - WALInsertNodeCache::getCacheRequestCount); - } - - @Override - public void unbindFrom(AbstractMetricService metricService) { - metricService.remove( - MetricType.AUTO_GAUGE, Metric.PIPE_WAL_INSERT_NODE_CACHE_HIT_RATE.toString()); - metricService.remove( - MetricType.AUTO_GAUGE, Metric.PIPE_WAL_INSERT_NODE_CACHE_HIT_COUNT.toString()); - metricService.remove( - MetricType.AUTO_GAUGE, Metric.PIPE_WAL_INSERT_NODE_CACHE_REQUEST_COUNT.toString()); - } - - //////////////////////////// singleton //////////////////////////// - - private static class PipeWALInsertNodeCacheMetricsHolder { - - private static final PipeWALInsertNodeCacheMetrics INSTANCE = - new PipeWALInsertNodeCacheMetrics(); - - private PipeWALInsertNodeCacheMetricsHolder() { - // empty constructor - } - } - - public static PipeWALInsertNodeCacheMetrics getInstance() { - return PipeWALInsertNodeCacheMetricsHolder.INSTANCE; - } - - private PipeWALInsertNodeCacheMetrics() { - // empty constructor - } -} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/PipeDataNodeResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/PipeDataNodeResourceManager.java index 573106e45c2e..4a3af9d781c2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/PipeDataNodeResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/PipeDataNodeResourceManager.java @@ -19,7 +19,6 @@ package org.apache.iotdb.db.pipe.resource; -import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.resource.log.PipeLogManager; import org.apache.iotdb.commons.pipe.resource.ref.PipePhantomReferenceManager; import org.apache.iotdb.commons.pipe.resource.snapshot.PipeSnapshotResourceManager; @@ -27,16 +26,10 @@ import org.apache.iotdb.db.pipe.resource.ref.PipeDataNodePhantomReferenceManager; import org.apache.iotdb.db.pipe.resource.snapshot.PipeDataNodeSnapshotResourceManager; import org.apache.iotdb.db.pipe.resource.tsfile.PipeTsFileResourceManager; -import org.apache.iotdb.db.pipe.resource.wal.PipeWALResourceManager; -import org.apache.iotdb.db.pipe.resource.wal.hardlink.PipeWALHardlinkResourceManager; -import org.apache.iotdb.db.pipe.resource.wal.selfhost.PipeWALSelfHostResourceManager; - -import java.util.concurrent.atomic.AtomicReference; public class PipeDataNodeResourceManager { private final PipeTsFileResourceManager pipeTsFileResourceManager; - private final AtomicReference pipeWALResourceManager; private final PipeSnapshotResourceManager pipeSnapshotResourceManager; private final PipeMemoryManager pipeMemoryManager; private final PipeLogManager pipeLogManager; @@ -46,20 +39,6 @@ public static PipeTsFileResourceManager tsfile() { return PipeResourceManagerHolder.INSTANCE.pipeTsFileResourceManager; } - public static PipeWALResourceManager wal() { - if (PipeResourceManagerHolder.INSTANCE.pipeWALResourceManager.get() == null) { - synchronized (PipeResourceManagerHolder.INSTANCE) { - if (PipeResourceManagerHolder.INSTANCE.pipeWALResourceManager.get() == null) { - PipeResourceManagerHolder.INSTANCE.pipeWALResourceManager.set( - PipeConfig.getInstance().getPipeHardLinkWALEnabled() - ? new PipeWALHardlinkResourceManager() - : new PipeWALSelfHostResourceManager()); - } - } - } - return PipeResourceManagerHolder.INSTANCE.pipeWALResourceManager.get(); - } - public static PipeSnapshotResourceManager snapshot() { return PipeResourceManagerHolder.INSTANCE.pipeSnapshotResourceManager; } @@ -80,7 +59,6 @@ public static PipePhantomReferenceManager ref() { private PipeDataNodeResourceManager() { pipeTsFileResourceManager = new PipeTsFileResourceManager(); - pipeWALResourceManager = new AtomicReference<>(); pipeSnapshotResourceManager = new PipeDataNodeSnapshotResourceManager(); pipeMemoryManager = new PipeMemoryManager(); pipeLogManager = new PipeLogManager(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/PipeWALResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/PipeWALResource.java index 9d1e530a19d1..19f681bb55bb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/PipeWALResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/PipeWALResource.java @@ -22,7 +22,6 @@ import org.apache.iotdb.commons.exception.pipe.PipeRuntimeCriticalException; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeNonCriticalException; import org.apache.iotdb.db.storageengine.dataregion.wal.exception.MemTablePinException; -import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALEntryHandler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/PipeWALResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/PipeWALResourceManager.java index 9c51d79daad4..83bb9939013e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/PipeWALResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/PipeWALResourceManager.java @@ -22,7 +22,6 @@ import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; -import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALEntryHandler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/hardlink/PipeWALHardlinkResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/hardlink/PipeWALHardlinkResource.java deleted file mode 100644 index f1ad513ccc4c..000000000000 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/hardlink/PipeWALHardlinkResource.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * 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.pipe.resource.wal.hardlink; - -import org.apache.iotdb.db.pipe.resource.wal.PipeWALResource; -import org.apache.iotdb.db.storageengine.dataregion.wal.exception.MemTablePinException; -import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALEntryHandler; - -public class PipeWALHardlinkResource extends PipeWALResource { - - private final PipeWALHardlinkResourceManager resourceManager; - - protected PipeWALHardlinkResource( - WALEntryHandler walEntryHandler, PipeWALHardlinkResourceManager resourceManager) { - super(walEntryHandler); - this.resourceManager = resourceManager; - } - - @Override - protected void pinInternal() throws MemTablePinException { - // TODO: hardlink - walEntryHandler.pinMemTable(); - } - - @Override - protected void unpinInternal() throws MemTablePinException { - // TODO: hardlink - walEntryHandler.unpinMemTable(); - } -} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/hardlink/PipeWALHardlinkResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/hardlink/PipeWALHardlinkResourceManager.java deleted file mode 100644 index eebf766dc367..000000000000 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/hardlink/PipeWALHardlinkResourceManager.java +++ /dev/null @@ -1,152 +0,0 @@ -/* - * 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.pipe.resource.wal.hardlink; - -import org.apache.iotdb.commons.conf.IoTDBConstant; -import org.apache.iotdb.commons.pipe.config.PipeConfig; -import org.apache.iotdb.commons.utils.FileUtils; -import org.apache.iotdb.commons.utils.TestOnly; -import org.apache.iotdb.db.pipe.resource.wal.PipeWALResourceManager; -import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALEntryHandler; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.util.HashMap; -import java.util.Map; - -public class PipeWALHardlinkResourceManager extends PipeWALResourceManager { - - @Override - protected void pinInternal(final long memTableId, final WALEntryHandler walEntryHandler) { - memtableIdToPipeWALResourceMap - .computeIfAbsent(memTableId, id -> new PipeWALHardlinkResource(walEntryHandler, this)) - .pin(); - } - - @Override - protected void unpinInternal(final long memTableId, final WALEntryHandler walEntryHandler) { - memtableIdToPipeWALResourceMap.get(memTableId).unpin(); - } - - //////////////////////////// hardlink related //////////////////////////// - - private final Map hardlinkToReferenceMap = new HashMap<>(); - - /** - * given a file, create a hardlink, maintain a reference count for the hardlink, and return the - * hardlink. - * - *

if the given file is already a hardlink, increase its reference count and return it. - * - *

if the given file is a wal, create a hardlink in pipe dir, increase the reference count of - * the hardlink and return it. - * - * @param file wal file. can be original file or the hardlink of original file - * @return the hardlink - * @throws IOException when create hardlink failed - */ - public synchronized File increaseFileReference(final File file) throws IOException { - // if the file is already a hardlink, just increase reference count and return it - if (increaseReferenceIfExists(file.getPath())) { - return file; - } - - // if the file is not a hardlink, check if there is a related hardlink in pipe dir. if so, - // increase reference count and return it. - final File hardlink = getHardlinkInPipeWALDir(file); - if (increaseReferenceIfExists(hardlink.getPath())) { - return hardlink; - } - - // if the file is a wal, and there is no related hardlink in pipe dir, create a hardlink to pipe - // dir, maintain a reference count for the hardlink, and return the hardlink. - hardlinkToReferenceMap.put(hardlink.getPath(), 1); - return FileUtils.createHardLink(file, hardlink); - } - - private boolean increaseReferenceIfExists(final String path) { - hardlinkToReferenceMap.computeIfPresent(path, (k, v) -> v + 1); - return hardlinkToReferenceMap.containsKey(path); - } - - // TODO: Check me! Make sure the file is not a hardlink. - // TODO: IF user specify a wal by config, will the method work? - private static File getHardlinkInPipeWALDir(final File file) throws IOException { - try { - return new File(getPipeWALDirPath(file), getRelativeFilePath(file)); - } catch (final Exception e) { - throw new IOException( - String.format( - "failed to get hardlink in pipe dir " + "for file %s, it is not a wal", - file.getPath()), - e); - } - } - - private static String getPipeWALDirPath(File file) throws IOException { - while (!file.getName().equals(IoTDBConstant.WAL_FOLDER_NAME)) { - file = file.getParentFile(); - } - - return file.getParentFile().getCanonicalPath() - + File.separator - + IoTDBConstant.DATA_FOLDER_NAME - + File.separator - + PipeConfig.getInstance().getPipeHardlinkBaseDirName() - + File.separator - + PipeConfig.getInstance().getPipeHardlinkWALDirName(); - } - - private static String getRelativeFilePath(File file) { - StringBuilder builder = new StringBuilder(file.getName()); - while (!file.getParentFile().getName().equals(IoTDBConstant.WAL_FOLDER_NAME)) { - file = file.getParentFile(); - builder = - new StringBuilder(file.getName()) - .append(IoTDBConstant.FILE_NAME_SEPARATOR) - .append(builder); - } - return builder.toString(); - } - - /** - * given a hardlink, decrease its reference count, if the reference count is 0, delete the file. - * if the given file is not a hardlink, do nothing. - * - * @param hardlink the hardlinked file - * @throws IOException when delete file failed - */ - public synchronized void decreaseFileReference(final File hardlink) throws IOException { - final Integer updatedReference = - hardlinkToReferenceMap.computeIfPresent( - hardlink.getPath(), (file, reference) -> reference - 1); - - if (updatedReference != null && updatedReference == 0) { - Files.deleteIfExists(hardlink.toPath()); - hardlinkToReferenceMap.remove(hardlink.getPath()); - } - } - - @TestOnly - public synchronized int getFileReferenceCount(final File hardlink) { - return hardlinkToReferenceMap.getOrDefault(hardlink.getPath(), 0); - } -} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/selfhost/PipeWALSelfHostResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/selfhost/PipeWALSelfHostResource.java index e8e03e64a162..d93bbd8eff36 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/selfhost/PipeWALSelfHostResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/selfhost/PipeWALSelfHostResource.java @@ -21,7 +21,6 @@ import org.apache.iotdb.db.pipe.resource.wal.PipeWALResource; import org.apache.iotdb.db.storageengine.dataregion.wal.exception.MemTablePinException; -import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALEntryHandler; public class PipeWALSelfHostResource extends PipeWALResource { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/selfhost/PipeWALSelfHostResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/selfhost/PipeWALSelfHostResourceManager.java deleted file mode 100644 index c7fe0accda2f..000000000000 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/selfhost/PipeWALSelfHostResourceManager.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * 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.pipe.resource.wal.selfhost; - -import org.apache.iotdb.db.pipe.resource.wal.PipeWALResourceManager; -import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALEntryHandler; - -public class PipeWALSelfHostResourceManager extends PipeWALResourceManager { - - @Override - protected void pinInternal(final long memTableId, final WALEntryHandler walEntryHandler) { - memtableIdToPipeWALResourceMap - .computeIfAbsent(memTableId, id -> new PipeWALSelfHostResource(walEntryHandler)) - .pin(); - } - - @Override - protected void unpinInternal(final long memTableId, final WALEntryHandler walEntryHandler) { - memtableIdToPipeWALResourceMap.get(memTableId).unpin(); - } -} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java index c91cb7838e34..f1d20a650806 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java @@ -321,10 +321,7 @@ public void insert(InsertRowNode insertRowNode, long[] costsForMetrics) } PipeInsertionDataNodeListener.getInstance() .listenToInsertNode( - dataRegionInfo.getDataRegion().getDataRegionId(), - walFlushListener.getWalEntryHandler(), - insertRowNode, - tsFileResource); + dataRegionInfo.getDataRegion().getDataRegionId(), insertRowNode, tsFileResource); int pointInserted; if (insertRowNode.isAligned()) { @@ -422,10 +419,7 @@ public void insert(InsertRowsNode insertRowsNode, long[] costsForMetrics) } PipeInsertionDataNodeListener.getInstance() .listenToInsertNode( - dataRegionInfo.getDataRegion().getDataRegionId(), - walFlushListener.getWalEntryHandler(), - insertRowsNode, - tsFileResource); + dataRegionInfo.getDataRegion().getDataRegionId(), insertRowsNode, tsFileResource); int pointInserted = 0; for (InsertRowNode insertRowNode : insertRowsNode.getInsertRowNodeList()) { @@ -540,10 +534,7 @@ public void insertTablet( } PipeInsertionDataNodeListener.getInstance() .listenToInsertNode( - dataRegionInfo.getDataRegion().getDataRegionId(), - walFlushListener.getWalEntryHandler(), - insertTabletNode, - tsFileResource); + dataRegionInfo.getDataRegion().getDataRegionId(), insertTabletNode, tsFileResource); int pointInserted; try { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALBuffer.java index edc30604a6e2..36c404faa7c2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALBuffer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALBuffer.java @@ -342,7 +342,6 @@ private void handleInfoEntry(WALEntry walEntry) { info.metaData.add(size, searchIndex, walEntry.getMemTableId()); info.memTableId2WalDiskUsage.compute( walEntry.getMemTableId(), (k, v) -> v == null ? size : v + size); - walEntry.getWalFlushListener().getWalEntryHandler().setSize(size); info.fsyncListeners.add(walEntry.getWalFlushListener()); } @@ -593,13 +592,8 @@ public void run() { // notify all waiting listeners if (forceSuccess) { - long position = lastFsyncPosition; for (WALFlushListener fsyncListener : info.fsyncListeners) { fsyncListener.succeed(); - if (fsyncListener.getWalEntryHandler() != null) { - fsyncListener.getWalEntryHandler().setEntryPosition(walFileVersionId, position); - position += fsyncListener.getWalEntryHandler().getSize(); - } } lastFsyncPosition = currentWALFileWriter.originalSize(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALEntry.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALEntry.java index a8e94e2780a3..18304e621b7e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALEntry.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALEntry.java @@ -77,14 +77,14 @@ protected WALEntry(long memTableId, WALEntryValue value, boolean wait) { } else { throw new RuntimeException("Unknown WALEntry type"); } - walFlushListener = new WALFlushListener(wait, value); + walFlushListener = new WALFlushListener(wait); } protected WALEntry(WALEntryType type, long memTableId, WALEntryValue value, boolean wait) { this.type = type; this.memTableId = memTableId; this.value = value; - this.walFlushListener = new WALFlushListener(wait, value); + this.walFlushListener = new WALFlushListener(wait); } public abstract void serialize(IWALByteBufferView buffer); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/checkpoint/MemTableInfo.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/checkpoint/MemTableInfo.java index 59c2a3be23b5..984006b25bc0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/checkpoint/MemTableInfo.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/checkpoint/MemTableInfo.java @@ -47,8 +47,7 @@ public class MemTableInfo implements WALEntryValue { // memTable private IMemTable memTable; - // memTable pin count - private int pinCount; + // memTable is flushed or not private boolean flushed; // data region id @@ -116,22 +115,6 @@ public IMemTable getMemTable() { return memTable; } - public void pin() { - this.pinCount++; - } - - public void unpin() { - this.pinCount--; - } - - public boolean isPinned() { - return pinCount > 0; - } - - public int getPinCount() { - return pinCount; - } - public boolean isFlushed() { return flushed; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/io/WALByteBufReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/io/WALByteBufReader.java index b03b27a69948..a022946ba90e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/io/WALByteBufReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/io/WALByteBufReader.java @@ -20,7 +20,6 @@ package org.apache.iotdb.db.storageengine.dataregion.wal.io; import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.WALEntry; -import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALEntryPosition; import java.io.Closeable; import java.io.DataInputStream; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALFakeNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALFakeNode.java index 38b69f1162b4..dfa7bf6bb9dd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALFakeNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALFakeNode.java @@ -41,9 +41,9 @@ private WALFakeNode(Status status) { public WALFakeNode(Status status, Exception cause) { this.status = status; - this.successListener = new WALFlushListener(false, null); + this.successListener = new WALFlushListener(false); this.successListener.succeed(); - this.failListener = new WALFlushListener(false, null); + this.failListener = new WALFlushListener(false); this.failListener.fail(cause); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALNode.java index 35432a8fd6ff..003c74763da7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALNode.java @@ -49,7 +49,6 @@ import org.apache.iotdb.db.storageengine.dataregion.wal.checkpoint.CheckpointManager; import org.apache.iotdb.db.storageengine.dataregion.wal.checkpoint.CheckpointType; import org.apache.iotdb.db.storageengine.dataregion.wal.checkpoint.MemTableInfo; -import org.apache.iotdb.db.storageengine.dataregion.wal.exception.MemTablePinException; import org.apache.iotdb.db.storageengine.dataregion.wal.io.WALByteBufReader; import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALFileStatus; import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALFileUtils; @@ -57,7 +56,6 @@ import org.apache.iotdb.db.storageengine.dataregion.wal.utils.listener.AbstractResultListener.Status; import org.apache.iotdb.db.storageengine.dataregion.wal.utils.listener.WALFlushListener; -import org.apache.commons.lang3.StringUtils; import org.apache.tsfile.fileSystem.FSFactoryProducer; import org.apache.tsfile.utils.TsFileUtils; import org.slf4j.Logger; @@ -183,7 +181,6 @@ private WALFlushListener log(WALEntry walEntry) { buffer.write(walEntry); // set handler for pipe - walEntry.getWalFlushListener().getWalEntryHandler().setWalNode(this, walEntry.getMemTableId()); return walEntry.getWalFlushListener(); } @@ -228,25 +225,6 @@ public void setDeleted(boolean deleted) { // region methods for pipe - /** - * Pin the wal files of the given memory table. Notice: cannot pin one memTable too long, - * otherwise the wal disk usage may too large. - * - * @throws MemTablePinException If the memTable has been flushed - */ - public void pinMemTable(long memTableId) throws MemTablePinException { - checkpointManager.pinMemTable(memTableId); - } - - /** - * Unpin the wal files of the given memory table. - * - * @throws MemTablePinException If there aren't corresponding pin operations - */ - public void unpinMemTable(long memTableId) throws MemTablePinException { - checkpointManager.unpinMemTable(memTableId); - } - // endregion // region Task to delete outdated .wal files @@ -270,8 +248,6 @@ private class DeleteOutdatedFileTask implements Runnable { // the effective information ratio private double effectiveInfoRatio = 1.0d; - private List pinnedMemTableIds; - private int fileIndexAfterFilterSafelyDeleteIndex = Integer.MAX_VALUE; private List successfullyDeleted; private long deleteFileSize; @@ -297,7 +273,6 @@ private boolean initAndCheckIfNeedContinue() { this.sortedWalFilesExcludingLast = Arrays.copyOfRange(allWalFilesOfOneNode, 0, allWalFilesOfOneNode.length - 1); this.activeOrPinnedMemTables = checkpointManager.activeOrPinnedMemTables(); - this.pinnedMemTableIds = initPinnedMemTableIds(); this.fileIndexAfterFilterSafelyDeleteIndex = initFileIndexAfterFilterSafelyDeleteIndex(); this.successfullyDeleted = new ArrayList<>(); this.deleteFileSize = 0; @@ -318,20 +293,6 @@ public void rollWalFileIfHaveNoActiveMemTable() { } } - private List initPinnedMemTableIds() { - List memTableInfos = checkpointManager.activeOrPinnedMemTables(); - if (memTableInfos.isEmpty()) { - return new ArrayList<>(); - } - List pinnedIds = new ArrayList<>(); - for (MemTableInfo memTableInfo : memTableInfos) { - if (memTableInfo.isFlushed() && memTableInfo.isPinned()) { - pinnedIds.add(memTableInfo.getMemTableId()); - } - } - return pinnedIds; - } - @Override public void run() { // The intent of the loop execution here is to try to get as many memTable flush or snapshot @@ -365,7 +326,7 @@ public void run() { private void updateEffectiveInfoRationAndUpdateMetric() { // calculate effective information ratio long costOfActiveMemTables = checkpointManager.getTotalCostOfActiveMemTables(); - MemTableInfo oldestUnpinnedMemTableInfo = checkpointManager.getOldestUnpinnedMemTableInfo(); + MemTableInfo oldestUnpinnedMemTableInfo = checkpointManager.getOldestMemTableInfo(); long avgFileSize = getFileNum() != 0 ? getDiskUsage() / getFileNum() @@ -389,45 +350,10 @@ private void updateEffectiveInfoRationAndUpdateMetric() { } private void summarizeExecuteResult() { - if (!pinnedMemTableIds.isEmpty() - || fileIndexAfterFilterSafelyDeleteIndex < sortedWalFilesExcludingLast.length) { - if (logger.isDebugEnabled()) { - StringBuilder summary = - new StringBuilder( - String.format( - "wal node-%s delete outdated files summary:the range is: [%d,%d], delete successful is [%s], safely delete file index is: [%s].The following reasons influenced the result: %s", - identifier, - WALFileUtils.parseVersionId(sortedWalFilesExcludingLast[0].getName()), - WALFileUtils.parseVersionId( - sortedWalFilesExcludingLast[sortedWalFilesExcludingLast.length - 1] - .getName()), - StringUtils.join(successfullyDeleted, ","), - fileIndexAfterFilterSafelyDeleteIndex, - System.lineSeparator())); - - if (!pinnedMemTableIds.isEmpty()) { - summary - .append("- MemTable has been flushed but pinned by PIPE, the MemTableId list is : ") - .append(StringUtils.join(pinnedMemTableIds, ",")) - .append(".") - .append(System.lineSeparator()); - } - if (fileIndexAfterFilterSafelyDeleteIndex < sortedWalFilesExcludingLast.length) { - summary.append( - String.format( - "- The data in the wal file was not consumed by the consensus group,current search index is %d, safely delete index is %d", - getCurrentSearchIndex(), safelyDeletedSearchIndex)); - } - String summaryLog = summary.toString(); - logger.debug(summaryLog); - } - - } else { - logger.debug( - "Successfully delete {} outdated wal files for wal node-{}", - successfullyDeleted.size(), - identifier); - } + logger.debug( + "Successfully delete {} outdated wal files for wal node-{}", + successfullyDeleted.size(), + identifier); } /** Delete obsolete wal files while recording which succeeded or failed */ @@ -475,20 +401,10 @@ private boolean trySnapshotOrFlushMemTable() { return false; } // find oldest memTable - MemTableInfo oldestMemTableInfo = checkpointManager.getOldestUnpinnedMemTableInfo(); + MemTableInfo oldestMemTableInfo = checkpointManager.getOldestMemTableInfo(); if (oldestMemTableInfo == null) { return false; } - if (oldestMemTableInfo.isPinned()) { - logger.warn( - "Pipe: Effective information ratio {} of wal node-{} is below wal min effective info ratio {}. But fail to delete memTable-{}'s wal files because they are pinned by the Pipe module. Pin count: {}.", - effectiveInfoRatio, - identifier, - config.getWalMinEffectiveInfoRatio(), - oldestMemTableInfo.getMemTableId(), - oldestMemTableInfo.getPinCount()); - return false; - } IMemTable oldestMemTable = oldestMemTableInfo.getMemTable(); if (oldestMemTable == null) { return false; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/WALEntryHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/WALEntryHandler.java deleted file mode 100644 index f5d7406f5a60..000000000000 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/WALEntryHandler.java +++ /dev/null @@ -1,233 +0,0 @@ -/* - * 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.storageengine.dataregion.wal.utils; - -import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode; -import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.WALEntry; -import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.WALEntryValue; -import org.apache.iotdb.db.storageengine.dataregion.wal.exception.MemTablePinException; -import org.apache.iotdb.db.storageengine.dataregion.wal.exception.WALPipeException; -import org.apache.iotdb.db.storageengine.dataregion.wal.node.WALNode; - -import org.apache.tsfile.utils.Pair; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.nio.ByteBuffer; -import java.util.concurrent.atomic.AtomicReference; - -/** - * This handler is used by the Pipe to find the corresponding {@link InsertNode}. Besides, it can - * try to pin/unpin the {@link WALEntry}s by the memTable id. - */ -public class WALEntryHandler { - - private static final Logger logger = LoggerFactory.getLogger(WALEntryHandler.class); - - private long memTableId = -1; - - // cached value, null after this value is flushed to wal successfully - @SuppressWarnings("squid:S3077") - private volatile WALEntryValue value; - - // wal entry's position in the wal, valid after the value is flushed to wal successfully - // it's safe to use volatile here to make this reference thread-safe. - @SuppressWarnings("squid:S3077") - private final WALEntryPosition walEntryPosition = new WALEntryPosition(); - - // wal node, null when wal is disabled - private WALNode walNode = null; - - private volatile boolean isHardlink = false; - private final AtomicReference hardlinkFile = new AtomicReference<>(); - - public WALEntryHandler(final WALEntryValue value) { - this.value = value; - } - - /** - * Pin the wal files of the given memory table. Notice: cannot pin one memTable too long, - * otherwise the wal disk usage may too large. - * - * @throws MemTablePinException If the memTable has been flushed - */ - public void pinMemTable() throws MemTablePinException { - if (walNode == null || memTableId < 0) { - throw new MemTablePinException("Fail to pin memTable because of internal error."); - } - walNode.pinMemTable(memTableId); - } - - /** - * Unpin the wal files of the given memory table. - * - * @throws MemTablePinException If there aren't corresponding pin operations - */ - public void unpinMemTable() throws MemTablePinException { - if (walNode == null || memTableId < 0) { - throw new MemTablePinException("Fail to pin memTable because of internal error."); - } - walNode.unpinMemTable(memTableId); - } - - public InsertNode getInsertNodeViaCacheIfPossible() { - try { - final WALEntryValue finalValue = value; - if (finalValue instanceof InsertNode) { - return (InsertNode) finalValue; - } - final Pair byteBufferInsertNodePair = - walEntryPosition.getByteBufferOrInsertNodeIfPossible(); - return byteBufferInsertNodePair == null ? null : byteBufferInsertNodePair.getRight(); - } catch (final Exception e) { - logger.warn("Fail to get insert node via cache. {}", this, e); - throw e; - } - } - - /** - * Get this handler's value. - * - * @throws WALPipeException when failing to get the value. - */ - public InsertNode getInsertNode() throws WALPipeException { - // return local cache - final WALEntryValue res = value; - if (res != null) { - if (res instanceof InsertNode) { - return (InsertNode) res; - } else { - throw new WALPipeException("Fail to get value because the entry type isn't InsertNode."); - } - } - - // wait until the position is ready - while (!walEntryPosition.canRead()) { - try { - synchronized (this) { - this.wait(); - } - } catch (final InterruptedException e) { - logger.warn("Interrupted when waiting for result.", e); - Thread.currentThread().interrupt(); - } - } - - final InsertNode node = isHardlink ? readFromHardlinkWALFile() : readFromOriginalWALFile(); - if (node == null) { - throw new WALPipeException( - String.format("Fail to get the wal value of the position %s.", walEntryPosition)); - } - return node; - } - - public ByteBuffer getByteBuffer() throws WALPipeException { - // wait until the position is ready - while (!walEntryPosition.canRead()) { - try { - synchronized (this) { - this.wait(); - } - } catch (InterruptedException e) { - logger.warn("Interrupted when waiting for result.", e); - Thread.currentThread().interrupt(); - } - } - - final ByteBuffer buffer = readByteBufferFromWALFile(); - if (buffer == null) { - throw new WALPipeException( - String.format("Fail to get the wal value of the position %s.", walEntryPosition)); - } - return buffer; - } - - private InsertNode readFromOriginalWALFile() throws WALPipeException { - try { - return walEntryPosition.readInsertNodeViaCacheAfterCanRead(); - } catch (Exception e) { - throw new WALPipeException("Fail to get value because the file content isn't correct.", e); - } - } - - private InsertNode readFromHardlinkWALFile() throws WALPipeException { - try { - return walEntryPosition.readInsertNodeViaCacheAfterCanRead(); - } catch (Exception e) { - throw new WALPipeException("Fail to get value because the file content isn't correct.", e); - } - } - - private ByteBuffer readByteBufferFromWALFile() throws WALPipeException { - try { - return walEntryPosition.readByteBufferViaCacheAfterCanRead(); - } catch (Exception e) { - throw new WALPipeException("Fail to get value because the file content isn't correct.", e); - } - } - - public void setWalNode(final WALNode walNode, final long memTableId) { - this.walNode = walNode; - this.memTableId = memTableId; - walEntryPosition.setWalNode(walNode, memTableId); - } - - public long getMemTableId() { - return memTableId; - } - - public void setEntryPosition(final long walFileVersionId, final long position) { - this.walEntryPosition.setEntryPosition(walFileVersionId, position, value); - this.value = null; - synchronized (this) { - this.notifyAll(); - } - } - - public WALEntryPosition getWalEntryPosition() { - return walEntryPosition; - } - - public int getSize() { - return walEntryPosition.getSize(); - } - - public void setSize(final int size) { - this.walEntryPosition.setSize(size); - } - - public void hardlinkTo(File hardlinkFile) { - isHardlink = true; - this.hardlinkFile.set(hardlinkFile); - } - - @Override - public String toString() { - return "WALEntryHandler{" - + "memTableId=" - + memTableId - + ", value=" - + value - + ", walEntryPosition=" - + walEntryPosition - + '}'; - } -} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/WALEntryPosition.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/WALEntryPosition.java deleted file mode 100644 index 4d71cb1030c3..000000000000 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/WALEntryPosition.java +++ /dev/null @@ -1,229 +0,0 @@ -/* - * 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.storageengine.dataregion.wal.utils; - -import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode; -import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.WALEntryValue; -import org.apache.iotdb.db.storageengine.dataregion.wal.io.WALInputStream; -import org.apache.iotdb.db.storageengine.dataregion.wal.node.WALNode; - -import org.apache.tsfile.utils.Pair; - -import java.io.File; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.channels.FileChannel; -import java.nio.file.StandardOpenOption; -import java.util.Objects; - -/** - * This class uses the tuple(identifier, file, position) to denote the position of the wal entry, - * and give some methods to read the content from the disk. - */ -public class WALEntryPosition { - private volatile String identifier = ""; - private volatile long walFileVersionId = -1; - private volatile long position; - private volatile int size; - // wal node, null when wal is disabled - private WALNode walNode = null; - // wal file is not null when openReadFileChannel method has been called - private File walFile = null; - // cache for wal entry - private WALInsertNodeCache cache = null; - - private static final String ENTRY_NOT_READY_MESSAGE = "This entry isn't ready for read."; - - public WALEntryPosition() {} - - public WALEntryPosition(String identifier, long walFileVersionId, long position, int size) { - this.identifier = identifier; - this.walFileVersionId = walFileVersionId; - this.position = position; - this.size = size; - } - - /** - * Try to read the wal entry directly from the cache. No need to check if the wal entry is ready - * for read. - */ - public Pair getByteBufferOrInsertNodeIfPossible() { - return cache.getByteBufferOrInsertNodeIfPossible(this); - } - - /** - * Read the wal entry and parse it to the InsertNode. Use LRU cache to accelerate read. - * - * @throws IOException failing to read. - */ - public InsertNode readInsertNodeViaCacheAfterCanRead() throws IOException { - if (!canRead()) { - throw new IOException(ENTRY_NOT_READY_MESSAGE); - } - return cache.getInsertNode(this); - } - - /** - * Read the wal entry and get the raw bytebuffer. Use LRU cache to accelerate read. - * - * @throws IOException failing to read. - */ - public ByteBuffer readByteBufferViaCacheAfterCanRead() throws IOException { - if (!canRead()) { - throw new IOException(ENTRY_NOT_READY_MESSAGE); - } - return cache.getByteBuffer(this); - } - - /** - * Read the byte buffer directly. - * - * @throws IOException failing to read. - */ - ByteBuffer read() throws IOException { - if (!canRead()) { - throw new IOException("Target file hasn't been specified."); - } - // TODO: Reuse the file stream - try (WALInputStream is = openReadFileStream()) { - is.skipToGivenLogicalPosition(position); - ByteBuffer buffer = ByteBuffer.allocate(size); - is.read(buffer); - return buffer; - } - } - - /** - * Open the read file channel for this wal entry, this method will retry automatically when the - * file is sealed when opening the file channel. - * - * @throws IOException failing to open the file channel. - */ - public FileChannel openReadFileChannel() throws IOException { - if (isInSealedFile()) { - walFile = walNode.getWALFile(walFileVersionId); - return FileChannel.open(walFile.toPath(), StandardOpenOption.READ); - } else { - try { - walFile = walNode.getWALFile(walFileVersionId); - return FileChannel.open(walFile.toPath(), StandardOpenOption.READ); - } catch (IOException e) { - // unsealed file may be renamed after sealed, so we should try again - if (isInSealedFile()) { - walFile = walNode.getWALFile(walFileVersionId); - return FileChannel.open(walFile.toPath(), StandardOpenOption.READ); - } else { - throw e; - } - } - } - } - - public WALInputStream openReadFileStream() throws IOException { - // TODO: Refactor this part of code - if (isInSealedFile()) { - walFile = walNode.getWALFile(walFileVersionId); - return new WALInputStream(walFile); - } else { - try { - walFile = walNode.getWALFile(walFileVersionId); - return new WALInputStream(walFile); - } catch (IOException e) { - // unsealed file may be renamed after sealed, so we should try again - if (isInSealedFile()) { - walFile = walNode.getWALFile(walFileVersionId); - return new WALInputStream(walFile); - } else { - throw e; - } - } - } - } - - public File getWalFile() { - return walFile; - } - - /** Return true only when the tuple(file, position, size) is ready. */ - public boolean canRead() { - return walFileVersionId >= 0; - } - - /** Return true only when this wal file is sealed. */ - public boolean isInSealedFile() { - if (walNode == null || !canRead()) { - throw new RuntimeException(ENTRY_NOT_READY_MESSAGE); - } - return walFileVersionId < walNode.getCurrentWALFileVersion(); - } - - public void setWalNode(WALNode walNode, long memTableId) { - this.walNode = walNode; - identifier = walNode.getIdentifier(); - cache = WALInsertNodeCache.getInstance(); - } - - public String getIdentifier() { - return identifier; - } - - public void setEntryPosition(long walFileVersionId, long position, WALEntryValue value) { - this.position = position; - this.walFileVersionId = walFileVersionId; - if (cache != null && value instanceof InsertNode) { - cache.cacheInsertNodeIfNeeded(this, (InsertNode) value); - } - } - - public long getPosition() { - return position; - } - - public long getWalFileVersionId() { - return walFileVersionId; - } - - public void setSize(int size) { - this.size = size; - } - - public int getSize() { - return size; - } - - @Override - public int hashCode() { - return Objects.hash(identifier, walFileVersionId, position); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - WALEntryPosition that = (WALEntryPosition) o; - return identifier.equals(that.identifier) - && walFileVersionId == that.walFileVersionId - && position == that.position; - } -} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/WALInsertNodeCache.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/WALInsertNodeCache.java deleted file mode 100644 index f69dc4a08263..000000000000 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/WALInsertNodeCache.java +++ /dev/null @@ -1,335 +0,0 @@ -/* - * 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.storageengine.dataregion.wal.utils; - -import org.apache.iotdb.commons.pipe.config.PipeConfig; -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.pipe.resource.PipeDataNodeResourceManager; -import org.apache.iotdb.db.pipe.resource.memory.InsertNodeMemoryEstimator; -import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryBlockType; -import org.apache.iotdb.db.pipe.resource.memory.PipeModelFixedMemoryBlock; -import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; -import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode; -import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.WALEntry; -import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.WALEntryType; -import org.apache.iotdb.db.storageengine.dataregion.wal.io.WALByteBufReader; - -import com.github.benmanes.caffeine.cache.CacheLoader; -import com.github.benmanes.caffeine.cache.Caffeine; -import com.github.benmanes.caffeine.cache.LoadingCache; -import com.github.benmanes.caffeine.cache.Weigher; -import org.apache.tsfile.utils.Pair; -import org.checkerframework.checker.nullness.qual.NonNull; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.HashMap; -import java.util.Map; -import java.util.Objects; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; - -/** This cache is used by {@link WALEntryPosition}. */ -public class WALInsertNodeCache { - - private static final Logger LOGGER = LoggerFactory.getLogger(WALInsertNodeCache.class); - private static final IoTDBConfig CONFIG = IoTDBDescriptor.getInstance().getConfig(); - private static final PipeConfig PIPE_CONFIG = PipeConfig.getInstance(); - - private static PipeModelFixedMemoryBlock walModelFixedMemory = null; - - // LRU cache, find Pair by WALEntryPosition - private final LoadingCache> lruCache; - - // ids of all pinned memTables - private final Set memTablesNeedSearch = ConcurrentHashMap.newKeySet(); - - private volatile boolean hasPipeRunning = false; - - private WALInsertNodeCache() { - if (walModelFixedMemory == null) { - init(); - } - - final long requestedAllocateSize = - (long) - (PipeDataNodeResourceManager.memory().getTotalNonFloatingMemorySizeInBytes() - * PIPE_CONFIG.getPipeDataStructureWalMemoryProportion()); - - lruCache = - Caffeine.newBuilder() - .maximumWeight(requestedAllocateSize) - .weigher( - (Weigher>) - (position, pair) -> { - long weightInLong = 0L; - if (pair.right != null) { - weightInLong = InsertNodeMemoryEstimator.sizeOf(pair.right); - } else { - weightInLong = position.getSize(); - } - if (weightInLong <= 0) { - return Integer.MAX_VALUE; - } - final int weightInInt = (int) weightInLong; - return weightInInt != weightInLong ? Integer.MAX_VALUE : weightInInt; - }) - .recordStats() - .build(new WALInsertNodeCacheLoader()); - } - - // please call this method at PipeLauncher - public static void init() { - if (walModelFixedMemory != null) { - return; - } - try { - // Allocate memory for the fixed memory block of WAL - walModelFixedMemory = - PipeDataNodeResourceManager.memory() - .forceAllocateForModelFixedMemoryBlock( - (long) - (PipeDataNodeResourceManager.memory().getTotalNonFloatingMemorySizeInBytes() - * PIPE_CONFIG.getPipeDataStructureWalMemoryProportion()), - PipeMemoryBlockType.WAL); - } catch (Exception e) { - LOGGER.error("Failed to initialize WAL model fixed memory block", e); - walModelFixedMemory = - PipeDataNodeResourceManager.memory() - .forceAllocateForModelFixedMemoryBlock(0, PipeMemoryBlockType.WAL); - } - } - - /////////////////////////// Getter & Setter /////////////////////////// - - public InsertNode getInsertNode(final WALEntryPosition position) { - final Pair pair = getByteBufferOrInsertNode(position); - - if (pair.getRight() != null) { - return pair.getRight(); - } - - if (pair.getLeft() == null) { - throw new IllegalStateException(); - } - - try { - // multi pipes may share the same wal entry, so we need to wrap the byte[] into - // different ByteBuffer for each pipe - final InsertNode insertNode = parse(ByteBuffer.wrap(pair.getLeft().array())); - pair.setRight(insertNode); - return insertNode; - } catch (final Exception e) { - LOGGER.error( - "Parsing failed when recovering insertNode from wal, walFile:{}, position:{}, size:{}, exception:", - position.getWalFile(), - position.getPosition(), - position.getSize(), - e); - throw e; - } - } - - private InsertNode parse(final ByteBuffer buffer) { - final PlanNode node = WALEntry.deserializeForConsensus(buffer); - if (node instanceof InsertNode) { - return (InsertNode) node; - } else { - return null; - } - } - - public ByteBuffer getByteBuffer(final WALEntryPosition position) { - Pair pair = getByteBufferOrInsertNode(position); - - if (pair.getLeft() != null) { - // multi pipes may share the same wal entry, so we need to wrap the byte[] into - // different ByteBuffer for each pipe - return ByteBuffer.wrap(pair.getLeft().array()); - } - - // forbid multi threads to invalidate and load the same entry - synchronized (this) { - lruCache.invalidate(position); - pair = getByteBufferOrInsertNode(position); - } - - if (pair.getLeft() == null) { - throw new IllegalStateException(); - } - - return ByteBuffer.wrap(pair.getLeft().array()); - } - - public Pair getByteBufferOrInsertNode(final WALEntryPosition position) { - hasPipeRunning = true; - - final Pair pair = lruCache.get(position); - - if (pair == null) { - throw new IllegalStateException(); - } - - return pair; - } - - public Pair getByteBufferOrInsertNodeIfPossible( - final WALEntryPosition position) { - hasPipeRunning = true; - return lruCache.getIfPresent(position); - } - - public void cacheInsertNodeIfNeeded( - final WALEntryPosition walEntryPosition, final InsertNode insertNode) { - // reduce memory usage - if (hasPipeRunning) { - lruCache.put(walEntryPosition, new Pair<>(null, insertNode)); - } - } - - //////////////////////////// APIs provided for metric framework //////////////////////////// - - public double getCacheHitRate() { - return Objects.nonNull(lruCache) ? lruCache.stats().hitRate() : 0; - } - - public double getCacheHitCount() { - return Objects.nonNull(lruCache) ? lruCache.stats().hitCount() : 0; - } - - public double getCacheRequestCount() { - return Objects.nonNull(lruCache) ? lruCache.stats().requestCount() : 0; - } - - /////////////////////////// MemTable /////////////////////////// - - public void addMemTable(final long memTableId) { - memTablesNeedSearch.add(memTableId); - } - - public void removeMemTable(final long memTableId) { - memTablesNeedSearch.remove(memTableId); - } - - /////////////////////////// Cache Loader /////////////////////////// - - class WALInsertNodeCacheLoader - implements CacheLoader> { - - @Override - public @Nullable Pair load(@NonNull final WALEntryPosition key) - throws Exception { - return new Pair<>(key.read(), null); - } - - /** Batch load all wal entries in the file when any one key is absent. */ - @Override - public @NonNull Map<@NonNull WALEntryPosition, @NonNull Pair> loadAll( - @NonNull final Iterable walEntryPositions) { - final Map> loadedEntries = new HashMap<>(); - - for (final WALEntryPosition walEntryPosition : walEntryPositions) { - if (loadedEntries.containsKey(walEntryPosition) || !walEntryPosition.canRead()) { - continue; - } - - final long walFileVersionId = walEntryPosition.getWalFileVersionId(); - - // load one when wal file is not sealed - if (!walEntryPosition.isInSealedFile()) { - try { - loadedEntries.put(walEntryPosition, load(walEntryPosition)); - } catch (final Exception e) { - LOGGER.info( - "Fail to cache wal entries from the wal file with version id {}", - walFileVersionId, - e); - } - continue; - } - - // batch load when wal file is sealed - long position = 0; - try (final WALByteBufReader walByteBufReader = new WALByteBufReader(walEntryPosition)) { - while (walByteBufReader.hasNext()) { - // see WALInfoEntry#serialize, entry type + memtable id + plan node type - final ByteBuffer buffer = walByteBufReader.next(); - - final int size = buffer.capacity(); - final WALEntryType type = WALEntryType.valueOf(buffer.get()); - final long memTableId = buffer.getLong(); - - if ((memTablesNeedSearch.contains(memTableId) - || walEntryPosition.getPosition() == position) - && type.needSearch()) { - buffer.clear(); - loadedEntries.put( - new WALEntryPosition( - walEntryPosition.getIdentifier(), walFileVersionId, position, size), - new Pair<>(buffer, null)); - } - - position += size; - } - } catch (final IOException e) { - LOGGER.info( - "Fail to cache wal entries from the wal file with version id {}", - walFileVersionId, - e); - } - } - - return loadedEntries; - } - } - - /////////////////////////// Singleton /////////////////////////// - - public static WALInsertNodeCache getInstance() { - return InstanceHolder.INSTANCE; - } - - private static class InstanceHolder { - - public static final WALInsertNodeCache INSTANCE = new WALInsertNodeCache(); - - private InstanceHolder() { - // forbidding instantiation - } - } - - /////////////////////////// Test Only /////////////////////////// - - @TestOnly - boolean contains(WALEntryPosition position) { - return lruCache.getIfPresent(position) != null; - } - - @TestOnly - public void clear() { - lruCache.invalidateAll(); - memTablesNeedSearch.clear(); - } -} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/listener/WALFlushListener.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/listener/WALFlushListener.java index 8c84a0cb0fe5..7896ea01dcfb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/listener/WALFlushListener.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/listener/WALFlushListener.java @@ -19,20 +19,10 @@ package org.apache.iotdb.db.storageengine.dataregion.wal.utils.listener; -import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.WALEntryValue; -import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALEntryHandler; - /** This class helps judge whether wal is flushed to the storage device. */ public class WALFlushListener extends AbstractResultListener { - // handler for pipeline, only exists when value is InsertNode - private final WALEntryHandler walEntryHandler; - public WALFlushListener(boolean wait, WALEntryValue value) { + public WALFlushListener(boolean wait) { super(wait); - walEntryHandler = new WALEntryHandler(value); - } - - public WALEntryHandler getWalEntryHandler() { - return walEntryHandler; } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipeRealtimeExtractTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipeRealtimeExtractTest.java index 46e957917068..edaa11c49439 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipeRealtimeExtractTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipeRealtimeExtractTest.java @@ -33,7 +33,6 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowNode; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; -import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALEntryHandler; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; import org.apache.iotdb.pipe.api.event.Event; @@ -63,8 +62,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Function; -import static org.mockito.Mockito.mock; - public class PipeRealtimeExtractTest { private static final Logger LOGGER = LoggerFactory.getLogger(PipeRealtimeExtractTest.class); @@ -282,7 +279,6 @@ private Future write2DataRegion(int writeNum, String dataRegionId, int startN PipeInsertionDataNodeListener.getInstance() .listenToInsertNode( dataRegionId, - mock(WALEntryHandler.class), new InsertRowNode( new PlanNodeId(String.valueOf(i)), new PartialPath(device), @@ -296,7 +292,6 @@ private Future write2DataRegion(int writeNum, String dataRegionId, int startN PipeInsertionDataNodeListener.getInstance() .listenToInsertNode( dataRegionId, - mock(WALEntryHandler.class), new InsertRowNode( new PlanNodeId(String.valueOf(i)), new PartialPath(device), diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeWALHardlinkResourceManagerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeWALHardlinkResourceManagerTest.java index 33e47af0a8dd..d403caf20d05 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeWALHardlinkResourceManagerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeWALHardlinkResourceManagerTest.java @@ -21,7 +21,6 @@ import org.apache.iotdb.commons.conf.IoTDBConstant; import org.apache.iotdb.commons.utils.FileUtils; -import org.apache.iotdb.db.pipe.resource.wal.hardlink.PipeWALHardlinkResourceManager; import org.junit.After; import org.junit.Assert; diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/checkpoint/CheckpointManagerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/checkpoint/CheckpointManagerTest.java deleted file mode 100644 index 9c9290f98529..000000000000 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/checkpoint/CheckpointManagerTest.java +++ /dev/null @@ -1,172 +0,0 @@ -/* - * 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.storageengine.dataregion.wal.checkpoint; - -import org.apache.iotdb.db.conf.IoTDBConfig; -import org.apache.iotdb.db.conf.IoTDBDescriptor; -import org.apache.iotdb.db.storageengine.dataregion.memtable.PrimitiveMemTable; -import org.apache.iotdb.db.storageengine.dataregion.wal.io.CheckpointReader; -import org.apache.iotdb.db.storageengine.dataregion.wal.recover.CheckpointRecoverUtils; -import org.apache.iotdb.db.storageengine.dataregion.wal.utils.CheckpointFileUtils; -import org.apache.iotdb.db.utils.EnvironmentUtils; -import org.apache.iotdb.db.utils.constant.TestConstant; - -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import java.io.File; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.Callable; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -public class CheckpointManagerTest { - private static final String database = "root.test"; - private static final String dataRegionId = "1"; - private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig(); - private static final String identifier = String.valueOf(Integer.MAX_VALUE); - private static final String logDirectory = TestConstant.BASE_OUTPUT_PATH.concat("wal-test"); - private CheckpointManager checkpointManager; - private long prevFileSize; - - @Before - public void setUp() throws Exception { - EnvironmentUtils.cleanDir(logDirectory); - prevFileSize = config.getCheckpointFileSizeThresholdInByte(); - config.setCheckpointFileSizeThresholdInByte(10 * 1024); - checkpointManager = new CheckpointManager(identifier, logDirectory); - } - - @After - public void tearDown() throws Exception { - checkpointManager.close(); - config.setCheckpointFileSizeThresholdInByte(prevFileSize); - EnvironmentUtils.cleanDir(logDirectory); - } - - @Test - public void testNewFile() { - Checkpoint initCheckpoint = - new Checkpoint(CheckpointType.GLOBAL_MEMORY_TABLE_INFO, Collections.emptyList()); - List expectedCheckpoints = Collections.singletonList(initCheckpoint); - CheckpointReader checkpointReader = - new CheckpointReader( - new File(logDirectory + File.separator + CheckpointFileUtils.getLogFileName(0))); - List actualCheckpoints = checkpointReader.getCheckpoints(); - assertEquals(expectedCheckpoints, actualCheckpoints); - } - - @Test - public void testConcurrentWrite() throws Exception { - // start write threads to write concurrently - int threadsNum = 5; - ExecutorService executorService = Executors.newFixedThreadPool(threadsNum); - List> futures = new ArrayList<>(); - Map expectedMemTableId2Info = new ConcurrentHashMap<>(); - Map versionId2memTableId = new ConcurrentHashMap<>(); - // create 10 memTables, and flush the first 5 of them - int memTablesNum = 10; - for (int i = 0; i < memTablesNum; ++i) { - long versionId = i; - Callable writeTask = - () -> { - String tsFilePath = logDirectory + File.separator + versionId + ".tsfile"; - MemTableInfo memTableInfo = - new MemTableInfo( - new PrimitiveMemTable(database, dataRegionId), tsFilePath, versionId); - versionId2memTableId.put(versionId, memTableInfo.getMemTableId()); - checkpointManager.makeCreateMemTableCPInMemory(memTableInfo); - checkpointManager.makeCreateMemTableCPOnDisk(memTableInfo.getMemTableId()); - if (versionId < memTablesNum / 2) { - checkpointManager.makeFlushMemTableCP(versionId2memTableId.get(versionId)); - } else { - expectedMemTableId2Info.put(memTableInfo.getMemTableId(), memTableInfo); - } - return null; - }; - Future future = executorService.submit(writeTask); - futures.add(future); - } - // wait until all write tasks are done - for (Future future : futures) { - future.get(); - } - executorService.shutdown(); - // check first valid version id - assertEquals(memTablesNum / 2, checkpointManager.getFirstValidWALVersionId()); - // recover info from checkpoint file - Map actualMemTableId2Info = - CheckpointRecoverUtils.recoverMemTableInfo(new File(logDirectory)).getMemTableId2Info(); - assertEquals(expectedMemTableId2Info, actualMemTableId2Info); - } - - @Test - public void testTriggerLogRoller() { - // create memTables until reach LOG_SIZE_LIMIT, and flush the first 5 of them - int size = 0; - long versionId = 0; - Map expectedMemTableId2Info = new HashMap<>(); - Map versionId2memTableId = new HashMap<>(); - while (size < config.getCheckpointFileSizeThresholdInByte()) { - ++versionId; - String tsFilePath = logDirectory + File.separator + versionId + ".tsfile"; - MemTableInfo memTableInfo = - new MemTableInfo(new PrimitiveMemTable(database, dataRegionId), tsFilePath, versionId); - versionId2memTableId.put(versionId, memTableInfo.getMemTableId()); - Checkpoint checkpoint = - new Checkpoint( - CheckpointType.CREATE_MEMORY_TABLE, Collections.singletonList(memTableInfo)); - size += checkpoint.serializedSize(); - checkpointManager.makeCreateMemTableCPInMemory(memTableInfo); - checkpointManager.makeCreateMemTableCPOnDisk(memTableInfo.getMemTableId()); - if (versionId < 5) { - checkpoint = - new Checkpoint( - CheckpointType.FLUSH_MEMORY_TABLE, Collections.singletonList(memTableInfo)); - size += checkpoint.serializedSize(); - checkpointManager.makeFlushMemTableCP(versionId2memTableId.get(versionId)); - } else { - expectedMemTableId2Info.put(memTableInfo.getMemTableId(), memTableInfo); - } - } - checkpointManager.fsyncCheckpointFile(); - // check first valid version id - assertEquals(5, checkpointManager.getFirstValidWALVersionId()); - // check checkpoint files - assertFalse( - new File(logDirectory + File.separator + CheckpointFileUtils.getLogFileName(0)).exists()); - assertTrue( - new File(logDirectory + File.separator + CheckpointFileUtils.getLogFileName(1)).exists()); - // recover info from checkpoint file - Map actualMemTableId2Info = - CheckpointRecoverUtils.recoverMemTableInfo(new File(logDirectory)).getMemTableId2Info(); - assertEquals(expectedMemTableId2Info, actualMemTableId2Info); - } -} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALEntryHandlerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALEntryHandlerTest.java deleted file mode 100644 index d5913e54355c..000000000000 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALEntryHandlerTest.java +++ /dev/null @@ -1,586 +0,0 @@ -/* - * 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.storageengine.dataregion.wal.node; - -import org.apache.iotdb.commons.exception.IllegalPathException; -import org.apache.iotdb.commons.path.PartialPath; -import org.apache.iotdb.consensus.iot.log.ConsensusReqReader; -import org.apache.iotdb.db.conf.IoTDBConfig; -import org.apache.iotdb.db.conf.IoTDBDescriptor; -import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; -import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowNode; -import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowsNode; -import org.apache.iotdb.db.storageengine.dataregion.memtable.IMemTable; -import org.apache.iotdb.db.storageengine.dataregion.memtable.PrimitiveMemTable; -import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.WALEntry; -import org.apache.iotdb.db.storageengine.dataregion.wal.checkpoint.CheckpointManager; -import org.apache.iotdb.db.storageengine.dataregion.wal.checkpoint.MemTableInfo; -import org.apache.iotdb.db.storageengine.dataregion.wal.exception.MemTablePinException; -import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALEntryHandler; -import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALInsertNodeCache; -import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALMode; -import org.apache.iotdb.db.storageengine.dataregion.wal.utils.listener.WALFlushListener; -import org.apache.iotdb.db.utils.EnvironmentUtils; -import org.apache.iotdb.db.utils.constant.TestConstant; - -import org.apache.tsfile.common.conf.TSFileConfig; -import org.apache.tsfile.enums.TSDataType; -import org.apache.tsfile.utils.Binary; -import org.apache.tsfile.write.schema.MeasurementSchema; -import org.awaitility.Awaitility; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -public class WALEntryHandlerTest { - private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig(); - private static final String identifier1 = String.valueOf(Integer.MAX_VALUE); - private static final String identifier2 = String.valueOf(Integer.MAX_VALUE - 1); - private static final String logDirectory1 = - TestConstant.BASE_OUTPUT_PATH.concat("wal-test" + identifier1); - private static final String logDirectory2 = - TestConstant.BASE_OUTPUT_PATH.concat("wal-test" + identifier2); - - private static final String databasePath = "root.test_sg"; - private static final String devicePath = databasePath + ".test_d"; - private static final String dataRegionId = "1"; - private WALMode prevMode; - private WALNode walNode1; - private WALNode walNode2; - - @Before - public void setUp() throws Exception { - EnvironmentUtils.cleanDir(logDirectory1); - EnvironmentUtils.cleanDir(logDirectory2); - prevMode = config.getWalMode(); - config.setWalMode(WALMode.SYNC); - walNode1 = new WALNode(identifier1, logDirectory1); - walNode2 = new WALNode(identifier2, logDirectory2); - } - - @After - public void tearDown() throws Exception { - walNode1.close(); - walNode2.close(); - config.setWalMode(prevMode); - EnvironmentUtils.cleanDir(logDirectory1); - EnvironmentUtils.cleanDir(logDirectory2); - WALInsertNodeCache.getInstance().clear(); - } - - @Test(expected = MemTablePinException.class) - public void pinDeletedMemTable1() throws Exception { - IMemTable memTable = new PrimitiveMemTable(databasePath, dataRegionId); - walNode1.onMemTableCreated(memTable, logDirectory1 + "/" + "fake.tsfile"); - WALFlushListener flushListener = - walNode1.log( - memTable.getMemTableId(), getInsertRowNode(devicePath, System.currentTimeMillis())); - walNode1.onMemTableFlushed(memTable); - Awaitility.await().until(() -> walNode1.isAllWALEntriesConsumed()); - // pin flushed memTable - WALEntryHandler handler = flushListener.getWalEntryHandler(); - handler.pinMemTable(); - } - - @Test(expected = MemTablePinException.class) - public void pinDeletedMemTable2() throws Exception { - IMemTable memTable = new PrimitiveMemTable(databasePath, dataRegionId); - walNode1.onMemTableCreated(memTable, logDirectory1 + "/" + "fake.tsfile"); - WALFlushListener flushListener = - walNode1.log( - memTable.getMemTableId(), getInsertRowsNode(devicePath, System.currentTimeMillis())); - walNode1.onMemTableFlushed(memTable); - Awaitility.await().until(() -> walNode1.isAllWALEntriesConsumed()); - // pin flushed memTable - WALEntryHandler handler = flushListener.getWalEntryHandler(); - handler.pinMemTable(); - } - - @Test - public void pinMemTable1() throws Exception { - IMemTable memTable = new PrimitiveMemTable(databasePath, dataRegionId); - walNode1.onMemTableCreated(memTable, logDirectory1 + "/" + "fake.tsfile"); - InsertRowNode node1 = getInsertRowNode(devicePath, System.currentTimeMillis()); - node1.setSearchIndex(1); - WALFlushListener flushListener = walNode1.log(memTable.getMemTableId(), node1); - // pin memTable - WALEntryHandler handler = flushListener.getWalEntryHandler(); - handler.pinMemTable(); - // roll wal file - walNode1.rollWALFile(); - InsertRowNode node2 = getInsertRowNode(devicePath, System.currentTimeMillis()); - node2.setSearchIndex(2); - walNode1.log(memTable.getMemTableId(), node2); - walNode1.onMemTableFlushed(memTable); - walNode1.rollWALFile(); - // find node1 - ConsensusReqReader.ReqIterator itr = walNode1.getReqIterator(1); - assertTrue(itr.hasNext()); - assertEquals( - node1, - WALEntry.deserializeForConsensus(itr.next().getRequests().get(0).serializeToByteBuffer())); - // try to delete flushed but pinned memTable - walNode1.deleteOutdatedFiles(); - // try to find node1 - itr = walNode1.getReqIterator(1); - assertTrue(itr.hasNext()); - assertEquals( - node1, - WALEntry.deserializeForConsensus(itr.next().getRequests().get(0).serializeToByteBuffer())); - } - - @Test - public void pinMemTable2() throws Exception { - IMemTable memTable = new PrimitiveMemTable(databasePath, dataRegionId); - walNode1.onMemTableCreated(memTable, logDirectory1 + "/" + "fake.tsfile"); - InsertRowsNode node1 = getInsertRowsNode(devicePath, System.currentTimeMillis()); - node1.setSearchIndex(1); - WALFlushListener flushListener = walNode1.log(memTable.getMemTableId(), node1); - // pin memTable - WALEntryHandler handler = flushListener.getWalEntryHandler(); - handler.pinMemTable(); - // roll wal file - walNode1.rollWALFile(); - InsertRowsNode node2 = getInsertRowsNode(devicePath, System.currentTimeMillis()); - node2.setSearchIndex(2); - walNode1.log(memTable.getMemTableId(), node2); - walNode1.onMemTableFlushed(memTable); - walNode1.rollWALFile(); - // find node1 - ConsensusReqReader.ReqIterator itr = walNode1.getReqIterator(1); - assertTrue(itr.hasNext()); - assertEquals( - node1, - WALEntry.deserializeForConsensus(itr.next().getRequests().get(0).serializeToByteBuffer())); - // try to delete flushed but pinned memTable - walNode1.deleteOutdatedFiles(); - // try to find node1 - itr = walNode1.getReqIterator(1); - assertTrue(itr.hasNext()); - assertEquals( - node1, - WALEntry.deserializeForConsensus(itr.next().getRequests().get(0).serializeToByteBuffer())); - } - - @Test(expected = MemTablePinException.class) - public void unpinDeletedMemTable1() throws Exception { - IMemTable memTable = new PrimitiveMemTable(databasePath, dataRegionId); - walNode1.onMemTableCreated(memTable, logDirectory1 + "/" + "fake.tsfile"); - WALFlushListener flushListener = - walNode1.log( - memTable.getMemTableId(), getInsertRowNode(devicePath, System.currentTimeMillis())); - walNode1.onMemTableFlushed(memTable); - // pin flushed memTable - WALEntryHandler handler = flushListener.getWalEntryHandler(); - handler.unpinMemTable(); - } - - @Test(expected = MemTablePinException.class) - public void unpinDeletedMemTable2() throws Exception { - IMemTable memTable = new PrimitiveMemTable(databasePath, dataRegionId); - walNode1.onMemTableCreated(memTable, logDirectory1 + "/" + "fake.tsfile"); - WALFlushListener flushListener = - walNode1.log( - memTable.getMemTableId(), getInsertRowsNode(devicePath, System.currentTimeMillis())); - walNode1.onMemTableFlushed(memTable); - // pin flushed memTable - WALEntryHandler handler = flushListener.getWalEntryHandler(); - handler.unpinMemTable(); - } - - @Test - public void unpinFlushedMemTable1() throws Exception { - IMemTable memTable = new PrimitiveMemTable(databasePath, dataRegionId); - walNode1.onMemTableCreated(memTable, logDirectory1 + "/" + "fake.tsfile"); - WALFlushListener flushListener = - walNode1.log( - memTable.getMemTableId(), getInsertRowNode(devicePath, System.currentTimeMillis())); - WALEntryHandler handler = flushListener.getWalEntryHandler(); - // pin twice - handler.pinMemTable(); - handler.pinMemTable(); - walNode1.onMemTableFlushed(memTable); - Awaitility.await().until(() -> walNode1.isAllWALEntriesConsumed()); - // unpin 1 - CheckpointManager checkpointManager = walNode1.getCheckpointManager(); - handler.unpinMemTable(); - MemTableInfo oldestMemTableInfo = checkpointManager.getOldestUnpinnedMemTableInfo(); - assertNull(oldestMemTableInfo); - // unpin 2 - handler.unpinMemTable(); - assertNull(checkpointManager.getOldestUnpinnedMemTableInfo()); - } - - @Test - public void unpinFlushedMemTable2() throws Exception { - IMemTable memTable = new PrimitiveMemTable(databasePath, dataRegionId); - walNode1.onMemTableCreated(memTable, logDirectory1 + "/" + "fake.tsfile"); - WALFlushListener flushListener = - walNode1.log( - memTable.getMemTableId(), getInsertRowsNode(devicePath, System.currentTimeMillis())); - WALEntryHandler handler = flushListener.getWalEntryHandler(); - // pin twice - handler.pinMemTable(); - handler.pinMemTable(); - walNode1.onMemTableFlushed(memTable); - Awaitility.await().until(() -> walNode1.isAllWALEntriesConsumed()); - // unpin 1 - CheckpointManager checkpointManager = walNode1.getCheckpointManager(); - handler.unpinMemTable(); - MemTableInfo oldestMemTableInfo = checkpointManager.getOldestUnpinnedMemTableInfo(); - assertNull(oldestMemTableInfo); - // unpin 2 - handler.unpinMemTable(); - assertNull(checkpointManager.getOldestUnpinnedMemTableInfo()); - } - - @Test - public void unpinMemTable1() throws Exception { - IMemTable memTable = new PrimitiveMemTable(databasePath, dataRegionId); - walNode1.onMemTableCreated(memTable, logDirectory1 + "/" + "fake.tsfile"); - InsertRowNode node1 = getInsertRowNode(devicePath, System.currentTimeMillis()); - node1.setSearchIndex(1); - WALFlushListener flushListener = walNode1.log(memTable.getMemTableId(), node1); - // pin memTable - WALEntryHandler handler = flushListener.getWalEntryHandler(); - handler.pinMemTable(); - walNode1.onMemTableFlushed(memTable); - // roll wal file - walNode1.rollWALFile(); - walNode1.rollWALFile(); - // find node1 - ConsensusReqReader.ReqIterator itr = walNode1.getReqIterator(1); - assertTrue(itr.hasNext()); - assertEquals( - node1, - WALEntry.deserializeForConsensus(itr.next().getRequests().get(0).serializeToByteBuffer())); - // unpin flushed memTable - handler.unpinMemTable(); - // try to delete flushed but pinned memTable - walNode1.deleteOutdatedFiles(); - // try to find node1 - itr = walNode1.getReqIterator(1); - assertFalse(itr.hasNext()); - } - - @Test - public void unpinMemTable2() throws Exception { - IMemTable memTable = new PrimitiveMemTable(databasePath, dataRegionId); - walNode1.onMemTableCreated(memTable, logDirectory1 + "/" + "fake.tsfile"); - InsertRowsNode node1 = getInsertRowsNode(devicePath, System.currentTimeMillis()); - node1.setSearchIndex(1); - WALFlushListener flushListener = walNode1.log(memTable.getMemTableId(), node1); - // pin memTable - WALEntryHandler handler = flushListener.getWalEntryHandler(); - handler.pinMemTable(); - walNode1.onMemTableFlushed(memTable); - // roll wal file - walNode1.rollWALFile(); - walNode1.rollWALFile(); - // find node1 - ConsensusReqReader.ReqIterator itr = walNode1.getReqIterator(1); - assertTrue(itr.hasNext()); - assertEquals( - node1, - WALEntry.deserializeForConsensus(itr.next().getRequests().get(0).serializeToByteBuffer())); - // unpin flushed memTable - handler.unpinMemTable(); - // try to delete flushed but pinned memTable - walNode1.deleteOutdatedFiles(); - // try to find node1 - itr = walNode1.getReqIterator(1); - assertFalse(itr.hasNext()); - } - - @Test - public void getUnFlushedValue1() throws Exception { - IMemTable memTable = new PrimitiveMemTable(databasePath, dataRegionId); - walNode1.onMemTableCreated(memTable, logDirectory1 + "/" + "fake.tsfile"); - InsertRowNode node1 = getInsertRowNode(devicePath, System.currentTimeMillis()); - node1.setSearchIndex(1); - WALFlushListener flushListener = walNode1.log(memTable.getMemTableId(), node1); - // pin memTable - WALEntryHandler handler = flushListener.getWalEntryHandler(); - handler.pinMemTable(); - walNode1.onMemTableFlushed(memTable); - assertEquals(node1, handler.getInsertNode()); - } - - @Test - public void getUnFlushedValue2() throws Exception { - IMemTable memTable = new PrimitiveMemTable(databasePath, dataRegionId); - walNode1.onMemTableCreated(memTable, logDirectory1 + "/" + "fake.tsfile"); - InsertRowsNode node1 = getInsertRowsNode(devicePath, System.currentTimeMillis()); - node1.setSearchIndex(1); - WALFlushListener flushListener = walNode1.log(memTable.getMemTableId(), node1); - // pin memTable - WALEntryHandler handler = flushListener.getWalEntryHandler(); - handler.pinMemTable(); - walNode1.onMemTableFlushed(memTable); - assertEquals(node1, handler.getInsertNode()); - } - - @Test - public void getFlushedValue1() throws Exception { - IMemTable memTable = new PrimitiveMemTable(databasePath, dataRegionId); - walNode1.onMemTableCreated(memTable, logDirectory1 + "/" + "fake.tsfile"); - InsertRowNode node1 = getInsertRowNode(devicePath, System.currentTimeMillis()); - node1.setSearchIndex(1); - WALFlushListener flushListener = walNode1.log(memTable.getMemTableId(), node1); - // pin memTable - WALEntryHandler handler = flushListener.getWalEntryHandler(); - handler.pinMemTable(); - walNode1.onMemTableFlushed(memTable); - // wait until wal flushed - Awaitility.await().until(() -> walNode1.isAllWALEntriesConsumed()); - assertEquals(node1, handler.getInsertNode()); - } - - @Test - public void getFlushedValue2() throws Exception { - IMemTable memTable = new PrimitiveMemTable(databasePath, dataRegionId); - walNode1.onMemTableCreated(memTable, logDirectory1 + "/" + "fake.tsfile"); - InsertRowsNode node1 = getInsertRowsNode(devicePath, System.currentTimeMillis()); - node1.setSearchIndex(1); - WALFlushListener flushListener = walNode1.log(memTable.getMemTableId(), node1); - // pin memTable - WALEntryHandler handler = flushListener.getWalEntryHandler(); - handler.pinMemTable(); - walNode1.onMemTableFlushed(memTable); - // wait until wal flushed - Awaitility.await().until(() -> walNode1.isAllWALEntriesConsumed()); - assertEquals(node1, handler.getInsertNode()); - } - - @Test - public void testConcurrentGetValue1() throws Exception { - int threadsNum = 10; - ExecutorService executorService = Executors.newFixedThreadPool(threadsNum); - List> futures = new ArrayList<>(); - for (int i = 0; i < threadsNum; ++i) { - WALNode walNode = i % 2 == 0 ? walNode1 : walNode2; - String logDirectory = i % 2 == 0 ? logDirectory1 : logDirectory2; - Callable writeTask = - () -> { - IMemTable memTable = new PrimitiveMemTable(databasePath, dataRegionId); - walNode.onMemTableCreated(memTable, logDirectory + "/" + "fake.tsfile"); - - List walFlushListeners = new ArrayList<>(); - List expectedInsertRowNodes = new ArrayList<>(); - try { - for (int j = 0; j < 1_000; ++j) { - long memTableId = memTable.getMemTableId(); - InsertRowNode node = - getInsertRowNode(devicePath + memTableId, System.currentTimeMillis()); - expectedInsertRowNodes.add(node); - WALFlushListener walFlushListener = walNode.log(memTableId, node); - walFlushListeners.add(walFlushListener); - } - } catch (IllegalPathException e) { - fail(); - } - - // wait until wal flushed - Awaitility.await().until(walNode::isAllWALEntriesConsumed); - - walFlushListeners.get(0).getWalEntryHandler().pinMemTable(); - walNode.onMemTableFlushed(memTable); - - for (int j = 0; j < expectedInsertRowNodes.size(); ++j) { - InsertRowNode expect = expectedInsertRowNodes.get(j); - InsertRowNode actual = - (InsertRowNode) walFlushListeners.get(j).getWalEntryHandler().getInsertNode(); - assertEquals(expect, actual); - } - - walFlushListeners.get(0).getWalEntryHandler().unpinMemTable(); - return null; - }; - Future future = executorService.submit(writeTask); - futures.add(future); - } - // wait until all write tasks are done - for (Future future : futures) { - future.get(); - } - executorService.shutdown(); - } - - @Test - public void testConcurrentGetValue2() throws Exception { - int threadsNum = 10; - ExecutorService executorService = Executors.newFixedThreadPool(threadsNum); - List> futures = new ArrayList<>(); - for (int i = 0; i < threadsNum; ++i) { - WALNode walNode = i % 2 == 0 ? walNode1 : walNode2; - String logDirectory = i % 2 == 0 ? logDirectory1 : logDirectory2; - Callable writeTask = - () -> { - IMemTable memTable = new PrimitiveMemTable(databasePath, dataRegionId); - walNode.onMemTableCreated(memTable, logDirectory + "/" + "fake.tsfile"); - - List walFlushListeners = new ArrayList<>(); - List expectedInsertRowsNodes = new ArrayList<>(); - try { - for (int j = 0; j < 1_000; ++j) { - long memTableId = memTable.getMemTableId(); - InsertRowsNode node = - getInsertRowsNode(devicePath + memTableId, System.currentTimeMillis()); - expectedInsertRowsNodes.add(node); - WALFlushListener walFlushListener = walNode.log(memTableId, node); - walFlushListeners.add(walFlushListener); - } - } catch (IllegalPathException e) { - fail(); - } - - // wait until wal flushed - Awaitility.await().until(walNode::isAllWALEntriesConsumed); - - walFlushListeners.get(0).getWalEntryHandler().pinMemTable(); - walNode.onMemTableFlushed(memTable); - - for (int j = 0; j < expectedInsertRowsNodes.size(); ++j) { - InsertRowsNode expect = expectedInsertRowsNodes.get(j); - InsertRowsNode actual = - (InsertRowsNode) walFlushListeners.get(j).getWalEntryHandler().getInsertNode(); - assertEquals(expect, actual); - } - - walFlushListeners.get(0).getWalEntryHandler().unpinMemTable(); - return null; - }; - Future future = executorService.submit(writeTask); - futures.add(future); - } - // wait until all write tasks are done - for (Future future : futures) { - future.get(); - } - executorService.shutdown(); - } - - private InsertRowNode getInsertRowNode(String devicePath, long time) throws IllegalPathException { - TSDataType[] dataTypes = - new TSDataType[] { - TSDataType.DOUBLE, - TSDataType.FLOAT, - TSDataType.INT64, - TSDataType.INT32, - TSDataType.BOOLEAN, - TSDataType.TEXT - }; - - Object[] columns = new Object[6]; - columns[0] = 1.0d; - columns[1] = 2f; - columns[2] = 10000L; - columns[3] = 100; - columns[4] = false; - columns[5] = new Binary("hh" + 0, TSFileConfig.STRING_CHARSET); - - InsertRowNode node = - new InsertRowNode( - new PlanNodeId(""), - new PartialPath(devicePath), - false, - new String[] {"s1", "s2", "s3", "s4", "s5", "s6"}, - dataTypes, - time, - columns, - false); - MeasurementSchema[] schemas = new MeasurementSchema[6]; - for (int i = 0; i < 6; i++) { - schemas[i] = new MeasurementSchema("s" + (i + 1), dataTypes[i]); - } - node.setMeasurementSchemas(schemas); - return node; - } - - private InsertRowsNode getInsertRowsNode(String devicePath, long firstTime) - throws IllegalPathException { - TSDataType[] dataTypes = - new TSDataType[] { - TSDataType.DOUBLE, - TSDataType.FLOAT, - TSDataType.INT64, - TSDataType.INT32, - TSDataType.BOOLEAN, - TSDataType.TEXT - }; - - Object[] columns = new Object[6]; - columns[0] = 1.0d; - columns[1] = 2f; - columns[2] = 10000L; - columns[3] = 100; - columns[4] = false; - columns[5] = new Binary("hh" + 0, TSFileConfig.STRING_CHARSET); - - InsertRowNode node = - new InsertRowNode( - new PlanNodeId(""), - new PartialPath(devicePath), - false, - new String[] {"s1", "s2", "s3", "s4", "s5", "s6"}, - dataTypes, - firstTime, - columns, - false); - MeasurementSchema[] schemas = new MeasurementSchema[6]; - for (int i = 0; i < 6; i++) { - schemas[i] = new MeasurementSchema("s" + (i + 1), dataTypes[i]); - } - node.setMeasurementSchemas(schemas); - - InsertRowsNode insertRowsNode = new InsertRowsNode(new PlanNodeId("")); - insertRowsNode.addOneInsertRowNode(node, 0); - - node = - new InsertRowNode( - new PlanNodeId(""), - new PartialPath(devicePath), - false, - new String[] {"s1", "s2", "s3", "s4", "s5", "s6"}, - dataTypes, - firstTime + 10, - columns, - false); - schemas = new MeasurementSchema[6]; - for (int i = 0; i < 6; i++) { - schemas[i] = new MeasurementSchema("s" + (i + 1), dataTypes[i]); - } - node.setMeasurementSchemas(schemas); - insertRowsNode.addOneInsertRowNode(node, 1); - return insertRowsNode; - } -} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALNodeTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALNodeTest.java index f72f55a3cb8a..df88c34ee0eb 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALNodeTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WALNodeTest.java @@ -66,7 +66,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -149,14 +148,6 @@ public void testConcurrentWrite() throws Exception { } } assertEquals(expectedInsertTabletNodes, actualInsertTabletNodes); - // check flush listeners - try { - for (WALFlushListener walFlushListener : walFlushListeners) { - assertNotEquals(WALFlushListener.Status.FAILURE, walFlushListener.waitForResult()); - } - } catch (NullPointerException e) { - // ignore - } } private void writeInsertTabletNode( @@ -329,13 +320,5 @@ public void testDeleteOutdatedFiles() throws Exception { + File.separator + WALFileUtils.getLogFileName(1, 0, WALFileStatus.CONTAINS_SEARCH_INDEX)) .exists()); - // check flush listeners - try { - for (WALFlushListener walFlushListener : walFlushListeners) { - assertNotEquals(WALFlushListener.Status.FAILURE, walFlushListener.waitForResult()); - } - } catch (NullPointerException e) { - // ignore - } } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WalDeleteOutdatedNewTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WalDeleteOutdatedNewTest.java index 593d25b6932c..6af5ed10cacc 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WalDeleteOutdatedNewTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WalDeleteOutdatedNewTest.java @@ -34,9 +34,7 @@ import org.apache.iotdb.db.storageengine.dataregion.memtable.IMemTable; import org.apache.iotdb.db.storageengine.dataregion.memtable.PrimitiveMemTable; import org.apache.iotdb.db.storageengine.dataregion.wal.exception.MemTablePinException; -import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALEntryHandler; import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALFileUtils; -import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALInsertNodeCache; import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALMode; import org.apache.iotdb.db.storageengine.dataregion.wal.utils.listener.WALFlushListener; import org.apache.iotdb.db.utils.EnvironmentUtils; @@ -88,7 +86,6 @@ public void tearDown() throws Exception { config.setDataRegionConsensusProtocolClass(prevConsensus); EnvironmentUtils.cleanDir(logDirectory1); StorageEngine.getInstance().reset(); - WALInsertNodeCache.getInstance().clear(); } /** @@ -308,8 +305,6 @@ public void test05() throws IllegalPathException, MemTablePinException { walNode1.rollWALFile(); // pin memTable - WALEntryHandler handler = listener.getWalEntryHandler(); - handler.pinMemTable(); walNode1.log( memTable0.getMemTableId(), generateInsertRowNode(devicePath, System.currentTimeMillis(), 2)); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/WALInsertNodeCacheTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/WALInsertNodeCacheTest.java deleted file mode 100644 index 552c8334f953..000000000000 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/utils/WALInsertNodeCacheTest.java +++ /dev/null @@ -1,190 +0,0 @@ -/* - * 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.storageengine.dataregion.wal.utils; - -import org.apache.iotdb.commons.exception.IllegalPathException; -import org.apache.iotdb.commons.path.PartialPath; -import org.apache.iotdb.db.conf.IoTDBConfig; -import org.apache.iotdb.db.conf.IoTDBDescriptor; -import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; -import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowNode; -import org.apache.iotdb.db.storageengine.dataregion.memtable.IMemTable; -import org.apache.iotdb.db.storageengine.dataregion.memtable.PrimitiveMemTable; -import org.apache.iotdb.db.storageengine.dataregion.wal.node.WALNode; -import org.apache.iotdb.db.storageengine.dataregion.wal.utils.listener.WALFlushListener; -import org.apache.iotdb.db.utils.EnvironmentUtils; -import org.apache.iotdb.db.utils.constant.TestConstant; - -import org.apache.tsfile.common.conf.TSFileConfig; -import org.apache.tsfile.enums.TSDataType; -import org.apache.tsfile.utils.Binary; -import org.apache.tsfile.write.schema.MeasurementSchema; -import org.awaitility.Awaitility; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; - -public class WALInsertNodeCacheTest { - private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig(); - private static final String identifier = String.valueOf(Integer.MAX_VALUE); - private static final String logDirectory = TestConstant.BASE_OUTPUT_PATH.concat("wal-test"); - private static final String databasePath = "root.test_sg"; - private static final String devicePath = databasePath + ".test_d"; - private static final String dataRegionId = "1"; - private static final WALInsertNodeCache cache = WALInsertNodeCache.getInstance(); - private WALMode prevMode; - private WALNode walNode; - - @Before - public void setUp() throws Exception { - EnvironmentUtils.cleanDir(logDirectory); - cache.clear(); - prevMode = config.getWalMode(); - config.setWalMode(WALMode.SYNC); - walNode = new WALNode(identifier, logDirectory); - } - - @After - public void tearDown() throws Exception { - walNode.close(); - cache.clear(); - config.setWalMode(prevMode); - EnvironmentUtils.cleanDir(logDirectory); - } - - @Test - public void testLoadAfterSyncBuffer() throws IllegalPathException { - try { - // Limit the wal buffer size to trigger sync Buffer when writing wal entry - walNode.setBufferSize(24); - // write memTable - IMemTable memTable = new PrimitiveMemTable(databasePath, dataRegionId); - walNode.onMemTableCreated(memTable, logDirectory + "/" + "fake.tsfile"); - InsertRowNode node1 = getInsertRowNode(System.currentTimeMillis()); - node1.setSearchIndex(1); - WALFlushListener flushListener = walNode.log(memTable.getMemTableId(), node1); - WALEntryPosition position = flushListener.getWalEntryHandler().getWalEntryPosition(); - // wait until wal flushed - walNode.rollWALFile(); - Awaitility.await().until(() -> walNode.isAllWALEntriesConsumed() && position.canRead()); - // load by cache - System.out.println(position.getPosition()); - assertEquals(node1, cache.getInsertNode(position)); - } finally { - walNode.setBufferSize(config.getWalBufferSize()); - } - } - - @Test - public void testGetInsertNodeInParallel() throws IllegalPathException { - // write memTable - IMemTable memTable = new PrimitiveMemTable(databasePath, dataRegionId); - walNode.onMemTableCreated(memTable, logDirectory + "/" + "fake.tsfile"); - InsertRowNode node1 = getInsertRowNode(System.currentTimeMillis()); - node1.setSearchIndex(1); - WALFlushListener flushListener = walNode.log(memTable.getMemTableId(), node1); - WALEntryPosition position = flushListener.getWalEntryHandler().getWalEntryPosition(); - // wait until wal flushed - walNode.rollWALFile(); - Awaitility.await().until(() -> walNode.isAllWALEntriesConsumed() && position.canRead()); - // Test getInsertNode in parallel to detect buffer concurrent problem - AtomicBoolean failure = new AtomicBoolean(false); - List threadList = new ArrayList<>(5); - for (int i = 0; i < 5; ++i) { - Thread getInsertNodeThread = - new Thread( - () -> { - if (!node1.equals(cache.getInsertNode(position))) { - failure.set(true); - } - }); - threadList.add(getInsertNodeThread); - getInsertNodeThread.start(); - } - Awaitility.await() - .until( - () -> { - for (Thread thread : threadList) { - if (thread.isAlive()) { - return false; - } - } - return true; - }); - assertFalse(failure.get()); - } - - @Test - public void testLoadUnsealedWALFile() throws Exception { - IMemTable memTable = new PrimitiveMemTable(databasePath, dataRegionId); - walNode.onMemTableCreated(memTable, logDirectory + "/" + "fake.tsfile"); - InsertRowNode node1 = getInsertRowNode(System.currentTimeMillis()); - node1.setSearchIndex(1); - WALFlushListener flushListener = walNode.log(memTable.getMemTableId(), node1); - WALEntryPosition position = flushListener.getWalEntryHandler().getWalEntryPosition(); - // wait until wal flushed - Awaitility.await().until(() -> walNode.isAllWALEntriesConsumed() && position.canRead()); - // load by cache - assertEquals(node1, cache.getInsertNode(position)); - } - - private InsertRowNode getInsertRowNode(long time) throws IllegalPathException { - TSDataType[] dataTypes = - new TSDataType[] { - TSDataType.DOUBLE, - TSDataType.FLOAT, - TSDataType.INT64, - TSDataType.INT32, - TSDataType.BOOLEAN, - TSDataType.TEXT - }; - - Object[] columns = new Object[6]; - columns[0] = 1.0d; - columns[1] = 2f; - columns[2] = 10000L; - columns[3] = 100; - columns[4] = false; - columns[5] = new Binary("hh" + 0, TSFileConfig.STRING_CHARSET); - - InsertRowNode node = - new InsertRowNode( - new PlanNodeId(""), - new PartialPath(WALInsertNodeCacheTest.devicePath), - false, - new String[] {"s1", "s2", "s3", "s4", "s5", "s6"}, - dataTypes, - time, - columns, - false); - MeasurementSchema[] schemas = new MeasurementSchema[6]; - for (int i = 0; i < 6; i++) { - schemas[i] = new MeasurementSchema("s" + (i + 1), dataTypes[i]); - } - node.setMeasurementSchemas(schemas); - return node; - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/Metric.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/Metric.java index 145b4adaf94e..39b22a3d7468 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/Metric.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/Metric.java @@ -161,7 +161,6 @@ public enum Metric { PIPE_WAL_INSERT_NODE_CACHE_REQUEST_COUNT("pipe_wal_insert_node_cache_request_count"), PIPE_EXTRACTOR_TSFILE_EPOCH_STATE("pipe_extractor_tsfile_epoch_state"), PIPE_MEM("pipe_mem"), - PIPE_PINNED_MEMTABLE_COUNT("pipe_pinned_memtable_count"), PIPE_LINKED_TSFILE_COUNT("pipe_linked_tsfile_count"), PIPE_LINKED_TSFILE_SIZE("pipe_linked_tsfile_size"), PIPE_PHANTOM_REFERENCE_COUNT("pipe_phantom_reference_count"), From 078a05724b4b96e82d9bdb4c8003d9704704b475 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 26 Jun 2025 15:30:30 +0800 Subject: [PATCH 003/185] continue deletion --- .../db/pipe/resource/wal/PipeWALResource.java | 168 ------------------ .../resource/wal/PipeWALResourceManager.java | 141 --------------- .../wal/selfhost/PipeWALSelfHostResource.java | 40 ----- .../PipeWALHardlinkResourceManagerTest.java | 113 ------------ 4 files changed, 462 deletions(-) delete mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/PipeWALResource.java delete mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/PipeWALResourceManager.java delete mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/selfhost/PipeWALSelfHostResource.java delete mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeWALHardlinkResourceManagerTest.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/PipeWALResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/PipeWALResource.java deleted file mode 100644 index 19f681bb55bb..000000000000 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/PipeWALResource.java +++ /dev/null @@ -1,168 +0,0 @@ -/* - * 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.pipe.resource.wal; - -import org.apache.iotdb.commons.exception.pipe.PipeRuntimeCriticalException; -import org.apache.iotdb.commons.exception.pipe.PipeRuntimeNonCriticalException; -import org.apache.iotdb.db.storageengine.dataregion.wal.exception.MemTablePinException; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.Closeable; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; - -public abstract class PipeWALResource implements Closeable { - - private static final Logger LOGGER = LoggerFactory.getLogger(PipeWALResource.class); - - protected final WALEntryHandler walEntryHandler; - - private final AtomicInteger referenceCount; - - public static final long WAL_MIN_TIME_TO_LIVE_IN_MS = 1000L * 20; - private final AtomicLong lastLogicalPinTime; - private final AtomicBoolean isPhysicallyPinned; - - protected PipeWALResource(WALEntryHandler walEntryHandler) { - this.walEntryHandler = walEntryHandler; - - referenceCount = new AtomicInteger(0); - - lastLogicalPinTime = new AtomicLong(0); - isPhysicallyPinned = new AtomicBoolean(false); - } - - public final void pin() throws PipeRuntimeNonCriticalException { - if (referenceCount.get() == 0) { - if (!isPhysicallyPinned.get()) { - try { - pinInternal(); - } catch (MemTablePinException e) { - throw new PipeRuntimeNonCriticalException( - String.format( - "failed to pin wal %d, because %s", - walEntryHandler.getMemTableId(), e.getMessage())); - } - isPhysicallyPinned.set(true); - LOGGER.info("wal {} is pinned by pipe engine", walEntryHandler.getMemTableId()); - } // else means the wal is already pinned, do nothing - - // no matter the wal is pinned or not, update the last pin time - lastLogicalPinTime.set(System.currentTimeMillis()); - } - - referenceCount.incrementAndGet(); - } - - protected abstract void pinInternal() - throws MemTablePinException, PipeRuntimeNonCriticalException; - - public final void unpin() throws PipeRuntimeNonCriticalException { - final int finalReferenceCount = referenceCount.get(); - - if (finalReferenceCount == 1) { - unpinPhysicallyIfOutOfTimeToLive(); - } else if (finalReferenceCount < 1) { - throw new PipeRuntimeCriticalException( - String.format( - "wal %d is unpinned more than pinned, this should not happen", - walEntryHandler.getMemTableId())); - } - - referenceCount.decrementAndGet(); - } - - protected abstract void unpinInternal() - throws MemTablePinException, PipeRuntimeNonCriticalException; - - /** - * Invalidate the wal if it is unpinned and out of time to live. - * - * @return true if the wal is invalidated, false otherwise - */ - public final boolean invalidateIfPossible() { - if (referenceCount.get() > 0) { - return false; - } - - // referenceCount.get() == 0 - return unpinPhysicallyIfOutOfTimeToLive(); - } - - /** - * Unpin the wal if it is out of time to live. - * - * @return true if the wal is unpinned physically (then it can be invalidated), false otherwise - * @throws PipeRuntimeNonCriticalException if failed to unpin WAL of memtable. - */ - private boolean unpinPhysicallyIfOutOfTimeToLive() { - if (isPhysicallyPinned.get()) { - if (System.currentTimeMillis() - lastLogicalPinTime.get() > WAL_MIN_TIME_TO_LIVE_IN_MS) { - try { - unpinInternal(); - } catch (MemTablePinException e) { - throw new PipeRuntimeNonCriticalException( - String.format( - "failed to unpin wal %d, because %s", - walEntryHandler.getMemTableId(), e.getMessage())); - } - isPhysicallyPinned.set(false); - LOGGER.info( - "wal {} is unpinned by pipe engine when checking time to live", - walEntryHandler.getMemTableId()); - return true; - } else { - return false; - } - } else { - LOGGER.info( - "wal {} is not pinned physically when checking time to live", - walEntryHandler.getMemTableId()); - return true; - } - } - - @Override - public final void close() { - if (isPhysicallyPinned.get()) { - try { - unpinInternal(); - } catch (MemTablePinException e) { - LOGGER.error( - "failed to unpin wal {} when closing pipe wal resource, because {}", - walEntryHandler.getMemTableId(), - e.getMessage()); - } - isPhysicallyPinned.set(false); - LOGGER.info( - "wal {} is unpinned by pipe engine when closing pipe wal resource", - walEntryHandler.getMemTableId()); - } - - referenceCount.set(0); - } - - public int getReferenceCount() { - return referenceCount.get(); - } -} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/PipeWALResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/PipeWALResourceManager.java deleted file mode 100644 index 83bb9939013e..000000000000 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/PipeWALResourceManager.java +++ /dev/null @@ -1,141 +0,0 @@ -/* - * 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.pipe.resource.wal; - -import org.apache.iotdb.commons.pipe.config.PipeConfig; -import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; -import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.ConcurrentModificationException; -import java.util.Iterator; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.locks.ReentrantLock; - -public abstract class PipeWALResourceManager { - - private static final Logger LOGGER = LoggerFactory.getLogger(PipeWALResourceManager.class); - - protected final Map memtableIdToPipeWALResourceMap; - - private static final int SEGMENT_LOCK_COUNT = 32; - private final ReentrantLock[] memTableIdSegmentLocks; - - protected PipeWALResourceManager() { - // memTableIdToPipeWALResourceMap can be concurrently accessed by multiple threads - memtableIdToPipeWALResourceMap = new ConcurrentHashMap<>(); - - memTableIdSegmentLocks = new ReentrantLock[SEGMENT_LOCK_COUNT]; - for (int i = 0; i < SEGMENT_LOCK_COUNT; i++) { - memTableIdSegmentLocks[i] = new ReentrantLock(); - } - - PipeDataNodeAgent.runtime() - .registerPeriodicalJob( - "PipeWALResourceManager#ttlCheck()", - this::ttlCheck, - Math.max(PipeWALResource.WAL_MIN_TIME_TO_LIVE_IN_MS / 1000, 1)); - } - - @SuppressWarnings("java:S2222") - private void ttlCheck() { - final Iterator> iterator = - memtableIdToPipeWALResourceMap.entrySet().iterator(); - final Optional logger = - PipeDataNodeResourceManager.log() - .schedule( - PipeWALResourceManager.class, - PipeConfig.getInstance().getPipeWalPinMaxLogNumPerRound(), - PipeConfig.getInstance().getPipeWalPinMaxLogIntervalRounds(), - memtableIdToPipeWALResourceMap.size()); - - final StringBuilder logBuilder = new StringBuilder(); - try { - while (iterator.hasNext()) { - final Map.Entry entry = iterator.next(); - final ReentrantLock lock = - memTableIdSegmentLocks[(int) (entry.getKey() % SEGMENT_LOCK_COUNT)]; - - lock.lock(); - try { - if (entry.getValue().invalidateIfPossible()) { - iterator.remove(); - } else { - logBuilder.append( - String.format( - "<%d , %d times> ", entry.getKey(), entry.getValue().getReferenceCount())); - } - } finally { - lock.unlock(); - } - } - } catch (final ConcurrentModificationException e) { - LOGGER.error( - "Concurrent modification issues happened, skipping the WAL in this round of ttl check", - e); - } finally { - if (logBuilder.length() > 0) { - logger.ifPresent(l -> l.info("WAL {}are still referenced", logBuilder)); - } - } - } - - public final void pin(final WALEntryHandler walEntryHandler) throws IOException { - final long memTableId = walEntryHandler.getMemTableId(); - final ReentrantLock lock = memTableIdSegmentLocks[(int) (memTableId % SEGMENT_LOCK_COUNT)]; - - lock.lock(); - try { - pinInternal(memTableId, walEntryHandler); - } finally { - lock.unlock(); - } - } - - protected abstract void pinInternal(final long memTableId, final WALEntryHandler walEntryHandler) - throws IOException; - - public final void unpin(final WALEntryHandler walEntryHandler) throws IOException { - final long memTableId = walEntryHandler.getMemTableId(); - final ReentrantLock lock = memTableIdSegmentLocks[(int) (memTableId % SEGMENT_LOCK_COUNT)]; - - lock.lock(); - try { - unpinInternal(memTableId, walEntryHandler); - } finally { - lock.unlock(); - } - } - - protected abstract void unpinInternal( - final long memTableId, final WALEntryHandler walEntryHandler) throws IOException; - - public int getPinnedWalCount() { - return Objects.nonNull(memtableIdToPipeWALResourceMap) - ? memtableIdToPipeWALResourceMap.size() - : 0; - } -} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/selfhost/PipeWALSelfHostResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/selfhost/PipeWALSelfHostResource.java deleted file mode 100644 index d93bbd8eff36..000000000000 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/wal/selfhost/PipeWALSelfHostResource.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * 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.pipe.resource.wal.selfhost; - -import org.apache.iotdb.db.pipe.resource.wal.PipeWALResource; -import org.apache.iotdb.db.storageengine.dataregion.wal.exception.MemTablePinException; - -public class PipeWALSelfHostResource extends PipeWALResource { - - public PipeWALSelfHostResource(WALEntryHandler walEntryHandler) { - super(walEntryHandler); - } - - @Override - protected void pinInternal() throws MemTablePinException { - walEntryHandler.pinMemTable(); - } - - @Override - protected void unpinInternal() throws MemTablePinException { - walEntryHandler.unpinMemTable(); - } -} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeWALHardlinkResourceManagerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeWALHardlinkResourceManagerTest.java deleted file mode 100644 index d403caf20d05..000000000000 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeWALHardlinkResourceManagerTest.java +++ /dev/null @@ -1,113 +0,0 @@ -/* - * 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.pipe.resource; - -import org.apache.iotdb.commons.conf.IoTDBConstant; -import org.apache.iotdb.commons.utils.FileUtils; - -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; - -public class PipeWALHardlinkResourceManagerTest { - private static final String ROOT_DIR = "target" + File.separator + "PipeWALHolderTest"; - - private static final String WAL_DIR = ROOT_DIR + File.separator + IoTDBConstant.WAL_FOLDER_NAME; - - private static final String WAL_NAME = WAL_DIR + File.separator + "test.wal"; - - private PipeWALHardlinkResourceManager pipeWALHardlinkResourceManager; - - @Before - public void setUp() throws Exception { - pipeWALHardlinkResourceManager = new PipeWALHardlinkResourceManager(); - - createWAL(); - } - - private void createWAL() { - File file = new File(WAL_NAME); - if (file.exists()) { - boolean ignored = file.delete(); - } - - try { - file.getParentFile().mkdirs(); - file.createNewFile(); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @After - public void tearDown() throws Exception { - File pipeFolder = new File(ROOT_DIR); - if (pipeFolder.exists()) { - FileUtils.deleteFileOrDirectory(pipeFolder); - } - } - - @Test - public void testIncreaseTsfile() throws IOException { - File originWALFile = new File(WAL_NAME); - Assert.assertEquals(0, pipeWALHardlinkResourceManager.getFileReferenceCount(originWALFile)); - - File pipeWALFile = pipeWALHardlinkResourceManager.increaseFileReference(originWALFile); - Assert.assertEquals(1, pipeWALHardlinkResourceManager.getFileReferenceCount(pipeWALFile)); - Assert.assertTrue(Files.exists(originWALFile.toPath())); - Assert.assertTrue(Files.exists(pipeWALFile.toPath())); - - // test use hardlinkTsFile to increase reference counts - pipeWALHardlinkResourceManager.increaseFileReference(pipeWALFile); - Assert.assertEquals(2, pipeWALHardlinkResourceManager.getFileReferenceCount(pipeWALFile)); - Assert.assertTrue(Files.exists(originWALFile.toPath())); - Assert.assertTrue(Files.exists(pipeWALFile.toPath())); - } - - @Test - public void testDecreaseTsfile() throws IOException { - File originFile = new File(WAL_NAME); - - pipeWALHardlinkResourceManager.decreaseFileReference(originFile); - Assert.assertEquals(0, pipeWALHardlinkResourceManager.getFileReferenceCount(originFile)); - - File pipeWALFile = pipeWALHardlinkResourceManager.increaseFileReference(originFile); - Assert.assertEquals(1, pipeWALHardlinkResourceManager.getFileReferenceCount(pipeWALFile)); - Assert.assertTrue(Files.exists(pipeWALFile.toPath())); - Assert.assertTrue(Files.exists(pipeWALFile.toPath())); - - Assert.assertTrue(originFile.delete()); - Assert.assertFalse(Files.exists(originFile.toPath())); - - Assert.assertEquals(1, pipeWALHardlinkResourceManager.getFileReferenceCount(pipeWALFile)); - Assert.assertFalse(Files.exists(originFile.toPath())); - Assert.assertTrue(Files.exists(pipeWALFile.toPath())); - - pipeWALHardlinkResourceManager.decreaseFileReference(pipeWALFile); - Assert.assertEquals(0, pipeWALHardlinkResourceManager.getFileReferenceCount(pipeWALFile)); - Assert.assertFalse(Files.exists(originFile.toPath())); - Assert.assertFalse(Files.exists(pipeWALFile.toPath())); - } -} From 4dd437d7665c6bc6bf70ed153d92e5f8111f3bc4 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 26 Jun 2025 15:32:29 +0800 Subject: [PATCH 004/185] Update CheckpointManager.java --- .../wal/checkpoint/CheckpointManager.java | 76 +------------------ 1 file changed, 1 insertion(+), 75 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/checkpoint/CheckpointManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/checkpoint/CheckpointManager.java index afa6651dfa48..77b7f347ae4c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/checkpoint/CheckpointManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/checkpoint/CheckpointManager.java @@ -25,11 +25,9 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.service.metrics.WritingMetrics; import org.apache.iotdb.db.storageengine.dataregion.memtable.TsFileProcessor; -import org.apache.iotdb.db.storageengine.dataregion.wal.exception.MemTablePinException; import org.apache.iotdb.db.storageengine.dataregion.wal.io.CheckpointWriter; import org.apache.iotdb.db.storageengine.dataregion.wal.io.ILogWriter; import org.apache.iotdb.db.storageengine.dataregion.wal.utils.CheckpointFileUtils; -import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALInsertNodeCache; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,7 +38,6 @@ import java.nio.file.Files; import java.util.ArrayList; import java.util.Collections; -import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -177,9 +174,7 @@ public void makeFlushMemTableCP(long memTableId) { return; } memTableInfo.setFlushed(); - if (!memTableInfo.isPinned()) { - memTableId2Info.remove(memTableId); - } + memTableId2Info.remove(memTableId); Checkpoint checkpoint = new Checkpoint( CheckpointType.FLUSH_MEMORY_TABLE, Collections.singletonList(memTableInfo)); @@ -261,75 +256,6 @@ private boolean tryRollingLogWriter() throws IOException { // endregion - // region methods for pipe - /** - * Pin the wal files of the given memory table. Notice: cannot pin one memTable too long, - * otherwise the wal disk usage may too large. - * - * @throws MemTablePinException If the memTable has been flushed - */ - public void pinMemTable(long memTableId) throws MemTablePinException { - infoLock.lock(); - try { - if (!memTableId2Info.containsKey(memTableId)) { - throw new MemTablePinException( - String.format( - "Fail to pin memTable-%d because this memTable doesn't exist in the wal.", - memTableId)); - } - MemTableInfo memTableInfo = memTableId2Info.get(memTableId); - if (!memTableInfo.isPinned()) { - WALInsertNodeCache.getInstance().addMemTable(memTableId); - } - memTableInfo.pin(); - } finally { - infoLock.unlock(); - } - } - - /** - * Unpin the wal files of the given memory table. - * - * @throws MemTablePinException If there aren't corresponding pin operations - */ - public void unpinMemTable(long memTableId) throws MemTablePinException { - infoLock.lock(); - try { - if (!memTableId2Info.containsKey(memTableId)) { - throw new MemTablePinException( - String.format( - "Fail to unpin memTable-%d because this memTable doesn't exist in the wal.", - memTableId)); - } - if (!memTableId2Info.get(memTableId).isPinned()) { - throw new MemTablePinException( - String.format( - "Fail to unpin memTable-%d because this memTable hasn't been pinned.", memTableId)); - } - MemTableInfo memTableInfo = memTableId2Info.get(memTableId); - memTableInfo.unpin(); - if (!memTableInfo.isPinned()) { - WALInsertNodeCache.getInstance().removeMemTable(memTableId); - if (memTableInfo.isFlushed()) { - memTableId2Info.remove(memTableId); - } - } - } finally { - infoLock.unlock(); - } - } - - // endregion - - /** Get MemTableInfo of oldest unpinned MemTable, whose first version id is smallest. */ - public MemTableInfo getOldestUnpinnedMemTableInfo() { - // find oldest memTable - return activeOrPinnedMemTables().stream() - .filter(memTableInfo -> !memTableInfo.isPinned()) - .min(Comparator.comparingLong(MemTableInfo::getMemTableId)) - .orElse(null); - } - /** * Get version id of first valid .wal file * From cb182d4ca8fb5fb4999605c2358abeab7e3b2f34 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 26 Jun 2025 15:38:31 +0800 Subject: [PATCH 005/185] Release memory --- .../common/tablet/PipeInsertNodeTabletInsertionEvent.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java index 12d5eb1f5118..faeb107f0327 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java @@ -75,7 +75,7 @@ public class PipeInsertNodeTabletInsertionEvent extends EnrichedEvent private List dataContainers; - private final InsertNode insertNode; + private InsertNode insertNode; private ProgressIndex progressIndex; @@ -142,6 +142,7 @@ public boolean internallyDecreaseResourceReferenceCount(final String holderMessa dataContainers.clear(); dataContainers = null; } + insertNode = null; close(); return true; } catch (final Exception e) { From f6b1cb38a896ee51a71367fa86b8c08f77813897 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 26 Jun 2025 16:15:44 +0800 Subject: [PATCH 006/185] Parital --- .../agent/task/PipeDataNodeTaskAgent.java | 28 ---------- ...PipeRealtimeDataRegionHybridExtractor.java | 54 ++----------------- .../iotdb/commons/conf/CommonConfig.java | 14 ----- .../iotdb/commons/pipe/config/PipeConfig.java | 5 -- 4 files changed, 5 insertions(+), 96 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java index d427da148490..c98f1b0a65af 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java @@ -707,22 +707,6 @@ && mayDeletedTsFileSizeReachDangerousThreshold()) { "Pipe {} needs to restart because too many TsFiles are out-of-date.", pipeMeta.getStaticMeta()); stuckPipes.add(pipeMeta); - continue; - } - - // Try to restart the stream mode pipes for releasing memTables. - if (extractors.get(0).isStreamMode()) { - if (extractors.stream().anyMatch(IoTDBDataRegionExtractor::hasConsumedAllHistoricalTsFiles) - && (mayMemTablePinnedCountReachDangerousThreshold() - || mayWalSizeReachThrottleThreshold())) { - // Extractors of this pipe may be stuck and is pinning too many MemTables. - LOGGER.warn( - "Pipe {} needs to restart because too many memTables are pinned or the WAL size is too large. mayMemTablePinnedCountReachDangerousThreshold: {}, mayWalSizeReachThrottleThreshold: {}", - pipeMeta.getStaticMeta(), - mayMemTablePinnedCountReachDangerousThreshold(), - mayWalSizeReachThrottleThreshold()); - stuckPipes.add(pipeMeta); - } } } @@ -754,18 +738,6 @@ private boolean mayDeletedTsFileSizeReachDangerousThreshold() { } } - private boolean mayMemTablePinnedCountReachDangerousThreshold() { - return PipeConfig.getInstance().getPipeMaxAllowedPinnedMemTableCount() != Integer.MAX_VALUE - && PipeDataNodeResourceManager.wal().getPinnedWalCount() - >= 5 - * PipeConfig.getInstance().getPipeMaxAllowedPinnedMemTableCount() - * StorageEngine.getInstance().getDataRegionNumber(); - } - - private boolean mayWalSizeReachThrottleThreshold() { - return 3 * WALManager.getInstance().getTotalDiskUsage() > 2 * CONFIG.getThrottleThreshold(); - } - private void restartStuckPipe(final PipeMeta pipeMeta) { LOGGER.warn( "Pipe {} will be restarted because it is stuck or has encountered issues such as data backlog or being stopped for too long.", diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java index c9d16163fc8c..7d1334ce623a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java @@ -22,7 +22,6 @@ import org.apache.iotdb.commons.exception.pipe.PipeRuntimeNonCriticalException; import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.event.ProgressReportEvent; -import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; @@ -35,8 +34,6 @@ import org.apache.iotdb.db.pipe.metric.source.PipeDataRegionExtractorMetrics; import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryManager; -import org.apache.iotdb.db.storageengine.StorageEngine; -import org.apache.iotdb.db.storageengine.dataregion.wal.WALManager; import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; import org.apache.iotdb.pipe.api.event.dml.insertion.TsFileInsertionEvent; @@ -227,12 +224,10 @@ private boolean canNotUseTabletAnyMore(final PipeRealtimeEvent event) { // represented by the tablet event should be carried by the following tsfile event: // 0. If the remaining insert event count is too large, we need to reduce the accumulated // tablets. - // 1. If Wal size > maximum size of wal buffer, // the write operation will be throttled, so we should not extract any more tablet events. - // 2. The shallow memory usage of the insert node has reached the dangerous threshold. - // 3. Deprecated logics (unused by default) + // 1. The shallow memory usage of the insert node has reached the dangerous threshold. + // 2. Deprecated logics (unused by default) return mayRemainingInsertNodeEventExceedLimit(event) - || mayWalSizeReachThrottleThreshold(event) || mayInsertNodeMemoryReachDangerousThreshold(event) || canNotUseTabletAnymoreDeprecated(event); } @@ -253,23 +248,6 @@ private boolean mayRemainingInsertNodeEventExceedLimit(final PipeRealtimeEvent e return mayRemainingInsertEventExceedLimit; } - private boolean mayWalSizeReachThrottleThreshold(final PipeRealtimeEvent event) { - final boolean mayWalSizeReachThrottleThreshold = - 3 * WALManager.getInstance().getTotalDiskUsage() - > IoTDBDescriptor.getInstance().getConfig().getThrottleThreshold(); - if (mayWalSizeReachThrottleThreshold && event.mayExtractorUseTablets(this)) { - logByLogManager( - l -> - l.info( - "Pipe task {}@{} canNotUseTabletAnyMore(1): Wal size {} has reached throttle threshold {}", - pipeName, - dataRegionId, - WALManager.getInstance().getTotalDiskUsage(), - IoTDBDescriptor.getInstance().getConfig().getThrottleThreshold() / 3.0d)); - } - return mayWalSizeReachThrottleThreshold; - } - private boolean mayInsertNodeMemoryReachDangerousThreshold(final PipeRealtimeEvent event) { final long floatingMemoryUsageInByte = PipeDataNodeAgent.task().getFloatingMemoryUsageInByte(pipeName); @@ -306,12 +284,10 @@ private boolean canNotUseTabletAnymoreDeprecated(final PipeRealtimeEvent event) // In the following 5 cases, we should not extract any more tablet events. all the data // represented by the tablet events should be carried by the following tsfile event: // 0. If the pipe task is currently restarted. - // 1. The number of pinned memTables has reached the dangerous threshold. - // 2. The number of historical tsFile events to transfer has exceeded the limit. - // 3. The number of realtime tsfile events to transfer has exceeded the limit. - // 4. The number of linked tsFiles has reached the dangerous threshold. + // 1. The number of historical tsFile events to transfer has exceeded the limit. + // 2. The number of realtime tsfile events to transfer has exceeded the limit. + // 3. The number of linked tsFiles has reached the dangerous threshold. return isPipeTaskCurrentlyRestarted(event) - || mayMemTablePinnedCountReachDangerousThreshold(event) || isHistoricalTsFileEventCountExceededLimit(event) || isRealtimeTsFileEventCountExceededLimit(event) || mayTsFileLinkedCountReachDangerousThreshold(event); @@ -333,26 +309,6 @@ private boolean isPipeTaskCurrentlyRestarted(final PipeRealtimeEvent event) { return isPipeTaskCurrentlyRestarted; } - private boolean mayMemTablePinnedCountReachDangerousThreshold(final PipeRealtimeEvent event) { - if (PipeConfig.getInstance().getPipeMaxAllowedPinnedMemTableCount() == Integer.MAX_VALUE) { - return false; - } - final boolean mayMemTablePinnedCountReachDangerousThreshold = - PipeDataNodeResourceManager.wal().getPinnedWalCount() - >= PipeConfig.getInstance().getPipeMaxAllowedPinnedMemTableCount() - * StorageEngine.getInstance().getDataRegionNumber(); - if (mayMemTablePinnedCountReachDangerousThreshold && event.mayExtractorUseTablets(this)) { - LOGGER.info( - "Pipe task {}@{} canNotUseTabletAnymoreDeprecated(1): The number of pinned memTables {} has reached the dangerous threshold {}", - pipeName, - dataRegionId, - PipeDataNodeResourceManager.wal().getPinnedWalCount(), - PipeConfig.getInstance().getPipeMaxAllowedPinnedMemTableCount() - * StorageEngine.getInstance().getDataRegionNumber()); - } - return mayMemTablePinnedCountReachDangerousThreshold; - } - private boolean isHistoricalTsFileEventCountExceededLimit(final PipeRealtimeEvent event) { if (PipeConfig.getInstance().getPipeMaxAllowedHistoricalTsFilePerDataRegion() == Integer.MAX_VALUE) { diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java index 10fe9d9ebb7f..46ff03633182 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java @@ -278,7 +278,6 @@ public class CommonConfig { private int pipeMaxAllowedHistoricalTsFilePerDataRegion = Integer.MAX_VALUE; // Deprecated private int pipeMaxAllowedPendingTsFileEpochPerDataRegion = Integer.MAX_VALUE; // Deprecated - private int pipeMaxAllowedPinnedMemTableCount = Integer.MAX_VALUE; // per data region private long pipeMaxAllowedLinkedTsFileCount = Long.MAX_VALUE; // Deprecated private float pipeMaxAllowedLinkedDeletedTsFileDiskUsagePercentage = 0.1F; private long pipeStuckRestartIntervalSeconds = 120; @@ -1510,19 +1509,6 @@ public void setPipeMaxAllowedPendingTsFileEpochPerDataRegion( pipeMaxAllowedPendingTsFileEpochPerDataRegion); } - public int getPipeMaxAllowedPinnedMemTableCount() { - return pipeMaxAllowedPinnedMemTableCount; - } - - public void setPipeMaxAllowedPinnedMemTableCount(int pipeMaxAllowedPinnedMemTableCount) { - if (this.pipeMaxAllowedPinnedMemTableCount == pipeMaxAllowedPinnedMemTableCount) { - return; - } - this.pipeMaxAllowedPinnedMemTableCount = pipeMaxAllowedPinnedMemTableCount; - logger.info( - "pipeMaxAllowedPinnedMemTableCount is set to {}", pipeMaxAllowedPinnedMemTableCount); - } - public long getPipeMaxAllowedLinkedTsFileCount() { return pipeMaxAllowedLinkedTsFileCount; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java index be0c70d7f422..06441c437456 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java @@ -339,10 +339,6 @@ public int getPipeMaxAllowedPendingTsFileEpochPerDataRegion() { return COMMON_CONFIG.getPipeMaxAllowedPendingTsFileEpochPerDataRegion(); } - public int getPipeMaxAllowedPinnedMemTableCount() { - return COMMON_CONFIG.getPipeMaxAllowedPinnedMemTableCount(); - } - public long getPipeMaxAllowedLinkedTsFileCount() { return COMMON_CONFIG.getPipeMaxAllowedLinkedTsFileCount(); } @@ -621,7 +617,6 @@ public void printAllConfigs() { LOGGER.info( "PipeMaxAllowedPendingTsFileEpochPerDataRegion: {}", getPipeMaxAllowedPendingTsFileEpochPerDataRegion()); - LOGGER.info("PipeMaxAllowedPinnedMemTableCount: {}", getPipeMaxAllowedPinnedMemTableCount()); LOGGER.info("PipeMaxAllowedLinkedTsFileCount: {}", getPipeMaxAllowedLinkedTsFileCount()); LOGGER.info( "PipeMaxAllowedLinkedDeletedTsFileDiskUsagePercentage: {}", From 12c21dd892aed1ae44ac0fc428ff23314bb3a141 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Thu, 26 Jun 2025 16:24:55 +0800 Subject: [PATCH 007/185] update PipeInsertNodeTabletInsertionEvent --- .../common/tablet/PipeInsertNodeTabletInsertionEvent.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java index faeb107f0327..6b8d1d8e7bc0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java @@ -30,6 +30,7 @@ import org.apache.iotdb.db.pipe.event.ReferenceTrackableEvent; import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeRemainingEventAndTimeMetrics; import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; +import org.apache.iotdb.db.pipe.resource.memory.InsertNodeMemoryEstimator; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryWeightUtil; import org.apache.iotdb.db.pipe.resource.memory.PipeTabletMemoryBlock; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode; @@ -97,7 +98,6 @@ private PipeInsertNodeTabletInsertionEvent( // Record device path here so there's no need to get it from InsertNode cache later. this.progressIndex = insertNode.getProgressIndex(); this.insertNode = insertNode; - this.allocatedMemoryBlock = new AtomicReference<>(); } @@ -408,7 +408,7 @@ public String coreReportMessage() { @Override public long ramBytesUsed() { return INSTANCE_SIZE - + (Objects.nonNull(devicePath) ? PartialPath.estimateSize(devicePath) : 0) + + (Objects.nonNull(insertNode) ? InsertNodeMemoryEstimator.sizeOf(insertNode) : 0) + (Objects.nonNull(progressIndex) ? progressIndex.ramBytesUsed() : 0); } From a078b1faa4478d7f8a5212c96413c6744c3706d7 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 26 Jun 2025 16:28:12 +0800 Subject: [PATCH 008/185] Remove useless --- .../resource/memory/PipeMemoryManager.java | 12 ---- .../iotdb/commons/conf/CommonConfig.java | 70 +------------------ .../iotdb/commons/pipe/config/PipeConfig.java | 28 -------- .../commons/pipe/config/PipeDescriptor.java | 28 -------- 4 files changed, 1 insertion(+), 137 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryManager.java index 690a7c540838..f01d7e6b1cc2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryManager.java @@ -104,18 +104,6 @@ private static double allowedMaxMemorySizeInBytesOfTsTiles() { * getTotalNonFloatingMemorySizeInBytes(); } - public long getAllocatedMemorySizeInBytesOfWAL() { - return (long) - (PipeConfig.getInstance().getPipeDataStructureWalMemoryProportion() - * getTotalNonFloatingMemorySizeInBytes()); - } - - public long getAllocatedMemorySizeInBytesOfBatch() { - return (long) - (PipeConfig.getInstance().getPipeDataStructureBatchMemoryProportion() - * getTotalNonFloatingMemorySizeInBytes()); - } - public boolean isEnough4TabletParsing() { return (double) usedMemorySizeInBytesOfTablets + (double) usedMemorySizeInBytesOfTsFiles < EXCEED_PROTECT_THRESHOLD * allowedMaxMemorySizeInBytesOfTabletsAndTsFiles() diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java index 46ff03633182..d776807431ab 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java @@ -201,10 +201,6 @@ public class CommonConfig { private String pipeProgressIndexPersistDirName = "progress"; - private String pipeHardlinkWALDirName = "wal"; - - private boolean pipeHardLinkWALEnabled = false; - private boolean pipeFileReceiverFsyncEnabled = true; private int pipeRealTimeQueuePollTsFileThreshold = 10; @@ -221,9 +217,8 @@ public class CommonConfig { private int pipeDataStructureTabletSizeInBytes = 2097152; private double pipeDataStructureTabletMemoryBlockAllocationRejectThreshold = 0.2; private double pipeDataStructureTsFileMemoryBlockAllocationRejectThreshold = 0.2; - private double pipeDataStructureWalMemoryProportion = 0.3; private double PipeDataStructureBatchMemoryProportion = 0.1; - private double pipeTotalFloatingMemoryProportion = 0.2; + private double pipeTotalFloatingMemoryProportion = 0.5; private int pipeSubtaskExecutorBasicCheckPointIntervalByConsumedEventCount = 10_000; private long pipeSubtaskExecutorBasicCheckPointIntervalByTimeDuration = 10 * 1000L; @@ -293,8 +288,6 @@ public class CommonConfig { private int pipeMetaReportMaxLogIntervalRounds = 36; private int pipeTsFilePinMaxLogNumPerRound = 10; private int pipeTsFilePinMaxLogIntervalRounds = 90; - private int pipeWalPinMaxLogNumPerRound = 10; - private int pipeWalPinMaxLogIntervalRounds = 90; private boolean pipeMemoryManagementEnabled = true; private long pipeMemoryAllocateRetryIntervalMs = 50; @@ -773,30 +766,6 @@ public void setPipeProgressIndexPersistDirName(String pipeProgressIndexPersistDi logger.info("pipeProgressIndexPersistDir is set to {}.", pipeProgressIndexPersistDirName); } - public String getPipeHardlinkWALDirName() { - return pipeHardlinkWALDirName; - } - - public void setPipeHardlinkWALDirName(String pipeWALDirName) { - if (Objects.equals(pipeWALDirName, this.pipeHardlinkWALDirName)) { - return; - } - this.pipeHardlinkWALDirName = pipeWALDirName; - logger.info("pipeHardlinkWALDirName is set to {}.", pipeWALDirName); - } - - public boolean getPipeHardLinkWALEnabled() { - return pipeHardLinkWALEnabled; - } - - public void setPipeHardLinkWALEnabled(boolean pipeHardLinkWALEnabled) { - if (this.pipeHardLinkWALEnabled == pipeHardLinkWALEnabled) { - return; - } - this.pipeHardLinkWALEnabled = pipeHardLinkWALEnabled; - logger.info("pipeHardLinkWALEnabled is set to {}.", pipeHardLinkWALEnabled); - } - public boolean getPipeFileReceiverFsyncEnabled() { return pipeFileReceiverFsyncEnabled; } @@ -868,19 +837,6 @@ public void setPipeDataStructureTsFileMemoryBlockAllocationRejectThreshold( pipeDataStructureTsFileMemoryBlockAllocationRejectThreshold); } - public double getPipeDataStructureWalMemoryProportion() { - return pipeDataStructureWalMemoryProportion; - } - - public void setPipeDataStructureWalMemoryProportion(double pipeDataStructureWalMemoryProportion) { - if (this.pipeDataStructureWalMemoryProportion == pipeDataStructureWalMemoryProportion) { - return; - } - this.pipeDataStructureWalMemoryProportion = pipeDataStructureWalMemoryProportion; - logger.info( - "pipeDataStructureWalMemoryProportion is set to {}.", pipeDataStructureWalMemoryProportion); - } - public double getPipeDataStructureBatchMemoryProportion() { return PipeDataStructureBatchMemoryProportion; } @@ -1700,30 +1656,6 @@ public void setPipeTsFilePinMaxLogIntervalRounds(int pipeTsFilePinMaxLogInterval "pipeTsFilePinMaxLogIntervalRounds is set to {}", pipeTsFilePinMaxLogIntervalRounds); } - public int getPipeWalPinMaxLogNumPerRound() { - return pipeWalPinMaxLogNumPerRound; - } - - public void setPipeWalPinMaxLogNumPerRound(int pipeWalPinMaxLogNumPerRound) { - if (this.pipeWalPinMaxLogNumPerRound == pipeWalPinMaxLogNumPerRound) { - return; - } - this.pipeWalPinMaxLogNumPerRound = pipeWalPinMaxLogNumPerRound; - logger.info("pipeWalPinMaxLogNumPerRound is set to {}", pipeWalPinMaxLogNumPerRound); - } - - public int getPipeWalPinMaxLogIntervalRounds() { - return pipeWalPinMaxLogIntervalRounds; - } - - public void setPipeWalPinMaxLogIntervalRounds(int pipeWalPinMaxLogIntervalRounds) { - if (this.pipeWalPinMaxLogIntervalRounds == pipeWalPinMaxLogIntervalRounds) { - return; - } - this.pipeWalPinMaxLogIntervalRounds = pipeWalPinMaxLogIntervalRounds; - logger.info("pipeWalPinMaxLogIntervalRounds is set to {}", pipeWalPinMaxLogIntervalRounds); - } - public boolean getPipeMemoryManagementEnabled() { return pipeMemoryManagementEnabled; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java index 06441c437456..984a0bc8ef7f 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java @@ -62,14 +62,6 @@ public long getPipeProgressIndexFlushIntervalMs() { return COMMON_CONFIG.getPipeProgressIndexFlushIntervalMs(); } - public String getPipeHardlinkWALDirName() { - return COMMON_CONFIG.getPipeHardlinkWALDirName(); - } - - public boolean getPipeHardLinkWALEnabled() { - return COMMON_CONFIG.getPipeHardLinkWALEnabled(); - } - public boolean getPipeFileReceiverFsyncEnabled() { return COMMON_CONFIG.getPipeFileReceiverFsyncEnabled(); } @@ -95,14 +87,6 @@ public double getPipeDataStructureTsFileMemoryBlockAllocationRejectThreshold() { return COMMON_CONFIG.getPipeDataStructureTsFileMemoryBlockAllocationRejectThreshold(); } - public double getPipeDataStructureWalMemoryProportion() { - return COMMON_CONFIG.getPipeDataStructureWalMemoryProportion(); - } - - public double getPipeDataStructureBatchMemoryProportion() { - return COMMON_CONFIG.getPipeDataStructureBatchMemoryProportion(); - } - public double getPipeTotalFloatingMemoryProportion() { return COMMON_CONFIG.getPipeTotalFloatingMemoryProportion(); } @@ -397,14 +381,6 @@ public int getPipeTsFilePinMaxLogIntervalRounds() { return COMMON_CONFIG.getPipeTsFilePinMaxLogIntervalRounds(); } - public int getPipeWalPinMaxLogNumPerRound() { - return COMMON_CONFIG.getPipeWalPinMaxLogNumPerRound(); - } - - public int getPipeWalPinMaxLogIntervalRounds() { - return COMMON_CONFIG.getPipeWalPinMaxLogIntervalRounds(); - } - /////////////////////////////// Memory /////////////////////////////// public boolean getPipeMemoryManagementEnabled() { @@ -475,8 +451,6 @@ public void printAllConfigs() { LOGGER.info( "PipeProgressIndexPersistCheckPointGap: {}", getPipeProgressIndexPersistCheckPointGap()); LOGGER.info("PipeProgressIndexFlushIntervalMs: {}", getPipeProgressIndexFlushIntervalMs()); - LOGGER.info("PipeHardlinkWALDirName: {}", getPipeHardlinkWALDirName()); - LOGGER.info("PipeHardLinkWALEnabled: {}", getPipeHardLinkWALEnabled()); LOGGER.info("PipeFileReceiverFsyncEnabled: {}", getPipeFileReceiverFsyncEnabled()); LOGGER.info("PipeDataStructureTabletRowSize: {}", getPipeDataStructureTabletRowSize()); @@ -641,8 +615,6 @@ public void printAllConfigs() { LOGGER.info("PipeMetaReportMaxLogIntervalRounds: {}", getPipeMetaReportMaxLogIntervalRounds()); LOGGER.info("PipeTsFilePinMaxLogNumPerRound: {}", getPipeTsFilePinMaxLogNumPerRound()); LOGGER.info("PipeTsFilePinMaxLogIntervalRounds: {}", getPipeTsFilePinMaxLogIntervalRounds()); - LOGGER.info("PipeWalPinMaxLogNumPerRound: {}", getPipeWalPinMaxLogNumPerRound()); - LOGGER.info("PipeWalPinMaxLogIntervalRounds: {}", getPipeWalPinMaxLogIntervalRounds()); LOGGER.info("PipeMemoryManagementEnabled: {}", getPipeMemoryManagementEnabled()); LOGGER.info("PipeMemoryAllocateMaxRetries: {}", getPipeMemoryAllocateMaxRetries()); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java index 7086ff731bfa..de11e5a6d516 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java @@ -49,13 +49,6 @@ public static void loadPipeStaticConfig(CommonConfig config, TrimProperties prop config.setPipeProgressIndexPersistDirName( properties.getProperty( "pipe_progress_index_persist_dir_name", config.getPipeProgressIndexPersistDirName())); - config.setPipeHardlinkWALDirName( - properties.getProperty("pipe_hardlink_wal_dir_name", config.getPipeHardlinkWALDirName())); - config.setPipeHardLinkWALEnabled( - Boolean.parseBoolean( - properties.getProperty( - "pipe_hardlink_wal_enabled", - Boolean.toString(config.getPipeHardLinkWALEnabled())))); int pipeSubtaskExecutorMaxThreadNum = Integer.parseInt( properties.getProperty( @@ -147,17 +140,6 @@ public static void loadPipeStaticConfig(CommonConfig config, TrimProperties prop properties.getProperty( "pipe_tsfile_pin_max_log_interval_rounds", String.valueOf(config.getPipeTsFilePinMaxLogIntervalRounds())))); - config.setPipeWalPinMaxLogNumPerRound( - Integer.parseInt( - properties.getProperty( - "pipe_wal_pin_max_log_num_per_round", - String.valueOf(config.getPipeWalPinMaxLogNumPerRound())))); - config.setPipeWalPinMaxLogIntervalRounds( - Integer.parseInt( - properties.getProperty( - "pipe_wal_pin_max_log_interval_rounds", - String.valueOf(config.getPipeWalPinMaxLogIntervalRounds())))); - config.setPipeMemoryManagementEnabled( Boolean.parseBoolean( properties.getProperty( @@ -253,11 +235,6 @@ public static void loadPipeInternalConfig(CommonConfig config, TrimProperties pr "pipe_data_structure_ts_file_memory_block_allocation_reject_threshold", String.valueOf( config.getPipeDataStructureTsFileMemoryBlockAllocationRejectThreshold())))); - config.setPipeDataStructureWalMemoryProportion( - Double.parseDouble( - properties.getProperty( - "pipe_data_structure_wal_memory_proportion", - String.valueOf(config.getPipeDataStructureWalMemoryProportion())))); config.setPipeDataStructureBatchMemoryProportion( Double.parseDouble( properties.getProperty( @@ -446,11 +423,6 @@ public static void loadPipeInternalConfig(CommonConfig config, TrimProperties pr properties.getProperty( "pipe_max_allowed_pending_tsfile_epoch_per_data_region", String.valueOf(config.getPipeMaxAllowedPendingTsFileEpochPerDataRegion())))); - config.setPipeMaxAllowedPinnedMemTableCount( - Integer.parseInt( - properties.getProperty( - "pipe_max_allowed_pinned_memtable_count", - String.valueOf(config.getPipeMaxAllowedPinnedMemTableCount())))); config.setPipeMaxAllowedLinkedTsFileCount( Long.parseLong( properties.getProperty( From 9241bcce695e1ef36fd30edf23a7c85ea38565d6 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 26 Jun 2025 16:33:02 +0800 Subject: [PATCH 009/185] Update PipeRealtimeDataRegionHybridExtractor.java --- .../realtime/PipeRealtimeDataRegionHybridExtractor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java index 7d1334ce623a..4c2c17ea72de 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java @@ -260,7 +260,7 @@ private boolean mayInsertNodeMemoryReachDangerousThreshold(final PipeRealtimeEve logByLogManager( l -> l.info( - "Pipe task {}@{} canNotUseTabletAnyMore(2): The shallow memory usage of the insert node {} has reached the dangerous threshold {}", + "Pipe task {}@{} canNotUseTabletAnyMore(2): The memory usage of the insert node {} has reached the dangerous threshold {}", pipeName, dataRegionId, floatingMemoryUsageInByte * pipeCount, From 09a0e61e8d0580d0faa91bf9cbcc9115fe7df723 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 26 Jun 2025 16:35:48 +0800 Subject: [PATCH 010/185] Update Metric.java --- .../org/apache/iotdb/commons/service/metric/enums/Metric.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/Metric.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/Metric.java index 39b22a3d7468..90183f8150b1 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/Metric.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/Metric.java @@ -156,9 +156,6 @@ public enum Metric { PIPE_CONNECTOR_TSFILE_TRANSFER("pipe_connector_tsfile_transfer"), PIPE_CONNECTOR_HEARTBEAT_TRANSFER("pipe_connector_heartbeat_transfer"), PIPE_HEARTBEAT_EVENT("pipe_heartbeat_event"), - PIPE_WAL_INSERT_NODE_CACHE_HIT_RATE("pipe_wal_insert_node_cache_hit_rate"), - PIPE_WAL_INSERT_NODE_CACHE_HIT_COUNT("pipe_wal_insert_node_cache_hit_count"), - PIPE_WAL_INSERT_NODE_CACHE_REQUEST_COUNT("pipe_wal_insert_node_cache_request_count"), PIPE_EXTRACTOR_TSFILE_EPOCH_STATE("pipe_extractor_tsfile_epoch_state"), PIPE_MEM("pipe_mem"), PIPE_LINKED_TSFILE_COUNT("pipe_linked_tsfile_count"), From ac7272a550804fbe47212ade51bb35bc299fc5f3 Mon Sep 17 00:00:00 2001 From: VGalaxies Date: Thu, 26 Jun 2025 00:27:06 +0800 Subject: [PATCH 011/185] setup --- .../env/cluster/config/MppCommonConfig.java | 6 ++++ .../cluster/config/MppSharedCommonConfig.java | 7 +++++ .../env/remote/config/RemoteCommonConfig.java | 5 +++ .../apache/iotdb/itbase/env/CommonConfig.java | 2 ++ .../cluster/IoTDBSubscriptionRestartIT.java | 4 +++ .../it/dual/AbstractSubscriptionDualIT.java | 4 +++ .../it/local/AbstractSubscriptionLocalIT.java | 2 ++ .../triple/AbstractSubscriptionTripleIT.java | 5 +++ .../iotdb/tools/it/ExportTsFileTestIT.java | 3 ++ .../org/apache/iotdb/rpc/TSStatusCode.java | 1 + .../consumer/SubscriptionProvider.java | 1 + .../execution/PipeSubtaskExecutorManager.java | 6 +++- .../impl/DataNodeInternalRPCServiceImpl.java | 26 ++++++++++++++++ .../executor/ClusterConfigTaskExecutor.java | 31 +++++++++++++++++++ .../agent/SubscriptionReceiverAgent.java | 16 ++++++++-- .../agent/SubscriptionRuntimeAgent.java | 4 +++ .../iotdb/commons/conf/CommonConfig.java | 10 ++++++ .../iotdb/commons/conf/CommonDescriptor.java | 5 +++ .../pipe/agent/task/PipeTaskAgent.java | 8 +++++ .../config/SubscriptionConfig.java | 6 ++++ 20 files changed, 149 insertions(+), 3 deletions(-) diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java index f1a820e21127..fe4190d7708e 100644 --- a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java +++ b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java @@ -502,6 +502,12 @@ public CommonConfig setQueryMemoryProportion(String queryMemoryProportion) { return this; } + @Override + public CommonConfig setSubscriptionEnabled(boolean subscriptionEnabled) { + setProperty("subscription_enabled", String.valueOf(subscriptionEnabled)); + return this; + } + @Override public CommonConfig setDefaultStorageGroupLevel(int defaultStorageGroupLevel) { setProperty("default_storage_group_level", String.valueOf(defaultStorageGroupLevel)); diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppSharedCommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppSharedCommonConfig.java index c81680c52300..c2ade6eace08 100644 --- a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppSharedCommonConfig.java +++ b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppSharedCommonConfig.java @@ -513,6 +513,13 @@ public CommonConfig setQueryMemoryProportion(String queryMemoryProportion) { return this; } + @Override + public CommonConfig setSubscriptionEnabled(boolean subscriptionEnabled) { + dnConfig.setSubscriptionEnabled(subscriptionEnabled); + cnConfig.setSubscriptionEnabled(subscriptionEnabled); + return this; + } + @Override public CommonConfig setDefaultStorageGroupLevel(int defaultStorageGroupLevel) { dnConfig.setDefaultStorageGroupLevel(defaultStorageGroupLevel); diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteCommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteCommonConfig.java index 3d57745ed68f..581c5f4345ed 100644 --- a/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteCommonConfig.java +++ b/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteCommonConfig.java @@ -360,4 +360,9 @@ public CommonConfig setPipeConnectorRequestSliceThresholdBytes( public CommonConfig setQueryMemoryProportion(String queryMemoryProportion) { return this; } + + @Override + public CommonConfig setSubscriptionEnabled(boolean subscriptionEnabled) { + return this; + } } diff --git a/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java index dbba83aed588..16d509542093 100644 --- a/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java +++ b/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java @@ -160,6 +160,8 @@ CommonConfig setPipeConnectorRequestSliceThresholdBytes( CommonConfig setQueryMemoryProportion(String queryMemoryProportion); + CommonConfig setSubscriptionEnabled(boolean subscriptionEnabled); + default CommonConfig setDefaultStorageGroupLevel(int defaultStorageGroupLevel) { return this; } diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/cluster/IoTDBSubscriptionRestartIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/cluster/IoTDBSubscriptionRestartIT.java index fee93bacdd5d..de17dec1ed42 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/cluster/IoTDBSubscriptionRestartIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/cluster/IoTDBSubscriptionRestartIT.java @@ -71,6 +71,10 @@ public class IoTDBSubscriptionRestartIT extends AbstractSubscriptionIT { public void setUp() throws Exception { super.setUp(); + // enable subscription + EnvFactory.getEnv().getConfig().getCommonConfig().setSubscriptionEnabled(true); + + // set cluster env EnvFactory.getEnv() .getConfig() .getCommonConfig() diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/AbstractSubscriptionDualIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/AbstractSubscriptionDualIT.java index 27d8c3c493f4..594f9efe6915 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/AbstractSubscriptionDualIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/AbstractSubscriptionDualIT.java @@ -47,6 +47,10 @@ public void setUp() throws Exception { } protected void setUpConfig() { + // enable subscription + senderEnv.getConfig().getCommonConfig().setSubscriptionEnabled(true); + receiverEnv.getConfig().getCommonConfig().setSubscriptionEnabled(true); + // enable auto create schema senderEnv.getConfig().getCommonConfig().setAutoCreateSchemaEnabled(true); receiverEnv.getConfig().getCommonConfig().setAutoCreateSchemaEnabled(true); diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/AbstractSubscriptionLocalIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/AbstractSubscriptionLocalIT.java index 5fa7c5808fa8..3f4150519287 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/AbstractSubscriptionLocalIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/AbstractSubscriptionLocalIT.java @@ -32,6 +32,8 @@ public abstract class AbstractSubscriptionLocalIT extends AbstractSubscriptionIT public void setUp() throws Exception { super.setUp(); + // enable subscription + EnvFactory.getEnv().getConfig().getCommonConfig().setSubscriptionEnabled(true); EnvFactory.getEnv().initClusterEnvironment(); } diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/triple/AbstractSubscriptionTripleIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/triple/AbstractSubscriptionTripleIT.java index 9622f7279080..3b49eb8ed131 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/triple/AbstractSubscriptionTripleIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/triple/AbstractSubscriptionTripleIT.java @@ -55,6 +55,11 @@ public void setUp() throws Exception { } protected void setUpConfig() { + // enable subscription + sender.getConfig().getCommonConfig().setSubscriptionEnabled(true); + receiver1.getConfig().getCommonConfig().setSubscriptionEnabled(true); + receiver2.getConfig().getCommonConfig().setSubscriptionEnabled(true); + // enable auto create schema sender.getConfig().getCommonConfig().setAutoCreateSchemaEnabled(true); receiver1.getConfig().getCommonConfig().setAutoCreateSchemaEnabled(true); diff --git a/integration-test/src/test/java/org/apache/iotdb/tools/it/ExportTsFileTestIT.java b/integration-test/src/test/java/org/apache/iotdb/tools/it/ExportTsFileTestIT.java index 57b76e55e814..6ad0c843e274 100644 --- a/integration-test/src/test/java/org/apache/iotdb/tools/it/ExportTsFileTestIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/tools/it/ExportTsFileTestIT.java @@ -52,7 +52,10 @@ public class ExportTsFileTestIT extends AbstractScriptIT { @BeforeClass public static void setUp() throws Exception { + // enable subscription + EnvFactory.getEnv().getConfig().getCommonConfig().setSubscriptionEnabled(true); EnvFactory.getEnv().initClusterEnvironment(); + ip = EnvFactory.getEnv().getIP(); port = EnvFactory.getEnv().getPort(); toolsPath = EnvFactory.getEnv().getToolsPath(); diff --git a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java index 22bdeb3114e7..34f9872bb10a 100644 --- a/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java +++ b/iotdb-client/service-rpc/src/main/java/org/apache/iotdb/rpc/TSStatusCode.java @@ -274,6 +274,7 @@ public enum TSStatusCode { SUBSCRIPTION_MISSING_CUSTOMER(1909), SHOW_SUBSCRIPTION_ERROR(1910), SUBSCRIPTION_PIPE_TIMEOUT_ERROR(1911), + SUBSCRIPTION_NOT_ENABLED_ERROR(1912), // Topic CREATE_TOPIC_ERROR(2000), diff --git a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/consumer/SubscriptionProvider.java b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/consumer/SubscriptionProvider.java index 8d50fda60709..03eefeaddba7 100644 --- a/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/consumer/SubscriptionProvider.java +++ b/iotdb-client/session/src/main/java/org/apache/iotdb/session/subscription/consumer/SubscriptionProvider.java @@ -413,6 +413,7 @@ private static void verifyPipeSubscribeSuccess(final TSStatus status) case 1900: // SUBSCRIPTION_VERSION_ERROR case 1901: // SUBSCRIPTION_TYPE_ERROR case 1909: // SUBSCRIPTION_MISSING_CUSTOMER + case 1912: // SUBSCRIPTION_NOT_ENABLED_ERROR default: { final String errorMessage = diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/execution/PipeSubtaskExecutorManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/execution/PipeSubtaskExecutorManager.java index eadb75463bb7..f0de4d8d58e5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/execution/PipeSubtaskExecutorManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/execution/PipeSubtaskExecutorManager.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.pipe.agent.task.execution; +import org.apache.iotdb.commons.subscription.config.SubscriptionConfig; import org.apache.iotdb.db.pipe.consensus.PipeConsensusSubtaskExecutor; import org.apache.iotdb.db.subscription.task.execution.SubscriptionSubtaskExecutor; @@ -53,7 +54,10 @@ public PipeConsensusSubtaskExecutor getConsensusExecutor() { private PipeSubtaskExecutorManager() { processorExecutor = new PipeProcessorSubtaskExecutor(); connectorExecutor = new PipeConnectorSubtaskExecutor(); - subscriptionExecutor = new SubscriptionSubtaskExecutor(); + subscriptionExecutor = + SubscriptionConfig.getInstance().getSubscriptionEnabled() + ? new SubscriptionSubtaskExecutor() + : null; consensusExecutor = new PipeConsensusSubtaskExecutor(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java index 2927980d0037..0be487aa7fd1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java @@ -63,6 +63,7 @@ import org.apache.iotdb.commons.schema.view.viewExpression.ViewExpression; import org.apache.iotdb.commons.service.metric.MetricService; import org.apache.iotdb.commons.service.metric.enums.Tag; +import org.apache.iotdb.commons.subscription.config.SubscriptionConfig; import org.apache.iotdb.commons.subscription.meta.consumer.ConsumerGroupMeta; import org.apache.iotdb.commons.subscription.meta.topic.TopicMeta; import org.apache.iotdb.commons.trigger.TriggerInformation; @@ -1185,6 +1186,11 @@ public TPushPipeMetaResp pushMultiPipeMeta(TPushMultiPipeMetaReq req) { @Override public TPushTopicMetaResp pushTopicMeta(TPushTopicMetaReq req) { + if (!SubscriptionConfig.getInstance().getSubscriptionEnabled()) { + return new TPushTopicMetaResp() + .setStatus(new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode())); + } + final List topicMetas = new ArrayList<>(); for (ByteBuffer byteBuffer : req.getTopicMetas()) { topicMetas.add(TopicMeta.deserialize(byteBuffer)); @@ -1208,6 +1214,11 @@ public TPushTopicMetaResp pushTopicMeta(TPushTopicMetaReq req) { @Override public TPushTopicMetaResp pushSingleTopicMeta(TPushSingleTopicMetaReq req) { + if (!SubscriptionConfig.getInstance().getSubscriptionEnabled()) { + return new TPushTopicMetaResp() + .setStatus(new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode())); + } + try { final TPushTopicMetaRespExceptionMessage exceptionMessage; if (req.isSetTopicNameToDrop()) { @@ -1236,6 +1247,11 @@ public TPushTopicMetaResp pushSingleTopicMeta(TPushSingleTopicMetaReq req) { @Override public TPushTopicMetaResp pushMultiTopicMeta(TPushMultiTopicMetaReq req) { + if (!SubscriptionConfig.getInstance().getSubscriptionEnabled()) { + return new TPushTopicMetaResp() + .setStatus(new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode())); + } + boolean hasException = false; // If there is any exception, we use the size of exceptionMessages to record the fail index List exceptionMessages = new ArrayList<>(); @@ -1283,6 +1299,11 @@ public TPushTopicMetaResp pushMultiTopicMeta(TPushMultiTopicMetaReq req) { @Override public TPushConsumerGroupMetaResp pushConsumerGroupMeta(TPushConsumerGroupMetaReq req) { + if (!SubscriptionConfig.getInstance().getSubscriptionEnabled()) { + return new TPushConsumerGroupMetaResp() + .setStatus(new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode())); + } + final List consumerGroupMetas = new ArrayList<>(); for (ByteBuffer byteBuffer : req.getConsumerGroupMetas()) { consumerGroupMetas.add(ConsumerGroupMeta.deserialize(byteBuffer)); @@ -1307,6 +1328,11 @@ public TPushConsumerGroupMetaResp pushConsumerGroupMeta(TPushConsumerGroupMetaRe @Override public TPushConsumerGroupMetaResp pushSingleConsumerGroupMeta( TPushSingleConsumerGroupMetaReq req) { + if (!SubscriptionConfig.getInstance().getSubscriptionEnabled()) { + return new TPushConsumerGroupMetaResp() + .setStatus(new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode())); + } + try { final TPushConsumerGroupMetaRespExceptionMessage exceptionMessage; if (req.isSetConsumerGroupNameToDrop()) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java index abdecf6ba739..c0436818de10 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java @@ -53,6 +53,7 @@ import org.apache.iotdb.commons.pipe.connector.payload.airgap.AirGapPseudoTPipeTransferRequest; import org.apache.iotdb.commons.schema.view.LogicalViewSchema; import org.apache.iotdb.commons.schema.view.viewExpression.ViewExpression; +import org.apache.iotdb.commons.subscription.config.SubscriptionConfig; import org.apache.iotdb.commons.subscription.meta.topic.TopicMeta; import org.apache.iotdb.commons.trigger.service.TriggerExecutableManager; import org.apache.iotdb.commons.udf.service.UDFClassLoader; @@ -311,6 +312,16 @@ private static final class ClusterConfigTaskExecutorHolder { private ClusterConfigTaskExecutorHolder() {} } + private static final SettableFuture SUBSCRIPTION_NOT_ENABLED_ERROR_FUTURE; + + static { + SUBSCRIPTION_NOT_ENABLED_ERROR_FUTURE = SettableFuture.create(); + SUBSCRIPTION_NOT_ENABLED_ERROR_FUTURE.setException( + new IoTDBException( + "Subscription not enabled, please set config `subscription_enabled` to true.", + TSStatusCode.SUBSCRIPTION_NOT_ENABLED_ERROR.getStatusCode())); + } + public static ClusterConfigTaskExecutor getInstance() { return ClusterConfigTaskExecutor.ClusterConfigTaskExecutorHolder.INSTANCE; } @@ -2080,6 +2091,10 @@ public SettableFuture showPipes(ShowPipesStatement showPipesSt @Override public SettableFuture showSubscriptions( ShowSubscriptionsStatement showSubscriptionsStatement) { + if (!SubscriptionConfig.getInstance().getSubscriptionEnabled()) { + return SUBSCRIPTION_NOT_ENABLED_ERROR_FUTURE; + } + final SettableFuture future = SettableFuture.create(); try (final ConfigNodeClient configNodeClient = @@ -2113,6 +2128,10 @@ public SettableFuture showSubscriptions( public SettableFuture dropSubscription( final DropSubscriptionStatement dropSubscriptionStatement) { + if (!SubscriptionConfig.getInstance().getSubscriptionEnabled()) { + return SUBSCRIPTION_NOT_ENABLED_ERROR_FUTURE; + } + final SettableFuture future = SettableFuture.create(); try (ConfigNodeClient configNodeClient = CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { @@ -2134,6 +2153,10 @@ public SettableFuture dropSubscription( @Override public SettableFuture createTopic(CreateTopicStatement createTopicStatement) { + if (!SubscriptionConfig.getInstance().getSubscriptionEnabled()) { + return SUBSCRIPTION_NOT_ENABLED_ERROR_FUTURE; + } + final SettableFuture future = SettableFuture.create(); final String topicName = createTopicStatement.getTopicName(); @@ -2198,6 +2221,10 @@ public SettableFuture createTopic(CreateTopicStatement createT @Override public SettableFuture dropTopic(DropTopicStatement dropTopicStatement) { + if (!SubscriptionConfig.getInstance().getSubscriptionEnabled()) { + return SUBSCRIPTION_NOT_ENABLED_ERROR_FUTURE; + } + final SettableFuture future = SettableFuture.create(); try (ConfigNodeClient configNodeClient = CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { @@ -2221,6 +2248,10 @@ public SettableFuture dropTopic(DropTopicStatement dropTopicSt @Override public SettableFuture showTopics(ShowTopicsStatement showTopicsStatement) { + if (!SubscriptionConfig.getInstance().getSubscriptionEnabled()) { + return SUBSCRIPTION_NOT_ENABLED_ERROR_FUTURE; + } + final SettableFuture future = SettableFuture.create(); try (final ConfigNodeClient configNodeClient = CONFIG_NODE_CLIENT_MANAGER.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionReceiverAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionReceiverAgent.java index 0eb32e21f5d4..2e2f27b0a02d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionReceiverAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionReceiverAgent.java @@ -42,17 +42,29 @@ public class SubscriptionReceiverAgent { private static final Logger LOGGER = LoggerFactory.getLogger(SubscriptionReceiverAgent.class); - private final ThreadLocal receiverThreadLocal = new ThreadLocal<>(); - private static final Map> RECEIVER_CONSTRUCTORS = new HashMap<>(); + private static final TPipeSubscribeResp SUBSCRIPTION_NOT_ENABLED_ERROR_RESP = + new TPipeSubscribeResp( + RpcUtils.getStatus( + TSStatusCode.SUBSCRIPTION_NOT_ENABLED_ERROR, + "Subscription not enabled, please set config `subscription_enabled` to true."), + PipeSubscribeResponseVersion.VERSION_1.getVersion(), + PipeSubscribeResponseType.ACK.getType()); + + private final ThreadLocal receiverThreadLocal = new ThreadLocal<>(); + SubscriptionReceiverAgent() { RECEIVER_CONSTRUCTORS.put( PipeSubscribeRequestVersion.VERSION_1.getVersion(), SubscriptionReceiverV1::new); } public TPipeSubscribeResp handle(final TPipeSubscribeReq req) { + if (!SubscriptionConfig.getInstance().getSubscriptionEnabled()) { + return SUBSCRIPTION_NOT_ENABLED_ERROR_RESP; + } + final byte reqVersion = req.getVersion(); if (RECEIVER_CONSTRUCTORS.containsKey(reqVersion)) { return getReceiver(reqVersion).handle(req); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionRuntimeAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionRuntimeAgent.java index 27c75b62dced..aec165684635 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionRuntimeAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/agent/SubscriptionRuntimeAgent.java @@ -62,6 +62,10 @@ SubscriptionTopicAgent topic() { @Override public void start() throws StartupException { + if (!SubscriptionConfig.getInstance().getSubscriptionEnabled()) { + return; + } + SubscriptionConfig.getInstance().printAllConfigs(); SubscriptionAgentLauncher.launchSubscriptionTopicAgent(); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java index d776807431ab..2411b2bc53c4 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java @@ -318,6 +318,8 @@ public class CommonConfig { private boolean pipeEventReferenceTrackingEnabled = true; private long pipeEventReferenceEliminateIntervalSeconds = 10; + private boolean subscriptionEnabled = false; + private float subscriptionCacheMemoryUsagePercentage = 0.2F; private int subscriptionSubtaskExecutorMaxThreadNum = 2; @@ -2068,6 +2070,14 @@ public void setPipeEventReferenceEliminateIntervalSeconds( pipeEventReferenceEliminateIntervalSeconds); } + public boolean getSubscriptionEnabled() { + return subscriptionEnabled; + } + + public void setSubscriptionEnabled(boolean subscriptionEnabled) { + this.subscriptionEnabled = subscriptionEnabled; + } + public float getSubscriptionCacheMemoryUsagePercentage() { return subscriptionCacheMemoryUsagePercentage; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonDescriptor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonDescriptor.java index c17723cde3c6..0432415637fa 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonDescriptor.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonDescriptor.java @@ -252,6 +252,11 @@ public void loadCommonProps(TrimProperties properties) throws IOException { } private void loadSubscriptionProps(TrimProperties properties) { + config.setSubscriptionEnabled( + Boolean.parseBoolean( + properties.getProperty( + "subscription_enabled", String.valueOf(config.getSubscriptionEnabled())))); + config.setSubscriptionCacheMemoryUsagePercentage( Float.parseFloat( properties.getProperty( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java index 031551753b2e..8b3c260f46d2 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java @@ -34,6 +34,7 @@ import org.apache.iotdb.commons.pipe.agent.task.progress.CommitterKey; import org.apache.iotdb.commons.pipe.agent.task.progress.PipeEventCommitManager; import org.apache.iotdb.commons.pipe.connector.limiter.PipeEndPointRateLimiter; +import org.apache.iotdb.commons.subscription.config.SubscriptionConfig; import org.apache.iotdb.mpp.rpc.thrift.TPipeHeartbeatReq; import org.apache.iotdb.mpp.rpc.thrift.TPipeHeartbeatResp; import org.apache.iotdb.mpp.rpc.thrift.TPushPipeMetaRespExceptionMessage; @@ -168,6 +169,13 @@ protected TPushPipeMetaRespExceptionMessage handleSinglePipeMetaChangesInternal( private void executeSinglePipeMetaChanges(final PipeMeta metaFromCoordinator) throws IllegalPathException { final String pipeName = metaFromCoordinator.getStaticMeta().getPipeName(); + + // Do nothing with the subscription pipe if disable subscription + if (PipeStaticMeta.isSubscriptionPipe(pipeName) + && !SubscriptionConfig.getInstance().getSubscriptionEnabled()) { + return; + } + final PipeMeta metaInAgent = pipeMetaKeeper.getPipeMeta(pipeName); // If pipe meta does not exist on local agent, create a new pipe diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/config/SubscriptionConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/config/SubscriptionConfig.java index a332b87f0ed1..c58ae7258888 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/config/SubscriptionConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/subscription/config/SubscriptionConfig.java @@ -29,6 +29,10 @@ public class SubscriptionConfig { private static final CommonConfig COMMON_CONFIG = CommonDescriptor.getInstance().getConfig(); + public boolean getSubscriptionEnabled() { + return COMMON_CONFIG.getSubscriptionEnabled(); + } + public float getSubscriptionCacheMemoryUsagePercentage() { return COMMON_CONFIG.getSubscriptionCacheMemoryUsagePercentage(); } @@ -138,6 +142,8 @@ public long getSubscriptionMetaSyncerSyncIntervalMinutes() { private static final Logger LOGGER = LoggerFactory.getLogger(SubscriptionConfig.class); public void printAllConfigs() { + LOGGER.info("SubscriptionEnabled: {}", getSubscriptionEnabled()); + LOGGER.info( "SubscriptionCacheMemoryUsagePercentage: {}", getSubscriptionCacheMemoryUsagePercentage()); LOGGER.info( From 31a3a023deedd02db07f0087d54e92d687ebf722 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 26 Jun 2025 16:45:12 +0800 Subject: [PATCH 012/185] Fix --- .../dataregion/wal/checkpoint/CheckpointManager.java | 8 ++++++++ .../dataregion/wal/io/WALByteBufReader.java | 12 ------------ 2 files changed, 8 insertions(+), 12 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/checkpoint/CheckpointManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/checkpoint/CheckpointManager.java index 77b7f347ae4c..6f801a78457c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/checkpoint/CheckpointManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/checkpoint/CheckpointManager.java @@ -38,6 +38,7 @@ import java.nio.file.Files; import java.util.ArrayList; import java.util.Collections; +import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -256,6 +257,13 @@ private boolean tryRollingLogWriter() throws IOException { // endregion + public MemTableInfo getOldestMemTableInfo() { + // find oldest memTable + return activeOrPinnedMemTables().stream() + .min(Comparator.comparingLong(MemTableInfo::getMemTableId)) + .orElse(null); + } + /** * Get version id of first valid .wal file * diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/io/WALByteBufReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/io/WALByteBufReader.java index a022946ba90e..2f257da9adc4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/io/WALByteBufReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/io/WALByteBufReader.java @@ -49,18 +49,6 @@ public WALByteBufReader(File logFile) throws IOException { } } - public WALByteBufReader(WALEntryPosition walEntryPosition) throws IOException { - WALInputStream walInputStream = walEntryPosition.openReadFileStream(); - try { - this.logStream = new DataInputStream(walInputStream); - this.metaData = walInputStream.getWALMetaData(); - this.sizeIterator = metaData.getBuffersSize().iterator(); - } catch (Exception e) { - walInputStream.close(); - throw e; - } - } - /** Like {@link Iterator#hasNext()}. */ public boolean hasNext() { return sizeIterator.hasNext(); From 56f97e89f1f4b86f2bdee478e8dd9fd04bb142d3 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Thu, 26 Jun 2025 16:52:46 +0800 Subject: [PATCH 013/185] modify pipeStuckRestartMinIntervalMs --- .../main/java/org/apache/iotdb/commons/conf/CommonConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java index 2411b2bc53c4..5caeeffe12b7 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java @@ -276,7 +276,7 @@ public class CommonConfig { private long pipeMaxAllowedLinkedTsFileCount = Long.MAX_VALUE; // Deprecated private float pipeMaxAllowedLinkedDeletedTsFileDiskUsagePercentage = 0.1F; private long pipeStuckRestartIntervalSeconds = 120; - private long pipeStuckRestartMinIntervalMs = 5 * 60 * 1000L; // 5 minutes + private long pipeStuckRestartMinIntervalMs = 10 * 60 * 1000L; // 5 minutes private boolean pipeEpochKeepTsFileAfterStuckRestartEnabled = false; private long pipeFlushAfterLastTerminateSeconds = 30; private long pipeFlushAfterTerminateCount = 30; From 2cb4692808f503e30315955556a15f633c954b91 Mon Sep 17 00:00:00 2001 From: Steve Yurong Su Date: Thu, 19 Jun 2025 12:05:09 +0800 Subject: [PATCH 014/185] Pipe/IoTV2: Persist progress index locally before shutdown to accurate recovery after restart (#15779) --- .../agent/task/PipeDataNodeTaskAgent.java | 24 ++++++++++++++++++ .../db/service/DataNodeShutdownHook.java | 4 +++ .../pipe/agent/task/meta/PipeRuntimeMeta.java | 9 +++++++ .../pipe/agent/task/meta/PipeTaskMeta.java | 25 ++++++++----------- 4 files changed, 48 insertions(+), 14 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java index c98f1b0a65af..67ca74976982 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java @@ -830,6 +830,30 @@ public void runPipeTasks( } } + ///////////////////////// Shutdown Logic ///////////////////////// + + public void persistAllProgressIndexLocally() { + if (!PipeConfig.getInstance().isPipeProgressIndexPersistEnabled()) { + LOGGER.info( + "Pipe progress index persist disabled. Skipping persist all progress index locally."); + return; + } + if (!tryReadLockWithTimeOut(10)) { + LOGGER.info("Failed to persist all progress index locally because of timeout."); + return; + } + try { + for (final PipeMeta pipeMeta : pipeMetaKeeper.getPipeMetaList()) { + pipeMeta.getRuntimeMeta().persistProgressIndex(); + } + LOGGER.info("Persist all progress index locally successfully."); + } catch (final Exception e) { + LOGGER.warn("Failed to record all progress index locally, because {}.", e.getMessage(), e); + } finally { + releaseReadLock(); + } + } + ///////////////////////// Pipe Consensus ///////////////////////// public ProgressIndex getPipeTaskProgressIndex(final String pipeName, final int consensusGroupId) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java index 13b87f5bb83c..191341a1ea81 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java @@ -29,6 +29,7 @@ import org.apache.iotdb.consensus.exception.ConsensusException; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.consensus.DataRegionConsensusImpl; +import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; import org.apache.iotdb.db.protocol.client.ConfigNodeClient; import org.apache.iotdb.db.protocol.client.ConfigNodeClientManager; import org.apache.iotdb.db.protocol.client.ConfigNodeInfo; @@ -86,6 +87,9 @@ public void run() { triggerSnapshotForAllDataRegion(); } + // Persist progress index before shutdown to accurate recovery after restart + PipeDataNodeAgent.task().persistAllProgressIndexLocally(); + // Shutdown pipe progressIndex background service PipePeriodicalJobExecutor.shutdownBackgroundService(); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/meta/PipeRuntimeMeta.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/meta/PipeRuntimeMeta.java index 752edae0cf0d..402a601e52b7 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/meta/PipeRuntimeMeta.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/meta/PipeRuntimeMeta.java @@ -140,6 +140,15 @@ public void setIsStoppedByRuntimeException(boolean isStoppedByRuntimeException) this.isStoppedByRuntimeException.set(isStoppedByRuntimeException); } + public void persistProgressIndex() { + // Iterate through all the task metas and persist their progress index + for (final PipeTaskMeta taskMeta : consensusGroupId2TaskMetaMap.values()) { + if (taskMeta.getProgressIndex() != null) { + taskMeta.persistProgressIndex(); + } + } + } + public ByteBuffer serialize() throws IOException { PublicBAOS byteArrayOutputStream = new PublicBAOS(); DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/meta/PipeTaskMeta.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/meta/PipeTaskMeta.java index 6a4ab25db7e8..4a753c0e5bfb 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/meta/PipeTaskMeta.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/meta/PipeTaskMeta.java @@ -113,34 +113,31 @@ public ProgressIndex getProgressIndex() { public ProgressIndex updateProgressIndex(final ProgressIndex updateIndex) { // only pipeTaskMeta that need to updateProgressIndex will persist progress index // isRegisterPersistTask is used to avoid multiple threads registering persist task concurrently - if (Objects.nonNull(progressIndexPersistFile) - && !isRegisterPersistTask.getAndSet(true) + if (PipeConfig.getInstance().isPipeProgressIndexPersistEnabled() && this.persistProgressIndexFuture == null - && PipeConfig.getInstance().isPipeProgressIndexPersistEnabled()) { + && !isRegisterPersistTask.getAndSet(true)) { this.persistProgressIndexFuture = PipePeriodicalJobExecutor.submitBackgroundJob( - () -> { - if (PipeConfig.getInstance().isPipeProgressIndexPersistEnabled()) { - persistProgressIndex(); - } - }, + this::persistProgressIndex, 0, PipeConfig.getInstance().getPipeProgressIndexFlushIntervalMs()); } progressIndex.updateAndGet( index -> index.updateToMinimumEqualOrIsAfterProgressIndex(updateIndex)); - if (Objects.nonNull(progressIndexPersistFile) - && updateCount.incrementAndGet() - lastPersistCount.get() > checkPointGap - && PipeConfig.getInstance().isPipeProgressIndexPersistEnabled()) { + + if (PipeConfig.getInstance().isPipeProgressIndexPersistEnabled() + && updateCount.incrementAndGet() - lastPersistCount.get() > checkPointGap) { persistProgressIndex(); } + return progressIndex.get(); } - private synchronized void persistProgressIndex() { - if (lastPersistCount.get() == updateCount.get()) { - // in case of multiple threads calling updateProgressIndex at the same time + public synchronized void persistProgressIndex() { + if (Objects.isNull(progressIndexPersistFile) + // in case of multiple threads calling updateProgressIndex at the same time + || lastPersistCount.get() == updateCount.get()) { return; } From 99239772f6b4b772b2e5597c56245a871e77b993 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Thu, 26 Jun 2025 17:20:20 +0800 Subject: [PATCH 015/185] add MaxWaitFinishTime --- ...eDataNodeRemainingEventAndTimeMetrics.java | 2 +- ...DataNodeRemainingEventAndTimeOperator.java | 6 ++--- .../db/service/DataNodeShutdownHook.java | 25 +++++++++++++++++++ .../iotdb/commons/conf/CommonConfig.java | 14 +++++++++++ .../iotdb/commons/pipe/config/PipeConfig.java | 6 +++++ .../commons/pipe/config/PipeDescriptor.java | 5 ++++ 6 files changed, 54 insertions(+), 4 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeMetrics.java index 354a980edfd2..ea31c879005b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeMetrics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeMetrics.java @@ -50,7 +50,7 @@ public class PipeDataNodeRemainingEventAndTimeMetrics implements IMetricSet { @SuppressWarnings("java:S3077") private volatile AbstractMetricService metricService; - private final Map + public final Map remainingEventAndTimeOperatorMap = new ConcurrentHashMap<>(); private static Histogram PIPE_DATANODE_INSERTNODE_TRANSFER_TIME_HISTOGRAM = diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeOperator.java index 86368acf353f..3cf7dd411d73 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeOperator.java @@ -40,7 +40,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; -class PipeDataNodeRemainingEventAndTimeOperator extends PipeRemainingOperator { +public class PipeDataNodeRemainingEventAndTimeOperator extends PipeRemainingOperator { // Calculate from schema region extractors directly for it requires less computation private final Set schemaRegionExtractors = @@ -107,7 +107,7 @@ void decreaseHeartbeatEventCount() { return insertNodeEventCountEMA.insertNodeEMAValue; } - long getRemainingEvents() { + public long getRemainingEvents() { final long remainingEvents = tsfileEventCount.get() + rawTabletEventCount.get() @@ -131,7 +131,7 @@ long getRemainingEvents() { * * @return The estimated remaining time */ - double getRemainingTime() { + public double getRemainingTime() { final PipeRateAverage pipeRemainingTimeCommitRateAverageTime = PipeConfig.getInstance().getPipeRemainingTimeCommitRateAverageTime(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java index 191341a1ea81..d384fce30146 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java @@ -25,11 +25,14 @@ import org.apache.iotdb.commons.concurrent.ThreadName; import org.apache.iotdb.commons.conf.CommonDescriptor; import org.apache.iotdb.commons.pipe.agent.runtime.PipePeriodicalJobExecutor; +import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.consensus.ConsensusFactory; import org.apache.iotdb.consensus.exception.ConsensusException; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.consensus.DataRegionConsensusImpl; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; +import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeRemainingEventAndTimeMetrics; +import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeRemainingEventAndTimeOperator; import org.apache.iotdb.db.protocol.client.ConfigNodeClient; import org.apache.iotdb.db.protocol.client.ConfigNodeClientManager; import org.apache.iotdb.db.protocol.client.ConfigNodeInfo; @@ -43,6 +46,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Map; + public class DataNodeShutdownHook extends Thread { private static final Logger logger = LoggerFactory.getLogger(DataNodeShutdownHook.class); @@ -87,6 +92,26 @@ public void run() { triggerSnapshotForAllDataRegion(); } + long startTime = System.currentTimeMillis(); + if (PipeDataNodeAgent.task().getPipeCount() != 0) { + for (Map.Entry entry : + PipeDataNodeRemainingEventAndTimeMetrics.getInstance() + .remainingEventAndTimeOperatorMap + .entrySet()) { + while (entry.getValue().getRemainingEvents() > 0) { + if (System.currentTimeMillis() - startTime + > PipeConfig.getInstance().getPipeMaxWaitFinishTime()) { + break; + } + try { + Thread.sleep(100); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + logger.info("Interrupted when waiting for pipe to finish"); + } + } + } + } // Persist progress index before shutdown to accurate recovery after restart PipeDataNodeAgent.task().persistAllProgressIndexLocally(); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java index 5caeeffe12b7..e352814f48ab 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java @@ -227,6 +227,8 @@ public class CommonConfig { private long pipeSubtaskExecutorCronHeartbeatEventIntervalSeconds = 20; private long pipeSubtaskExecutorForcedRestartIntervalMs = Long.MAX_VALUE; + private long pipeMaxWaitFinishTime = 2 * 60 * 1000; + private int pipeExtractorAssignerDisruptorRingBufferSize = 65536; private long pipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes = 50; // 50B private int pipeExtractorMatcherCacheSize = 1024; @@ -1335,6 +1337,18 @@ public void setPipeSubtaskExecutorForcedRestartIntervalMs( pipeSubtaskExecutorForcedRestartIntervalMs); } + public long getPipeMaxWaitFinishTime() { + return pipeMaxWaitFinishTime; + } + + public void setPipeMaxWaitFinishTime(long pipeMaxWaitFinishTime) { + if (this.pipeMaxWaitFinishTime == pipeMaxWaitFinishTime) { + return; + } + this.pipeMaxWaitFinishTime = pipeMaxWaitFinishTime; + logger.info("pipeMaxWaitFinishTime is set to {}.", pipeMaxWaitFinishTime); + } + public int getPipeRealTimeQueuePollTsFileThreshold() { return pipeRealTimeQueuePollTsFileThreshold; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java index 984a0bc8ef7f..2458d0b8c26c 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java @@ -131,6 +131,12 @@ public long getPipeSubtaskExecutorForcedRestartIntervalMs() { return COMMON_CONFIG.getPipeSubtaskExecutorForcedRestartIntervalMs(); } + public long getPipeMaxWaitFinishTime() { + { + return COMMON_CONFIG.getPipeMaxWaitFinishTime(); + } + } + /////////////////////////////// Extractor /////////////////////////////// public int getPipeExtractorAssignerDisruptorRingBufferSize() { diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java index de11e5a6d516..4fd5a7479649 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java @@ -563,6 +563,11 @@ public static void loadPipeInternalConfig(CommonConfig config, TrimProperties pr "pipe_threshold_allocation_strategy_high_usage_threshold", String.valueOf( config.getPipeThresholdAllocationStrategyFixedMemoryHighUsageThreshold())))); + + config.setPipeMaxWaitFinishTime( + Long.parseLong( + properties.getProperty( + "pipe_max_wait_finish_time", String.valueOf(config.getPipeMaxWaitFinishTime())))); } public static void loadPipeExternalConfig( From 3ff163de8352667debadd69f8b48dcff14c39a61 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Thu, 26 Jun 2025 17:25:18 +0800 Subject: [PATCH 016/185] update PipeConfig --- .../java/org/apache/iotdb/commons/pipe/config/PipeConfig.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java index 2458d0b8c26c..8dd3e3bc7cd2 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java @@ -132,9 +132,7 @@ public long getPipeSubtaskExecutorForcedRestartIntervalMs() { } public long getPipeMaxWaitFinishTime() { - { - return COMMON_CONFIG.getPipeMaxWaitFinishTime(); - } + return COMMON_CONFIG.getPipeMaxWaitFinishTime(); } /////////////////////////////// Extractor /////////////////////////////// From 66d1631f01c299e7c58fb4669c03ca1d3a1f688a Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 26 Jun 2025 17:28:21 +0800 Subject: [PATCH 017/185] Fix --- ...ipeDataNodeRemainingEventAndTimeOperator.java | 16 ++++++++++++++++ .../iotdb/db/service/DataNodeShutdownHook.java | 2 +- 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeOperator.java index 3cf7dd411d73..b81c22eda309 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeOperator.java @@ -107,6 +107,22 @@ void decreaseHeartbeatEventCount() { return insertNodeEventCountEMA.insertNodeEMAValue; } + public long getRemainingNonHeartbeatEvents() { + final long remainingEvents = + tsfileEventCount.get() + + rawTabletEventCount.get() + + insertNodeEventCount.get() + + schemaRegionExtractors.stream() + .map(IoTDBSchemaRegionExtractor::getUnTransferredEventCount) + .reduce(Long::sum) + .orElse(0L); + + // There are cases where the indicator is negative. For example, after the Pipe is restarted, + // the Processor SubTask is still collecting Events, resulting in a negative count. This + // situation cannot be avoided because the Pipe may be restarted internally. + return remainingEvents >= 0 ? remainingEvents : 0; + } + public long getRemainingEvents() { final long remainingEvents = tsfileEventCount.get() diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java index d384fce30146..57d98df49c4a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java @@ -98,7 +98,7 @@ public void run() { PipeDataNodeRemainingEventAndTimeMetrics.getInstance() .remainingEventAndTimeOperatorMap .entrySet()) { - while (entry.getValue().getRemainingEvents() > 0) { + while (entry.getValue().getRemainingNonHeartbeatEvents() > 0) { if (System.currentTimeMillis() - startTime > PipeConfig.getInstance().getPipeMaxWaitFinishTime()) { break; From 6e7ff7f4a8504235d2669328723ec9539580891d Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 26 Jun 2025 18:01:20 +0800 Subject: [PATCH 018/185] Update WalDeleteOutdatedNewTest.java --- .../wal/node/WalDeleteOutdatedNewTest.java | 60 ------------------- 1 file changed, 60 deletions(-) diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WalDeleteOutdatedNewTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WalDeleteOutdatedNewTest.java index 6af5ed10cacc..c132d268c7d3 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WalDeleteOutdatedNewTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/node/WalDeleteOutdatedNewTest.java @@ -33,10 +33,8 @@ import org.apache.iotdb.db.storageengine.dataregion.DataRegionTest; import org.apache.iotdb.db.storageengine.dataregion.memtable.IMemTable; import org.apache.iotdb.db.storageengine.dataregion.memtable.PrimitiveMemTable; -import org.apache.iotdb.db.storageengine.dataregion.wal.exception.MemTablePinException; import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALFileUtils; import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALMode; -import org.apache.iotdb.db.storageengine.dataregion.wal.utils.listener.WALFlushListener; import org.apache.iotdb.db.utils.EnvironmentUtils; import org.apache.iotdb.db.utils.constant.TestConstant; @@ -279,64 +277,6 @@ public void test04() throws IllegalPathException { Assert.assertEquals(1, WALFileUtils.listAllWALFiles(new File(logDirectory1)).length); } - /** - * Ensure that wal pinned to memtable cannot be deleted:
- * 1. _0-0-1.wal: memTable0
- * 2. pin memTable0
- * 3. memTable0 flush
- * 4. roll wal file
- * 5. _1-1-1.wal: memTable0、memTable1
- * 6. roll wal file
- * 7. _2-1-1.wal: memTable1
- * 8. roll wal file
- * 9. _2-1-1.wal: memTable1
- * 10. wait until all walEntry consumed
- * 11. memTable0 flush, memTable1 flush
- * 12. delete outdated wal files - */ - @Test - public void test05() throws IllegalPathException, MemTablePinException { - IMemTable memTable0 = new PrimitiveMemTable(databasePath, dataRegionId); - walNode1.onMemTableCreated(memTable0, logDirectory1 + "/" + "fake.tsfile"); - WALFlushListener listener = - walNode1.log( - memTable0.getMemTableId(), - generateInsertRowNode(devicePath, System.currentTimeMillis(), 1)); - walNode1.rollWALFile(); - - // pin memTable - walNode1.log( - memTable0.getMemTableId(), - generateInsertRowNode(devicePath, System.currentTimeMillis(), 2)); - IMemTable memTable1 = new PrimitiveMemTable(databasePath, dataRegionId); - walNode1.onMemTableCreated(memTable1, logDirectory1 + "/" + "fake.tsfile"); - walNode1.log( - memTable1.getMemTableId(), - generateInsertRowNode(devicePath, System.currentTimeMillis(), 3)); - walNode1.rollWALFile(); - - walNode1.log( - memTable1.getMemTableId(), - generateInsertRowNode(devicePath, System.currentTimeMillis(), 4)); - walNode1.rollWALFile(); - - walNode1.log( - memTable1.getMemTableId(), - generateInsertRowNode(devicePath, System.currentTimeMillis(), 5)); - walNode1.onMemTableFlushed(memTable0); - walNode1.onMemTableFlushed(memTable1); - Awaitility.await().until(() -> walNode1.isAllWALEntriesConsumed()); - - Map> memTableIdsOfWal = walNode1.getWALBuffer().getMemTableIdsOfWal(); - Assert.assertEquals(4, memTableIdsOfWal.size()); - Assert.assertEquals(4, WALFileUtils.listAllWALFiles(new File(logDirectory1)).length); - - walNode1.deleteOutdatedFiles(); - Map> memTableIdsOfWalAfter = walNode1.getWALBuffer().getMemTableIdsOfWal(); - Assert.assertEquals(3, memTableIdsOfWalAfter.size()); - Assert.assertEquals(3, WALFileUtils.listAllWALFiles(new File(logDirectory1)).length); - } - /** * Ensure that the flushed wal related to memtable cannot be deleted:
* 1. _0-0-1.wal: memTable0
From 213236d76520c01feafca1cf3048f09a86304b75 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 26 Jun 2025 18:11:43 +0800 Subject: [PATCH 019/185] Update PipeMemoryManager.java --- .../db/pipe/resource/memory/PipeMemoryManager.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryManager.java index f01d7e6b1cc2..3f7f7b235079 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryManager.java @@ -54,9 +54,6 @@ public class PipeMemoryManager { private volatile long usedMemorySizeInBytesOfTsFiles; - private static final double FLOATING_MEMORY_RATIO = - PipeConfig.getInstance().getPipeTotalFloatingMemoryProportion(); - // Only non-zero memory blocks will be added to this set. private final Set allocatedBlocks = new HashSet<>(); @@ -641,10 +638,14 @@ public long getFreeMemorySizeInBytes() { } public static long getTotalNonFloatingMemorySizeInBytes() { - return (long) (TOTAL_MEMORY_SIZE_IN_BYTES * (1 - FLOATING_MEMORY_RATIO)); + return (long) + (TOTAL_MEMORY_SIZE_IN_BYTES + * (1 - PipeConfig.getInstance().getPipeTotalFloatingMemoryProportion())); } public static long getTotalFloatingMemorySizeInBytes() { - return (long) (TOTAL_MEMORY_SIZE_IN_BYTES * FLOATING_MEMORY_RATIO); + return (long) + (TOTAL_MEMORY_SIZE_IN_BYTES + * PipeConfig.getInstance().getPipeTotalFloatingMemoryProportion()); } } From a8cfd04f42ba054743c62ce783157a0e77c1a828 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 26 Jun 2025 18:12:21 +0800 Subject: [PATCH 020/185] Update CommonConfig.java --- .../main/java/org/apache/iotdb/commons/conf/CommonConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java index e352814f48ab..46c3d6927d7c 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java @@ -218,7 +218,7 @@ public class CommonConfig { private double pipeDataStructureTabletMemoryBlockAllocationRejectThreshold = 0.2; private double pipeDataStructureTsFileMemoryBlockAllocationRejectThreshold = 0.2; private double PipeDataStructureBatchMemoryProportion = 0.1; - private double pipeTotalFloatingMemoryProportion = 0.5; + private volatile double pipeTotalFloatingMemoryProportion = 0.5; private int pipeSubtaskExecutorBasicCheckPointIntervalByConsumedEventCount = 10_000; private long pipeSubtaskExecutorBasicCheckPointIntervalByTimeDuration = 10 * 1000L; From 4dc20723c088a9438094864f67480be843bb6648 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 26 Jun 2025 19:03:06 +0800 Subject: [PATCH 021/185] Update PipeInsertNodeTabletInsertionEvent.java --- .../event/common/tablet/PipeInsertNodeTabletInsertionEvent.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java index 6b8d1d8e7bc0..791e9a554993 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java @@ -142,7 +142,6 @@ public boolean internallyDecreaseResourceReferenceCount(final String holderMessa dataContainers.clear(); dataContainers = null; } - insertNode = null; close(); return true; } catch (final Exception e) { @@ -152,6 +151,7 @@ public boolean internallyDecreaseResourceReferenceCount(final String holderMessa } finally { if (Objects.nonNull(pipeName)) { PipeDataNodeAgent.task().decreaseFloatingMemoryUsageInByte(pipeName, ramBytesUsed()); + insertNode = null; PipeDataNodeRemainingEventAndTimeMetrics.getInstance() .decreaseInsertNodeEventCount(pipeName, creationTime, System.nanoTime() - extractTime); } From 2a3c8618485750cb83a37ba016f6d5a60b676b8c Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 26 Jun 2025 19:06:02 +0800 Subject: [PATCH 022/185] Update PipeInsertNodeTabletInsertionEvent.java --- .../event/common/tablet/PipeInsertNodeTabletInsertionEvent.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java index 791e9a554993..da71eddb461c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java @@ -151,10 +151,10 @@ public boolean internallyDecreaseResourceReferenceCount(final String holderMessa } finally { if (Objects.nonNull(pipeName)) { PipeDataNodeAgent.task().decreaseFloatingMemoryUsageInByte(pipeName, ramBytesUsed()); - insertNode = null; PipeDataNodeRemainingEventAndTimeMetrics.getInstance() .decreaseInsertNodeEventCount(pipeName, creationTime, System.nanoTime() - extractTime); } + insertNode = null; } } From 0b35ba65cbf4631970bebe89c426c6ddb5b12d67 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 26 Jun 2025 20:12:22 +0800 Subject: [PATCH 023/185] Bug fix --- .../apache/iotdb/pipe/it/autocreate/IoTDBPipeDataSinkIT.java | 1 - .../apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java | 2 +- .../org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java | 2 +- 3 files changed, 2 insertions(+), 3 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeDataSinkIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeDataSinkIT.java index 7fbb0f3a7179..9a8ec2fc18e3 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeDataSinkIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeDataSinkIT.java @@ -459,7 +459,6 @@ public void testAsyncLoadTsFileWithoutVerify() throws Exception { testLoadTsFileWithoutVerify("async"); } - @Test private void testLoadTsFileWithoutVerify(final String loadTsFileStrategy) throws Exception { final DataNodeWrapper receiverDataNode = receiverEnv.getDataNodeWrapper(0); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java index 67ca74976982..738a7ef5bae8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java @@ -404,7 +404,7 @@ public void stopAllPipesWithCriticalException() { public void collectPipeMetaList(final TDataNodeHeartbeatResp resp) throws TException { if (!tryReadLockWithTimeOut( - CommonDescriptor.getInstance().getConfig().getCnConnectionTimeoutInMS())) { + CommonDescriptor.getInstance().getConfig().getDnConnectionTimeoutInMS() * 2L / 3)) { return; } try { diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java index 8b3c260f46d2..2bfde4173b29 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java @@ -346,7 +346,7 @@ protected TPushPipeMetaRespExceptionMessage handleDropPipeInternal(final String public List handlePipeMetaChanges( final List pipeMetaListFromCoordinator) { - if (tryWriteLockWithTimeOut( + if (!tryWriteLockWithTimeOut( CommonDescriptor.getInstance().getConfig().getDnConnectionTimeoutInMS() * 2L / 3)) { return null; } From bbd8f7b88b575e48376c2737101ad1ddb1eabfe4 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 26 Jun 2025 20:36:27 +0800 Subject: [PATCH 024/185] Update PipeConnectorConstant.java --- .../commons/pipe/config/constant/PipeConnectorConstant.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeConnectorConstant.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeConnectorConstant.java index 394ff7ae9d6a..c67809347037 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeConnectorConstant.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/constant/PipeConnectorConstant.java @@ -75,7 +75,7 @@ public class PipeConnectorConstant { public static final String CONNECTOR_IOTDB_BATCH_SIZE_KEY = "connector.batch.size-bytes"; public static final String SINK_IOTDB_BATCH_SIZE_KEY = "sink.batch.size-bytes"; public static final long CONNECTOR_IOTDB_PLAIN_BATCH_SIZE_DEFAULT_VALUE = MB; - public static final long CONNECTOR_IOTDB_TS_FILE_BATCH_SIZE_DEFAULT_VALUE = 2 * MB; + public static final long CONNECTOR_IOTDB_TS_FILE_BATCH_SIZE_DEFAULT_VALUE = MB; public static final String CONNECTOR_IOTDB_USER_KEY = "connector.user"; public static final String SINK_IOTDB_USER_KEY = "sink.user"; From 5b3e27527256bce2ced1758709d9bf64101e0f71 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Thu, 26 Jun 2025 21:06:46 +0800 Subject: [PATCH 025/185] add PipeDynamicMemoryBlock --- .../evolvable/batch/PipeTabletEventBatch.java | 35 +++++++++++++------ .../resource/memory/PipeMemoryManager.java | 6 ++++ .../iotdb/commons/pipe/config/PipeConfig.java | 4 +++ 3 files changed, 35 insertions(+), 10 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventBatch.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventBatch.java index 1cb9f50d92d9..2b9fa68ef302 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventBatch.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventBatch.java @@ -22,6 +22,7 @@ import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.db.pipe.connector.protocol.thrift.async.IoTDBDataRegionAsyncConnector; import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; +import org.apache.iotdb.db.pipe.resource.memory.PipeDynamicMemoryBlock; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryBlockType; import org.apache.iotdb.db.pipe.resource.memory.PipeModelFixedMemoryBlock; import org.apache.iotdb.db.storageengine.dataregion.wal.exception.WALPipeException; @@ -47,7 +48,7 @@ public abstract class PipeTabletEventBatch implements AutoCloseable { private long firstEventProcessingTime = Long.MIN_VALUE; protected long totalBufferSize = 0; - private final PipeModelFixedMemoryBlock allocatedMemoryBlock; + private final PipeDynamicMemoryBlock allocatedMemoryBlock; protected volatile boolean isClosed = false; @@ -60,11 +61,9 @@ protected PipeTabletEventBatch(final int maxDelayInMs, final long requestMaxBatc // limit in buffer size this.allocatedMemoryBlock = - PipeDataNodeResourceManager.memory() - .forceAllocateForModelFixedMemoryBlock( - requestMaxBatchSizeInBytes, PipeMemoryBlockType.BATCH); + pipeModelFixedMemoryBlock.registerPipeBatchMemoryBlock(requestMaxBatchSizeInBytes); - if (getMaxBatchSizeInBytes() != requestMaxBatchSizeInBytes) { + if (getMaxBatchSizeInBytes() != allocatedMemoryBlock.getMemoryUsageInBytes()) { LOGGER.info( "PipeTabletEventBatch: the max batch size is adjusted from {} to {} due to the " + "memory restriction", @@ -127,8 +126,12 @@ protected abstract boolean constructBatch(final TabletInsertionEvent event) throws WALPipeException, IOException; public boolean shouldEmit() { - return totalBufferSize >= getMaxBatchSizeInBytes() - || System.currentTimeMillis() - firstEventProcessingTime >= maxDelayInMs; + final long diff = System.currentTimeMillis() - firstEventProcessingTime; + if (totalBufferSize >= getMaxBatchSizeInBytes() || diff >= maxDelayInMs) { + allocatedMemoryBlock.updateCurrentMemoryEfficiencyAdjustMem((double) diff / maxDelayInMs); + return true; + } + return false; } private long getMaxBatchSizeInBytes() { @@ -194,9 +197,21 @@ public static void init() { } try { - pipeModelFixedMemoryBlock = - PipeDataNodeResourceManager.memory() - .forceAllocateForModelFixedMemoryBlock(0L, PipeMemoryBlockType.BATCH); + long batchSize = PipeDataNodeResourceManager.memory().getAllocatedMemorySizeInBytesOfBatch(); + for (long i = batchSize; i > 0; i = i / 2) { + try { + pipeModelFixedMemoryBlock = + PipeDataNodeResourceManager.memory() + .forceAllocateForModelFixedMemoryBlock(i, PipeMemoryBlockType.BATCH); + + LOGGER.info("pipe model fixed memory block initialized with size: {} bytes", i); + } catch (Exception ignore) { + // ignore the exception and try to allocate a smaller size + LOGGER.info( + "pipe model fixed memory block initialized with size: {} bytes failed, try smaller size", + i); + } + } } catch (Exception e) { LOGGER.error("init pipe model fixed memory block failed", e); // If the allocation fails, we still need to create a default memory block to avoid NPE. diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryManager.java index 3f7f7b235079..1afef9cf9ff4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryManager.java @@ -633,6 +633,12 @@ public long getUsedMemorySizeInBytesOfTsFiles() { return usedMemorySizeInBytesOfTsFiles; } + public long getAllocatedMemorySizeInBytesOfBatch() { + return (long) + (PipeConfig.getInstance().getPipeDataStructureBatchMemoryProportion() + * getTotalNonFloatingMemorySizeInBytes()); + } + public long getFreeMemorySizeInBytes() { return TOTAL_MEMORY_SIZE_IN_BYTES - usedMemorySizeInBytes; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java index 8dd3e3bc7cd2..2ecedde44f12 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java @@ -91,6 +91,10 @@ public double getPipeTotalFloatingMemoryProportion() { return COMMON_CONFIG.getPipeTotalFloatingMemoryProportion(); } + public double getPipeDataStructureBatchMemoryProportion() { + return COMMON_CONFIG.getPipeDataStructureBatchMemoryProportion(); + } + /////////////////////////////// Subtask Connector /////////////////////////////// public int getPipeRealTimeQueuePollTsFileThreshold() { From f12da9b2bd7d7e4caec334945910d5cd47bcd0f8 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Thu, 26 Jun 2025 21:10:21 +0800 Subject: [PATCH 026/185] fix --- .../connector/payload/evolvable/batch/PipeTabletEventBatch.java | 1 + 1 file changed, 1 insertion(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventBatch.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventBatch.java index 2b9fa68ef302..78a237a4102e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventBatch.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventBatch.java @@ -205,6 +205,7 @@ public static void init() { .forceAllocateForModelFixedMemoryBlock(i, PipeMemoryBlockType.BATCH); LOGGER.info("pipe model fixed memory block initialized with size: {} bytes", i); + return; } catch (Exception ignore) { // ignore the exception and try to allocate a smaller size LOGGER.info( From 3ffb268d5e952f9da95d6e8d5fa873e2a6139cc3 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 26 Jun 2025 21:11:20 +0800 Subject: [PATCH 027/185] defaultval --- .../java/org/apache/iotdb/commons/conf/CommonConfig.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java index 46c3d6927d7c..e23ec301e878 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java @@ -215,9 +215,9 @@ public class CommonConfig { private int pipeDataStructureTabletRowSize = 2048; private int pipeDataStructureTabletSizeInBytes = 2097152; - private double pipeDataStructureTabletMemoryBlockAllocationRejectThreshold = 0.2; - private double pipeDataStructureTsFileMemoryBlockAllocationRejectThreshold = 0.2; - private double PipeDataStructureBatchMemoryProportion = 0.1; + private double pipeDataStructureTabletMemoryBlockAllocationRejectThreshold = 0.3; + private double pipeDataStructureTsFileMemoryBlockAllocationRejectThreshold = 0.3; + private double PipeDataStructureBatchMemoryProportion = 0.2; private volatile double pipeTotalFloatingMemoryProportion = 0.5; private int pipeSubtaskExecutorBasicCheckPointIntervalByConsumedEventCount = 10_000; From 07bba6638774352e2bf0102b4588e940709eaff5 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 27 Jun 2025 09:45:46 +0800 Subject: [PATCH 028/185] Update IoTDBDataRegionExtractor.java --- .../dataregion/IoTDBDataRegionExtractor.java | 28 ------------------- 1 file changed, 28 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java index 808df12dfacc..fb5f12c1d85a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java @@ -38,7 +38,6 @@ import org.apache.iotdb.db.pipe.metric.overview.PipeTsFileToTabletsMetrics; import org.apache.iotdb.db.pipe.metric.source.PipeDataRegionExtractorMetrics; import org.apache.iotdb.db.storageengine.StorageEngine; -import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALMode; import org.apache.iotdb.pipe.api.customizer.configuration.PipeExtractorRuntimeConfiguration; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; @@ -47,7 +46,6 @@ import org.apache.iotdb.pipe.api.event.dml.insertion.TsFileInsertionEvent; import org.apache.iotdb.pipe.api.exception.PipeException; -import org.apache.tsfile.file.metadata.enums.CompressionType; import org.apache.tsfile.utils.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -270,7 +268,6 @@ private void constructRealtimeExtractor(final PipeParameters parameters) // Use hybrid mode by default if (!parameters.hasAnyAttributes(EXTRACTOR_REALTIME_MODE_KEY, SOURCE_REALTIME_MODE_KEY)) { - checkWalEnableAndSetUncompressed(parameters); realtimeExtractor = new PipeRealtimeDataRegionHybridExtractor(); LOGGER.info( "Pipe: '{}' is not set, use hybrid mode by default.", EXTRACTOR_REALTIME_MODE_KEY); @@ -285,15 +282,12 @@ private void constructRealtimeExtractor(final PipeParameters parameters) case EXTRACTOR_REALTIME_MODE_HYBRID_VALUE: case EXTRACTOR_REALTIME_MODE_LOG_VALUE: case EXTRACTOR_REALTIME_MODE_STREAM_MODE_VALUE: - checkWalEnableAndSetUncompressed(parameters); realtimeExtractor = new PipeRealtimeDataRegionHybridExtractor(); break; case EXTRACTOR_REALTIME_MODE_FORCED_LOG_VALUE: - checkWalEnableAndSetUncompressed(parameters); realtimeExtractor = new PipeRealtimeDataRegionLogExtractor(); break; default: - checkWalEnableAndSetUncompressed(parameters); realtimeExtractor = new PipeRealtimeDataRegionHybridExtractor(); if (LOGGER.isWarnEnabled()) { LOGGER.warn( @@ -303,28 +297,6 @@ private void constructRealtimeExtractor(final PipeParameters parameters) } } - private void checkWalEnableAndSetUncompressed(final PipeParameters parameters) - throws IllegalPathException { - if (Boolean.TRUE.equals( - DataRegionListeningFilter.parseInsertionDeletionListeningOptionPair(parameters) - .getLeft()) - && IoTDBDescriptor.getInstance().getConfig().getWalMode().equals(WALMode.DISABLE)) { - throw new PipeException( - "The pipe cannot transfer realtime insertion if data region disables wal. Please set 'realtime.mode'='batch' in source parameters when enabling realtime transmission."); - } - - if (!IoTDBDescriptor.getInstance() - .getConfig() - .getWALCompressionAlgorithm() - .equals(CompressionType.UNCOMPRESSED)) { - LOGGER.info( - "The pipe prefers uncompressed wal, and may introduce certain delay in realtime insert syncing without it. Hence, we change it to uncompressed."); - IoTDBDescriptor.getInstance() - .getConfig() - .setWALCompressionAlgorithm(CompressionType.UNCOMPRESSED); - } - } - @Override public void customize( final PipeParameters parameters, final PipeExtractorRuntimeConfiguration configuration) From d09466e1ee2834a0fa59d09ccdff6adc08e8eab4 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 27 Jun 2025 11:09:58 +0800 Subject: [PATCH 029/185] Fix --- ...PipeRealtimeDataRegionHybridExtractor.java | 22 +--------------- ...eDataNodeRemainingEventAndTimeMetrics.java | 26 ------------------- ...DataNodeRemainingEventAndTimeOperator.java | 20 -------------- .../iotdb/commons/conf/CommonConfig.java | 18 ------------- .../iotdb/commons/pipe/config/PipeConfig.java | 7 ----- .../commons/pipe/config/PipeDescriptor.java | 5 ---- 6 files changed, 1 insertion(+), 97 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java index 4c2c17ea72de..c86bad92426b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java @@ -30,7 +30,6 @@ import org.apache.iotdb.db.pipe.extractor.dataregion.IoTDBDataRegionExtractor; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.assigner.PipeTsFileEpochProgressIndexKeeper; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.epoch.TsFileEpoch; -import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeRemainingEventAndTimeMetrics; import org.apache.iotdb.db.pipe.metric.source.PipeDataRegionExtractorMetrics; import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryManager; @@ -222,32 +221,13 @@ private void extractTsFileInsertion(final PipeRealtimeEvent event) { private boolean canNotUseTabletAnyMore(final PipeRealtimeEvent event) { // In the following 4 cases, we should not extract this tablet event. all the data // represented by the tablet event should be carried by the following tsfile event: - // 0. If the remaining insert event count is too large, we need to reduce the accumulated - // tablets. // the write operation will be throttled, so we should not extract any more tablet events. // 1. The shallow memory usage of the insert node has reached the dangerous threshold. // 2. Deprecated logics (unused by default) - return mayRemainingInsertNodeEventExceedLimit(event) - || mayInsertNodeMemoryReachDangerousThreshold(event) + return mayInsertNodeMemoryReachDangerousThreshold(event) || canNotUseTabletAnymoreDeprecated(event); } - private boolean mayRemainingInsertNodeEventExceedLimit(final PipeRealtimeEvent event) { - final boolean mayRemainingInsertEventExceedLimit = - PipeDataNodeRemainingEventAndTimeMetrics.getInstance() - .mayRemainingInsertEventExceedLimit(pipeID); - if (mayRemainingInsertEventExceedLimit && event.mayExtractorUseTablets(this)) { - logByLogManager( - l -> - l.info( - "Pipe task {}@{} canNotUseTabletAnyMore(0): remaining insert event has reached max allowed insert event count {}", - pipeName, - dataRegionId, - PipeConfig.getInstance().getPipeMaxAllowedRemainingInsertEventCountPerPipe())); - } - return mayRemainingInsertEventExceedLimit; - } - private boolean mayInsertNodeMemoryReachDangerousThreshold(final PipeRealtimeEvent event) { final long floatingMemoryUsageInByte = PipeDataNodeAgent.task().getFloatingMemoryUsageInByte(pipeName); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeMetrics.java index ea31c879005b..c26acc1ffe7a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeMetrics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeMetrics.java @@ -20,7 +20,6 @@ package org.apache.iotdb.db.pipe.metric.overview; import org.apache.iotdb.commons.pipe.agent.task.progress.PipeEventCommitManager; -import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.service.metric.enums.Metric; import org.apache.iotdb.commons.service.metric.enums.Tag; import org.apache.iotdb.db.pipe.extractor.dataregion.IoTDBDataRegionExtractor; @@ -119,31 +118,6 @@ private void createAutoGauge(final String pipeID) { operator.getPipeName())); } - public boolean mayRemainingInsertEventExceedLimit(final String pipeID) { - if (Objects.isNull(metricService)) { - return true; - } - - if (remainingEventAndTimeOperatorMap.values().stream() - .map(PipeDataNodeRemainingEventAndTimeOperator::getRemainingInsertEventSmoothingCount) - .reduce(0d, Double::sum) - > PipeConfig.getInstance().getPipeMaxAllowedTotalRemainingInsertEventCount()) { - return true; - } - - final PipeDataNodeRemainingEventAndTimeOperator operator = - remainingEventAndTimeOperatorMap.get(pipeID); - if (Objects.isNull(operator)) { - LOGGER.warn( - "Failed to get remaining insert event, RemainingEventAndTimeOperator({}) does not exist, will degrade anyway", - pipeID); - return true; - } - - return operator.getRemainingInsertEventSmoothingCount() - > PipeConfig.getInstance().getPipeMaxAllowedRemainingInsertEventCountPerPipe(); - } - @Override public void unbindFrom(final AbstractMetricService metricService) { ImmutableSet.copyOf(remainingEventAndTimeOperatorMap.keySet()).forEach(this::deregister); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeOperator.java index b81c22eda309..87c7c6731c99 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeOperator.java @@ -59,8 +59,6 @@ public class PipeDataNodeRemainingEventAndTimeOperator extends PipeRemainingOper private Timer insertNodeTransferTimer = DoNothingMetricManager.DO_NOTHING_TIMER; private Timer tsfileTransferTimer = DoNothingMetricManager.DO_NOTHING_TIMER; - private final InsertNodeEMA insertNodeEventCountEMA = new InsertNodeEMA(); - private double lastDataRegionCommitSmoothingValue = Long.MAX_VALUE; private double lastSchemaRegionCommitSmoothingValue = Long.MAX_VALUE; @@ -102,11 +100,6 @@ void decreaseHeartbeatEventCount() { heartbeatEventCount.decrementAndGet(); } - double getRemainingInsertEventSmoothingCount() { - insertNodeEventCountEMA.update(insertNodeEventCount.get()); - return insertNodeEventCountEMA.insertNodeEMAValue; - } - public long getRemainingNonHeartbeatEvents() { final long remainingEvents = tsfileEventCount.get() @@ -282,17 +275,4 @@ public synchronized void freezeRate(final boolean isStopPipe) { dataRegionCommitMeter.set(null); schemaRegionCommitMeter.set(null); } - - private static class InsertNodeEMA { - private double insertNodeEMAValue; - - public void update(final double newValue) { - final double alpha = PipeConfig.getInstance().getPipeRemainingInsertNodeCountEMAAlpha(); - if (insertNodeEMAValue == 0) { - insertNodeEMAValue = newValue; - } else { - insertNodeEMAValue = alpha * newValue + (1 - alpha) * insertNodeEMAValue; - } - } - } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java index e23ec301e878..8db59b55f588 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java @@ -283,7 +283,6 @@ public class CommonConfig { private long pipeFlushAfterLastTerminateSeconds = 30; private long pipeFlushAfterTerminateCount = 30; private long pipeStorageEngineFlushTimeIntervalMs = Long.MAX_VALUE; - private int pipeMaxAllowedRemainingInsertEventCountPerPipe = 10000; private int pipeMaxAllowedTotalRemainingInsertEventCount = 50000; private int pipeMetaReportMaxLogNumPerRound = 10; @@ -1526,23 +1525,6 @@ public long getPipeStorageEngineFlushTimeIntervalMs() { return pipeStorageEngineFlushTimeIntervalMs; } - public int getPipeMaxAllowedRemainingInsertEventCountPerPipe() { - return pipeMaxAllowedRemainingInsertEventCountPerPipe; - } - - public void setPipeMaxAllowedRemainingInsertEventCountPerPipe( - int pipeMaxAllowedRemainingInsertEventCountPerPipe) { - if (this.pipeMaxAllowedRemainingInsertEventCountPerPipe - == pipeMaxAllowedRemainingInsertEventCountPerPipe) { - return; - } - this.pipeMaxAllowedRemainingInsertEventCountPerPipe = - pipeMaxAllowedRemainingInsertEventCountPerPipe; - logger.info( - "pipeMaxAllowedRemainingInsertEventCount is set to {}", - pipeMaxAllowedRemainingInsertEventCountPerPipe); - } - public int getPipeMaxAllowedTotalRemainingInsertEventCount() { return pipeMaxAllowedTotalRemainingInsertEventCount; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java index 2ecedde44f12..3c3f429beeac 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java @@ -363,10 +363,6 @@ public long getPipeStorageEngineFlushTimeIntervalMs() { return COMMON_CONFIG.getPipeStorageEngineFlushTimeIntervalMs(); } - public int getPipeMaxAllowedRemainingInsertEventCountPerPipe() { - return COMMON_CONFIG.getPipeMaxAllowedRemainingInsertEventCountPerPipe(); - } - public int getPipeMaxAllowedTotalRemainingInsertEventCount() { return COMMON_CONFIG.getPipeMaxAllowedTotalRemainingInsertEventCount(); } @@ -612,9 +608,6 @@ public void printAllConfigs() { LOGGER.info("PipeFlushAfterLastTerminateSeconds: {}", getPipeFlushAfterLastTerminateSeconds()); LOGGER.info( "PipeStorageEngineFlushTimeIntervalMs: {}", getPipeStorageEngineFlushTimeIntervalMs()); - LOGGER.info( - "PipeMaxAllowedRemainingInsertEventCountPerPipe: {}", - getPipeMaxAllowedRemainingInsertEventCountPerPipe()); LOGGER.info( "PipeMaxAllowedTotalRemainingInsertEventCount: {}", getPipeMaxAllowedTotalRemainingInsertEventCount()); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java index 4fd5a7479649..f5eda9bf27e4 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java @@ -438,11 +438,6 @@ public static void loadPipeInternalConfig(CommonConfig config, TrimProperties pr properties.getProperty( "pipe_stuck_restart_interval_seconds", String.valueOf(config.getPipeStuckRestartIntervalSeconds())))); - config.setPipeMaxAllowedRemainingInsertEventCountPerPipe( - Integer.parseInt( - properties.getProperty( - "pipe_max_allowed_remaining_insert_event_count_per_pipe", - String.valueOf(config.getPipeMaxAllowedRemainingInsertEventCountPerPipe())))); config.setPipeMaxAllowedTotalRemainingInsertEventCount( Integer.parseInt( properties.getProperty( From 22080332d3474ff095b58b3a941f0f9497acc565 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 27 Jun 2025 12:14:06 +0800 Subject: [PATCH 030/185] Total removal --- .../dataregion/IoTDBDataRegionExtractor.java | 4 +--- .../apache/iotdb/commons/conf/CommonConfig.java | 14 +------------- .../iotdb/commons/pipe/config/PipeConfig.java | 7 ------- 3 files changed, 2 insertions(+), 23 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java index fb5f12c1d85a..98504128a2e5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java @@ -20,7 +20,6 @@ package org.apache.iotdb.db.pipe.extractor.dataregion; import org.apache.iotdb.commons.consensus.DataRegionId; -import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.pipe.datastructure.pattern.IoTDBPipePattern; import org.apache.iotdb.commons.pipe.datastructure.pattern.PipePattern; import org.apache.iotdb.commons.pipe.extractor.IoTDBExtractor; @@ -239,8 +238,7 @@ private void constructHistoricalExtractor() { historicalExtractor = new PipeHistoricalDataRegionTsFileExtractor(); } - private void constructRealtimeExtractor(final PipeParameters parameters) - throws IllegalPathException { + private void constructRealtimeExtractor(final PipeParameters parameters) { // Use heartbeat only extractor if disable realtime extractor if (!parameters.getBooleanOrDefault( Arrays.asList(EXTRACTOR_REALTIME_ENABLE_KEY, SOURCE_REALTIME_ENABLE_KEY), diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java index 8db59b55f588..c4437d41b041 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java @@ -1525,23 +1525,11 @@ public long getPipeStorageEngineFlushTimeIntervalMs() { return pipeStorageEngineFlushTimeIntervalMs; } + public int getPipeMaxAllowedTotalRemainingInsertEventCount() { return pipeMaxAllowedTotalRemainingInsertEventCount; } - public void setPipeMaxAllowedTotalRemainingInsertEventCount( - int pipeMaxAllowedTotalRemainingInsertEventCount) { - if (this.pipeMaxAllowedTotalRemainingInsertEventCount - == pipeMaxAllowedTotalRemainingInsertEventCount) { - return; - } - this.pipeMaxAllowedTotalRemainingInsertEventCount = - pipeMaxAllowedTotalRemainingInsertEventCount; - logger.info( - "pipeMaxAllowedTotalRemainingInsertEventCount is set to {}", - pipeMaxAllowedTotalRemainingInsertEventCount); - } - public void setPipeStuckRestartIntervalSeconds(long pipeStuckRestartIntervalSeconds) { if (this.pipeStuckRestartIntervalSeconds == pipeStuckRestartIntervalSeconds) { return; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java index 3c3f429beeac..5b4210abfd8a 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java @@ -363,10 +363,6 @@ public long getPipeStorageEngineFlushTimeIntervalMs() { return COMMON_CONFIG.getPipeStorageEngineFlushTimeIntervalMs(); } - public int getPipeMaxAllowedTotalRemainingInsertEventCount() { - return COMMON_CONFIG.getPipeMaxAllowedTotalRemainingInsertEventCount(); - } - /////////////////////////////// Logger /////////////////////////////// public int getPipeMetaReportMaxLogNumPerRound() { @@ -608,9 +604,6 @@ public void printAllConfigs() { LOGGER.info("PipeFlushAfterLastTerminateSeconds: {}", getPipeFlushAfterLastTerminateSeconds()); LOGGER.info( "PipeStorageEngineFlushTimeIntervalMs: {}", getPipeStorageEngineFlushTimeIntervalMs()); - LOGGER.info( - "PipeMaxAllowedTotalRemainingInsertEventCount: {}", - getPipeMaxAllowedTotalRemainingInsertEventCount()); LOGGER.info("PipeMetaReportMaxLogNumPerRound: {}", getPipeMetaReportMaxLogNumPerRound()); LOGGER.info("PipeMetaReportMaxLogIntervalRounds: {}", getPipeMetaReportMaxLogIntervalRounds()); From 1fe1bd642c08e073ae9f8eb38d94e3644e9fb375 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 27 Jun 2025 12:14:38 +0800 Subject: [PATCH 031/185] Update CommonConfig.java --- .../main/java/org/apache/iotdb/commons/conf/CommonConfig.java | 1 - 1 file changed, 1 deletion(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java index c4437d41b041..4523f72b5cac 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java @@ -1525,7 +1525,6 @@ public long getPipeStorageEngineFlushTimeIntervalMs() { return pipeStorageEngineFlushTimeIntervalMs; } - public int getPipeMaxAllowedTotalRemainingInsertEventCount() { return pipeMaxAllowedTotalRemainingInsertEventCount; } From 0326633060a66c1bcccaff249def0f6f73696594 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 27 Jun 2025 09:45:46 +0800 Subject: [PATCH 032/185] revert Update IoTDBDataRegionExtractor.java Fix --- .../runtime/PipeDataNodeRuntimeAgent.java | 5 +++ .../common/terminate/PipeTerminateEvent.java | 44 +++++++++++++++++++ 2 files changed, 49 insertions(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java index 4fd2a68fe60d..5a1f9d8b8456 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java @@ -35,6 +35,7 @@ import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; +import org.apache.iotdb.db.pipe.event.common.terminate.PipeTerminateEvent; import org.apache.iotdb.db.pipe.extractor.schemaregion.SchemaRegionListeningQueue; import org.apache.iotdb.db.pipe.resource.PipeDataNodeHardlinkOrCopiedFileDirStartupCleaner; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode; @@ -89,6 +90,10 @@ public synchronized void start() throws StartupException { "PipeTaskAgent#restartAllStuckPipes", PipeDataNodeAgent.task()::restartAllStuckPipes, PipeConfig.getInstance().getPipeStuckRestartIntervalSeconds()); + registerPeriodicalJob( + "PipeTaskAgent#flushDataRegionIfNeeded", + PipeTerminateEvent::flushDataRegionIfNeeded, + PipeConfig.getInstance().getPipeFlushAfterLastTerminateSeconds()); pipePeriodicalJobExecutor.start(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/terminate/PipeTerminateEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/terminate/PipeTerminateEvent.java index 91d38cf3361d..ca78dd7d50ad 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/terminate/PipeTerminateEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/terminate/PipeTerminateEvent.java @@ -19,16 +19,23 @@ package org.apache.iotdb.db.pipe.event.common.terminate; +import org.apache.iotdb.common.rpc.thrift.TFlushReq; import org.apache.iotdb.commons.consensus.index.ProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; import org.apache.iotdb.commons.pipe.agent.task.meta.PipeTaskMeta; +import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.datastructure.pattern.PipePattern; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; import org.apache.iotdb.db.pipe.agent.task.PipeDataNodeTask; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; +import org.apache.iotdb.db.storageengine.StorageEngine; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicLong; /** * The {@link PipeTerminateEvent} is an {@link EnrichedEvent} that controls the termination of pipe, @@ -38,6 +45,40 @@ */ public class PipeTerminateEvent extends EnrichedEvent { + private static final Logger LOGGER = LoggerFactory.getLogger(PipeTerminateEvent.class); + + private static final AtomicLong PROGRESS_REPORT_COUNT = new AtomicLong(0); + private static final AtomicLong LAST_PROGRESS_REPORT_TIME = new AtomicLong(0); + + public static void flushDataRegionIfNeeded() { + if (PROGRESS_REPORT_COUNT.get() > 0 + && PROGRESS_REPORT_COUNT.get() + > PipeConfig.getInstance().getPipeFlushAfterTerminateCount()) { + flushDataRegion(); + return; + } + + if (LAST_PROGRESS_REPORT_TIME.get() > 0 + && System.currentTimeMillis() - LAST_PROGRESS_REPORT_TIME.get() + > PipeConfig.getInstance().getPipeFlushAfterLastTerminateSeconds() * 1000L) { + flushDataRegion(); + } + } + + private static void flushDataRegion() { + try { + StorageEngine.getInstance().operateFlush(new TFlushReq()); + PROGRESS_REPORT_COUNT.set(0); + LAST_PROGRESS_REPORT_TIME.set(0); + LOGGER.info("Force flush all data regions because of last progress report time."); + } catch (final Exception e) { + LOGGER.warn( + "Failed to flush all data regions, please check the error message: {}", + e.getMessage(), + e); + } + } + private final int dataRegionId; public PipeTerminateEvent( @@ -94,6 +135,9 @@ public boolean mayEventPathsOverlappedWithPattern() { @Override public void reportProgress() { + PROGRESS_REPORT_COUNT.incrementAndGet(); + LAST_PROGRESS_REPORT_TIME.set(System.currentTimeMillis()); + // To avoid deadlock CompletableFuture.runAsync( () -> PipeDataNodeAgent.task().markCompleted(pipeName, dataRegionId)); From 25ca24fe2d4764885dd065f68e18edb7b2f6f749 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 27 Jun 2025 18:04:44 +0800 Subject: [PATCH 033/185] Update ThreadName.java --- .../java/org/apache/iotdb/commons/concurrent/ThreadName.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java index f5b853e590c7..11fdd2ca1175 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java @@ -243,7 +243,8 @@ public enum ThreadName { PBTREE_RELEASE_MONITOR, SCHEMA_FORCE_MLOG, PBTREE_FLUSH_MONITOR, - PBTREE_WORKER_POOL)); + PBTREE_WORKER_POOL, + SCHEMA_PARALLEL_POOL)); private static final Set clientServiceThreadNames = new HashSet<>(Arrays.asList(CLIENT_RPC_SERVICE, CLIENT_RPC_PROCESSOR)); @@ -300,6 +301,7 @@ public enum ThreadName { PIPE_RECEIVER_AIR_GAP_AGENT, PIPE_AIR_GAP_RECEIVER, PIPE_PROGRESS_INDEX_BACKGROUND_SERVICE, + PIPE_PARALLEL_EXECUTION_POOL, SUBSCRIPTION_EXECUTOR_POOL, SUBSCRIPTION_RUNTIME_META_SYNCER, WINDOW_EVALUATION_SERVICE, From cfc38e25681263ebcde76af922609ef84f1f3b66 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 27 Jun 2025 18:31:49 +0800 Subject: [PATCH 034/185] Update PipeResourceMetrics.java --- .../metric/overview/PipeResourceMetrics.java | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeResourceMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeResourceMetrics.java index b8f47e29698d..d3c823187d5a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeResourceMetrics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeResourceMetrics.java @@ -22,6 +22,7 @@ import org.apache.iotdb.commons.pipe.resource.ref.PipePhantomReferenceManager; import org.apache.iotdb.commons.service.metric.enums.Metric; import org.apache.iotdb.commons.service.metric.enums.Tag; +import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryManager; import org.apache.iotdb.db.pipe.resource.tsfile.PipeTsFileResourceManager; @@ -33,6 +34,7 @@ public class PipeResourceMetrics implements IMetricSet { private static final String PIPE_USED_MEMORY = "PipeUsedMemory"; + private static final String PIPE_USED_FLOATING_MEMORY = "PipeUsedFloatingMemory"; private static final String PIPE_TABLET_USED_MEMORY = "PipeTabletUsedMemory"; @@ -40,6 +42,8 @@ public class PipeResourceMetrics implements IMetricSet { private static final String PIPE_TOTAL_MEMORY = "PipeTotalMemory"; + private static final String PIPE_FLOATING_MEMORY = "PipeFloatingMemory"; + //////////////////////////// bindTo & unbindFrom (metric framework) //////////////////////////// @Override @@ -73,6 +77,20 @@ public void bindTo(final AbstractMetricService metricService) { o -> PipeMemoryManager.getTotalNonFloatingMemorySizeInBytes(), Tag.NAME.toString(), PIPE_TOTAL_MEMORY); + metricService.createAutoGauge( + Metric.PIPE_MEM.toString(), + MetricLevel.IMPORTANT, + PipeDataNodeResourceManager.memory(), + o -> PipeMemoryManager.getTotalFloatingMemorySizeInBytes(), + Tag.NAME.toString(), + PIPE_FLOATING_MEMORY); + metricService.createAutoGauge( + Metric.PIPE_MEM.toString(), + MetricLevel.IMPORTANT, + PipeDataNodeResourceManager.memory(), + o -> PipeDataNodeAgent.task().getAllFloatingMemoryUsageInByte(), + Tag.NAME.toString(), + PIPE_USED_FLOATING_MEMORY); // resource reference count metricService.createAutoGauge( Metric.PIPE_LINKED_TSFILE_COUNT.toString(), @@ -109,6 +127,16 @@ public void unbindFrom(final AbstractMetricService metricService) { PIPE_TS_FILE_USED_MEMORY); metricService.remove( MetricType.AUTO_GAUGE, Metric.PIPE_MEM.toString(), Tag.NAME.toString(), PIPE_TOTAL_MEMORY); + metricService.remove( + MetricType.AUTO_GAUGE, + Metric.PIPE_MEM.toString(), + Tag.NAME.toString(), + PIPE_FLOATING_MEMORY); + metricService.remove( + MetricType.AUTO_GAUGE, + Metric.PIPE_MEM.toString(), + Tag.NAME.toString(), + PIPE_USED_FLOATING_MEMORY); // resource reference count metricService.remove(MetricType.AUTO_GAUGE, Metric.PIPE_LINKED_TSFILE_COUNT.toString()); metricService.remove(MetricType.AUTO_GAUGE, Metric.PIPE_LINKED_TSFILE_SIZE.toString()); From 26717acb4d43e20e39328421d0699be4a0eda3f2 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 27 Jun 2025 18:36:37 +0800 Subject: [PATCH 035/185] Update PipeDescriptor.java --- .../org/apache/iotdb/commons/pipe/config/PipeDescriptor.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java index f5eda9bf27e4..e2f3ee8bddc6 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java @@ -438,11 +438,6 @@ public static void loadPipeInternalConfig(CommonConfig config, TrimProperties pr properties.getProperty( "pipe_stuck_restart_interval_seconds", String.valueOf(config.getPipeStuckRestartIntervalSeconds())))); - config.setPipeMaxAllowedTotalRemainingInsertEventCount( - Integer.parseInt( - properties.getProperty( - "pipe_max_allowed_total_remaining_insert_event_count", - String.valueOf(config.getPipeMaxAllowedTotalRemainingInsertEventCount())))); config.setPipeStuckRestartMinIntervalMs( Long.parseLong( properties.getProperty( From 1e241c2a9784ec824ca6774aecdc30e2d2ef94a9 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Fri, 27 Jun 2025 18:40:39 +0800 Subject: [PATCH 036/185] add PIPE_FLOATING_MEMORY_USAGE --- ...ipeDataNodeRemainingEventAndTimeMetrics.java | 17 +++++++++++++++++ .../commons/service/metric/enums/Metric.java | 1 + 2 files changed, 18 insertions(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeMetrics.java index c26acc1ffe7a..4a3e3ee86547 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeMetrics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeMetrics.java @@ -22,6 +22,7 @@ import org.apache.iotdb.commons.pipe.agent.task.progress.PipeEventCommitManager; import org.apache.iotdb.commons.service.metric.enums.Metric; import org.apache.iotdb.commons.service.metric.enums.Tag; +import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; import org.apache.iotdb.db.pipe.extractor.dataregion.IoTDBDataRegionExtractor; import org.apache.iotdb.db.pipe.extractor.schemaregion.IoTDBSchemaRegionExtractor; import org.apache.iotdb.metrics.AbstractMetricService; @@ -102,6 +103,15 @@ private void createAutoGauge(final String pipeID) { operator.getPipeName(), Tag.CREATION_TIME.toString(), String.valueOf(operator.getCreationTime())); + metricService.createAutoGauge( + Metric.PIPE_FLOATING_MEMORY_USAGE.toString(), + MetricLevel.IMPORTANT, + PipeDataNodeAgent.task(), + a -> a.getFloatingMemoryUsageInByte(operator.getPipeName()), + Tag.NAME.toString(), + operator.getPipeName(), + Tag.CREATION_TIME.toString(), + String.valueOf(operator.getCreationTime())); operator.setInsertNodeTransferTimer( metricService.getOrCreateTimer( @@ -159,6 +169,13 @@ private void removeAutoGauge(final String pipeID) { operator.getPipeName(), Tag.CREATION_TIME.toString(), String.valueOf(operator.getCreationTime())); + metricService.remove( + MetricType.AUTO_GAUGE, + Metric.PIPE_DATANODE_REMAINING_TIME.toString(), + Tag.NAME.toString(), + operator.getPipeName(), + Tag.CREATION_TIME.toString(), + String.valueOf(operator.getCreationTime())); metricService.remove( MetricType.TIMER, Metric.PIPE_INSERT_NODE_EVENT_TRANSFER_TIME.toString(), diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/Metric.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/Metric.java index 90183f8150b1..1b177ee27e17 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/Metric.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/Metric.java @@ -173,6 +173,7 @@ public enum Metric { PIPE_INSERT_NODE_EVENT_TRANSFER_TIME("pipe_insert_node_event_transfer_time"), PIPE_TSFILE_EVENT_TRANSFER_TIME("pipe_tsfile_event_transfer_time"), PIPE_DATANODE_EVENT_TRANSFER("pipe_datanode_event_transfer"), + PIPE_FLOATING_MEMORY_USAGE("pipe_floating_memory_usage"), PIPE_CONFIG_LINKED_QUEUE_SIZE("pipe_config_linked_queue_size"), UNTRANSFERRED_CONFIG_COUNT("untransferred_config_count"), PIPE_CONNECTOR_CONFIG_TRANSFER("pipe_connector_config_transfer"), From e8a3a721f42a0cd325eeb690020a984956bbe8af Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Fri, 27 Jun 2025 18:44:13 +0800 Subject: [PATCH 037/185] fix PIPE_FLOATING_MEMORY_USAGE --- .../overview/PipeDataNodeRemainingEventAndTimeMetrics.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeMetrics.java index 4a3e3ee86547..2e1bab9faf06 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeMetrics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeMetrics.java @@ -171,7 +171,7 @@ private void removeAutoGauge(final String pipeID) { String.valueOf(operator.getCreationTime())); metricService.remove( MetricType.AUTO_GAUGE, - Metric.PIPE_DATANODE_REMAINING_TIME.toString(), + Metric.PIPE_FLOATING_MEMORY_USAGE.toString(), Tag.NAME.toString(), operator.getPipeName(), Tag.CREATION_TIME.toString(), From 5ef82ccc1c8498e4016deea4efa664a74a96e0f1 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 30 Jun 2025 11:43:45 +0800 Subject: [PATCH 038/185] Change listen --- .../iotdb/db/storageengine/dataregion/DataRegion.java | 8 ++++++++ .../dataregion/memtable/TsFileProcessor.java | 7 ------- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java index 5e9b25bdda11..4b7cbe456baf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java @@ -2682,6 +2682,14 @@ private void closeUnsealedTsFileProcessorCallBack(TsFileProcessor tsFileProcesso } if (isEmptyFile || isValidateTsFileFailed) { tsFileManager.remove(tsFileProcessor.getTsFileResource(), tsFileProcessor.isSequence()); + } else { + PipeInsertionDataNodeListener.getInstance() + .listenToTsFile( + dataRegionInfo.getDataRegion().getDataRegionId(), + tsFileProcessor.getTsFileResource(), + false, + tsFileProcessor.getWorkMemTable() != null + && tsFileProcessor.getWorkMemTable().isTotallyGeneratedByPipe()); } // closingSequenceTsFileProcessor is a thread safety class. diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java index f1d20a650806..40f9c4112c7a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java @@ -1139,13 +1139,6 @@ public Future asyncClose() { IMemTable tmpMemTable = workMemTable == null ? new NotifyFlushMemTable() : workMemTable; try { - PipeInsertionDataNodeListener.getInstance() - .listenToTsFile( - dataRegionInfo.getDataRegion().getDataRegionId(), - tsFileResource, - false, - tmpMemTable.isTotallyGeneratedByPipe()); - // When invoke closing TsFile after insert data to memTable, we shouldn't flush until invoke // flushing memTable in System module. Future future = addAMemtableIntoFlushingList(tmpMemTable); From 874b57b8a11b1c0fe0a548e1a98fbd0ecc13d518 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 30 Jun 2025 11:48:37 +0800 Subject: [PATCH 039/185] Changed isEmpty --- .../task/connection/PipeEventCollector.java | 2 +- .../IoTDBDataRegionAirGapConnector.java | 2 +- .../legacy/IoTDBLegacyPipeConnector.java | 2 +- .../tsfile/PipeTsFileInsertionEvent.java | 72 ++----------------- .../plugin/TwoStageCountProcessor.java | 2 +- 5 files changed, 9 insertions(+), 71 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollector.java index 66bc5ab2a508..97855856747d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollector.java @@ -122,7 +122,7 @@ private void parseAndCollectEvent(final PipeRawTabletInsertionEvent sourceEvent) } private void parseAndCollectEvent(final PipeTsFileInsertionEvent sourceEvent) throws Exception { - if (!sourceEvent.waitForTsFileClose()) { + if (sourceEvent.isEmpty()) { LOGGER.warn( "Pipe skipping temporary TsFile which shouldn't be transferred: {}", sourceEvent.getTsFile()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataRegionAirGapConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataRegionAirGapConnector.java index bc4805e8c8ef..a47b0c463802 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataRegionAirGapConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataRegionAirGapConnector.java @@ -99,7 +99,7 @@ public void transfer(final TsFileInsertionEvent tsFileInsertionEvent) throws Exc return; } - if (!((PipeTsFileInsertionEvent) tsFileInsertionEvent).waitForTsFileClose()) { + if (((PipeTsFileInsertionEvent) tsFileInsertionEvent).isEmpty()) { LOGGER.warn( "Pipe skipping temporary TsFile which shouldn't be transferred: {}", ((PipeTsFileInsertionEvent) tsFileInsertionEvent).getTsFile()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/legacy/IoTDBLegacyPipeConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/legacy/IoTDBLegacyPipeConnector.java index 58a929d30d0e..f4e2889744af 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/legacy/IoTDBLegacyPipeConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/legacy/IoTDBLegacyPipeConnector.java @@ -290,7 +290,7 @@ public void transfer(final TsFileInsertionEvent tsFileInsertionEvent) throws Exc "IoTDBLegacyPipeConnector only support PipeTsFileInsertionEvent."); } - if (!((PipeTsFileInsertionEvent) tsFileInsertionEvent).waitForTsFileClose()) { + if (((PipeTsFileInsertionEvent) tsFileInsertionEvent).isEmpty()) { LOGGER.warn( "Pipe skipping temporary TsFile which shouldn't be transferred: {}", ((PipeTsFileInsertionEvent) tsFileInsertionEvent).getTsFile()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java index 3aa9b4a6b13b..13ca2ab62fbf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java @@ -78,7 +78,6 @@ public class PipeTsFileInsertionEvent extends EnrichedEvent private final boolean isGeneratedByPipeConsensus; private final boolean isGeneratedByHistoricalExtractor; - private final AtomicBoolean isClosed; private final AtomicReference dataContainer; // The point count of the TsFile. Used for metrics on PipeConsensus' receiver side. @@ -130,42 +129,6 @@ public PipeTsFileInsertionEvent( this.isGeneratedByPipeConsensus = resource.isGeneratedByPipeConsensus(); this.isGeneratedByHistoricalExtractor = isGeneratedByHistoricalExtractor; - isClosed = new AtomicBoolean(resource.isClosed()); - // Register close listener if TsFile is not closed - if (!isClosed.get()) { - final TsFileProcessor processor = resource.getProcessor(); - if (processor != null) { - processor.addCloseFileListener( - o -> { - synchronized (isClosed) { - isClosed.set(true); - isClosed.notifyAll(); - - // Update flushPointCount after TsFile is closed - flushPointCount = processor.getMemTableFlushPointCount(); - } - }); - } - } - // Check again after register close listener in case TsFile is closed during the process - // TsFile flushing steps: - // 1. Flush tsFile - // 2. First listener (Set resource status "closed" -> Set processor == null -> processor == null - // is seen) - // 3. Other listeners (Set "closed" status for events) - // Then we can imply that: - // 1. If the listener cannot be executed because all listeners passed, then resources status is - // set "closed" and can be set here - // 2. If the listener cannot be executed because processor == null is seen, then resources - // status is set "closed" and can be set here - // Then we know: - // 1. The status in the event can be closed eventually. - // 2. If the status is "closed", then the resource status is "closed". - // Then we know: - // If the status is "closed", then the resource status is "closed", the tsFile won't be altered - // and can be sent. - isClosed.set(resource.isClosed()); - this.dataContainer = new AtomicReference<>(null); } @@ -173,35 +136,10 @@ public PipeTsFileInsertionEvent( * @return {@code false} if this file can't be sent by pipe because it is empty. {@code true} * otherwise. */ - public boolean waitForTsFileClose() throws InterruptedException { - if (!isClosed.get()) { - isClosed.set(resource.isClosed()); - - synchronized (isClosed) { - while (!isClosed.get()) { - isClosed.wait(100); - - final boolean isClosedNow = resource.isClosed(); - if (isClosedNow) { - isClosed.set(true); - isClosed.notifyAll(); - - // Update flushPointCount after TsFile is closed - final TsFileProcessor processor = resource.getProcessor(); - if (processor != null) { - flushPointCount = processor.getMemTableFlushPointCount(); - } - - break; - } - } - } - } - - // From illustrations above we know If the status is "closed", then the tsFile is flushed - // And here we guarantee that the isEmpty() is set before flushing if tsFile is empty + public boolean isEmpty() { + // Here we guarantee that the isEmpty() is set before flushing if tsFile is empty // Then we know: "isClosed" --> tsFile flushed --> (isEmpty() <--> tsFile is empty) - return !resource.isEmpty(); + return resource.isEmpty(); } public File getTsFile() { @@ -304,7 +242,7 @@ public void bindProgressIndex(final ProgressIndex overridingProgressIndex) { @Override public ProgressIndex getProgressIndex() { try { - if (!waitForTsFileClose()) { + if (isEmpty()) { LOGGER.warn( "Skipping temporary TsFile {}'s progressIndex, will report MinimumProgressIndex", tsFile); @@ -471,7 +409,7 @@ public Iterable toTabletInsertionEvents() throws PipeExcep public Iterable toTabletInsertionEvents(final long timeoutMs) throws PipeException { try { - if (!waitForTsFileClose()) { + if (isEmpty()) { LOGGER.warn( "Pipe skipping temporary TsFile's parsing which shouldn't be transferred: {}", tsFile); return Collections.emptyList(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/twostage/plugin/TwoStageCountProcessor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/twostage/plugin/TwoStageCountProcessor.java index 16a20a5f509a..df1925e59838 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/twostage/plugin/TwoStageCountProcessor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/twostage/plugin/TwoStageCountProcessor.java @@ -189,7 +189,7 @@ public void process(TsFileInsertionEvent tsFileInsertionEvent, EventCollector ev final PipeTsFileInsertionEvent event = (PipeTsFileInsertionEvent) tsFileInsertionEvent; event.skipReportOnCommit(); - if (!event.waitForTsFileClose()) { + if (event.isEmpty()) { LOGGER.warn("Ignored TsFileInsertionEvent is empty: {}", event); return; } From 3be74a09e0359792078a701f6d66ff2e0068d09e Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 30 Jun 2025 11:57:00 +0800 Subject: [PATCH 040/185] Fix & use async flush --- .../tsfile/PipeTsFileInsertionEvent.java | 33 +++++++------------ ...peHistoricalDataRegionTsFileExtractor.java | 2 +- .../storageengine/dataregion/DataRegion.java | 2 +- 3 files changed, 13 insertions(+), 24 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java index 13ca2ab62fbf..94ac5141fd12 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java @@ -241,24 +241,15 @@ public void bindProgressIndex(final ProgressIndex overridingProgressIndex) { @Override public ProgressIndex getProgressIndex() { - try { - if (isEmpty()) { - LOGGER.warn( - "Skipping temporary TsFile {}'s progressIndex, will report MinimumProgressIndex", - tsFile); - return MinimumProgressIndex.INSTANCE; - } - if (Objects.nonNull(overridingProgressIndex)) { - return overridingProgressIndex; - } - return resource.getMaxProgressIndexAfterClose(); - } catch (final InterruptedException e) { + if (isEmpty()) { LOGGER.warn( - String.format( - "Interrupted when waiting for closing TsFile %s.", resource.getTsFilePath())); - Thread.currentThread().interrupt(); + "Skipping temporary TsFile {}'s progressIndex, will report MinimumProgressIndex", tsFile); return MinimumProgressIndex.INSTANCE; } + if (Objects.nonNull(overridingProgressIndex)) { + return overridingProgressIndex; + } + return resource.getMaxProgressIndexAfterClose(); } /** @@ -320,9 +311,7 @@ public boolean isGeneratedByPipe() { public boolean mayEventTimeOverlappedWithTimeRange() { // If the tsFile is not closed the resource.getFileEndTime() will be Long.MIN_VALUE // In that case we only judge the resource.getFileStartTime() to avoid losing data - return isClosed.get() - ? startTime <= resource.getFileEndTime() && resource.getFileStartTime() <= endTime - : resource.getFileStartTime() <= endTime; + return startTime <= resource.getFileEndTime() && resource.getFileStartTime() <= endTime; } @Override @@ -555,8 +544,8 @@ public void close() { @Override public String toString() { return String.format( - "PipeTsFileInsertionEvent{resource=%s, tsFile=%s, isLoaded=%s, isGeneratedByPipe=%s, isClosed=%s, dataContainer=%s}", - resource, tsFile, isLoaded, isGeneratedByPipe, isClosed.get(), dataContainer) + "PipeTsFileInsertionEvent{resource=%s, tsFile=%s, isLoaded=%s, isGeneratedByPipe=%s, dataContainer=%s}", + resource, tsFile, isLoaded, isGeneratedByPipe, dataContainer) + " - " + super.toString(); } @@ -564,8 +553,8 @@ public String toString() { @Override public String coreReportMessage() { return String.format( - "PipeTsFileInsertionEvent{resource=%s, tsFile=%s, isLoaded=%s, isGeneratedByPipe=%s, isClosed=%s}", - resource, tsFile, isLoaded, isGeneratedByPipe, isClosed.get()) + "PipeTsFileInsertionEvent{resource=%s, tsFile=%s, isLoaded=%s, isGeneratedByPipe=%s}", + resource, tsFile, isLoaded, isGeneratedByPipe) + " - " + super.coreReportMessage(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java index acec7573a2a4..3b14d436650e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java @@ -404,7 +404,7 @@ public synchronized void start() { final long lastFlushedByPipeTime = DATA_REGION_ID_TO_PIPE_FLUSHED_TIME_MAP.get(dataRegionId); if (System.currentTimeMillis() - lastFlushedByPipeTime >= PIPE_MIN_FLUSH_INTERVAL_IN_MS) { - dataRegion.syncCloseAllWorkingTsFileProcessors(); + dataRegion.asyncCloseAllWorkingTsFileProcessors(); DATA_REGION_ID_TO_PIPE_FLUSHED_TIME_MAP.replace( dataRegionId, System.currentTimeMillis()); LOGGER.info( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java index 4b7cbe456baf..6883743e6537 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java @@ -1898,7 +1898,7 @@ private void waitClosingTsFileProcessorFinished() throws InterruptedException { } /** close all working tsfile processors */ - private List> asyncCloseAllWorkingTsFileProcessors() { + public List> asyncCloseAllWorkingTsFileProcessors() { writeLock("asyncCloseAllWorkingTsFileProcessors"); List> futures = new ArrayList<>(); int count = 0; From e3df0d6b32749d213fde79f4289af400ee371896 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 30 Jun 2025 12:05:34 +0800 Subject: [PATCH 041/185] Remove restart logic --- .../agent/task/PipeDataNodeTaskAgent.java | 245 ------------------ 1 file changed, 245 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java index 738a7ef5bae8..bf7fb280a803 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java @@ -109,11 +109,6 @@ public class PipeDataNodeTaskAgent extends PipeTaskAgent { protected static final IoTDBConfig CONFIG = IoTDBDescriptor.getInstance().getConfig(); - private static final AtomicLong LAST_FORCED_RESTART_TIME = - new AtomicLong(System.currentTimeMillis()); - private static final Map PIPE_NAME_TO_LAST_RESTART_TIME_MAP = - new ConcurrentHashMap<>(); - private final ExecutorService pipeExecutor = new WrappedThreadPoolExecutor( 0, @@ -139,54 +134,6 @@ protected Map buildPipeTasks(final PipeMeta pipeMetaFromConfi return new PipeDataNodeBuilder(pipeMetaFromConfigNode).build(); } - ////////////////////////// Manage by Pipe Name ////////////////////////// - - @Override - protected void startPipe(final String pipeName, final long creationTime) { - final PipeMeta existedPipeMeta = pipeMetaKeeper.getPipeMeta(pipeName); - final PipeStatus status = existedPipeMeta.getRuntimeMeta().getStatus().get(); - if (PipeStatus.STOPPED.equals(status) || status == null) { - restartPipeToReloadResourceIfNeeded(existedPipeMeta); - } - - super.startPipe(pipeName, creationTime); - } - - private void restartPipeToReloadResourceIfNeeded(final PipeMeta pipeMeta) { - if (System.currentTimeMillis() - pipeMeta.getStaticMeta().getCreationTime() - < PipeConfig.getInstance().getPipeStuckRestartMinIntervalMs()) { - return; - } - - final AtomicLong lastRestartTime = - PIPE_NAME_TO_LAST_RESTART_TIME_MAP.get(pipeMeta.getStaticMeta().getPipeName()); - if (lastRestartTime != null - && System.currentTimeMillis() - lastRestartTime.get() - < PipeConfig.getInstance().getPipeStuckRestartMinIntervalMs()) { - LOGGER.info( - "Skipping reload resource for stopped pipe {} before starting it because reloading resource is too frequent.", - pipeMeta.getStaticMeta().getPipeName()); - return; - } - - if (PIPE_NAME_TO_LAST_RESTART_TIME_MAP.isEmpty()) { - LOGGER.info( - "Flushing storage engine before restarting pipe {}.", - pipeMeta.getStaticMeta().getPipeName()); - final long currentTime = System.currentTimeMillis(); - StorageEngine.getInstance().syncCloseAllProcessor(); - WALManager.getInstance().syncDeleteOutdatedFilesInWALNodes(); - LOGGER.info( - "Finished flushing storage engine, time cost: {} ms.", - System.currentTimeMillis() - currentTime); - } - - restartStuckPipe(pipeMeta); - LOGGER.info( - "Reloaded resource for stopped pipe {} before starting it.", - pipeMeta.getStaticMeta().getPipeName()); - } - ///////////////////////// Manage by regionGroupId ///////////////////////// @Override @@ -577,198 +524,6 @@ protected void collectPipeMetaListInternal( PipeInsertionDataNodeListener.getInstance().listenToHeartbeat(true); } - ///////////////////////// Restart Logic ///////////////////////// - - public void restartAllStuckPipes() { - final List removedPipeName = removeOutdatedPipeInfoFromLastRestartTimeMap(); - if (!removedPipeName.isEmpty()) { - final long currentTime = System.currentTimeMillis(); - LOGGER.info( - "Pipes {} now can dynamically adjust their extraction strategies. " - + "Start to flush storage engine to trigger the adjustment.", - removedPipeName); - StorageEngine.getInstance().syncCloseAllProcessor(); - WALManager.getInstance().syncDeleteOutdatedFilesInWALNodes(); - LOGGER.info( - "Finished flushing storage engine, time cost: {} ms.", - System.currentTimeMillis() - currentTime); - LOGGER.info("Skipping restarting pipes this round because of the dynamic flushing."); - return; - } - - if (!tryWriteLockWithTimeOut(5)) { - return; - } - - final Set stuckPipes; - try { - stuckPipes = findAllStuckPipes(); - } finally { - releaseWriteLock(); - } - - // If the pipe has been restarted recently, skip it. - stuckPipes.removeIf( - pipeMeta -> { - final AtomicLong lastRestartTime = - PIPE_NAME_TO_LAST_RESTART_TIME_MAP.get(pipeMeta.getStaticMeta().getPipeName()); - return lastRestartTime != null - && System.currentTimeMillis() - lastRestartTime.get() - < PipeConfig.getInstance().getPipeStuckRestartMinIntervalMs(); - }); - - // Restart all stuck pipes. - // Note that parallelStream cannot be used here. The method PipeTaskAgent#dropPipe also uses - // parallelStream. If parallelStream is used here, the subtasks generated inside the dropPipe - // may not be scheduled by the worker thread of ForkJoinPool because of less available threads, - // and the parent task will wait for the completion of the subtasks in ForkJoinPool forever, - // causing the deadlock. - stuckPipes.forEach(this::restartStuckPipe); - } - - private List removeOutdatedPipeInfoFromLastRestartTimeMap() { - final List removedPipeName = new ArrayList<>(); - PIPE_NAME_TO_LAST_RESTART_TIME_MAP - .entrySet() - .removeIf( - entry -> { - final AtomicLong lastRestartTime = entry.getValue(); - final boolean shouldRemove = - lastRestartTime == null - || PipeConfig.getInstance().getPipeStuckRestartMinIntervalMs() - <= System.currentTimeMillis() - lastRestartTime.get(); - if (shouldRemove) { - removedPipeName.add(entry.getKey()); - } - return shouldRemove; - }); - return removedPipeName; - } - - private Set findAllStuckPipes() { - final Set stuckPipes = new HashSet<>(); - - if (System.currentTimeMillis() - LAST_FORCED_RESTART_TIME.get() - > PipeConfig.getInstance().getPipeSubtaskExecutorForcedRestartIntervalMs()) { - LAST_FORCED_RESTART_TIME.set(System.currentTimeMillis()); - for (final PipeMeta pipeMeta : pipeMetaKeeper.getPipeMetaList()) { - stuckPipes.add(pipeMeta); - } - if (!stuckPipes.isEmpty()) { - LOGGER.warn( - "All {} pipe(s) will be restarted because of forced restart policy.", - stuckPipes.size()); - } - return stuckPipes; - } - - final long totalLinkedButDeletedTsFileResourceRamSize = - PipeDataNodeResourceManager.tsfile().getTotalLinkedButDeletedTsFileResourceRamSize(); - final long totalInsertNodeFloatingMemoryUsageInBytes = getAllFloatingMemoryUsageInByte(); - final long totalFloatingMemorySizeInBytes = - PipeMemoryManager.getTotalFloatingMemorySizeInBytes(); - if (totalInsertNodeFloatingMemoryUsageInBytes + totalLinkedButDeletedTsFileResourceRamSize - >= totalFloatingMemorySizeInBytes) { - for (final PipeMeta pipeMeta : pipeMetaKeeper.getPipeMetaList()) { - stuckPipes.add(pipeMeta); - } - if (!stuckPipes.isEmpty()) { - LOGGER.warn( - "All {} pipe(s) will be restarted because linked but deleted tsFiles' resource size {} and all insertNode's size {} exceeds limit {}.", - stuckPipes.size(), - totalLinkedButDeletedTsFileResourceRamSize, - totalInsertNodeFloatingMemoryUsageInBytes, - totalFloatingMemorySizeInBytes); - } - return stuckPipes; - } - - final Map taskId2ExtractorMap = - PipeDataRegionExtractorMetrics.getInstance().getExtractorMap(); - for (final PipeMeta pipeMeta : pipeMetaKeeper.getPipeMetaList()) { - final String pipeName = pipeMeta.getStaticMeta().getPipeName(); - final List extractors = - taskId2ExtractorMap.values().stream() - .filter(e -> e.getPipeName().equals(pipeName) && e.shouldExtractInsertion()) - .collect(Collectors.toList()); - - if (extractors.isEmpty()) { - continue; - } - - // Extractors of this pipe might not pin too much MemTables, - // still need to check if linked-and-deleted TsFile count exceeds limit. - // Typically, if deleted tsFiles are too abundant all pipes may need to restart. - if ((CONFIG.isEnableSeqSpaceCompaction() - || CONFIG.isEnableUnseqSpaceCompaction() - || CONFIG.isEnableCrossSpaceCompaction()) - && mayDeletedTsFileSizeReachDangerousThreshold()) { - LOGGER.warn( - "Pipe {} needs to restart because too many TsFiles are out-of-date.", - pipeMeta.getStaticMeta()); - stuckPipes.add(pipeMeta); - } - } - - return stuckPipes; - } - - private boolean mayDeletedTsFileSizeReachDangerousThreshold() { - try { - final long linkedButDeletedTsFileSize = - PipeDataNodeResourceManager.tsfile().getTotalLinkedButDeletedTsfileSize(); - final double totalDisk = - MetricService.getInstance() - .getAutoGauge( - SystemMetric.SYS_DISK_TOTAL_SPACE.toString(), - MetricLevel.CORE, - Tag.NAME.toString(), - // This "system" should stay the same with the one in - // DataNodeInternalRPCServiceImpl. - "system") - .getValue(); - return linkedButDeletedTsFileSize > 0 - && totalDisk > 0 - && linkedButDeletedTsFileSize - > PipeConfig.getInstance().getPipeMaxAllowedLinkedDeletedTsFileDiskUsagePercentage() - * totalDisk; - } catch (final Exception e) { - LOGGER.warn("Failed to judge if deleted TsFile size reaches dangerous threshold.", e); - return false; - } - } - - private void restartStuckPipe(final PipeMeta pipeMeta) { - LOGGER.warn( - "Pipe {} will be restarted because it is stuck or has encountered issues such as data backlog or being stopped for too long.", - pipeMeta.getStaticMeta()); - acquireWriteLock(); - try { - final long startTime = System.currentTimeMillis(); - final PipeMeta originalPipeMeta = pipeMeta.deepCopy4TaskAgent(); - handleDropPipe(pipeMeta.getStaticMeta().getPipeName()); - - final long restartTime = System.currentTimeMillis(); - PIPE_NAME_TO_LAST_RESTART_TIME_MAP - .computeIfAbsent(pipeMeta.getStaticMeta().getPipeName(), k -> new AtomicLong(restartTime)) - .set(restartTime); - handleSinglePipeMetaChanges(originalPipeMeta); - - LOGGER.warn( - "Pipe {} was restarted because of stuck or data backlog, time cost: {} ms.", - originalPipeMeta.getStaticMeta(), - System.currentTimeMillis() - startTime); - } catch (final Exception e) { - LOGGER.warn("Failed to restart stuck pipe {}.", pipeMeta.getStaticMeta(), e); - } finally { - releaseWriteLock(); - } - } - - public boolean isPipeTaskCurrentlyRestarted(final String pipeName) { - return PIPE_NAME_TO_LAST_RESTART_TIME_MAP.containsKey(pipeName); - } - ///////////////////////// Terminate Logic ///////////////////////// public void markCompleted(final String pipeName, final int regionId) { From d801fa9348144c24accfac916e832bfeb245a7c9 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 30 Jun 2025 14:55:17 +0800 Subject: [PATCH 042/185] Sync delete tsFile & Added pipeName --- .../agent/task/PipeDataNodeTaskAgent.java | 10 -- .../tsfile/PipeTsFileInsertionEvent.java | 20 ++-- ...peHistoricalDataRegionTsFileExtractor.java | 6 +- .../resource/tsfile/PipeTsFileResource.java | 26 ++--- .../tsfile/PipeTsFileResourceManager.java | 101 ++++-------------- .../PipeTsFileResourceManagerTest.java | 33 +++--- 6 files changed, 61 insertions(+), 135 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java index bf7fb280a803..1a9d9ff9d566 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java @@ -38,8 +38,6 @@ import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant; import org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant; -import org.apache.iotdb.commons.service.metric.MetricService; -import org.apache.iotdb.commons.service.metric.enums.Tag; import org.apache.iotdb.consensus.exception.ConsensusException; import org.apache.iotdb.consensus.pipe.consensuspipe.ConsensusPipeName; import org.apache.iotdb.db.conf.IoTDBConfig; @@ -49,22 +47,16 @@ import org.apache.iotdb.db.pipe.agent.task.builder.PipeDataNodeBuilder; import org.apache.iotdb.db.pipe.agent.task.builder.PipeDataNodeTaskBuilder; import org.apache.iotdb.db.pipe.extractor.dataregion.DataRegionListeningFilter; -import org.apache.iotdb.db.pipe.extractor.dataregion.IoTDBDataRegionExtractor; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.listener.PipeInsertionDataNodeListener; import org.apache.iotdb.db.pipe.extractor.schemaregion.SchemaRegionListeningFilter; import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeRemainingEventAndTimeMetrics; import org.apache.iotdb.db.pipe.metric.overview.PipeTsFileToTabletsMetrics; -import org.apache.iotdb.db.pipe.metric.source.PipeDataRegionExtractorMetrics; import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; -import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryManager; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.pipe.PipeOperateSchemaQueueNode; import org.apache.iotdb.db.schemaengine.SchemaEngine; import org.apache.iotdb.db.storageengine.StorageEngine; -import org.apache.iotdb.db.storageengine.dataregion.wal.WALManager; import org.apache.iotdb.db.subscription.agent.SubscriptionAgent; -import org.apache.iotdb.metrics.utils.MetricLevel; -import org.apache.iotdb.metrics.utils.SystemMetric; import org.apache.iotdb.mpp.rpc.thrift.TDataNodeHeartbeatResp; import org.apache.iotdb.mpp.rpc.thrift.TPipeHeartbeatReq; import org.apache.iotdb.mpp.rpc.thrift.TPipeHeartbeatResp; @@ -92,13 +84,11 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.ArrayBlockingQueue; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; import java.util.stream.Collectors; import java.util.stream.StreamSupport; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java index 94ac5141fd12..9b60836c3c47 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java @@ -190,9 +190,13 @@ public long getTimePartitionId() { public boolean internallyIncreaseResourceReferenceCount(final String holderMessage) { extractTime = System.nanoTime(); try { - tsFile = PipeDataNodeResourceManager.tsfile().increaseFileReference(tsFile, true, resource); + tsFile = + PipeDataNodeResourceManager.tsfile() + .increaseFileReference(tsFile, true, resource, pipeName); if (isWithMod) { - modFile = PipeDataNodeResourceManager.tsfile().increaseFileReference(modFile, false, null); + modFile = + PipeDataNodeResourceManager.tsfile() + .increaseFileReference(modFile, false, null, pipeName); } return true; } catch (final Exception e) { @@ -213,9 +217,9 @@ public boolean internallyIncreaseResourceReferenceCount(final String holderMessa @Override public boolean internallyDecreaseResourceReferenceCount(final String holderMessage) { try { - PipeDataNodeResourceManager.tsfile().decreaseFileReference(tsFile); + PipeDataNodeResourceManager.tsfile().decreaseFileReference(tsFile, pipeName); if (isWithMod) { - PipeDataNodeResourceManager.tsfile().decreaseFileReference(modFile); + PipeDataNodeResourceManager.tsfile().decreaseFileReference(modFile, pipeName); } close(); return true; @@ -571,6 +575,7 @@ public PipeEventResource eventResourceBuilder() { return new PipeTsFileInsertionEventResource( this.isReleased, this.referenceCount, + this.pipeName, this.tsFile, this.isWithMod, this.modFile, @@ -583,15 +588,18 @@ private static class PipeTsFileInsertionEventResource extends PipeEventResource private final boolean isWithMod; private final File modFile; private final AtomicReference dataContainer; + private final String pipeName; private PipeTsFileInsertionEventResource( final AtomicBoolean isReleased, final AtomicInteger referenceCount, + final String pipeName, final File tsFile, final boolean isWithMod, final File modFile, final AtomicReference dataContainer) { super(isReleased, referenceCount); + this.pipeName = pipeName; this.tsFile = tsFile; this.isWithMod = isWithMod; this.modFile = modFile; @@ -602,9 +610,9 @@ private PipeTsFileInsertionEventResource( protected void finalizeResource() { try { // decrease reference count - PipeDataNodeResourceManager.tsfile().decreaseFileReference(tsFile); + PipeDataNodeResourceManager.tsfile().decreaseFileReference(tsFile, pipeName); if (isWithMod) { - PipeDataNodeResourceManager.tsfile().decreaseFileReference(modFile); + PipeDataNodeResourceManager.tsfile().decreaseFileReference(modFile, pipeName); } // close data container diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java index 3b14d436650e..ac8fc50b8baa 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java @@ -478,7 +478,7 @@ && mayTsFileResourceOverlappedWithPattern(resource))) // Will unpin it after the PipeTsFileInsertionEvent is created and pinned. try { PipeDataNodeResourceManager.tsfile() - .pinTsFileResource(resource, shouldTransferModFile); + .pinTsFileResource(resource, shouldTransferModFile, pipeName); return false; } catch (final IOException e) { LOGGER.warn("Pipe: failed to pin TsFileResource {}", resource.getTsFilePath(), e); @@ -639,7 +639,7 @@ public synchronized Event supply() { return isReferenceCountIncreased ? event : null; } finally { try { - PipeDataNodeResourceManager.tsfile().unpinTsFileResource(resource); + PipeDataNodeResourceManager.tsfile().unpinTsFileResource(resource, pipeName); } catch (final IOException e) { LOGGER.warn( "Pipe {}@{}: failed to unpin TsFileResource after creating event, original path: {}", @@ -669,7 +669,7 @@ public synchronized void close() { pendingQueue.forEach( resource -> { try { - PipeDataNodeResourceManager.tsfile().unpinTsFileResource(resource); + PipeDataNodeResourceManager.tsfile().unpinTsFileResource(resource, pipeName); } catch (final IOException e) { LOGGER.warn( "Pipe {}@{}: failed to unpin TsFileResource after dropping pipe, original path: {}", diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResource.java index bf789b9732d6..3d4154bc1b7f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResource.java @@ -42,7 +42,6 @@ import java.util.Map; import java.util.Objects; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; public class PipeTsFileResource implements AutoCloseable { @@ -60,7 +59,6 @@ public class PipeTsFileResource implements AutoCloseable { private volatile long fileSize = -1L; private final AtomicInteger referenceCount; - private final AtomicLong lastUnpinToZeroTime; private PipeMemoryBlock allocatedMemoryBlock; private Map> deviceMeasurementsMap = null; private Map deviceIsAlignedMap = null; @@ -75,7 +73,6 @@ public PipeTsFileResource( this.tsFileResource = tsFileResource; referenceCount = new AtomicInteger(1); - lastUnpinToZeroTime = new AtomicLong(Long.MAX_VALUE); } public File getFile() { @@ -107,31 +104,20 @@ public int getReferenceCount() { return referenceCount.get(); } - public int increaseAndGetReference() { - return referenceCount.addAndGet(1); + public void increaseReferenceCount() { + referenceCount.addAndGet(1); } - public int decreaseAndGetReference() { + public boolean decreaseReferenceCount() { final int finalReferenceCount = referenceCount.addAndGet(-1); if (finalReferenceCount == 0) { - lastUnpinToZeroTime.set(System.currentTimeMillis()); + close(); + return true; } if (finalReferenceCount < 0) { LOGGER.warn("PipeTsFileResource's reference count is decreased to below 0."); } - return finalReferenceCount; - } - - public synchronized boolean closeIfOutOfTimeToLive() { - if (referenceCount.get() <= 0 - && (deviceMeasurementsMap == null // Not cached yet. - || System.currentTimeMillis() - lastUnpinToZeroTime.get() - > TSFILE_MIN_TIME_TO_LIVE_IN_MS)) { - close(); - return true; - } else { - return false; - } + return false; } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java index d00318ae5616..82ed257fc897 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java @@ -22,8 +22,6 @@ import org.apache.iotdb.commons.conf.IoTDBConstant; import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.utils.FileUtils; -import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; -import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; import org.apache.iotdb.db.storageengine.dataregion.modification.ModificationFile; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; @@ -32,14 +30,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.io.File; import java.io.IOException; -import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.TimeUnit; public class PipeTsFileResourceManager { @@ -49,72 +46,6 @@ public class PipeTsFileResourceManager { new ConcurrentHashMap<>(); private final PipeTsFileResourceSegmentLock segmentLock = new PipeTsFileResourceSegmentLock(); - public PipeTsFileResourceManager() { - PipeDataNodeAgent.runtime() - .registerPeriodicalJob( - "PipeTsFileResourceManager#ttlCheck()", - this::tryTtlCheck, - Math.max(PipeTsFileResource.TSFILE_MIN_TIME_TO_LIVE_IN_MS / 1000, 1)); - } - - private void tryTtlCheck() { - try { - ttlCheck(); - } catch (final InterruptedException e) { - Thread.currentThread().interrupt(); - LOGGER.warn("failed to try lock when checking TTL because of interruption", e); - } catch (final Exception e) { - LOGGER.warn("failed to check TTL of PipeTsFileResource: ", e); - } - } - - private void ttlCheck() throws InterruptedException { - final Iterator> iterator = - hardlinkOrCopiedFileToPipeTsFileResourceMap.entrySet().iterator(); - final long timeout = - PipeConfig.getInstance().getPipeSubtaskExecutorCronHeartbeatEventIntervalSeconds() >> 1; - final Optional logger = - PipeDataNodeResourceManager.log() - .schedule( - PipeTsFileResourceManager.class, - PipeConfig.getInstance().getPipeTsFilePinMaxLogNumPerRound(), - PipeConfig.getInstance().getPipeTsFilePinMaxLogIntervalRounds(), - hardlinkOrCopiedFileToPipeTsFileResourceMap.size()); - final StringBuilder logBuilder = new StringBuilder(); - while (iterator.hasNext()) { - final Map.Entry entry = iterator.next(); - - final String hardlinkOrCopiedFile = entry.getKey(); - if (!segmentLock.tryLock(new File(hardlinkOrCopiedFile), timeout, TimeUnit.SECONDS)) { - LOGGER.warn( - "failed to try lock when checking TTL for file {} because of timeout ({}s)", - hardlinkOrCopiedFile, - timeout); - continue; - } - - try { - if (entry.getValue().closeIfOutOfTimeToLive()) { - iterator.remove(); - } else { - logBuilder.append( - String.format( - "<%s , %d times, %d bytes> ", - entry.getKey(), - entry.getValue().getReferenceCount(), - entry.getValue().getFileSize())); - } - } catch (final Exception e) { - LOGGER.warn("failed to close PipeTsFileResource when checking TTL: ", e); - } finally { - segmentLock.unlock(new File(hardlinkOrCopiedFile)); - } - } - if (logBuilder.length() > 0) { - logger.ifPresent(l -> l.info("Pipe file {}are still referenced", logBuilder)); - } - } - /** * Given a file, create a hardlink or copy it to pipe dir, maintain a reference count for the * hardlink or copied file, and return the hardlink or copied file. @@ -137,7 +68,10 @@ private void ttlCheck() throws InterruptedException { * @throws IOException when create hardlink or copy file failed */ public File increaseFileReference( - final File file, final boolean isTsFile, final TsFileResource tsFileResource) + final File file, + final boolean isTsFile, + final TsFileResource tsFileResource, + final @Nullable String pipeName) throws IOException { // If the file is already a hardlink or copied file, // just increase reference count and return it @@ -183,7 +117,7 @@ private boolean increaseReferenceIfExists(final File file) { final PipeTsFileResource resource = hardlinkOrCopiedFileToPipeTsFileResourceMap.get(file.getPath()); if (resource != null) { - resource.increaseAndGetReference(); + resource.increaseReferenceCount(); return true; } return false; @@ -233,13 +167,14 @@ private static String getRelativeFilePath(File file) { * * @param hardlinkOrCopiedFile the copied or hardlinked file */ - public void decreaseFileReference(final File hardlinkOrCopiedFile) { + public void decreaseFileReference( + final File hardlinkOrCopiedFile, final @Nullable String pipeName) { segmentLock.lock(hardlinkOrCopiedFile); try { final String filePath = hardlinkOrCopiedFile.getPath(); final PipeTsFileResource resource = hardlinkOrCopiedFileToPipeTsFileResourceMap.get(filePath); - if (resource != null) { - resource.decreaseAndGetReference(); + if (resource != null && resource.decreaseReferenceCount()) { + hardlinkOrCopiedFileToPipeTsFileResourceMap.remove(filePath); } } finally { segmentLock.unlock(hardlinkOrCopiedFile); @@ -316,21 +251,23 @@ public Map getMeasurementDataTypeMapFromCache( } } - public void pinTsFileResource(final TsFileResource resource, final boolean withMods) + public void pinTsFileResource( + final TsFileResource resource, final boolean withMods, final String pipeName) throws IOException { - increaseFileReference(resource.getTsFile(), true, resource); + increaseFileReference(resource.getTsFile(), true, resource, pipeName); if (withMods && resource.getModFile().exists()) { - increaseFileReference(new File(resource.getModFile().getFilePath()), false, null); + increaseFileReference(new File(resource.getModFile().getFilePath()), false, null, pipeName); } } - public void unpinTsFileResource(final TsFileResource resource) throws IOException { + public void unpinTsFileResource(final TsFileResource resource, final String pipeName) + throws IOException { final File pinnedFile = getHardlinkOrCopiedFileInPipeDir(resource.getTsFile()); - decreaseFileReference(pinnedFile); + decreaseFileReference(pinnedFile, pipeName); final File modFile = new File(pinnedFile + ModificationFile.FILE_SUFFIX); if (modFile.exists()) { - decreaseFileReference(modFile); + decreaseFileReference(modFile, pipeName); } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java index 0281bb035194..f54b10e09538 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java @@ -59,6 +59,7 @@ public class PipeTsFileResourceManagerTest { ROOT_DIR + File.separator + IoTDBConstant.SEQUENCE_FOLDER_NAME; private static final String TS_FILE_NAME = SEQUENCE_DIR + File.separator + "test.tsfile"; private static final String MODS_FILE_NAME = TS_FILE_NAME + ".mods"; + private static final String PIPE_NAME = "pipe"; private PipeTsFileResourceManager pipeTsFileResourceManager; @@ -162,8 +163,10 @@ public void testIncreaseTsfile() throws IOException { Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(originTsfile)); Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(originModFile)); - File pipeTsfile = pipeTsFileResourceManager.increaseFileReference(originTsfile, true, null); - File pipeModFile = pipeTsFileResourceManager.increaseFileReference(originModFile, false, null); + File pipeTsfile = + pipeTsFileResourceManager.increaseFileReference(originTsfile, true, null, PIPE_NAME); + File pipeModFile = + pipeTsFileResourceManager.increaseFileReference(originModFile, false, null, PIPE_NAME); Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile)); Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile)); Assert.assertTrue(Files.exists(originTsfile.toPath())); @@ -171,36 +174,38 @@ public void testIncreaseTsfile() throws IOException { Assert.assertTrue(Files.exists(pipeTsfile.toPath())); Assert.assertTrue(Files.exists(pipeModFile.toPath())); - pipeTsFileResourceManager.increaseFileReference(originTsfile, true, null); - pipeTsFileResourceManager.increaseFileReference(originModFile, false, null); + pipeTsFileResourceManager.increaseFileReference(originTsfile, true, null, PIPE_NAME); + pipeTsFileResourceManager.increaseFileReference(originModFile, false, null, PIPE_NAME); Assert.assertEquals(2, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile)); Assert.assertEquals(2, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile)); // test use hardlinkTsFile to increase reference counts - pipeTsFileResourceManager.increaseFileReference(pipeTsfile, true, null); - Assert.assertEquals(3, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile)); + // test null, shall not reuse the pipe's tsFile + pipeTsFileResourceManager.increaseFileReference(pipeTsfile, true, null, null); + Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile)); Assert.assertTrue(Files.exists(originTsfile.toPath())); Assert.assertTrue(Files.exists(pipeTsfile.toPath())); // test use copyFile to increase reference counts - pipeTsFileResourceManager.increaseFileReference(pipeModFile, false, null); + pipeTsFileResourceManager.increaseFileReference(pipeModFile, false, null, PIPE_NAME); Assert.assertEquals(3, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile)); Assert.assertTrue(Files.exists(originModFile.toPath())); Assert.assertTrue(Files.exists(pipeModFile.toPath())); } @Test - public void testDecreaseTsfile() throws IOException { + public void testDecreaseTsFile() throws IOException { File originFile = new File(TS_FILE_NAME); File originModFile = new File(MODS_FILE_NAME); - pipeTsFileResourceManager.decreaseFileReference(originFile); - pipeTsFileResourceManager.decreaseFileReference(originModFile); + pipeTsFileResourceManager.decreaseFileReference(originFile, null); + pipeTsFileResourceManager.decreaseFileReference(originModFile, null); Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(originFile)); Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(originModFile)); - File pipeTsfile = pipeTsFileResourceManager.increaseFileReference(originFile, true, null); - File pipeModFile = pipeTsFileResourceManager.increaseFileReference(originModFile, false, null); + File pipeTsfile = pipeTsFileResourceManager.increaseFileReference(originFile, true, null, null); + File pipeModFile = + pipeTsFileResourceManager.increaseFileReference(originModFile, false, null, null); Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile)); Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile)); Assert.assertTrue(Files.exists(pipeTsfile.toPath())); @@ -220,8 +225,8 @@ public void testDecreaseTsfile() throws IOException { Assert.assertTrue(Files.exists(pipeTsfile.toPath())); Assert.assertTrue(Files.exists(pipeModFile.toPath())); - pipeTsFileResourceManager.decreaseFileReference(pipeTsfile); - pipeTsFileResourceManager.decreaseFileReference(pipeModFile); + pipeTsFileResourceManager.decreaseFileReference(pipeTsfile, null); + pipeTsFileResourceManager.decreaseFileReference(pipeModFile, null); Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile)); Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile)); Assert.assertFalse(Files.exists(originFile.toPath())); From 4196fc302ad0cc1d3d4a4cc940879fbae32f4a98 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 30 Jun 2025 15:36:36 +0800 Subject: [PATCH 043/185] try using pipe dir --- .../common/tsfile/PipeTsFileInsertionEvent.java | 3 ++- .../PipeHistoricalDataRegionTsFileExtractor.java | 3 ++- .../tsfile/PipeTsFileResourceManager.java | 15 +++++++++------ 3 files changed, 13 insertions(+), 8 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java index 9b60836c3c47..b76de24dc1c6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java @@ -328,7 +328,8 @@ public boolean mayEventPathsOverlappedWithPattern() { final Map deviceIsAlignedMap = PipeDataNodeResourceManager.tsfile() .getDeviceIsAlignedMapFromCache( - PipeTsFileResourceManager.getHardlinkOrCopiedFileInPipeDir(resource.getTsFile()), + PipeTsFileResourceManager.getHardlinkOrCopiedFileInPipeDir( + resource.getTsFile(), pipeName), false); final Set deviceSet = Objects.nonNull(deviceIsAlignedMap) ? deviceIsAlignedMap.keySet() : resource.getDevices(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java index ac8fc50b8baa..01b8c0100dc3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java @@ -532,7 +532,8 @@ private boolean mayTsFileResourceOverlappedWithPattern(final TsFileResource reso final Map deviceIsAlignedMap = PipeDataNodeResourceManager.tsfile() .getDeviceIsAlignedMapFromCache( - PipeTsFileResourceManager.getHardlinkOrCopiedFileInPipeDir(resource.getTsFile()), + PipeTsFileResourceManager.getHardlinkOrCopiedFileInPipeDir( + resource.getTsFile(), pipeName), false); deviceSet = Objects.nonNull(deviceIsAlignedMap) ? deviceIsAlignedMap.keySet() : resource.getDevices(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java index 82ed257fc897..a61f59af8b27 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java @@ -36,6 +36,7 @@ import java.io.IOException; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; public class PipeTsFileResourceManager { @@ -86,7 +87,7 @@ public File increaseFileReference( // If the file is not a hardlink or copied file, check if there is a related hardlink or // copied file in pipe dir. if so, increase reference count and return it - final File hardlinkOrCopiedFile = getHardlinkOrCopiedFileInPipeDir(file); + final File hardlinkOrCopiedFile = getHardlinkOrCopiedFileInPipeDir(file, pipeName); segmentLock.lock(hardlinkOrCopiedFile); try { if (increaseReferenceIfExists(hardlinkOrCopiedFile)) { @@ -123,9 +124,10 @@ private boolean increaseReferenceIfExists(final File file) { return false; } - public static File getHardlinkOrCopiedFileInPipeDir(final File file) throws IOException { + public static File getHardlinkOrCopiedFileInPipeDir(final File file, final String pipeName) + throws IOException { try { - return new File(getPipeTsFileDirPath(file), getRelativeFilePath(file)); + return new File(getPipeTsFileDirPath(file, pipeName), getRelativeFilePath(file)); } catch (final Exception e) { throw new IOException( String.format( @@ -136,7 +138,7 @@ public static File getHardlinkOrCopiedFileInPipeDir(final File file) throws IOEx } } - private static String getPipeTsFileDirPath(File file) throws IOException { + private static String getPipeTsFileDirPath(File file, final String pipeName) throws IOException { while (!file.getName().equals(IoTDBConstant.SEQUENCE_FOLDER_NAME) && !file.getName().equals(IoTDBConstant.UNSEQUENCE_FOLDER_NAME)) { file = file.getParentFile(); @@ -145,7 +147,8 @@ private static String getPipeTsFileDirPath(File file) throws IOException { + File.separator + PipeConfig.getInstance().getPipeHardlinkBaseDirName() + File.separator - + PipeConfig.getInstance().getPipeHardlinkTsFileDirName(); + + PipeConfig.getInstance().getPipeHardlinkTsFileDirName() + + (Objects.nonNull(pipeName) ? File.separator + pipeName : ""); } private static String getRelativeFilePath(File file) { @@ -262,7 +265,7 @@ public void pinTsFileResource( public void unpinTsFileResource(final TsFileResource resource, final String pipeName) throws IOException { - final File pinnedFile = getHardlinkOrCopiedFileInPipeDir(resource.getTsFile()); + final File pinnedFile = getHardlinkOrCopiedFileInPipeDir(resource.getTsFile(), pipeName); decreaseFileReference(pinnedFile, pipeName); final File modFile = new File(pinnedFile + ModificationFile.FILE_SUFFIX); From 55d63558f03f4e3062b95c6662285ad2dc87e73c Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 30 Jun 2025 15:55:05 +0800 Subject: [PATCH 044/185] Fixed the concurrent problem of floating memory --- .../tablet/PipeInsertNodeTabletInsertionEvent.java | 6 ++++-- .../iotdb/commons/pipe/agent/task/PipeTaskAgent.java | 12 ++++++++---- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java index da71eddb461c..6849699ae4a9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java @@ -124,7 +124,8 @@ public boolean internallyIncreaseResourceReferenceCount(final String holderMessa if (Objects.nonNull(pipeName)) { PipeDataNodeRemainingEventAndTimeMetrics.getInstance() .increaseInsertNodeEventCount(pipeName, creationTime); - PipeDataNodeAgent.task().addFloatingMemoryUsageInByte(pipeName, ramBytesUsed()); + PipeDataNodeAgent.task() + .addFloatingMemoryUsageInByte(pipeName, creationTime, ramBytesUsed()); } return true; } catch (final Exception e) { @@ -150,7 +151,8 @@ public boolean internallyDecreaseResourceReferenceCount(final String holderMessa return false; } finally { if (Objects.nonNull(pipeName)) { - PipeDataNodeAgent.task().decreaseFloatingMemoryUsageInByte(pipeName, ramBytesUsed()); + PipeDataNodeAgent.task() + .decreaseFloatingMemoryUsageInByte(pipeName, creationTime, ramBytesUsed()); PipeDataNodeRemainingEventAndTimeMetrics.getInstance() .decreaseInsertNodeEventCount(pipeName, creationTime, System.nanoTime() - extractTime); } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java index 2bfde4173b29..4ae678ae0754 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java @@ -1112,17 +1112,21 @@ public long getFloatingMemoryUsageInByte(final String pipeName) { : ((PipeTemporaryMetaInAgent) pipeMeta.getTemporaryMeta()).getFloatingMemoryUsageInByte(); } - public void addFloatingMemoryUsageInByte(final String pipeName, final long sizeInByte) { + public void addFloatingMemoryUsageInByte( + final String pipeName, final long creationTime, final long sizeInByte) { final PipeMeta pipeMeta = pipeMetaKeeper.getPipeMeta(pipeName); - if (Objects.nonNull(pipeMeta)) { + // To avoid stale pipe before alter + if (Objects.nonNull(pipeMeta) && pipeMeta.getStaticMeta().getCreationTime() == creationTime) { ((PipeTemporaryMetaInAgent) pipeMeta.getTemporaryMeta()) .addFloatingMemoryUsageInByte(sizeInByte); } } - public void decreaseFloatingMemoryUsageInByte(final String pipeName, final long sizeInByte) { + public void decreaseFloatingMemoryUsageInByte( + final String pipeName, final long creationTime, final long sizeInByte) { final PipeMeta pipeMeta = pipeMetaKeeper.getPipeMeta(pipeName); - if (Objects.nonNull(pipeMeta)) { + // To avoid stale pipe before alter + if (Objects.nonNull(pipeMeta) && pipeMeta.getStaticMeta().getCreationTime() == creationTime) { ((PipeTemporaryMetaInAgent) pipeMeta.getTemporaryMeta()) .decreaseFloatingMemoryUsageInByte(sizeInByte); } From d467454df7faba0e7fa0f7080d66efdcbca07881 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 30 Jun 2025 16:02:13 +0800 Subject: [PATCH 045/185] remove unnecessary flushes & restart parameters --- .../runtime/PipeDataNodeRuntimeAgent.java | 10 -- .../common/terminate/PipeTerminateEvent.java | 44 ------- ...PipeRealtimeDataRegionHybridExtractor.java | 20 +--- .../iotdb/db/storageengine/StorageEngine.java | 8 -- .../iotdb/commons/conf/CommonConfig.java | 107 ------------------ .../iotdb/commons/pipe/config/PipeConfig.java | 40 ------- .../commons/pipe/config/PipeDescriptor.java | 35 ------ 7 files changed, 1 insertion(+), 263 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java index 5a1f9d8b8456..afb05fc2c39b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java @@ -35,7 +35,6 @@ import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; -import org.apache.iotdb.db.pipe.event.common.terminate.PipeTerminateEvent; import org.apache.iotdb.db.pipe.extractor.schemaregion.SchemaRegionListeningQueue; import org.apache.iotdb.db.pipe.resource.PipeDataNodeHardlinkOrCopiedFileDirStartupCleaner; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode; @@ -86,15 +85,6 @@ public synchronized void start() throws StartupException { PipeConfig.getInstance().printAllConfigs(); PipeAgentLauncher.launchPipeTaskAgent(); - registerPeriodicalJob( - "PipeTaskAgent#restartAllStuckPipes", - PipeDataNodeAgent.task()::restartAllStuckPipes, - PipeConfig.getInstance().getPipeStuckRestartIntervalSeconds()); - registerPeriodicalJob( - "PipeTaskAgent#flushDataRegionIfNeeded", - PipeTerminateEvent::flushDataRegionIfNeeded, - PipeConfig.getInstance().getPipeFlushAfterLastTerminateSeconds()); - pipePeriodicalJobExecutor.start(); if (PipeConfig.getInstance().getPipeEventReferenceTrackingEnabled()) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/terminate/PipeTerminateEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/terminate/PipeTerminateEvent.java index ca78dd7d50ad..91d38cf3361d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/terminate/PipeTerminateEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/terminate/PipeTerminateEvent.java @@ -19,23 +19,16 @@ package org.apache.iotdb.db.pipe.event.common.terminate; -import org.apache.iotdb.common.rpc.thrift.TFlushReq; import org.apache.iotdb.commons.consensus.index.ProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; import org.apache.iotdb.commons.pipe.agent.task.meta.PipeTaskMeta; -import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.datastructure.pattern.PipePattern; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; import org.apache.iotdb.db.pipe.agent.task.PipeDataNodeTask; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; -import org.apache.iotdb.db.storageengine.StorageEngine; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.atomic.AtomicLong; /** * The {@link PipeTerminateEvent} is an {@link EnrichedEvent} that controls the termination of pipe, @@ -45,40 +38,6 @@ */ public class PipeTerminateEvent extends EnrichedEvent { - private static final Logger LOGGER = LoggerFactory.getLogger(PipeTerminateEvent.class); - - private static final AtomicLong PROGRESS_REPORT_COUNT = new AtomicLong(0); - private static final AtomicLong LAST_PROGRESS_REPORT_TIME = new AtomicLong(0); - - public static void flushDataRegionIfNeeded() { - if (PROGRESS_REPORT_COUNT.get() > 0 - && PROGRESS_REPORT_COUNT.get() - > PipeConfig.getInstance().getPipeFlushAfterTerminateCount()) { - flushDataRegion(); - return; - } - - if (LAST_PROGRESS_REPORT_TIME.get() > 0 - && System.currentTimeMillis() - LAST_PROGRESS_REPORT_TIME.get() - > PipeConfig.getInstance().getPipeFlushAfterLastTerminateSeconds() * 1000L) { - flushDataRegion(); - } - } - - private static void flushDataRegion() { - try { - StorageEngine.getInstance().operateFlush(new TFlushReq()); - PROGRESS_REPORT_COUNT.set(0); - LAST_PROGRESS_REPORT_TIME.set(0); - LOGGER.info("Force flush all data regions because of last progress report time."); - } catch (final Exception e) { - LOGGER.warn( - "Failed to flush all data regions, please check the error message: {}", - e.getMessage(), - e); - } - } - private final int dataRegionId; public PipeTerminateEvent( @@ -135,9 +94,6 @@ public boolean mayEventPathsOverlappedWithPattern() { @Override public void reportProgress() { - PROGRESS_REPORT_COUNT.incrementAndGet(); - LAST_PROGRESS_REPORT_TIME.set(System.currentTimeMillis()); - // To avoid deadlock CompletableFuture.runAsync( () -> PipeDataNodeAgent.task().markCompleted(pipeName, dataRegionId)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java index c86bad92426b..df3d0ad18fc1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java @@ -263,32 +263,14 @@ PipeRealtimeDataRegionHybridExtractor.class, getTaskID(), Integer.MAX_VALUE, 100 private boolean canNotUseTabletAnymoreDeprecated(final PipeRealtimeEvent event) { // In the following 5 cases, we should not extract any more tablet events. all the data // represented by the tablet events should be carried by the following tsfile event: - // 0. If the pipe task is currently restarted. // 1. The number of historical tsFile events to transfer has exceeded the limit. // 2. The number of realtime tsfile events to transfer has exceeded the limit. // 3. The number of linked tsFiles has reached the dangerous threshold. - return isPipeTaskCurrentlyRestarted(event) - || isHistoricalTsFileEventCountExceededLimit(event) + return isHistoricalTsFileEventCountExceededLimit(event) || isRealtimeTsFileEventCountExceededLimit(event) || mayTsFileLinkedCountReachDangerousThreshold(event); } - private boolean isPipeTaskCurrentlyRestarted(final PipeRealtimeEvent event) { - if (!PipeConfig.getInstance().isPipeEpochKeepTsFileAfterStuckRestartEnabled()) { - return false; - } - - final boolean isPipeTaskCurrentlyRestarted = - PipeDataNodeAgent.task().isPipeTaskCurrentlyRestarted(pipeName); - if (isPipeTaskCurrentlyRestarted && event.mayExtractorUseTablets(this)) { - LOGGER.info( - "Pipe task {}@{} canNotUseTabletAnymoreDeprecated(0): Pipe task is currently restarted", - pipeName, - dataRegionId); - } - return isPipeTaskCurrentlyRestarted; - } - private boolean isHistoricalTsFileEventCountExceededLimit(final PipeRealtimeEvent event) { if (PipeConfig.getInstance().getPipeMaxAllowedHistoricalTsFilePerDataRegion() == Integer.MAX_VALUE) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/StorageEngine.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/StorageEngine.java index f923fb2f3413..6fbf870617a7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/StorageEngine.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/StorageEngine.java @@ -38,7 +38,6 @@ import org.apache.iotdb.commons.exception.ShutdownException; import org.apache.iotdb.commons.exception.StartupException; import org.apache.iotdb.commons.file.SystemFileFactory; -import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.schema.ttl.TTLCache; import org.apache.iotdb.commons.service.IService; import org.apache.iotdb.commons.service.ServiceType; @@ -54,7 +53,6 @@ import org.apache.iotdb.db.exception.WriteProcessRejectException; import org.apache.iotdb.db.exception.load.LoadReadOnlyException; import org.apache.iotdb.db.exception.runtime.StorageEngineFailureException; -import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; import org.apache.iotdb.db.queryengine.plan.analyze.cache.schema.DataNodeTTLCache; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.load.LoadTsFilePieceNode; import org.apache.iotdb.db.queryengine.plan.scheduler.load.LoadTsFileScheduler; @@ -219,12 +217,6 @@ private void asyncRecoverDataRegion() throws StartupException { LOGGER.info( "Storage Engine recover cost: {}s.", (System.currentTimeMillis() - startRecoverTime) / 1000); - - PipeDataNodeAgent.runtime() - .registerPeriodicalJob( - "StorageEngine#operateFlush", - () -> operateFlush(new TFlushReq()), - PipeConfig.getInstance().getPipeStorageEngineFlushTimeIntervalMs() / 1000); }, ThreadName.STORAGE_ENGINE_RECOVER_TRIGGER.getName()); recoverEndTrigger.start(); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java index 4523f72b5cac..476170874bcc 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java @@ -276,14 +276,6 @@ public class CommonConfig { private int pipeMaxAllowedHistoricalTsFilePerDataRegion = Integer.MAX_VALUE; // Deprecated private int pipeMaxAllowedPendingTsFileEpochPerDataRegion = Integer.MAX_VALUE; // Deprecated private long pipeMaxAllowedLinkedTsFileCount = Long.MAX_VALUE; // Deprecated - private float pipeMaxAllowedLinkedDeletedTsFileDiskUsagePercentage = 0.1F; - private long pipeStuckRestartIntervalSeconds = 120; - private long pipeStuckRestartMinIntervalMs = 10 * 60 * 1000L; // 5 minutes - private boolean pipeEpochKeepTsFileAfterStuckRestartEnabled = false; - private long pipeFlushAfterLastTerminateSeconds = 30; - private long pipeFlushAfterTerminateCount = 30; - private long pipeStorageEngineFlushTimeIntervalMs = Long.MAX_VALUE; - private int pipeMaxAllowedTotalRemainingInsertEventCount = 50000; private int pipeMetaReportMaxLogNumPerRound = 10; private int pipeMetaReportMaxLogIntervalRounds = 36; @@ -1492,105 +1484,6 @@ public void setPipeMaxAllowedLinkedTsFileCount(long pipeMaxAllowedLinkedTsFileCo logger.info("pipeMaxAllowedLinkedTsFileCount is set to {}", pipeMaxAllowedLinkedTsFileCount); } - public float getPipeMaxAllowedLinkedDeletedTsFileDiskUsagePercentage() { - return pipeMaxAllowedLinkedDeletedTsFileDiskUsagePercentage; - } - - public void setPipeMaxAllowedLinkedDeletedTsFileDiskUsagePercentage( - float pipeMaxAllowedLinkedDeletedTsFileDiskUsagePercentage) { - if (this.pipeMaxAllowedLinkedDeletedTsFileDiskUsagePercentage - == pipeMaxAllowedLinkedDeletedTsFileDiskUsagePercentage) { - return; - } - this.pipeMaxAllowedLinkedDeletedTsFileDiskUsagePercentage = - pipeMaxAllowedLinkedDeletedTsFileDiskUsagePercentage; - logger.info( - "pipeMaxAllowedLinkedDeletedTsFileDiskUsagePercentage is set to {}", - pipeMaxAllowedLinkedDeletedTsFileDiskUsagePercentage); - } - - public long getPipeStuckRestartIntervalSeconds() { - return pipeStuckRestartIntervalSeconds; - } - - public long getPipeStuckRestartMinIntervalMs() { - return pipeStuckRestartMinIntervalMs; - } - - public boolean isPipeEpochKeepTsFileAfterStuckRestartEnabled() { - return pipeEpochKeepTsFileAfterStuckRestartEnabled; - } - - public long getPipeStorageEngineFlushTimeIntervalMs() { - return pipeStorageEngineFlushTimeIntervalMs; - } - - public int getPipeMaxAllowedTotalRemainingInsertEventCount() { - return pipeMaxAllowedTotalRemainingInsertEventCount; - } - - public void setPipeStuckRestartIntervalSeconds(long pipeStuckRestartIntervalSeconds) { - if (this.pipeStuckRestartIntervalSeconds == pipeStuckRestartIntervalSeconds) { - return; - } - this.pipeStuckRestartIntervalSeconds = pipeStuckRestartIntervalSeconds; - logger.info("pipeStuckRestartIntervalSeconds is set to {}", pipeStuckRestartIntervalSeconds); - } - - public void setPipeStuckRestartMinIntervalMs(long pipeStuckRestartMinIntervalMs) { - if (this.pipeStuckRestartMinIntervalMs == pipeStuckRestartMinIntervalMs) { - return; - } - this.pipeStuckRestartMinIntervalMs = pipeStuckRestartMinIntervalMs; - logger.info("pipeStuckRestartMinIntervalMs is set to {}", pipeStuckRestartMinIntervalMs); - } - - public void setPipeEpochKeepTsFileAfterStuckRestartEnabled( - boolean pipeEpochKeepTsFileAfterStuckRestartEnabled) { - if (this.pipeEpochKeepTsFileAfterStuckRestartEnabled - == pipeEpochKeepTsFileAfterStuckRestartEnabled) { - return; - } - this.pipeEpochKeepTsFileAfterStuckRestartEnabled = pipeEpochKeepTsFileAfterStuckRestartEnabled; - logger.info( - "pipeEpochKeepTsFileAfterStuckRestartEnabled is set to {}", - pipeEpochKeepTsFileAfterStuckRestartEnabled); - } - - public void setPipeStorageEngineFlushTimeIntervalMs(long pipeStorageEngineFlushTimeIntervalMs) { - if (this.pipeStorageEngineFlushTimeIntervalMs == pipeStorageEngineFlushTimeIntervalMs) { - return; - } - this.pipeStorageEngineFlushTimeIntervalMs = pipeStorageEngineFlushTimeIntervalMs; - logger.info( - "pipeStorageEngineFlushTimeIntervalMs is set to {}", pipeStorageEngineFlushTimeIntervalMs); - } - - public long getPipeFlushAfterLastTerminateSeconds() { - return pipeFlushAfterLastTerminateSeconds; - } - - public void setPipeFlushAfterLastTerminateSeconds(long pipeFlushAfterLastTerminateSeconds) { - if (this.pipeFlushAfterLastTerminateSeconds == pipeFlushAfterLastTerminateSeconds) { - return; - } - this.pipeFlushAfterLastTerminateSeconds = pipeFlushAfterLastTerminateSeconds; - logger.info( - "pipeFlushAfterLastTerminateSeconds is set to {}", pipeFlushAfterLastTerminateSeconds); - } - - public long getPipeFlushAfterTerminateCount() { - return pipeFlushAfterTerminateCount; - } - - public void setPipeFlushAfterTerminateCount(long pipeFlushAfterTerminateCount) { - if (this.pipeFlushAfterTerminateCount == pipeFlushAfterTerminateCount) { - return; - } - this.pipeFlushAfterTerminateCount = pipeFlushAfterTerminateCount; - logger.info("pipeFlushAfterTerminateCount is set to {}", pipeFlushAfterTerminateCount); - } - public int getPipeMetaReportMaxLogNumPerRound() { return pipeMetaReportMaxLogNumPerRound; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java index 5b4210abfd8a..6fedce58f9d7 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java @@ -335,34 +335,6 @@ public long getPipeMaxAllowedLinkedTsFileCount() { return COMMON_CONFIG.getPipeMaxAllowedLinkedTsFileCount(); } - public float getPipeMaxAllowedLinkedDeletedTsFileDiskUsagePercentage() { - return COMMON_CONFIG.getPipeMaxAllowedLinkedDeletedTsFileDiskUsagePercentage(); - } - - public long getPipeStuckRestartIntervalSeconds() { - return COMMON_CONFIG.getPipeStuckRestartIntervalSeconds(); - } - - public long getPipeStuckRestartMinIntervalMs() { - return COMMON_CONFIG.getPipeStuckRestartMinIntervalMs(); - } - - public boolean isPipeEpochKeepTsFileAfterStuckRestartEnabled() { - return COMMON_CONFIG.isPipeEpochKeepTsFileAfterStuckRestartEnabled(); - } - - public long getPipeFlushAfterTerminateCount() { - return COMMON_CONFIG.getPipeFlushAfterTerminateCount(); - } - - public long getPipeFlushAfterLastTerminateSeconds() { - return COMMON_CONFIG.getPipeFlushAfterLastTerminateSeconds(); - } - - public long getPipeStorageEngineFlushTimeIntervalMs() { - return COMMON_CONFIG.getPipeStorageEngineFlushTimeIntervalMs(); - } - /////////////////////////////// Logger /////////////////////////////// public int getPipeMetaReportMaxLogNumPerRound() { @@ -592,18 +564,6 @@ public void printAllConfigs() { "PipeMaxAllowedPendingTsFileEpochPerDataRegion: {}", getPipeMaxAllowedPendingTsFileEpochPerDataRegion()); LOGGER.info("PipeMaxAllowedLinkedTsFileCount: {}", getPipeMaxAllowedLinkedTsFileCount()); - LOGGER.info( - "PipeMaxAllowedLinkedDeletedTsFileDiskUsagePercentage: {}", - getPipeMaxAllowedLinkedDeletedTsFileDiskUsagePercentage()); - LOGGER.info("PipeStuckRestartIntervalSeconds: {}", getPipeStuckRestartIntervalSeconds()); - LOGGER.info("PipeStuckRestartMinIntervalMs: {}", getPipeStuckRestartMinIntervalMs()); - LOGGER.info( - "PipeEpochKeepTsFileAfterStuckRestartEnabled: {}", - isPipeEpochKeepTsFileAfterStuckRestartEnabled()); - LOGGER.info("PipeFlushAfterTerminateCount: {}", getPipeFlushAfterTerminateCount()); - LOGGER.info("PipeFlushAfterLastTerminateSeconds: {}", getPipeFlushAfterLastTerminateSeconds()); - LOGGER.info( - "PipeStorageEngineFlushTimeIntervalMs: {}", getPipeStorageEngineFlushTimeIntervalMs()); LOGGER.info("PipeMetaReportMaxLogNumPerRound: {}", getPipeMetaReportMaxLogNumPerRound()); LOGGER.info("PipeMetaReportMaxLogIntervalRounds: {}", getPipeMetaReportMaxLogIntervalRounds()); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java index e2f3ee8bddc6..b1937005eff3 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java @@ -428,41 +428,6 @@ public static void loadPipeInternalConfig(CommonConfig config, TrimProperties pr properties.getProperty( "pipe_max_allowed_linked_tsfile_count", String.valueOf(config.getPipeMaxAllowedLinkedTsFileCount())))); - config.setPipeMaxAllowedLinkedDeletedTsFileDiskUsagePercentage( - Float.parseFloat( - properties.getProperty( - "pipe_max_allowed_linked_deleted_tsfile_disk_usage_percentage", - String.valueOf(config.getPipeMaxAllowedLinkedDeletedTsFileDiskUsagePercentage())))); - config.setPipeStuckRestartIntervalSeconds( - Long.parseLong( - properties.getProperty( - "pipe_stuck_restart_interval_seconds", - String.valueOf(config.getPipeStuckRestartIntervalSeconds())))); - config.setPipeStuckRestartMinIntervalMs( - Long.parseLong( - properties.getProperty( - "pipe_stuck_restart_min_interval_ms", - String.valueOf(config.getPipeStuckRestartMinIntervalMs())))); - config.setPipeFlushAfterLastTerminateSeconds( - Long.parseLong( - properties.getProperty( - "pipe_flush_after_last_terminate_seconds", - String.valueOf(config.getPipeFlushAfterLastTerminateSeconds())))); - config.setPipeFlushAfterTerminateCount( - Long.parseLong( - properties.getProperty( - "pipe_flush_after_terminate_count", - String.valueOf(config.getPipeFlushAfterTerminateCount())))); - config.setPipeEpochKeepTsFileAfterStuckRestartEnabled( - Boolean.parseBoolean( - properties.getProperty( - "pipe_epoch_keep_tsfile_after_stuck_restart_enabled", - String.valueOf(config.isPipeEpochKeepTsFileAfterStuckRestartEnabled())))); - config.setPipeStorageEngineFlushTimeIntervalMs( - Long.parseLong( - properties.getProperty( - "pipe_storage_engine_flush_time_interval_ms", - String.valueOf(config.getPipeStorageEngineFlushTimeIntervalMs())))); config.setPipeMemoryAllocateMaxRetries( Integer.parseInt( From 05777b32cce7eafbeb50e99e5885188546a62ada Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 30 Jun 2025 16:09:38 +0800 Subject: [PATCH 046/185] Only closed --- ...peHistoricalDataRegionTsFileExtractor.java | 32 +++++++++---------- 1 file changed, 15 insertions(+), 17 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java index 01b8c0100dc3..5815cb2cfaf3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java @@ -445,32 +445,30 @@ public synchronized void start() { // Some resource is marked as deleted but not removed from the list. !resource.isDeleted() && ( - // Some resource may not be closed due to the control of - // PIPE_MIN_FLUSH_INTERVAL_IN_MS. We simply ignore them. - !resource.isClosed() - || mayTsFileContainUnprocessedData(resource) - && isTsFileResourceOverlappedWithTimeRange(resource) - && isTsFileGeneratedAfterExtractionTimeLowerBound(resource) - && mayTsFileResourceOverlappedWithPattern(resource))) + // Some resource may not be closed because the flush may be incomplete. + resource.isClosed() + && mayTsFileContainUnprocessedData(resource) + && isTsFileResourceOverlappedWithTimeRange(resource) + && isTsFileGeneratedAfterExtractionTimeLowerBound(resource) + && mayTsFileResourceOverlappedWithPattern(resource))) .collect(Collectors.toList()); resourceList.addAll(sequenceTsFileResources); - final Collection unsequenceTsFileResources = + final Collection unSequenceTsFileResources = tsFileManager.getTsFileList(false).stream() .filter( resource -> // Some resource is marked as deleted but not removed from the list. !resource.isDeleted() && ( - // Some resource may not be closed due to the control of - // PIPE_MIN_FLUSH_INTERVAL_IN_MS. We simply ignore them. - !resource.isClosed() - || mayTsFileContainUnprocessedData(resource) - && isTsFileResourceOverlappedWithTimeRange(resource) - && isTsFileGeneratedAfterExtractionTimeLowerBound(resource) - && mayTsFileResourceOverlappedWithPattern(resource))) + // Some resource may not be closed because the flush may be incomplete. + resource.isClosed() + && mayTsFileContainUnprocessedData(resource) + && isTsFileResourceOverlappedWithTimeRange(resource) + && isTsFileGeneratedAfterExtractionTimeLowerBound(resource) + && mayTsFileResourceOverlappedWithPattern(resource))) .collect(Collectors.toList()); - resourceList.addAll(unsequenceTsFileResources); + resourceList.addAll(unSequenceTsFileResources); resourceList.removeIf( resource -> { @@ -500,7 +498,7 @@ && mayTsFileResourceOverlappedWithPattern(resource))) dataRegionId, sequenceTsFileResources.size(), originalSequenceTsFileCount, - unsequenceTsFileResources.size(), + unSequenceTsFileResources.size(), originalUnsequenceTsFileCount, resourceList.size(), originalSequenceTsFileCount + originalUnsequenceTsFileCount, From e50a35c6c91912b9fad851cba30a823c41ff74bc Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 30 Jun 2025 17:38:26 +0800 Subject: [PATCH 047/185] Refactor historical flush & realtie listening --- ...peHistoricalDataRegionTsFileExtractor.java | 92 +------------------ ...PipeRealtimeDataRegionHybridExtractor.java | 27 +----- 2 files changed, 5 insertions(+), 114 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java index 5815cb2cfaf3..2a0d2c1a30da 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java @@ -24,7 +24,6 @@ import org.apache.iotdb.commons.consensus.index.impl.StateProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.TimeWindowStateProgressIndex; import org.apache.iotdb.commons.exception.IllegalPathException; -import org.apache.iotdb.commons.pipe.agent.task.meta.PipeStaticMeta; import org.apache.iotdb.commons.pipe.agent.task.meta.PipeTaskMeta; import org.apache.iotdb.commons.pipe.config.constant.SystemConstant; import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskExtractorRuntimeEnvironment; @@ -214,17 +213,14 @@ public void validate(final PipeParameterValidator validator) { try { historicalDataExtractionStartTime = - isHistoricalExtractorEnabled - && parameters.hasAnyAttributes( - EXTRACTOR_HISTORY_START_TIME_KEY, SOURCE_HISTORY_START_TIME_KEY) + parameters.hasAnyAttributes( + EXTRACTOR_HISTORY_START_TIME_KEY, SOURCE_HISTORY_START_TIME_KEY) ? DateTimeUtils.convertTimestampOrDatetimeStrToLongWithDefaultZone( parameters.getStringByKeys( EXTRACTOR_HISTORY_START_TIME_KEY, SOURCE_HISTORY_START_TIME_KEY)) : Long.MIN_VALUE; historicalDataExtractionEndTime = - isHistoricalExtractorEnabled - && parameters.hasAnyAttributes( - EXTRACTOR_HISTORY_END_TIME_KEY, SOURCE_HISTORY_END_TIME_KEY) + parameters.hasAnyAttributes(EXTRACTOR_HISTORY_END_TIME_KEY, SOURCE_HISTORY_END_TIME_KEY) ? DateTimeUtils.convertTimestampOrDatetimeStrToLongWithDefaultZone( parameters.getStringByKeys( EXTRACTOR_HISTORY_END_TIME_KEY, SOURCE_HISTORY_END_TIME_KEY)) @@ -295,34 +291,6 @@ public void customize( // PipeHistoricalDataRegionExtractor from implementation perspective. : environment.getCreationTime(); - // Only invoke flushDataRegionAllTsFiles() when the pipe runs in the realtime only mode. - // realtime only mode -> (historicalDataExtractionTimeLowerBound != Long.MIN_VALUE) - // - // Ensure that all data in the data region is flushed to disk before extracting data. - // This ensures the generation time of all newly generated TsFiles (realtime data) after the - // invocation of flushDataRegionAllTsFiles() is later than the creationTime of the pipe - // (historicalDataExtractionTimeLowerBound). - // - // Note that: the generation time of the TsFile is the time when the TsFile is created, not - // the time when the data is flushed to the TsFile. - // - // Then we can use the generation time of the TsFile to determine whether the data in the - // TsFile should be extracted by comparing the generation time of the TsFile with the - // historicalDataExtractionTimeLowerBound when starting the pipe in realtime only mode. - // - // If we don't invoke flushDataRegionAllTsFiles() in the realtime only mode, the data generated - // between the creation time of the pipe the time when the pipe starts will be lost. - if (historicalDataExtractionTimeLowerBound != Long.MIN_VALUE) { - synchronized (DATA_REGION_ID_TO_PIPE_FLUSHED_TIME_MAP) { - final long lastFlushedByPipeTime = - DATA_REGION_ID_TO_PIPE_FLUSHED_TIME_MAP.get(dataRegionId); - if (System.currentTimeMillis() - lastFlushedByPipeTime >= PIPE_MIN_FLUSH_INTERVAL_IN_MS) { - flushDataRegionAllTsFiles(); - DATA_REGION_ID_TO_PIPE_FLUSHED_TIME_MAP.replace(dataRegionId, System.currentTimeMillis()); - } - } - } - shouldTransferModFile = parameters.getBooleanOrDefault( Arrays.asList(SOURCE_MODS_ENABLE_KEY, EXTRACTOR_MODS_ENABLE_KEY), @@ -345,24 +313,9 @@ public void customize( } } - private void flushDataRegionAllTsFiles() { - final DataRegion dataRegion = - StorageEngine.getInstance().getDataRegion(new DataRegionId(dataRegionId)); - if (Objects.isNull(dataRegion)) { - return; - } - - dataRegion.writeLock("Pipe: create historical TsFile extractor"); - try { - dataRegion.syncCloseAllWorkingTsFileProcessors(); - } finally { - dataRegion.writeUnlock(); - } - } - @Override public synchronized void start() { - if (!shouldExtractInsertion) { + if (!shouldExtractInsertion || !isHistoricalExtractorEnabled) { hasBeenStarted = true; return; } @@ -385,43 +338,6 @@ public synchronized void start() { dataRegion.writeLock("Pipe: start to extract historical TsFile"); final long startHistoricalExtractionTime = System.currentTimeMillis(); try { - LOGGER.info("Pipe {}@{}: start to flush data region", pipeName, dataRegionId); - - // Consider the scenario: a consensus pipe comes to the same region, followed by another pipe - // **immediately**, the latter pipe will skip the flush operation. - // Since a large number of consensus pipes are not created at the same time, resulting in no - // serious waiting for locks. Therefore, the flush operation is always performed for the - // consensus pipe, and the lastFlushed timestamp is not updated here. - if (pipeName.startsWith(PipeStaticMeta.CONSENSUS_PIPE_PREFIX)) { - dataRegion.syncCloseAllWorkingTsFileProcessors(); - LOGGER.info( - "Pipe {}@{}: finish to flush data region, took {} ms", - pipeName, - dataRegionId, - System.currentTimeMillis() - startHistoricalExtractionTime); - } else { - synchronized (DATA_REGION_ID_TO_PIPE_FLUSHED_TIME_MAP) { - final long lastFlushedByPipeTime = - DATA_REGION_ID_TO_PIPE_FLUSHED_TIME_MAP.get(dataRegionId); - if (System.currentTimeMillis() - lastFlushedByPipeTime >= PIPE_MIN_FLUSH_INTERVAL_IN_MS) { - dataRegion.asyncCloseAllWorkingTsFileProcessors(); - DATA_REGION_ID_TO_PIPE_FLUSHED_TIME_MAP.replace( - dataRegionId, System.currentTimeMillis()); - LOGGER.info( - "Pipe {}@{}: finish to flush data region, took {} ms", - pipeName, - dataRegionId, - System.currentTimeMillis() - startHistoricalExtractionTime); - } else { - LOGGER.info( - "Pipe {}@{}: skip to flush data region, last flushed time {} ms ago", - pipeName, - dataRegionId, - System.currentTimeMillis() - lastFlushedByPipeTime); - } - } - } - final TsFileManager tsFileManager = dataRegion.getTsFileManager(); tsFileManager.readLock(); try { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java index df3d0ad18fc1..1dfcce363ca3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java @@ -25,7 +25,6 @@ import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; -import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; import org.apache.iotdb.db.pipe.extractor.dataregion.IoTDBDataRegionExtractor; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.assigner.PipeTsFileEpochProgressIndexKeeper; @@ -151,31 +150,7 @@ private void extractTsFileInsertion(final PipeRealtimeEvent event) { case USING_TSFILE: return TsFileEpoch.State.USING_TSFILE; case USING_TABLET: - if (((PipeTsFileInsertionEvent) event.getEvent()).getFileStartTime() - < event.getTsFileEpoch().getInsertNodeMinTime()) { - // Some insert nodes in the tsfile epoch are not captured by pipe, so we should - // capture the tsfile event to make sure all data in the tsfile epoch can be - // extracted. - // - // The situation can be caused by the following operations: - // 1. PipeA: start historical data extraction with flush - // 2. Data insertion - // 3. PipeB: start realtime data extraction - // 4. PipeB: start historical data extraction without flush - // 5. Data inserted in the step2 is not captured by PipeB, and if its tsfile - // epoch's state is USING_TABLET, the tsfile event will be ignored, which - // will cause the data loss in the tsfile epoch. - LOGGER.info( - "The tsFile {}'s epoch's start time {} is smaller than the captured insertNodes' min time {}, will regard it as data loss or un-sequential, will extract the tsFile", - ((PipeTsFileInsertionEvent) event.getEvent()).getTsFile(), - ((PipeTsFileInsertionEvent) event.getEvent()).getFileStartTime(), - event.getTsFileEpoch().getInsertNodeMinTime()); - return TsFileEpoch.State.USING_BOTH; - } else { - // All data in the tsfile epoch has been extracted in tablet mode, so we should - // simply keep the state of the tsfile epoch and discard the tsfile event. - return TsFileEpoch.State.USING_TABLET; - } + return TsFileEpoch.State.USING_TABLET; case USING_BOTH: default: return canNotUseTabletAnyMore(event) From 734eb1f9171a165e8854b3af34fdb5fc8d025471 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 30 Jun 2025 17:58:45 +0800 Subject: [PATCH 048/185] Nullable historical --- .../dataregion/IoTDBDataRegionExtractor.java | 67 ++++++++++--------- 1 file changed, 34 insertions(+), 33 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java index 98504128a2e5..913e0507b528 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.pipe.extractor.dataregion; import org.apache.iotdb.commons.consensus.DataRegionId; +import org.apache.iotdb.commons.pipe.config.constant.SystemConstant; import org.apache.iotdb.commons.pipe.datastructure.pattern.IoTDBPipePattern; import org.apache.iotdb.commons.pipe.datastructure.pattern.PipePattern; import org.apache.iotdb.commons.pipe.extractor.IoTDBExtractor; @@ -49,6 +50,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.util.Arrays; import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; @@ -92,13 +95,12 @@ public class IoTDBDataRegionExtractor extends IoTDBExtractor { private static final Logger LOGGER = LoggerFactory.getLogger(IoTDBDataRegionExtractor.class); - private PipeHistoricalDataRegionExtractor historicalExtractor; + private @Nullable PipeHistoricalDataRegionExtractor historicalExtractor; private PipeRealtimeDataRegionExtractor realtimeExtractor; private DataRegionWatermarkInjector watermarkInjector; private boolean hasNoExtractionNeed = true; - private boolean shouldExtractInsertion = false; private boolean shouldExtractDeletion = false; @Override @@ -113,7 +115,6 @@ public void validate(final PipeParameterValidator validator) throws Exception { return; } hasNoExtractionNeed = false; - shouldExtractInsertion = insertionDeletionListeningOptionPair.getLeft(); shouldExtractDeletion = insertionDeletionListeningOptionPair.getRight(); if (insertionDeletionListeningOptionPair.getLeft().equals(true) @@ -210,7 +211,17 @@ public void validate(final PipeParameterValidator validator) throws Exception { EXTRACTOR_HISTORY_END_TIME_KEY); } - constructHistoricalExtractor(); + if (validator + .getParameters() + .getBooleanOrDefault(SystemConstant.RESTART_KEY, SystemConstant.RESTART_DEFAULT_VALUE) + || validator + .getParameters() + .getBooleanOrDefault( + Arrays.asList(EXTRACTOR_HISTORY_ENABLE_KEY, SOURCE_HISTORY_ENABLE_KEY), + EXTRACTOR_HISTORY_ENABLE_DEFAULT_VALUE)) { + // Do not flush or open historical extractor when historical tsFile is disabled + constructHistoricalExtractor(); + } constructRealtimeExtractor(validator.getParameters()); historicalExtractor.validate(validator); @@ -234,7 +245,6 @@ private void validatePattern(final PipePattern pattern) { } private void constructHistoricalExtractor() { - // Enable historical extractor by default historicalExtractor = new PipeHistoricalDataRegionTsFileExtractor(); } @@ -305,7 +315,9 @@ public void customize( super.customize(parameters, configuration); - historicalExtractor.customize(parameters, configuration); + if (Objects.nonNull(historicalExtractor)) { + historicalExtractor.customize(parameters, configuration); + } realtimeExtractor.customize(parameters, configuration); // Set watermark injector @@ -342,7 +354,9 @@ public void start() throws Exception { "Pipe {}@{}: Starting historical extractor {} and realtime extractor {}.", pipeName, regionId, - historicalExtractor.getClass().getSimpleName(), + Objects.nonNull(historicalExtractor) + ? historicalExtractor.getClass().getSimpleName() + : null, realtimeExtractor.getClass().getSimpleName()); super.start(); @@ -395,14 +409,18 @@ private void startHistoricalExtractorAndRealtimeExtractor( // There can still be writing when tsFile events are added. If we start // realtimeExtractor after the process, then this part of data will be lost. realtimeExtractor.start(); - historicalExtractor.start(); + if (Objects.nonNull(historicalExtractor)) { + historicalExtractor.start(); + } } catch (final Exception e) { exceptionHolder.set(e); LOGGER.warn( "Pipe {}@{}: Start historical extractor {} and realtime extractor {} error.", pipeName, regionId, - historicalExtractor.getClass().getSimpleName(), + Objects.nonNull(historicalExtractor) + ? historicalExtractor.getClass().getSimpleName() + : null, realtimeExtractor.getClass().getSimpleName(), e); } @@ -421,7 +439,7 @@ public Event supply() throws Exception { } Event event = null; - if (!historicalExtractor.hasConsumedAll()) { + if (Objects.nonNull(historicalExtractor) && !historicalExtractor.hasConsumedAll()) { event = historicalExtractor.supply(); } else { if (Objects.nonNull(watermarkInjector)) { @@ -451,32 +469,21 @@ public void close() throws Exception { return; } - historicalExtractor.close(); + if (Objects.nonNull(historicalExtractor)) { + historicalExtractor.close(); + } realtimeExtractor.close(); if (Objects.nonNull(taskID)) { PipeDataRegionExtractorMetrics.getInstance().deregister(taskID); } } - //////////////////////////// APIs provided for detecting stuck //////////////////////////// - - public boolean shouldExtractInsertion() { - return shouldExtractInsertion; - } - - public boolean isStreamMode() { - return realtimeExtractor instanceof PipeRealtimeDataRegionHybridExtractor - || realtimeExtractor instanceof PipeRealtimeDataRegionLogExtractor; - } - - public boolean hasConsumedAllHistoricalTsFiles() { - return historicalExtractor.hasConsumedAll(); - } - //////////////////////////// APIs provided for metric framework //////////////////////////// public int getHistoricalTsFileInsertionEventCount() { - return hasBeenStarted.get() ? historicalExtractor.getPendingQueueSize() : 0; + return hasBeenStarted.get() && Objects.nonNull(historicalExtractor) + ? historicalExtractor.getPendingQueueSize() + : 0; } public int getTabletInsertionEventCount() { @@ -490,10 +497,4 @@ public int getRealtimeTsFileInsertionEventCount() { public int getPipeHeartbeatEventCount() { return hasBeenStarted.get() ? realtimeExtractor.getPipeHeartbeatEventCount() : 0; } - - public int getEventCount() { - return hasBeenStarted.get() - ? (historicalExtractor.getPendingQueueSize() + realtimeExtractor.getEventCount()) - : 0; - } } From eb023a269c8f8c7915fb2afa1cb6d1787bb31b09 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 30 Jun 2025 18:01:00 +0800 Subject: [PATCH 049/185] Update PipeHistoricalDataRegionTsFileExtractor.java --- ...peHistoricalDataRegionTsFileExtractor.java | 27 +++++++++++++------ 1 file changed, 19 insertions(+), 8 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java index 2a0d2c1a30da..e61deec119d4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java @@ -24,6 +24,7 @@ import org.apache.iotdb.commons.consensus.index.impl.StateProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.TimeWindowStateProgressIndex; import org.apache.iotdb.commons.exception.IllegalPathException; +import org.apache.iotdb.commons.pipe.agent.task.meta.PipeStaticMeta; import org.apache.iotdb.commons.pipe.agent.task.meta.PipeTaskMeta; import org.apache.iotdb.commons.pipe.config.constant.SystemConstant; import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskExtractorRuntimeEnvironment; @@ -56,7 +57,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -90,9 +90,6 @@ public class PipeHistoricalDataRegionTsFileExtractor implements PipeHistoricalDa private static final Logger LOGGER = LoggerFactory.getLogger(PipeHistoricalDataRegionTsFileExtractor.class); - private static final Map DATA_REGION_ID_TO_PIPE_FLUSHED_TIME_MAP = new HashMap<>(); - private static final long PIPE_MIN_FLUSH_INTERVAL_IN_MS = 2000; - private String pipeName; private long creationTime; @@ -264,10 +261,6 @@ public void customize( startIndex = environment.getPipeTaskMeta().restoreProgressIndex(); dataRegionId = environment.getRegionId(); - synchronized (DATA_REGION_ID_TO_PIPE_FLUSHED_TIME_MAP) { - DATA_REGION_ID_TO_PIPE_FLUSHED_TIME_MAP.putIfAbsent(dataRegionId, 0L); - } - pipePattern = PipePattern.parsePipePatternFromSourceParameters(parameters); final DataRegion dataRegion = @@ -338,6 +331,24 @@ public synchronized void start() { dataRegion.writeLock("Pipe: start to extract historical TsFile"); final long startHistoricalExtractionTime = System.currentTimeMillis(); try { + LOGGER.info("Pipe {}@{}: start to flush data region", pipeName, dataRegionId); + + // Consider the scenario: a consensus pipe comes to the same region, followed by another pipe + // **immediately**, the latter pipe will skip the flush operation. + // Since a large number of consensus pipes are not created at the same time, resulting in no + // serious waiting for locks. Therefore, the flush operation is always performed for the + // consensus pipe, and the lastFlushed timestamp is not updated here. + if (pipeName.startsWith(PipeStaticMeta.CONSENSUS_PIPE_PREFIX)) { + dataRegion.syncCloseAllWorkingTsFileProcessors(); + } else { + dataRegion.asyncCloseAllWorkingTsFileProcessors(); + } + LOGGER.info( + "Pipe {}@{}: finish to flush data region, took {} ms", + pipeName, + dataRegionId, + System.currentTimeMillis() - startHistoricalExtractionTime); + final TsFileManager tsFileManager = dataRegion.getTsFileManager(); tsFileManager.readLock(); try { From 47c6b26fb04718834e6b5d3bf986a93bc9d92ffc Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 30 Jun 2025 19:05:06 +0800 Subject: [PATCH 050/185] Delete file only --- ...deHardlinkOrCopiedFileDirStartupCleaner.java | 9 ++++++--- .../apache/iotdb/commons/utils/FileUtils.java | 17 +++++++++++++++++ 2 files changed, 23 insertions(+), 3 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/PipeDataNodeHardlinkOrCopiedFileDirStartupCleaner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/PipeDataNodeHardlinkOrCopiedFileDirStartupCleaner.java index b58d934988f1..ec3f9f55ccc6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/PipeDataNodeHardlinkOrCopiedFileDirStartupCleaner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/PipeDataNodeHardlinkOrCopiedFileDirStartupCleaner.java @@ -21,9 +21,9 @@ import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.resource.snapshot.PipeSnapshotResourceManager; +import org.apache.iotdb.commons.utils.FileUtils; import org.apache.iotdb.db.conf.IoTDBDescriptor; -import org.apache.commons.io.FileUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -52,7 +52,10 @@ private static void cleanTsFileDir() { LOGGER.info( "Pipe hardlink dir found, deleting it: {}, result: {}", pipeHardLinkDir, - FileUtils.deleteQuietly(pipeHardLinkDir)); + // For child directories, we need them to recover each pipe's progress + // Hence we do not delete them here, only delete the child files (from older version || + // assigner pinned tsFiles) + FileUtils.deleteFileChildrenQuietly(pipeHardLinkDir)); } } } @@ -65,7 +68,7 @@ private static void cleanSnapshotDir() { + PipeSnapshotResourceManager.PIPE_SNAPSHOT_DIR_NAME); if (pipeConsensusDir.isDirectory()) { LOGGER.info("Pipe snapshot dir found, deleting it: {},", pipeConsensusDir); - org.apache.iotdb.commons.utils.FileUtils.deleteFileOrDirectory(pipeConsensusDir); + FileUtils.deleteFileOrDirectory(pipeConsensusDir); } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/FileUtils.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/FileUtils.java index 7597b0826dfd..04577f423b28 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/FileUtils.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/FileUtils.java @@ -114,6 +114,23 @@ public static void deleteFileOrDirectoryWithRetry(File file) { } } + public static boolean deleteFileChildrenQuietly(final File file) { + if (!file.exists() || !file.isDirectory()) { + return false; + } + + boolean result = true; + final File[] items = file.listFiles(); + if (items != null) { + for (final File item : items) { + if (item.isFile()) { + result &= org.apache.commons.io.FileUtils.deleteQuietly(item); + } + } + } + return result; + } + public static void deleteDirectoryAndEmptyParent(File folder) { deleteFileOrDirectory(folder); final File parentFolder = folder.getParentFile(); From 3a911b680a48a4363c060143b7e6c890fb57c6eb Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 1 Jul 2025 09:56:44 +0800 Subject: [PATCH 051/185] Update IoTDBDataRegionExtractor.java --- .../pipe/extractor/dataregion/IoTDBDataRegionExtractor.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java index 913e0507b528..a37f268d486d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java @@ -224,7 +224,9 @@ public void validate(final PipeParameterValidator validator) throws Exception { } constructRealtimeExtractor(validator.getParameters()); - historicalExtractor.validate(validator); + if (Objects.nonNull(historicalExtractor)) { + historicalExtractor.validate(validator); + } realtimeExtractor.validate(validator); } From 47ee0bbfd6bf48be1bcfb0736525313bbd5542dc Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 1 Jul 2025 10:15:15 +0800 Subject: [PATCH 052/185] partial --- .../PipeRealtimeDataRegionHybridExtractor.java | 10 ++++++++-- .../realtime/PipeRealtimeDataRegionLogExtractor.java | 5 +---- 2 files changed, 9 insertions(+), 6 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java index 1dfcce363ca3..92212eaad63a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java @@ -25,6 +25,7 @@ import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; +import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; import org.apache.iotdb.db.pipe.extractor.dataregion.IoTDBDataRegionExtractor; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.assigner.PipeTsFileEpochProgressIndexKeeper; @@ -140,6 +141,8 @@ private void extractTabletInsertion(final PipeRealtimeEvent event) { } private void extractTsFileInsertion(final PipeRealtimeEvent event) { + // Notice that, if the tsFile is partially extracted because the pipe is not opened before, the + // former data won't be extracted event .getTsFileEpoch() .migrateState( @@ -147,10 +150,13 @@ private void extractTsFileInsertion(final PipeRealtimeEvent event) { state -> { switch (state) { case EMPTY: - case USING_TSFILE: - return TsFileEpoch.State.USING_TSFILE; + return ((PipeTsFileInsertionEvent) event.getEvent()).isLoaded() + ? TsFileEpoch.State.USING_TSFILE + : TsFileEpoch.State.USING_TABLET; case USING_TABLET: return TsFileEpoch.State.USING_TABLET; + case USING_TSFILE: + return TsFileEpoch.State.USING_TSFILE; case USING_BOTH: default: return canNotUseTabletAnyMore(event) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionLogExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionLogExtractor.java index 4b300355c80e..937d8aa53890 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionLogExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionLogExtractor.java @@ -82,10 +82,7 @@ private void extractTabletInsertion(PipeRealtimeEvent event) { private void extractTsFileInsertion(PipeRealtimeEvent event) { final PipeTsFileInsertionEvent tsFileInsertionEvent = (PipeTsFileInsertionEvent) event.getEvent(); - if (!(tsFileInsertionEvent.isLoaded() - // some insert nodes in the tsfile epoch are not captured by pipe - || tsFileInsertionEvent.getFileStartTime() - < event.getTsFileEpoch().getInsertNodeMinTime())) { + if (!tsFileInsertionEvent.isLoaded()) { // All data in the tsfile epoch has been extracted in tablet mode, so we should // simply ignore this event. event.decreaseReferenceCount(PipeRealtimeDataRegionLogExtractor.class.getName(), false); From 6ca8bb664d21ff5ab28c15ca085d38dd46667231 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 1 Jul 2025 10:20:21 +0800 Subject: [PATCH 053/185] simple refactor --- .../PipeHistoricalDataRegionTsFileExtractor.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java index e61deec119d4..cc27e4281af5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java @@ -353,16 +353,16 @@ public synchronized void start() { tsFileManager.readLock(); try { final int originalSequenceTsFileCount = tsFileManager.size(true); - final int originalUnsequenceTsFileCount = tsFileManager.size(false); + final int originalUnSequenceTsFileCount = tsFileManager.size(false); final List resourceList = - new ArrayList<>(originalSequenceTsFileCount + originalUnsequenceTsFileCount); + new ArrayList<>(originalSequenceTsFileCount + originalUnSequenceTsFileCount); LOGGER.info( "Pipe {}@{}: start to extract historical TsFile, original sequence file count {}, " - + "original unsequence file count {}, start progress index {}", + + "original unSequence file count {}, start progress index {}", pipeName, dataRegionId, originalSequenceTsFileCount, - originalUnsequenceTsFileCount, + originalUnSequenceTsFileCount, startIndex); final Collection sequenceTsFileResources = @@ -426,9 +426,9 @@ && mayTsFileResourceOverlappedWithPattern(resource))) sequenceTsFileResources.size(), originalSequenceTsFileCount, unSequenceTsFileResources.size(), - originalUnsequenceTsFileCount, + originalUnSequenceTsFileCount, resourceList.size(), - originalSequenceTsFileCount + originalUnsequenceTsFileCount, + originalSequenceTsFileCount + originalUnSequenceTsFileCount, System.currentTimeMillis() - startHistoricalExtractionTime); } finally { tsFileManager.readUnlock(); From fb5308bff3e5439f4d841b54a1a6262f90639ed7 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 1 Jul 2025 10:24:57 +0800 Subject: [PATCH 054/185] Restore tsFile wait Close --- .../task/connection/PipeEventCollector.java | 2 +- .../IoTDBDataRegionAirGapConnector.java | 2 +- .../legacy/IoTDBLegacyPipeConnector.java | 2 +- .../tsfile/PipeTsFileInsertionEvent.java | 95 ++++++++++++++++--- .../plugin/TwoStageCountProcessor.java | 2 +- 5 files changed, 84 insertions(+), 19 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollector.java index 97855856747d..66bc5ab2a508 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollector.java @@ -122,7 +122,7 @@ private void parseAndCollectEvent(final PipeRawTabletInsertionEvent sourceEvent) } private void parseAndCollectEvent(final PipeTsFileInsertionEvent sourceEvent) throws Exception { - if (sourceEvent.isEmpty()) { + if (!sourceEvent.waitForTsFileClose()) { LOGGER.warn( "Pipe skipping temporary TsFile which shouldn't be transferred: {}", sourceEvent.getTsFile()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataRegionAirGapConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataRegionAirGapConnector.java index a47b0c463802..bc4805e8c8ef 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataRegionAirGapConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataRegionAirGapConnector.java @@ -99,7 +99,7 @@ public void transfer(final TsFileInsertionEvent tsFileInsertionEvent) throws Exc return; } - if (((PipeTsFileInsertionEvent) tsFileInsertionEvent).isEmpty()) { + if (!((PipeTsFileInsertionEvent) tsFileInsertionEvent).waitForTsFileClose()) { LOGGER.warn( "Pipe skipping temporary TsFile which shouldn't be transferred: {}", ((PipeTsFileInsertionEvent) tsFileInsertionEvent).getTsFile()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/legacy/IoTDBLegacyPipeConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/legacy/IoTDBLegacyPipeConnector.java index f4e2889744af..58a929d30d0e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/legacy/IoTDBLegacyPipeConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/legacy/IoTDBLegacyPipeConnector.java @@ -290,7 +290,7 @@ public void transfer(final TsFileInsertionEvent tsFileInsertionEvent) throws Exc "IoTDBLegacyPipeConnector only support PipeTsFileInsertionEvent."); } - if (((PipeTsFileInsertionEvent) tsFileInsertionEvent).isEmpty()) { + if (!((PipeTsFileInsertionEvent) tsFileInsertionEvent).waitForTsFileClose()) { LOGGER.warn( "Pipe skipping temporary TsFile which shouldn't be transferred: {}", ((PipeTsFileInsertionEvent) tsFileInsertionEvent).getTsFile()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java index b76de24dc1c6..d5238f854533 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java @@ -78,6 +78,7 @@ public class PipeTsFileInsertionEvent extends EnrichedEvent private final boolean isGeneratedByPipeConsensus; private final boolean isGeneratedByHistoricalExtractor; + private final AtomicBoolean isClosed; private final AtomicReference dataContainer; // The point count of the TsFile. Used for metrics on PipeConsensus' receiver side. @@ -129,6 +130,42 @@ public PipeTsFileInsertionEvent( this.isGeneratedByPipeConsensus = resource.isGeneratedByPipeConsensus(); this.isGeneratedByHistoricalExtractor = isGeneratedByHistoricalExtractor; + isClosed = new AtomicBoolean(resource.isClosed()); + // Register close listener if TsFile is not closed + if (!isClosed.get()) { + final TsFileProcessor processor = resource.getProcessor(); + if (processor != null) { + processor.addCloseFileListener( + o -> { + synchronized (isClosed) { + isClosed.set(true); + isClosed.notifyAll(); + + // Update flushPointCount after TsFile is closed + flushPointCount = processor.getMemTableFlushPointCount(); + } + }); + } + } + // Check again after register close listener in case TsFile is closed during the process + // TsFile flushing steps: + // 1. Flush tsFile + // 2. First listener (Set resource status "closed" -> Set processor == null -> processor == null + // is seen) + // 3. Other listeners (Set "closed" status for events) + // Then we can imply that: + // 1. If the listener cannot be executed because all listeners passed, then resources status is + // set "closed" and can be set here + // 2. If the listener cannot be executed because processor == null is seen, then resources + // status is set "closed" and can be set here + // Then we know: + // 1. The status in the event can be closed eventually. + // 2. If the status is "closed", then the resource status is "closed". + // Then we know: + // If the status is "closed", then the resource status is "closed", the tsFile won't be altered + // and can be sent. + isClosed.set(resource.isClosed()); + this.dataContainer = new AtomicReference<>(null); } @@ -136,10 +173,35 @@ public PipeTsFileInsertionEvent( * @return {@code false} if this file can't be sent by pipe because it is empty. {@code true} * otherwise. */ - public boolean isEmpty() { - // Here we guarantee that the isEmpty() is set before flushing if tsFile is empty + public boolean waitForTsFileClose() throws InterruptedException { + if (!isClosed.get()) { + isClosed.set(resource.isClosed()); + + synchronized (isClosed) { + while (!isClosed.get()) { + isClosed.wait(100); + + final boolean isClosedNow = resource.isClosed(); + if (isClosedNow) { + isClosed.set(true); + isClosed.notifyAll(); + + // Update flushPointCount after TsFile is closed + final TsFileProcessor processor = resource.getProcessor(); + if (processor != null) { + flushPointCount = processor.getMemTableFlushPointCount(); + } + + break; + } + } + } + } + + // From illustrations above we know If the status is "closed", then the tsFile is flushed + // And here we guarantee that the isEmpty() is set before flushing if tsFile is empty // Then we know: "isClosed" --> tsFile flushed --> (isEmpty() <--> tsFile is empty) - return resource.isEmpty(); + return !resource.isEmpty(); } public File getTsFile() { @@ -164,10 +226,6 @@ public boolean isLoaded() { return isLoaded; } - public long getFileStartTime() { - return resource.getFileStartTime(); - } - /** * Only used for metrics on PipeConsensus' receiver side. If the event is recovered after data * node's restart, the flushPointCount can be not set. It's totally fine for the PipeConsensus' @@ -245,15 +303,22 @@ public void bindProgressIndex(final ProgressIndex overridingProgressIndex) { @Override public ProgressIndex getProgressIndex() { - if (isEmpty()) { + try { + if (!waitForTsFileClose()) { + LOGGER.warn( + "Skipping temporary TsFile {}'s progressIndex, will report MinimumProgressIndex", + tsFile); + return MinimumProgressIndex.INSTANCE; + } + if (Objects.nonNull(overridingProgressIndex)) { + return overridingProgressIndex; + } + return resource.getMaxProgressIndexAfterClose(); + } catch (final InterruptedException e) { LOGGER.warn( "Skipping temporary TsFile {}'s progressIndex, will report MinimumProgressIndex", tsFile); return MinimumProgressIndex.INSTANCE; } - if (Objects.nonNull(overridingProgressIndex)) { - return overridingProgressIndex; - } - return resource.getMaxProgressIndexAfterClose(); } /** @@ -313,8 +378,8 @@ public boolean isGeneratedByPipe() { @Override public boolean mayEventTimeOverlappedWithTimeRange() { - // If the tsFile is not closed the resource.getFileEndTime() will be Long.MIN_VALUE - // In that case we only judge the resource.getFileStartTime() to avoid losing data + // Notice that this is only called at realtime extraction, and the tsFile is always closed + // Thus we can use the end time to judge the overlap return startTime <= resource.getFileEndTime() && resource.getFileStartTime() <= endTime; } @@ -403,7 +468,7 @@ public Iterable toTabletInsertionEvents() throws PipeExcep public Iterable toTabletInsertionEvents(final long timeoutMs) throws PipeException { try { - if (isEmpty()) { + if (!waitForTsFileClose()) { LOGGER.warn( "Pipe skipping temporary TsFile's parsing which shouldn't be transferred: {}", tsFile); return Collections.emptyList(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/twostage/plugin/TwoStageCountProcessor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/twostage/plugin/TwoStageCountProcessor.java index df1925e59838..16a20a5f509a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/twostage/plugin/TwoStageCountProcessor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/twostage/plugin/TwoStageCountProcessor.java @@ -189,7 +189,7 @@ public void process(TsFileInsertionEvent tsFileInsertionEvent, EventCollector ev final PipeTsFileInsertionEvent event = (PipeTsFileInsertionEvent) tsFileInsertionEvent; event.skipReportOnCommit(); - if (event.isEmpty()) { + if (!event.waitForTsFileClose()) { LOGGER.warn("Ignored TsFileInsertionEvent is empty: {}", event); return; } From 8bbc14f1508474cf395e51368b3dafc91c7f5030 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 1 Jul 2025 10:41:48 +0800 Subject: [PATCH 055/185] Refactor max progress index --- .../tsfile/PipeTsFileInsertionEvent.java | 17 +------- ...peHistoricalDataRegionTsFileExtractor.java | 40 ++++++++++++------- .../InsertionCrossSpaceCompactionTask.java | 4 +- .../execute/utils/CompactionUtils.java | 4 +- .../dataregion/tsfile/TsFileResource.java | 8 ---- .../TsFileResourceProgressIndexTest.java | 23 ++++------- 6 files changed, 39 insertions(+), 57 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java index d5238f854533..17fa19083a51 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java @@ -303,22 +303,7 @@ public void bindProgressIndex(final ProgressIndex overridingProgressIndex) { @Override public ProgressIndex getProgressIndex() { - try { - if (!waitForTsFileClose()) { - LOGGER.warn( - "Skipping temporary TsFile {}'s progressIndex, will report MinimumProgressIndex", - tsFile); - return MinimumProgressIndex.INSTANCE; - } - if (Objects.nonNull(overridingProgressIndex)) { - return overridingProgressIndex; - } - return resource.getMaxProgressIndexAfterClose(); - } catch (final InterruptedException e) { - LOGGER.warn( - "Skipping temporary TsFile {}'s progressIndex, will report MinimumProgressIndex", tsFile); - return MinimumProgressIndex.INSTANCE; - } + return resource.getMaxProgressIndex(); } /** diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java index cc27e4281af5..0408e4f3a039 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java @@ -36,6 +36,7 @@ import org.apache.iotdb.db.pipe.resource.tsfile.PipeTsFileResourceManager; import org.apache.iotdb.db.storageengine.StorageEngine; import org.apache.iotdb.db.storageengine.dataregion.DataRegion; +import org.apache.iotdb.db.storageengine.dataregion.memtable.TsFileProcessor; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileManager; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.iotdb.db.storageengine.dataregion.tsfile.generator.TsFileNameGenerator; @@ -60,6 +61,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.Queue; import java.util.Set; import java.util.stream.Collectors; @@ -372,12 +374,17 @@ public synchronized void start() { // Some resource is marked as deleted but not removed from the list. !resource.isDeleted() && ( - // Some resource may not be closed because the flush may be incomplete. - resource.isClosed() - && mayTsFileContainUnprocessedData(resource) - && isTsFileResourceOverlappedWithTimeRange(resource) - && isTsFileGeneratedAfterExtractionTimeLowerBound(resource) - && mayTsFileResourceOverlappedWithPattern(resource))) + // If the tsFile is not already marked closing, it is not captured by + // the pipe realtime module. Thus, we can wait for the realtime sync + // module to handle this, to avoid blocking the pipe sync process. + !resource.isClosed() + && Optional.ofNullable(resource.getProcessor()) + .map(TsFileProcessor::alreadyMarkedClosing) + .orElse(true) + || mayTsFileContainUnprocessedData(resource) + && isTsFileResourceOverlappedWithTimeRange(resource) + && isTsFileGeneratedAfterExtractionTimeLowerBound(resource) + && mayTsFileResourceOverlappedWithPattern(resource))) .collect(Collectors.toList()); resourceList.addAll(sequenceTsFileResources); @@ -388,12 +395,17 @@ && mayTsFileResourceOverlappedWithPattern(resource))) // Some resource is marked as deleted but not removed from the list. !resource.isDeleted() && ( - // Some resource may not be closed because the flush may be incomplete. - resource.isClosed() - && mayTsFileContainUnprocessedData(resource) - && isTsFileResourceOverlappedWithTimeRange(resource) - && isTsFileGeneratedAfterExtractionTimeLowerBound(resource) - && mayTsFileResourceOverlappedWithPattern(resource))) + // If the tsFile is not already marked closing, it is not captured by + // the pipe realtime module. Thus, we can wait for the realtime sync + // module to handle this, to avoid blocking the pipe sync process. + !resource.isClosed() + && Optional.ofNullable(resource.getProcessor()) + .map(TsFileProcessor::alreadyMarkedClosing) + .orElse(true) + || mayTsFileContainUnprocessedData(resource) + && isTsFileResourceOverlappedWithTimeRange(resource) + && isTsFileGeneratedAfterExtractionTimeLowerBound(resource) + && mayTsFileResourceOverlappedWithPattern(resource))) .collect(Collectors.toList()); resourceList.addAll(unSequenceTsFileResources); @@ -447,8 +459,8 @@ private boolean mayTsFileContainUnprocessedData(final TsFileResource resource) { if (startIndex instanceof StateProgressIndex) { startIndex = ((StateProgressIndex) startIndex).getInnerProgressIndex(); } - return !startIndex.isAfter(resource.getMaxProgressIndexAfterClose()) - && !startIndex.equals(resource.getMaxProgressIndexAfterClose()); + return !startIndex.isAfter(resource.getMaxProgressIndex()) + && !startIndex.equals(resource.getMaxProgressIndex()); } private boolean mayTsFileResourceOverlappedWithPattern(final TsFileResource resource) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/InsertionCrossSpaceCompactionTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/InsertionCrossSpaceCompactionTask.java index c4fba1fb2154..992b5406708e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/InsertionCrossSpaceCompactionTask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/task/InsertionCrossSpaceCompactionTask.java @@ -227,9 +227,9 @@ private void prepareTargetFiles() throws IOException { new File(targetTsFile.getPath() + ModificationFile.FILE_SUFFIX).toPath(), new File(sourceTsFile.getPath() + ModificationFile.FILE_SUFFIX).toPath()); } - targetFile.setProgressIndex(unseqFileToInsert.getMaxProgressIndexAfterClose()); + targetFile.setProgressIndex(unseqFileToInsert.getMaxProgressIndex()); targetFile.deserialize(); - targetFile.setProgressIndex(unseqFileToInsert.getMaxProgressIndexAfterClose()); + targetFile.setProgressIndex(unseqFileToInsert.getMaxProgressIndex()); } private boolean recoverTaskInfoFromLogFile() throws IOException { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/CompactionUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/CompactionUtils.java index afbea98ec299..3b251a0c909a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/CompactionUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/CompactionUtils.java @@ -288,7 +288,7 @@ public static void updateProgressIndexAndMark( List unseqResources) { for (TsFileResource targetResource : targetResources) { for (TsFileResource unseqResource : unseqResources) { - targetResource.updateProgressIndex(unseqResource.getMaxProgressIndexAfterClose()); + targetResource.updateProgressIndex(unseqResource.getMaxProgressIndex()); targetResource.setGeneratedByPipe( unseqResource.isGeneratedByPipe() && targetResource.isGeneratedByPipe()); targetResource.setGeneratedByPipeConsensus( @@ -296,7 +296,7 @@ public static void updateProgressIndexAndMark( && targetResource.isGeneratedByPipeConsensus()); } for (TsFileResource seqResource : seqResources) { - targetResource.updateProgressIndex(seqResource.getMaxProgressIndexAfterClose()); + targetResource.updateProgressIndex(seqResource.getMaxProgressIndex()); targetResource.setGeneratedByPipe( seqResource.isGeneratedByPipe() && targetResource.isGeneratedByPipe()); targetResource.setGeneratedByPipeConsensus( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java index 7761ea5c3b16..ee05d2f1c74d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java @@ -1218,14 +1218,6 @@ public void setProgressIndex(ProgressIndex progressIndex) { .updateProgressIndex(getDataRegionId(), getTsFilePath(), maxProgressIndex); } - public ProgressIndex getMaxProgressIndexAfterClose() throws IllegalStateException { - if (getStatus().equals(TsFileResourceStatus.UNCLOSED)) { - throw new IllegalStateException( - "Should not get progress index from a unclosing TsFileResource."); - } - return getMaxProgressIndex(); - } - public ProgressIndex getMaxProgressIndex() { return maxProgressIndex == null ? MinimumProgressIndex.INSTANCE : maxProgressIndex; } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/TsFileResourceProgressIndexTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/TsFileResourceProgressIndexTest.java index 33bf80115a3c..87b25883fb17 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/TsFileResourceProgressIndexTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/TsFileResourceProgressIndexTest.java @@ -128,30 +128,23 @@ public void testProgressIndexRecorder() { Assert.assertTrue( hybridProgressIndex.isAfter(new RecoverProgressIndex(3, new SimpleProgressIndex(5, 4)))); - Assert.assertTrue( - new MockProgressIndex(0).isAfter(tsFileResource.getMaxProgressIndexAfterClose())); + Assert.assertTrue(new MockProgressIndex(0).isAfter(tsFileResource.getMaxProgressIndex())); indexList.forEach(tsFileResource::updateProgressIndex); + Assert.assertFalse(new MockProgressIndex(-1).isAfter(tsFileResource.getMaxProgressIndex())); + Assert.assertFalse(new MockProgressIndex(0).isAfter(tsFileResource.getMaxProgressIndex())); + Assert.assertFalse(new MockProgressIndex(1).isAfter(tsFileResource.getMaxProgressIndex())); Assert.assertFalse( - new MockProgressIndex(-1).isAfter(tsFileResource.getMaxProgressIndexAfterClose())); - Assert.assertFalse( - new MockProgressIndex(0).isAfter(tsFileResource.getMaxProgressIndexAfterClose())); - Assert.assertFalse( - new MockProgressIndex(1).isAfter(tsFileResource.getMaxProgressIndexAfterClose())); - Assert.assertFalse( - new MockProgressIndex(INDEX_NUM - 1) - .isAfter(tsFileResource.getMaxProgressIndexAfterClose())); + new MockProgressIndex(INDEX_NUM - 1).isAfter(tsFileResource.getMaxProgressIndex())); Assert.assertTrue( - new MockProgressIndex(INDEX_NUM).isAfter(tsFileResource.getMaxProgressIndexAfterClose())); + new MockProgressIndex(INDEX_NUM).isAfter(tsFileResource.getMaxProgressIndex())); Assert.assertTrue( - new MockProgressIndex(Integer.MAX_VALUE) - .isAfter(tsFileResource.getMaxProgressIndexAfterClose())); + new MockProgressIndex(Integer.MAX_VALUE).isAfter(tsFileResource.getMaxProgressIndex())); Assert.assertFalse( - new MockProgressIndex(1, INDEX_NUM - 1) - .isAfter(tsFileResource.getMaxProgressIndexAfterClose())); + new MockProgressIndex(1, INDEX_NUM - 1).isAfter(tsFileResource.getMaxProgressIndex())); } @Test From 6237535a2050e696ca14e6573dda6dab2986ceab Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 1 Jul 2025 12:07:29 +0800 Subject: [PATCH 056/185] Refactor resource Metrics && Fixed the metric multiple creation problem --- .../agent/task/PipeDataNodeTaskAgent.java | 15 ++-- .../processor/PipeProcessorSubtask.java | 4 +- .../common/heartbeat/PipeHeartbeatEvent.java | 6 +- .../PipeInsertNodeTabletInsertionEvent.java | 6 +- .../tablet/PipeRawTabletInsertionEvent.java | 6 +- .../tsfile/PipeTsFileInsertionEvent.java | 6 +- .../dataregion/IoTDBDataRegionExtractor.java | 4 +- ...PipeRealtimeDataRegionHybridExtractor.java | 4 +- .../IoTDBSchemaRegionExtractor.java | 4 +- .../db/pipe/metric/PipeDataNodeMetrics.java | 6 +- ...ava => PipeDataNodeSinglePipeMetrics.java} | 74 ++++++++++++----- .../metric/overview/PipeResourceMetrics.java | 12 --- .../overview/PipeTsFileToTabletsMetrics.java | 3 +- .../tsfile/PipeTsFileResourceManager.java | 83 ++++++------------- .../config/sys/pipe/ShowPipeTask.java | 4 +- .../db/service/DataNodeShutdownHook.java | 6 +- 16 files changed, 116 insertions(+), 127 deletions(-) rename iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/{PipeDataNodeRemainingEventAndTimeMetrics.java => PipeDataNodeSinglePipeMetrics.java} (86%) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java index 1a9d9ff9d566..a05abaf236db 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java @@ -49,7 +49,7 @@ import org.apache.iotdb.db.pipe.extractor.dataregion.DataRegionListeningFilter; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.listener.PipeInsertionDataNodeListener; import org.apache.iotdb.db.pipe.extractor.schemaregion.SchemaRegionListeningFilter; -import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeRemainingEventAndTimeMetrics; +import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeSinglePipeMetrics; import org.apache.iotdb.db.pipe.metric.overview.PipeTsFileToTabletsMetrics; import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; @@ -267,13 +267,12 @@ private void closeSchemaRegionListeningQueueIfNecessary( @Override protected void thawRate(final String pipeName, final long creationTime) { - PipeDataNodeRemainingEventAndTimeMetrics.getInstance().thawRate(pipeName + "_" + creationTime); + PipeDataNodeSinglePipeMetrics.getInstance().thawRate(pipeName + "_" + creationTime); } @Override protected void freezeRate(final String pipeName, final long creationTime) { - PipeDataNodeRemainingEventAndTimeMetrics.getInstance() - .freezeRate(pipeName + "_" + creationTime); + PipeDataNodeSinglePipeMetrics.getInstance().freezeRate(pipeName + "_" + creationTime); } @Override @@ -284,7 +283,7 @@ protected boolean dropPipe(final String pipeName, final long creationTime) { final String taskId = pipeName + "_" + creationTime; PipeTsFileToTabletsMetrics.getInstance().deregister(taskId); - PipeDataNodeRemainingEventAndTimeMetrics.getInstance().deregister(taskId); + PipeDataNodeSinglePipeMetrics.getInstance().deregister(taskId); return true; } @@ -312,7 +311,7 @@ protected boolean dropPipe(final String pipeName) { final long creationTime = pipeMeta.getStaticMeta().getCreationTime(); final String taskId = pipeName + "_" + creationTime; PipeTsFileToTabletsMetrics.getInstance().deregister(taskId); - PipeDataNodeRemainingEventAndTimeMetrics.getInstance().deregister(taskId); + PipeDataNodeSinglePipeMetrics.getInstance().deregister(taskId); // When the pipe contains no pipe tasks, there is no corresponding prefetching queue for the // subscribed pipe, so the subscription needs to be manually marked as completed. if (!hasPipeTasks && PipeStaticMeta.isSubscriptionPipe(pipeName)) { @@ -405,7 +404,7 @@ private void collectPipeMetaListInternal(final TDataNodeHeartbeatResp resp) thro final boolean isCompleted = isAllDataRegionCompleted && includeDataAndNeedDrop; final Pair remainingEventAndTime = - PipeDataNodeRemainingEventAndTimeMetrics.getInstance() + PipeDataNodeSinglePipeMetrics.getInstance() .getRemainingEventAndTime(staticMeta.getPipeName(), staticMeta.getCreationTime()); pipeCompletedList.add(isCompleted); pipeRemainingEventCountList.add(remainingEventAndTime.getLeft()); @@ -488,7 +487,7 @@ protected void collectPipeMetaListInternal( final boolean isCompleted = isAllDataRegionCompleted && includeDataAndNeedDrop; final Pair remainingEventAndTime = - PipeDataNodeRemainingEventAndTimeMetrics.getInstance() + PipeDataNodeSinglePipeMetrics.getInstance() .getRemainingEventAndTime(staticMeta.getPipeName(), staticMeta.getCreationTime()); pipeCompletedList.add(isCompleted); pipeRemainingEventCountList.add(remainingEventAndTime.getLeft()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/processor/PipeProcessorSubtask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/processor/PipeProcessorSubtask.java index c8a0acf32faf..d39feb96fe09 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/processor/PipeProcessorSubtask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/processor/PipeProcessorSubtask.java @@ -31,7 +31,7 @@ import org.apache.iotdb.db.pipe.agent.task.connection.PipeEventCollector; import org.apache.iotdb.db.pipe.event.UserDefinedEnrichedEvent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; -import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeRemainingEventAndTimeMetrics; +import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeSinglePipeMetrics; import org.apache.iotdb.db.pipe.metric.processor.PipeProcessorMetrics; import org.apache.iotdb.db.pipe.processor.pipeconsensus.PipeConsensusProcessor; import org.apache.iotdb.db.storageengine.StorageEngine; @@ -140,7 +140,7 @@ protected boolean executeOnce() throws Exception { } else if (event instanceof TsFileInsertionEvent) { pipeProcessor.process((TsFileInsertionEvent) event, outputEventCollector); PipeProcessorMetrics.getInstance().markTsFileEvent(taskID); - PipeDataNodeRemainingEventAndTimeMetrics.getInstance() + PipeDataNodeSinglePipeMetrics.getInstance() .markTsFileCollectInvocationCount( pipeNameWithCreationTime, outputEventCollector.getCollectInvocationCount()); } else if (event instanceof PipeHeartbeatEvent) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/heartbeat/PipeHeartbeatEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/heartbeat/PipeHeartbeatEvent.java index 19c3b0ca9963..ad29b2854424 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/heartbeat/PipeHeartbeatEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/heartbeat/PipeHeartbeatEvent.java @@ -25,7 +25,7 @@ import org.apache.iotdb.commons.pipe.agent.task.meta.PipeTaskMeta; import org.apache.iotdb.commons.pipe.datastructure.pattern.PipePattern; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; -import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeRemainingEventAndTimeMetrics; +import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeSinglePipeMetrics; import org.apache.iotdb.db.pipe.metric.overview.PipeHeartbeatEventMetrics; import org.apache.iotdb.db.utils.DateTimeUtils; import org.apache.iotdb.pipe.api.event.Event; @@ -83,7 +83,7 @@ public PipeHeartbeatEvent( @Override public boolean internallyIncreaseResourceReferenceCount(final String holderMessage) { if (Objects.nonNull(pipeName)) { - PipeDataNodeRemainingEventAndTimeMetrics.getInstance() + PipeDataNodeSinglePipeMetrics.getInstance() .increaseHeartbeatEventCount(pipeName, creationTime); } return true; @@ -94,7 +94,7 @@ public boolean internallyDecreaseResourceReferenceCount(final String holderMessa // PipeName == null indicates that the event is the raw event at disruptor, // not the event copied and passed to the extractor if (Objects.nonNull(pipeName)) { - PipeDataNodeRemainingEventAndTimeMetrics.getInstance() + PipeDataNodeSinglePipeMetrics.getInstance() .decreaseHeartbeatEventCount(pipeName, creationTime); if (shouldPrintMessage && LOGGER.isDebugEnabled()) { LOGGER.debug(this.toString()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java index 6849699ae4a9..6951a02e6b08 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java @@ -28,7 +28,7 @@ import org.apache.iotdb.commons.pipe.resource.ref.PipePhantomReferenceManager.PipeEventResource; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; import org.apache.iotdb.db.pipe.event.ReferenceTrackableEvent; -import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeRemainingEventAndTimeMetrics; +import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeSinglePipeMetrics; import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; import org.apache.iotdb.db.pipe.resource.memory.InsertNodeMemoryEstimator; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryWeightUtil; @@ -122,7 +122,7 @@ public boolean internallyIncreaseResourceReferenceCount(final String holderMessa extractTime = System.nanoTime(); try { if (Objects.nonNull(pipeName)) { - PipeDataNodeRemainingEventAndTimeMetrics.getInstance() + PipeDataNodeSinglePipeMetrics.getInstance() .increaseInsertNodeEventCount(pipeName, creationTime); PipeDataNodeAgent.task() .addFloatingMemoryUsageInByte(pipeName, creationTime, ramBytesUsed()); @@ -153,7 +153,7 @@ public boolean internallyDecreaseResourceReferenceCount(final String holderMessa if (Objects.nonNull(pipeName)) { PipeDataNodeAgent.task() .decreaseFloatingMemoryUsageInByte(pipeName, creationTime, ramBytesUsed()); - PipeDataNodeRemainingEventAndTimeMetrics.getInstance() + PipeDataNodeSinglePipeMetrics.getInstance() .decreaseInsertNodeEventCount(pipeName, creationTime, System.nanoTime() - extractTime); } insertNode = null; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeRawTabletInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeRawTabletInsertionEvent.java index 95a532708e02..9f3ad7a1bdf5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeRawTabletInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeRawTabletInsertionEvent.java @@ -28,7 +28,7 @@ import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.db.pipe.event.ReferenceTrackableEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; -import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeRemainingEventAndTimeMetrics; +import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeSinglePipeMetrics; import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryWeightUtil; import org.apache.iotdb.db.pipe.resource.memory.PipeTabletMemoryBlock; @@ -130,7 +130,7 @@ public boolean internallyIncreaseResourceReferenceCount(final String holderMessa allocatedMemoryBlock, PipeMemoryWeightUtil.calculateTabletSizeInBytes(tablet) + INSTANCE_SIZE); if (Objects.nonNull(pipeName)) { - PipeDataNodeRemainingEventAndTimeMetrics.getInstance() + PipeDataNodeSinglePipeMetrics.getInstance() .increaseRawTabletEventCount(pipeName, creationTime); } return true; @@ -139,7 +139,7 @@ public boolean internallyIncreaseResourceReferenceCount(final String holderMessa @Override public boolean internallyDecreaseResourceReferenceCount(final String holderMessage) { if (Objects.nonNull(pipeName)) { - PipeDataNodeRemainingEventAndTimeMetrics.getInstance() + PipeDataNodeSinglePipeMetrics.getInstance() .decreaseRawTabletEventCount(pipeName, creationTime); } allocatedMemoryBlock.close(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java index 17fa19083a51..8ae8fbd059b9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java @@ -32,7 +32,7 @@ import org.apache.iotdb.db.pipe.event.common.tsfile.container.TsFileInsertionDataContainer; import org.apache.iotdb.db.pipe.event.common.tsfile.container.TsFileInsertionDataContainerProvider; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.assigner.PipeTsFileEpochProgressIndexKeeper; -import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeRemainingEventAndTimeMetrics; +import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeSinglePipeMetrics; import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryManager; import org.apache.iotdb.db.pipe.resource.tsfile.PipeTsFileResourceManager; @@ -266,7 +266,7 @@ public boolean internallyIncreaseResourceReferenceCount(final String holderMessa return false; } finally { if (Objects.nonNull(pipeName)) { - PipeDataNodeRemainingEventAndTimeMetrics.getInstance() + PipeDataNodeSinglePipeMetrics.getInstance() .increaseTsFileEventCount(pipeName, creationTime); } } @@ -290,7 +290,7 @@ public boolean internallyDecreaseResourceReferenceCount(final String holderMessa return false; } finally { if (Objects.nonNull(pipeName)) { - PipeDataNodeRemainingEventAndTimeMetrics.getInstance() + PipeDataNodeSinglePipeMetrics.getInstance() .decreaseTsFileEventCount(pipeName, creationTime, System.nanoTime() - extractTime); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java index a37f268d486d..3d5b6160dacf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java @@ -34,7 +34,7 @@ import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionHybridExtractor; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionLogExtractor; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionTsFileExtractor; -import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeRemainingEventAndTimeMetrics; +import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeSinglePipeMetrics; import org.apache.iotdb.db.pipe.metric.overview.PipeTsFileToTabletsMetrics; import org.apache.iotdb.db.pipe.metric.source.PipeDataRegionExtractorMetrics; import org.apache.iotdb.db.storageengine.StorageEngine; @@ -342,7 +342,7 @@ public void customize( // register metric after generating taskID PipeDataRegionExtractorMetrics.getInstance().register(this); PipeTsFileToTabletsMetrics.getInstance().register(this); - PipeDataNodeRemainingEventAndTimeMetrics.getInstance().register(this); + PipeDataNodeSinglePipeMetrics.getInstance().register(this); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java index 92212eaad63a..7a6177007941 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java @@ -298,14 +298,14 @@ private boolean mayTsFileLinkedCountReachDangerousThreshold(final PipeRealtimeEv return false; } final boolean mayTsFileLinkedCountReachDangerousThreshold = - PipeDataNodeResourceManager.tsfile().getLinkedTsfileCount() + PipeDataNodeResourceManager.tsfile().getLinkedTsFileCount() >= PipeConfig.getInstance().getPipeMaxAllowedLinkedTsFileCount(); if (mayTsFileLinkedCountReachDangerousThreshold && event.mayExtractorUseTablets(this)) { LOGGER.info( "Pipe task {}@{} canNotUseTabletAnymoreDeprecated(4): The number of linked tsFiles {} has reached the dangerous threshold {}", pipeName, dataRegionId, - PipeDataNodeResourceManager.tsfile().getLinkedTsfileCount(), + PipeDataNodeResourceManager.tsfile().getLinkedTsFileCount(), PipeConfig.getInstance().getPipeMaxAllowedLinkedTsFileCount()); } return mayTsFileLinkedCountReachDangerousThreshold; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/schemaregion/IoTDBSchemaRegionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/schemaregion/IoTDBSchemaRegionExtractor.java index b1153071bab4..92fc23775e31 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/schemaregion/IoTDBSchemaRegionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/schemaregion/IoTDBSchemaRegionExtractor.java @@ -32,7 +32,7 @@ import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionSnapshotEvent; import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; -import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeRemainingEventAndTimeMetrics; +import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeSinglePipeMetrics; import org.apache.iotdb.db.pipe.metric.schema.PipeSchemaRegionExtractorMetrics; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; @@ -75,7 +75,7 @@ public void customize( listenedTypeSet = SchemaRegionListeningFilter.parseListeningPlanTypeSet(parameters); PipeSchemaRegionExtractorMetrics.getInstance().register(this); - PipeDataNodeRemainingEventAndTimeMetrics.getInstance().register(this); + PipeDataNodeSinglePipeMetrics.getInstance().register(this); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeMetrics.java index 1358a0bc08f1..d7f1577a85c6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeMetrics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeMetrics.java @@ -20,7 +20,7 @@ package org.apache.iotdb.db.pipe.metric; import org.apache.iotdb.commons.pipe.metric.PipeEventCommitMetrics; -import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeRemainingEventAndTimeMetrics; +import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeSinglePipeMetrics; import org.apache.iotdb.db.pipe.metric.overview.PipeHeartbeatEventMetrics; import org.apache.iotdb.db.pipe.metric.overview.PipeResourceMetrics; import org.apache.iotdb.db.pipe.metric.overview.PipeTsFileToTabletsMetrics; @@ -51,7 +51,7 @@ public void bindTo(final AbstractMetricService metricService) { PipeSchemaRegionListenerMetrics.getInstance().bindTo(metricService); PipeSchemaRegionExtractorMetrics.getInstance().bindTo(metricService); PipeSchemaRegionConnectorMetrics.getInstance().bindTo(metricService); - PipeDataNodeRemainingEventAndTimeMetrics.getInstance().bindTo(metricService); + PipeDataNodeSinglePipeMetrics.getInstance().bindTo(metricService); PipeDataNodeReceiverMetrics.getInstance().bindTo(metricService); PipeTsFileToTabletsMetrics.getInstance().bindTo(metricService); } @@ -68,7 +68,7 @@ public void unbindFrom(final AbstractMetricService metricService) { PipeSchemaRegionListenerMetrics.getInstance().unbindFrom(metricService); PipeSchemaRegionExtractorMetrics.getInstance().unbindFrom(metricService); PipeSchemaRegionConnectorMetrics.getInstance().unbindFrom(metricService); - PipeDataNodeRemainingEventAndTimeMetrics.getInstance().unbindFrom(metricService); + PipeDataNodeSinglePipeMetrics.getInstance().unbindFrom(metricService); PipeDataNodeReceiverMetrics.getInstance().unbindFrom(metricService); PipeTsFileToTabletsMetrics.getInstance().unbindFrom(metricService); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeSinglePipeMetrics.java similarity index 86% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeMetrics.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeSinglePipeMetrics.java index 2e1bab9faf06..534958e90112 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeRemainingEventAndTimeMetrics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeSinglePipeMetrics.java @@ -25,6 +25,7 @@ import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; import org.apache.iotdb.db.pipe.extractor.dataregion.IoTDBDataRegionExtractor; import org.apache.iotdb.db.pipe.extractor.schemaregion.IoTDBSchemaRegionExtractor; +import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; import org.apache.iotdb.metrics.AbstractMetricService; import org.apache.iotdb.metrics.impl.DoNothingMetricManager; import org.apache.iotdb.metrics.metricsets.IMetricSet; @@ -42,10 +43,9 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; -public class PipeDataNodeRemainingEventAndTimeMetrics implements IMetricSet { +public class PipeDataNodeSinglePipeMetrics implements IMetricSet { - private static final Logger LOGGER = - LoggerFactory.getLogger(PipeDataNodeRemainingEventAndTimeMetrics.class); + private static final Logger LOGGER = LoggerFactory.getLogger(PipeDataNodeSinglePipeMetrics.class); @SuppressWarnings("java:S3077") private volatile AbstractMetricService metricService; @@ -103,6 +103,8 @@ private void createAutoGauge(final String pipeID) { operator.getPipeName(), Tag.CREATION_TIME.toString(), String.valueOf(operator.getCreationTime())); + + // Resources metricService.createAutoGauge( Metric.PIPE_FLOATING_MEMORY_USAGE.toString(), MetricLevel.IMPORTANT, @@ -112,6 +114,24 @@ private void createAutoGauge(final String pipeID) { operator.getPipeName(), Tag.CREATION_TIME.toString(), String.valueOf(operator.getCreationTime())); + metricService.createAutoGauge( + Metric.PIPE_LINKED_TSFILE_COUNT.toString(), + MetricLevel.IMPORTANT, + PipeDataNodeResourceManager.tsfile(), + a -> a.getLinkedTsFileCount(operator.getPipeName()), + Tag.NAME.toString(), + operator.getPipeName(), + Tag.CREATION_TIME.toString(), + String.valueOf(operator.getCreationTime())); + metricService.createAutoGauge( + Metric.PIPE_LINKED_TSFILE_SIZE.toString(), + MetricLevel.IMPORTANT, + PipeDataNodeResourceManager.tsfile(), + a -> a.getTotalLinkedTsFileSize(operator.getPipeName()), + Tag.NAME.toString(), + operator.getPipeName(), + Tag.CREATION_TIME.toString(), + String.valueOf(operator.getCreationTime())); operator.setInsertNodeTransferTimer( metricService.getOrCreateTimer( @@ -176,6 +196,20 @@ private void removeAutoGauge(final String pipeID) { operator.getPipeName(), Tag.CREATION_TIME.toString(), String.valueOf(operator.getCreationTime())); + metricService.remove( + MetricType.AUTO_GAUGE, + Metric.PIPE_LINKED_TSFILE_COUNT.toString(), + Tag.NAME.toString(), + operator.getPipeName(), + Tag.CREATION_TIME.toString(), + String.valueOf(operator.getCreationTime())); + metricService.remove( + MetricType.AUTO_GAUGE, + Metric.PIPE_LINKED_TSFILE_SIZE.toString(), + Tag.NAME.toString(), + operator.getPipeName(), + Tag.CREATION_TIME.toString(), + String.valueOf(operator.getCreationTime())); metricService.remove( MetricType.TIMER, Metric.PIPE_INSERT_NODE_EVENT_TRANSFER_TIME.toString(), @@ -196,12 +230,13 @@ public void register(final IoTDBDataRegionExtractor extractor) { final String pipeID = extractor.getPipeName() + "_" + extractor.getCreationTime(); remainingEventAndTimeOperatorMap.computeIfAbsent( pipeID, - k -> - new PipeDataNodeRemainingEventAndTimeOperator( - extractor.getPipeName(), extractor.getCreationTime())); - if (Objects.nonNull(metricService)) { - createMetrics(pipeID); - } + k -> { + if (Objects.nonNull(metricService)) { + createMetrics(pipeID); + } + return new PipeDataNodeRemainingEventAndTimeOperator( + extractor.getPipeName(), extractor.getCreationTime()); + }); } public void register(final IoTDBSchemaRegionExtractor extractor) { @@ -210,13 +245,14 @@ public void register(final IoTDBSchemaRegionExtractor extractor) { remainingEventAndTimeOperatorMap .computeIfAbsent( pipeID, - k -> - new PipeDataNodeRemainingEventAndTimeOperator( - extractor.getPipeName(), extractor.getCreationTime())) + k -> { + if (Objects.nonNull(metricService)) { + createMetrics(pipeID); + } + return new PipeDataNodeRemainingEventAndTimeOperator( + extractor.getPipeName(), extractor.getCreationTime()); + }) .register(extractor); - if (Objects.nonNull(metricService)) { - createMetrics(pipeID); - } } public void increaseInsertNodeEventCount(final String pipeName, final long creationTime) { @@ -372,19 +408,19 @@ public Pair getRemainingEventAndTime( private static class PipeDataNodeRemainingEventAndTimeMetricsHolder { - private static final PipeDataNodeRemainingEventAndTimeMetrics INSTANCE = - new PipeDataNodeRemainingEventAndTimeMetrics(); + private static final PipeDataNodeSinglePipeMetrics INSTANCE = + new PipeDataNodeSinglePipeMetrics(); private PipeDataNodeRemainingEventAndTimeMetricsHolder() { // Empty constructor } } - public static PipeDataNodeRemainingEventAndTimeMetrics getInstance() { + public static PipeDataNodeSinglePipeMetrics getInstance() { return PipeDataNodeRemainingEventAndTimeMetricsHolder.INSTANCE; } - private PipeDataNodeRemainingEventAndTimeMetrics() { + private PipeDataNodeSinglePipeMetrics() { PipeEventCommitManager.getInstance().setCommitRateMarker(this::markRegionCommit); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeResourceMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeResourceMetrics.java index d3c823187d5a..37f8eb5e26aa 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeResourceMetrics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeResourceMetrics.java @@ -25,7 +25,6 @@ import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryManager; -import org.apache.iotdb.db.pipe.resource.tsfile.PipeTsFileResourceManager; import org.apache.iotdb.metrics.AbstractMetricService; import org.apache.iotdb.metrics.metricsets.IMetricSet; import org.apache.iotdb.metrics.utils.MetricLevel; @@ -91,17 +90,6 @@ public void bindTo(final AbstractMetricService metricService) { o -> PipeDataNodeAgent.task().getAllFloatingMemoryUsageInByte(), Tag.NAME.toString(), PIPE_USED_FLOATING_MEMORY); - // resource reference count - metricService.createAutoGauge( - Metric.PIPE_LINKED_TSFILE_COUNT.toString(), - MetricLevel.IMPORTANT, - PipeDataNodeResourceManager.tsfile(), - PipeTsFileResourceManager::getLinkedTsfileCount); - metricService.createAutoGauge( - Metric.PIPE_LINKED_TSFILE_SIZE.toString(), - MetricLevel.IMPORTANT, - PipeDataNodeResourceManager.tsfile(), - PipeTsFileResourceManager::getTotalLinkedTsfileSize); // phantom reference count metricService.createAutoGauge( Metric.PIPE_PHANTOM_REFERENCE_COUNT.toString(), diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeTsFileToTabletsMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeTsFileToTabletsMetrics.java index 04015fe6f82d..16108b36c5f4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeTsFileToTabletsMetrics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeTsFileToTabletsMetrics.java @@ -104,8 +104,7 @@ private void removeMetrics(final String pipeID) { public void register(final IoTDBDataRegionExtractor extractor) { final String pipeID = extractor.getPipeName() + "_" + extractor.getCreationTime(); - pipe.add(pipeID); - if (Objects.nonNull(metricService)) { + if (Objects.nonNull(metricService) && pipe.add(pipeID)) { createMetrics(pipeID); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java index a61f59af8b27..4eaac14c574b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java @@ -43,8 +43,10 @@ public class PipeTsFileResourceManager { private static final Logger LOGGER = LoggerFactory.getLogger(PipeTsFileResourceManager.class); - private final Map hardlinkOrCopiedFileToPipeTsFileResourceMap = + private final Map hardlinkOrCopiedFileToAssignerTsFileResourceMap = new ConcurrentHashMap<>(); + private final Map> + hardlinkOrCopiedFileToPipeTsFileResourceMap = new ConcurrentHashMap<>(); private final PipeTsFileResourceSegmentLock segmentLock = new PipeTsFileResourceSegmentLock(); /** @@ -78,7 +80,7 @@ public File increaseFileReference( // just increase reference count and return it segmentLock.lock(file); try { - if (increaseReferenceIfExists(file)) { + if (increaseReferenceIfExists(file, pipeName)) { return file; } } finally { @@ -90,8 +92,8 @@ public File increaseFileReference( final File hardlinkOrCopiedFile = getHardlinkOrCopiedFileInPipeDir(file, pipeName); segmentLock.lock(hardlinkOrCopiedFile); try { - if (increaseReferenceIfExists(hardlinkOrCopiedFile)) { - return hardlinkOrCopiedFileToPipeTsFileResourceMap + if (increaseReferenceIfExists(hardlinkOrCopiedFile, pipeName)) { + return getHardlinkOrCopiedFile2TsFileResourceMap(pipeName) .get(hardlinkOrCopiedFile.getPath()) .getFile(); } @@ -106,17 +108,17 @@ public File increaseFileReference( // If the file is not a hardlink or copied file, and there is no related hardlink or copied // file in pipe dir, create a hardlink or copy it to pipe dir, maintain a reference count for // the hardlink or copied file, and return the hardlink or copied file. - hardlinkOrCopiedFileToPipeTsFileResourceMap.put( - resultFile.getPath(), new PipeTsFileResource(resultFile, isTsFile, tsFileResource)); + getHardlinkOrCopiedFile2TsFileResourceMap(pipeName) + .put(resultFile.getPath(), new PipeTsFileResource(resultFile, isTsFile, tsFileResource)); return resultFile; } finally { segmentLock.unlock(hardlinkOrCopiedFile); } } - private boolean increaseReferenceIfExists(final File file) { + private boolean increaseReferenceIfExists(final File file, final String pipeName) { final PipeTsFileResource resource = - hardlinkOrCopiedFileToPipeTsFileResourceMap.get(file.getPath()); + getHardlinkOrCopiedFile2TsFileResourceMap(pipeName).get(file.getPath()); if (resource != null) { resource.increaseReferenceCount(); return true; @@ -175,9 +177,10 @@ public void decreaseFileReference( segmentLock.lock(hardlinkOrCopiedFile); try { final String filePath = hardlinkOrCopiedFile.getPath(); - final PipeTsFileResource resource = hardlinkOrCopiedFileToPipeTsFileResourceMap.get(filePath); + final PipeTsFileResource resource = + getHardlinkOrCopiedFile2TsFileResourceMap(pipeName).get(filePath); if (resource != null && resource.decreaseReferenceCount()) { - hardlinkOrCopiedFileToPipeTsFileResourceMap.remove(filePath); + getHardlinkOrCopiedFile2TsFileResourceMap(pipeName).remove(filePath); } } finally { segmentLock.unlock(hardlinkOrCopiedFile); @@ -190,11 +193,12 @@ public void decreaseFileReference( * @param hardlinkOrCopiedFile the copied or hardlinked file * @return the reference count of the file */ - public int getFileReferenceCount(final File hardlinkOrCopiedFile) { + public int getFileReferenceCount(final File hardlinkOrCopiedFile, final String pipeName) { segmentLock.lock(hardlinkOrCopiedFile); try { final String filePath = hardlinkOrCopiedFile.getPath(); - final PipeTsFileResource resource = hardlinkOrCopiedFileToPipeTsFileResourceMap.get(filePath); + final PipeTsFileResource resource = + getHardlinkOrCopiedFile2TsFileResourceMap(pipeName).get(filePath); return resource != null ? resource.getReferenceCount() : 0; } finally { segmentLock.unlock(hardlinkOrCopiedFile); @@ -274,12 +278,12 @@ public void unpinTsFileResource(final TsFileResource resource, final String pipe } } - public int getLinkedTsfileCount() { - return hardlinkOrCopiedFileToPipeTsFileResourceMap.size(); + public int getLinkedTsFileCount(final String pipeName) { + return getHardlinkOrCopiedFile2TsFileResourceMap(pipeName).size(); } - public long getTotalLinkedTsfileSize() { - return hardlinkOrCopiedFileToPipeTsFileResourceMap.values().stream() + public long getTotalLinkedTsFileSize(final String pipeName) { + return getHardlinkOrCopiedFile2TsFileResourceMap(pipeName).values().stream() .mapToLong( resource -> { try { @@ -292,46 +296,11 @@ public long getTotalLinkedTsfileSize() { .sum(); } - /** - * Get the total size of linked TsFiles whose original TsFile is deleted (by compaction or else) - */ - public long getTotalLinkedButDeletedTsfileSize() { - try { - return hardlinkOrCopiedFileToPipeTsFileResourceMap.values().parallelStream() - .filter(PipeTsFileResource::isOriginalTsFileDeleted) - .mapToLong( - resource -> { - try { - return resource.getFileSize(); - } catch (Exception e) { - LOGGER.warn( - "failed to get file size of linked but deleted TsFile {}: ", resource, e); - return 0; - } - }) - .sum(); - } catch (final Exception e) { - LOGGER.warn("failed to get total size of linked but deleted TsFiles: ", e); - return 0; - } - } - - public long getTotalLinkedButDeletedTsFileResourceRamSize() { - long totalLinkedButDeletedTsfileResourceRamSize = 0; - try { - for (final Map.Entry resourceEntry : - hardlinkOrCopiedFileToPipeTsFileResourceMap.entrySet()) { - final PipeTsFileResource pipeTsFileResource = resourceEntry.getValue(); - // If the original TsFile is not deleted, the memory of the resource is not counted - // because the memory of the resource is controlled by TsFileResourceManager. - if (pipeTsFileResource.isOriginalTsFileDeleted()) { - totalLinkedButDeletedTsfileResourceRamSize += pipeTsFileResource.getTsFileResourceSize(); - } - } - return totalLinkedButDeletedTsfileResourceRamSize; - } catch (final Exception e) { - LOGGER.warn("failed to get total size of linked but deleted TsFiles resource ram size: ", e); - return totalLinkedButDeletedTsfileResourceRamSize; - } + private Map getHardlinkOrCopiedFile2TsFileResourceMap( + final String pipeName) { + return Objects.nonNull(pipeName) + ? hardlinkOrCopiedFileToPipeTsFileResourceMap.computeIfAbsent( + pipeName, pipe -> new ConcurrentHashMap<>()) + : hardlinkOrCopiedFileToAssignerTsFileResourceMap; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/sys/pipe/ShowPipeTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/sys/pipe/ShowPipeTask.java index 687a438f9b3f..16ee005a089b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/sys/pipe/ShowPipeTask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/sys/pipe/ShowPipeTask.java @@ -20,7 +20,7 @@ package org.apache.iotdb.db.queryengine.plan.execution.config.sys.pipe; import org.apache.iotdb.confignode.rpc.thrift.TShowPipeInfo; -import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeRemainingEventAndTimeMetrics; +import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeSinglePipeMetrics; import org.apache.iotdb.db.queryengine.common.header.ColumnHeader; import org.apache.iotdb.db.queryengine.common.header.ColumnHeaderConstant; import org.apache.iotdb.db.queryengine.common.header.DatasetHeader; @@ -97,7 +97,7 @@ public static void buildTSBlock( if (remainingEventCount == -1 && remainingTime == -1) { final Pair remainingEventAndTime = - PipeDataNodeRemainingEventAndTimeMetrics.getInstance() + PipeDataNodeSinglePipeMetrics.getInstance() .getRemainingEventAndTime(tPipeInfo.getId(), tPipeInfo.getCreationTime()); remainingEventCount = remainingEventAndTime.getLeft(); remainingTime = remainingEventAndTime.getRight(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java index 57d98df49c4a..3c4cc1b7f3d5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java @@ -31,8 +31,8 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.consensus.DataRegionConsensusImpl; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; -import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeRemainingEventAndTimeMetrics; import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeRemainingEventAndTimeOperator; +import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeSinglePipeMetrics; import org.apache.iotdb.db.protocol.client.ConfigNodeClient; import org.apache.iotdb.db.protocol.client.ConfigNodeClientManager; import org.apache.iotdb.db.protocol.client.ConfigNodeInfo; @@ -95,9 +95,7 @@ public void run() { long startTime = System.currentTimeMillis(); if (PipeDataNodeAgent.task().getPipeCount() != 0) { for (Map.Entry entry : - PipeDataNodeRemainingEventAndTimeMetrics.getInstance() - .remainingEventAndTimeOperatorMap - .entrySet()) { + PipeDataNodeSinglePipeMetrics.getInstance().remainingEventAndTimeOperatorMap.entrySet()) { while (entry.getValue().getRemainingNonHeartbeatEvents() > 0) { if (System.currentTimeMillis() - startTime > PipeConfig.getInstance().getPipeMaxWaitFinishTime()) { From e030052084077637fa9259e7dc7492ac2d66b397 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 1 Jul 2025 12:11:07 +0800 Subject: [PATCH 057/185] Fixed tsFile manager test --- .../PipeTsFileResourceManagerTest.java | 34 ++++++++++--------- 1 file changed, 18 insertions(+), 16 deletions(-) diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java index f54b10e09538..f9aeb299f071 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java @@ -160,15 +160,17 @@ public void tearDown() throws Exception { public void testIncreaseTsfile() throws IOException { File originTsfile = new File(TS_FILE_NAME); File originModFile = new File(MODS_FILE_NAME); - Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(originTsfile)); - Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(originModFile)); + Assert.assertEquals( + 0, pipeTsFileResourceManager.getFileReferenceCount(originTsfile, PIPE_NAME)); + Assert.assertEquals( + 0, pipeTsFileResourceManager.getFileReferenceCount(originModFile, PIPE_NAME)); File pipeTsfile = pipeTsFileResourceManager.increaseFileReference(originTsfile, true, null, PIPE_NAME); File pipeModFile = pipeTsFileResourceManager.increaseFileReference(originModFile, false, null, PIPE_NAME); - Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile)); - Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile)); + Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, PIPE_NAME)); + Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, PIPE_NAME)); Assert.assertTrue(Files.exists(originTsfile.toPath())); Assert.assertTrue(Files.exists(originModFile.toPath())); Assert.assertTrue(Files.exists(pipeTsfile.toPath())); @@ -176,19 +178,19 @@ public void testIncreaseTsfile() throws IOException { pipeTsFileResourceManager.increaseFileReference(originTsfile, true, null, PIPE_NAME); pipeTsFileResourceManager.increaseFileReference(originModFile, false, null, PIPE_NAME); - Assert.assertEquals(2, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile)); - Assert.assertEquals(2, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile)); + Assert.assertEquals(2, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, PIPE_NAME)); + Assert.assertEquals(2, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, PIPE_NAME)); // test use hardlinkTsFile to increase reference counts // test null, shall not reuse the pipe's tsFile pipeTsFileResourceManager.increaseFileReference(pipeTsfile, true, null, null); - Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile)); + Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, null)); Assert.assertTrue(Files.exists(originTsfile.toPath())); Assert.assertTrue(Files.exists(pipeTsfile.toPath())); // test use copyFile to increase reference counts pipeTsFileResourceManager.increaseFileReference(pipeModFile, false, null, PIPE_NAME); - Assert.assertEquals(3, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile)); + Assert.assertEquals(3, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, PIPE_NAME)); Assert.assertTrue(Files.exists(originModFile.toPath())); Assert.assertTrue(Files.exists(pipeModFile.toPath())); } @@ -200,14 +202,14 @@ public void testDecreaseTsFile() throws IOException { pipeTsFileResourceManager.decreaseFileReference(originFile, null); pipeTsFileResourceManager.decreaseFileReference(originModFile, null); - Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(originFile)); - Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(originModFile)); + Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(originFile, null)); + Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(originModFile, null)); File pipeTsfile = pipeTsFileResourceManager.increaseFileReference(originFile, true, null, null); File pipeModFile = pipeTsFileResourceManager.increaseFileReference(originModFile, false, null, null); - Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile)); - Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile)); + Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, null)); + Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, null)); Assert.assertTrue(Files.exists(pipeTsfile.toPath())); Assert.assertTrue(Files.exists(pipeModFile.toPath())); Assert.assertTrue(Files.exists(pipeTsfile.toPath())); @@ -218,8 +220,8 @@ public void testDecreaseTsFile() throws IOException { Assert.assertFalse(Files.exists(originFile.toPath())); Assert.assertFalse(Files.exists(originModFile.toPath())); - Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile)); - Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile)); + Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, null)); + Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, null)); Assert.assertFalse(Files.exists(originFile.toPath())); Assert.assertFalse(Files.exists(originModFile.toPath())); Assert.assertTrue(Files.exists(pipeTsfile.toPath())); @@ -227,8 +229,8 @@ public void testDecreaseTsFile() throws IOException { pipeTsFileResourceManager.decreaseFileReference(pipeTsfile, null); pipeTsFileResourceManager.decreaseFileReference(pipeModFile, null); - Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile)); - Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile)); + Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, null)); + Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, null)); Assert.assertFalse(Files.exists(originFile.toPath())); Assert.assertFalse(Files.exists(originModFile.toPath())); // Pipe TsFile will be cleaned by a timed thread, so we wait some time here. From da1752ee9d8606bf975f809aac2e8f9a5ce2b9a8 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 1 Jul 2025 14:16:05 +0800 Subject: [PATCH 058/185] try refactor --- .../tsfile/PipeTsFileInsertionEvent.java | 7 ++---- .../resource/tsfile/PipeTsFileResource.java | 19 +------------- .../tsfile/PipeTsFileResourceManager.java | 25 ++++++++----------- .../PipeTsFileResourceManagerTest.java | 19 +++++++------- 4 files changed, 23 insertions(+), 47 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java index 8ae8fbd059b9..cf16fd2b32ea 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java @@ -248,13 +248,10 @@ public long getTimePartitionId() { public boolean internallyIncreaseResourceReferenceCount(final String holderMessage) { extractTime = System.nanoTime(); try { - tsFile = - PipeDataNodeResourceManager.tsfile() - .increaseFileReference(tsFile, true, resource, pipeName); + tsFile = PipeDataNodeResourceManager.tsfile().increaseFileReference(tsFile, true, pipeName); if (isWithMod) { modFile = - PipeDataNodeResourceManager.tsfile() - .increaseFileReference(modFile, false, null, pipeName); + PipeDataNodeResourceManager.tsfile().increaseFileReference(modFile, false, pipeName); } return true; } catch (final Exception e) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResource.java index 3d4154bc1b7f..4e05d1c80160 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResource.java @@ -23,8 +23,6 @@ import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryBlock; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryWeightUtil; -import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; -import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResourceStatus; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.file.metadata.IDeviceID; @@ -53,9 +51,6 @@ public class PipeTsFileResource implements AutoCloseable { private final File hardlinkOrCopiedFile; private final boolean isTsFile; - /** this TsFileResource is used to track the {@link TsFileResourceStatus} of original TsFile. * */ - private final TsFileResource tsFileResource; - private volatile long fileSize = -1L; private final AtomicInteger referenceCount; @@ -64,13 +59,9 @@ public class PipeTsFileResource implements AutoCloseable { private Map deviceIsAlignedMap = null; private Map measurementDataTypeMap = null; - public PipeTsFileResource( - final File hardlinkOrCopiedFile, - final boolean isTsFile, - final TsFileResource tsFileResource) { + public PipeTsFileResource(final File hardlinkOrCopiedFile, final boolean isTsFile) { this.hardlinkOrCopiedFile = hardlinkOrCopiedFile; this.isTsFile = isTsFile; - this.tsFileResource = tsFileResource; referenceCount = new AtomicInteger(1); } @@ -79,10 +70,6 @@ public File getFile() { return hardlinkOrCopiedFile; } - public boolean isOriginalTsFileDeleted() { - return isTsFile && Objects.nonNull(tsFileResource) && tsFileResource.isDeleted(); - } - public long getFileSize() { if (fileSize == -1L) { synchronized (this) { @@ -94,10 +81,6 @@ public long getFileSize() { return fileSize; } - public long getTsFileResourceSize() { - return Objects.nonNull(tsFileResource) ? tsFileResource.calculateRamSize() : 0; - } - ///////////////////// Reference Count ///////////////////// public int getReferenceCount() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java index 4eaac14c574b..17397eedcc81 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java @@ -43,6 +43,9 @@ public class PipeTsFileResourceManager { private static final Logger LOGGER = LoggerFactory.getLogger(PipeTsFileResourceManager.class); + // This is used to hold the assigner pinned tsFiles. + // Also, it is used to provide metadata cache of the tsFile, and is shared by all the pipe's + // tsFiles. private final Map hardlinkOrCopiedFileToAssignerTsFileResourceMap = new ConcurrentHashMap<>(); private final Map> @@ -65,17 +68,11 @@ public class PipeTsFileResourceManager { * @param file tsfile, resource file or mod file. can be original file or hardlink/copy of * original file * @param isTsFile {@code true} to create hardlink, {@code false} to copy file - * @param tsFileResource the TsFileResource of original TsFile. Ignored if {@param isTsFile} is - * {@code false}. * @return the hardlink or copied file * @throws IOException when create hardlink or copy file failed */ public File increaseFileReference( - final File file, - final boolean isTsFile, - final TsFileResource tsFileResource, - final @Nullable String pipeName) - throws IOException { + final File file, final boolean isTsFile, final @Nullable String pipeName) throws IOException { // If the file is already a hardlink or copied file, // just increase reference count and return it segmentLock.lock(file); @@ -109,7 +106,7 @@ public File increaseFileReference( // file in pipe dir, create a hardlink or copy it to pipe dir, maintain a reference count for // the hardlink or copied file, and return the hardlink or copied file. getHardlinkOrCopiedFile2TsFileResourceMap(pipeName) - .put(resultFile.getPath(), new PipeTsFileResource(resultFile, isTsFile, tsFileResource)); + .put(resultFile.getPath(), new PipeTsFileResource(resultFile, isTsFile)); return resultFile; } finally { segmentLock.unlock(hardlinkOrCopiedFile); @@ -215,7 +212,7 @@ public boolean cacheObjectsIfAbsent(final File hardlinkOrCopiedTsFile) throws IO segmentLock.lock(hardlinkOrCopiedTsFile); try { final PipeTsFileResource resource = - hardlinkOrCopiedFileToPipeTsFileResourceMap.get(hardlinkOrCopiedTsFile.getPath()); + hardlinkOrCopiedFileToAssignerTsFileResourceMap.get(hardlinkOrCopiedTsFile.getPath()); return resource != null && resource.cacheObjectsIfAbsent(); } finally { segmentLock.unlock(hardlinkOrCopiedTsFile); @@ -227,7 +224,7 @@ public Map> getDeviceMeasurementsMapFromCache( segmentLock.lock(hardlinkOrCopiedTsFile); try { final PipeTsFileResource resource = - hardlinkOrCopiedFileToPipeTsFileResourceMap.get(hardlinkOrCopiedTsFile.getPath()); + hardlinkOrCopiedFileToAssignerTsFileResourceMap.get(hardlinkOrCopiedTsFile.getPath()); return resource == null ? null : resource.tryGetDeviceMeasurementsMap(); } finally { segmentLock.unlock(hardlinkOrCopiedTsFile); @@ -239,7 +236,7 @@ public Map getDeviceIsAlignedMapFromCache( segmentLock.lock(hardlinkOrCopiedTsFile); try { final PipeTsFileResource resource = - hardlinkOrCopiedFileToPipeTsFileResourceMap.get(hardlinkOrCopiedTsFile.getPath()); + hardlinkOrCopiedFileToAssignerTsFileResourceMap.get(hardlinkOrCopiedTsFile.getPath()); return resource == null ? null : resource.tryGetDeviceIsAlignedMap(cacheOtherMetadata); } finally { segmentLock.unlock(hardlinkOrCopiedTsFile); @@ -251,7 +248,7 @@ public Map getMeasurementDataTypeMapFromCache( segmentLock.lock(hardlinkOrCopiedTsFile); try { final PipeTsFileResource resource = - hardlinkOrCopiedFileToPipeTsFileResourceMap.get(hardlinkOrCopiedTsFile.getPath()); + hardlinkOrCopiedFileToAssignerTsFileResourceMap.get(hardlinkOrCopiedTsFile.getPath()); return resource == null ? null : resource.tryGetMeasurementDataTypeMap(); } finally { segmentLock.unlock(hardlinkOrCopiedTsFile); @@ -261,9 +258,9 @@ public Map getMeasurementDataTypeMapFromCache( public void pinTsFileResource( final TsFileResource resource, final boolean withMods, final String pipeName) throws IOException { - increaseFileReference(resource.getTsFile(), true, resource, pipeName); + increaseFileReference(resource.getTsFile(), true, pipeName); if (withMods && resource.getModFile().exists()) { - increaseFileReference(new File(resource.getModFile().getFilePath()), false, null, pipeName); + increaseFileReference(new File(resource.getModFile().getFilePath()), false, pipeName); } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java index f9aeb299f071..ec00651d30d8 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java @@ -157,7 +157,7 @@ public void tearDown() throws Exception { } @Test - public void testIncreaseTsfile() throws IOException { + public void testIncreaseTsFile() throws IOException { File originTsfile = new File(TS_FILE_NAME); File originModFile = new File(MODS_FILE_NAME); Assert.assertEquals( @@ -166,9 +166,9 @@ public void testIncreaseTsfile() throws IOException { 0, pipeTsFileResourceManager.getFileReferenceCount(originModFile, PIPE_NAME)); File pipeTsfile = - pipeTsFileResourceManager.increaseFileReference(originTsfile, true, null, PIPE_NAME); + pipeTsFileResourceManager.increaseFileReference(originTsfile, true, PIPE_NAME); File pipeModFile = - pipeTsFileResourceManager.increaseFileReference(originModFile, false, null, PIPE_NAME); + pipeTsFileResourceManager.increaseFileReference(originModFile, false, PIPE_NAME); Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, PIPE_NAME)); Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, PIPE_NAME)); Assert.assertTrue(Files.exists(originTsfile.toPath())); @@ -176,20 +176,20 @@ public void testIncreaseTsfile() throws IOException { Assert.assertTrue(Files.exists(pipeTsfile.toPath())); Assert.assertTrue(Files.exists(pipeModFile.toPath())); - pipeTsFileResourceManager.increaseFileReference(originTsfile, true, null, PIPE_NAME); - pipeTsFileResourceManager.increaseFileReference(originModFile, false, null, PIPE_NAME); + pipeTsFileResourceManager.increaseFileReference(originTsfile, true, PIPE_NAME); + pipeTsFileResourceManager.increaseFileReference(originModFile, false, PIPE_NAME); Assert.assertEquals(2, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, PIPE_NAME)); Assert.assertEquals(2, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, PIPE_NAME)); // test use hardlinkTsFile to increase reference counts // test null, shall not reuse the pipe's tsFile - pipeTsFileResourceManager.increaseFileReference(pipeTsfile, true, null, null); + pipeTsFileResourceManager.increaseFileReference(pipeTsfile, true, null); Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, null)); Assert.assertTrue(Files.exists(originTsfile.toPath())); Assert.assertTrue(Files.exists(pipeTsfile.toPath())); // test use copyFile to increase reference counts - pipeTsFileResourceManager.increaseFileReference(pipeModFile, false, null, PIPE_NAME); + pipeTsFileResourceManager.increaseFileReference(pipeModFile, false, PIPE_NAME); Assert.assertEquals(3, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, PIPE_NAME)); Assert.assertTrue(Files.exists(originModFile.toPath())); Assert.assertTrue(Files.exists(pipeModFile.toPath())); @@ -205,9 +205,8 @@ public void testDecreaseTsFile() throws IOException { Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(originFile, null)); Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(originModFile, null)); - File pipeTsfile = pipeTsFileResourceManager.increaseFileReference(originFile, true, null, null); - File pipeModFile = - pipeTsFileResourceManager.increaseFileReference(originModFile, false, null, null); + File pipeTsfile = pipeTsFileResourceManager.increaseFileReference(originFile, true, null); + File pipeModFile = pipeTsFileResourceManager.increaseFileReference(originModFile, false, null); Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, null)); Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, null)); Assert.assertTrue(Files.exists(pipeTsfile.toPath())); From ebf297d65c287ed96d83851bbab922b5d2b259f8 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 1 Jul 2025 14:24:32 +0800 Subject: [PATCH 059/185] Update PipeRealtimeDataRegionHybridExtractor.java --- .../realtime/PipeRealtimeDataRegionHybridExtractor.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java index 7a6177007941..07eefabfb85d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java @@ -298,14 +298,14 @@ private boolean mayTsFileLinkedCountReachDangerousThreshold(final PipeRealtimeEv return false; } final boolean mayTsFileLinkedCountReachDangerousThreshold = - PipeDataNodeResourceManager.tsfile().getLinkedTsFileCount() + PipeDataNodeResourceManager.tsfile().getLinkedTsFileCount(pipeName) >= PipeConfig.getInstance().getPipeMaxAllowedLinkedTsFileCount(); if (mayTsFileLinkedCountReachDangerousThreshold && event.mayExtractorUseTablets(this)) { LOGGER.info( "Pipe task {}@{} canNotUseTabletAnymoreDeprecated(4): The number of linked tsFiles {} has reached the dangerous threshold {}", pipeName, dataRegionId, - PipeDataNodeResourceManager.tsfile().getLinkedTsFileCount(), + PipeDataNodeResourceManager.tsfile().getLinkedTsFileCount(pipeName), PipeConfig.getInstance().getPipeMaxAllowedLinkedTsFileCount()); } return mayTsFileLinkedCountReachDangerousThreshold; From aef786608d698603371d4bcffed939143b82b5e0 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 1 Jul 2025 14:40:55 +0800 Subject: [PATCH 060/185] May complete --- .../common/tsfile/PipeTsFileInsertionEvent.java | 10 +++++++++- .../PipeHistoricalDataRegionTsFileExtractor.java | 1 + .../resource/tsfile/PipeTsFileResourceManager.java | 13 ++++++++++++- 3 files changed, 22 insertions(+), 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java index cf16fd2b32ea..12f716463516 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java @@ -94,6 +94,7 @@ public PipeTsFileInsertionEvent( // The modFile must be copied before the event is assigned to the listening pipes this( resource, + null, true, isLoaded, isGeneratedByHistoricalExtractor, @@ -107,6 +108,7 @@ public PipeTsFileInsertionEvent( public PipeTsFileInsertionEvent( final TsFileResource resource, + final File tsFile, final boolean isWithMod, final boolean isLoaded, final boolean isGeneratedByHistoricalExtractor, @@ -119,7 +121,12 @@ public PipeTsFileInsertionEvent( super(pipeName, creationTime, pipeTaskMeta, pattern, startTime, endTime); this.resource = resource; - tsFile = resource.getTsFile(); + + // For events created at assigner or historical extractor, the tsFile is get from the resource + // For events created for source, the tsFile is inherited from the assigner, because the + // original tsFile may be gone, and we need to get the assigner's hard-linked tsFile to + // hard-link it to each pipe dir + this.tsFile = Objects.isNull(tsFile) ? resource.getTsFile() : tsFile; final ModificationFile modFile = resource.getModFile(); this.isWithMod = isWithMod && modFile.exists(); @@ -342,6 +349,7 @@ public PipeTsFileInsertionEvent shallowCopySelfAndBindPipeTaskMetaForProgressRep final long endTime) { return new PipeTsFileInsertionEvent( resource, + tsFile, isWithMod, isLoaded, isGeneratedByHistoricalExtractor, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java index 0408e4f3a039..12e76afa4a65 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java @@ -547,6 +547,7 @@ public synchronized Event supply() { final PipeTsFileInsertionEvent event = new PipeTsFileInsertionEvent( resource, + null, shouldTransferModFile, false, true, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java index 17397eedcc81..ab57811187e1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java @@ -120,6 +120,10 @@ private boolean increaseReferenceIfExists(final File file, final String pipeName resource.increaseReferenceCount(); return true; } + // Increase the assigner's file to avoid hard-link or memory cache cleaning + if (Objects.nonNull(pipeName)) { + hardlinkOrCopiedFileToAssignerTsFileResourceMap.get(file.getPath()).increaseReferenceCount(); + } return false; } @@ -167,7 +171,7 @@ private static String getRelativeFilePath(File file) { * Given a hardlink or copied file, decrease its reference count, if the reference count is 0, * delete the file. if the given file is not a hardlink or copied file, do nothing. * - * @param hardlinkOrCopiedFile the copied or hardlinked file + * @param hardlinkOrCopiedFile the copied or hard-linked file */ public void decreaseFileReference( final File hardlinkOrCopiedFile, final @Nullable String pipeName) { @@ -179,6 +183,13 @@ public void decreaseFileReference( if (resource != null && resource.decreaseReferenceCount()) { getHardlinkOrCopiedFile2TsFileResourceMap(pipeName).remove(filePath); } + // Decrease the assigner's file to clear hard-link and memory cache + if (Objects.nonNull(pipeName) + && hardlinkOrCopiedFileToAssignerTsFileResourceMap + .get(hardlinkOrCopiedFile.getPath()) + .decreaseReferenceCount()) { + hardlinkOrCopiedFileToPipeTsFileResourceMap.remove(filePath); + } } finally { segmentLock.unlock(hardlinkOrCopiedFile); } From 472d1c089277a14f2c1e639e42376f05860f80e4 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 1 Jul 2025 14:49:07 +0800 Subject: [PATCH 061/185] Update TsFileInsertionDataContainerTest.java --- .../event/TsFileInsertionDataContainerTest.java | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/TsFileInsertionDataContainerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/TsFileInsertionDataContainerTest.java index d97f2da68e6d..50816ebf9688 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/TsFileInsertionDataContainerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/TsFileInsertionDataContainerTest.java @@ -24,7 +24,6 @@ import org.apache.iotdb.commons.pipe.datastructure.pattern.PipePattern; import org.apache.iotdb.commons.pipe.datastructure.pattern.PrefixPipePattern; import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; -import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.container.TsFileInsertionDataContainer; import org.apache.iotdb.db.pipe.event.common.tsfile.container.query.TsFileInsertionQueryDataContainer; import org.apache.iotdb.db.pipe.event.common.tsfile.container.scan.TsFileInsertionScanDataContainer; @@ -558,18 +557,6 @@ private void testTsFilePointNum( final long endTime, final boolean isQuery, final int expectedCount) { - PipeTsFileInsertionEvent tsFileInsertionEvent = - new PipeTsFileInsertionEvent( - new TsFileResource(tsFile), - true, - false, - false, - null, - 0, - null, - null, - Long.MIN_VALUE, - Long.MAX_VALUE); try (final TsFileInsertionDataContainer tsFileContainer = isQuery ? new TsFileInsertionQueryDataContainer(tsFile, pattern, startTime, endTime) From 567ec0207f79eeca220372a006ee1006a4f4d40d Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 1 Jul 2025 15:30:17 +0800 Subject: [PATCH 062/185] Update PipeTsFileResourceManager.java --- .../db/pipe/resource/tsfile/PipeTsFileResourceManager.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java index ab57811187e1..638f26a1fec1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java @@ -48,6 +48,8 @@ public class PipeTsFileResourceManager { // tsFiles. private final Map hardlinkOrCopiedFileToAssignerTsFileResourceMap = new ConcurrentHashMap<>(); + + // PipeName -> TsFilePath -> PipeTsFileResource private final Map> hardlinkOrCopiedFileToPipeTsFileResourceMap = new ConcurrentHashMap<>(); private final PipeTsFileResourceSegmentLock segmentLock = new PipeTsFileResourceSegmentLock(); From ffeb881bd8f633cc435c653aba252c84f9453dd4 Mon Sep 17 00:00:00 2001 From: Steve Yurong Su Date: Tue, 1 Jul 2025 16:12:14 +0800 Subject: [PATCH 063/185] introduce PipeCompactionManager for tsfile resource replacement in pipe module --- .../PipeConnectorSubtaskLifeCycle.java | 5 ++ .../resource/PipeDataNodeResourceManager.java | 7 ++ .../tsfile/PipeCompactionManager.java | 69 +++++++++++++++++++ .../dataregion/tsfile/TsFileManager.java | 10 +++ .../UnboundedBlockingPendingQueue.java | 4 ++ 5 files changed, 95 insertions(+) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeCompactionManager.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeConnectorSubtaskLifeCycle.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeConnectorSubtaskLifeCycle.java index ecbbc641e4b9..6e4a858a3705 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeConnectorSubtaskLifeCycle.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeConnectorSubtaskLifeCycle.java @@ -21,6 +21,7 @@ import org.apache.iotdb.commons.pipe.agent.task.connection.UnboundedBlockingPendingQueue; import org.apache.iotdb.db.pipe.agent.task.execution.PipeConnectorSubtaskExecutor; +import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; import org.apache.iotdb.pipe.api.event.Event; import org.slf4j.Logger; @@ -65,6 +66,8 @@ public synchronized void register() { if (registeredTaskCount == 0) { executor.register(subtask); runningTaskCount = 0; + + PipeDataNodeResourceManager.compaction().registerPipeConnectorSubtaskLifeCycle(this); } registeredTaskCount++; @@ -152,5 +155,7 @@ public synchronized void stop() { @Override public synchronized void close() { executor.deregister(subtask.getTaskID()); + + PipeDataNodeResourceManager.compaction().deregisterPipeConnectorSubtaskLifeCycle(this); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/PipeDataNodeResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/PipeDataNodeResourceManager.java index 4a3af9d781c2..aaf9eff454b8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/PipeDataNodeResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/PipeDataNodeResourceManager.java @@ -25,11 +25,13 @@ import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryManager; import org.apache.iotdb.db.pipe.resource.ref.PipeDataNodePhantomReferenceManager; import org.apache.iotdb.db.pipe.resource.snapshot.PipeDataNodeSnapshotResourceManager; +import org.apache.iotdb.db.pipe.resource.tsfile.PipeCompactionManager; import org.apache.iotdb.db.pipe.resource.tsfile.PipeTsFileResourceManager; public class PipeDataNodeResourceManager { private final PipeTsFileResourceManager pipeTsFileResourceManager; + private final PipeCompactionManager pipeCompactionManager; private final PipeSnapshotResourceManager pipeSnapshotResourceManager; private final PipeMemoryManager pipeMemoryManager; private final PipeLogManager pipeLogManager; @@ -39,6 +41,10 @@ public static PipeTsFileResourceManager tsfile() { return PipeResourceManagerHolder.INSTANCE.pipeTsFileResourceManager; } + public static PipeCompactionManager compaction() { + return PipeResourceManagerHolder.INSTANCE.pipeCompactionManager; + } + public static PipeSnapshotResourceManager snapshot() { return PipeResourceManagerHolder.INSTANCE.pipeSnapshotResourceManager; } @@ -59,6 +65,7 @@ public static PipePhantomReferenceManager ref() { private PipeDataNodeResourceManager() { pipeTsFileResourceManager = new PipeTsFileResourceManager(); + pipeCompactionManager = new PipeCompactionManager(); pipeSnapshotResourceManager = new PipeDataNodeSnapshotResourceManager(); pipeMemoryManager = new PipeMemoryManager(); pipeLogManager = new PipeLogManager(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeCompactionManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeCompactionManager.java new file mode 100644 index 000000000000..970e819ea91e --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeCompactionManager.java @@ -0,0 +1,69 @@ +/* + * 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.pipe.resource.tsfile; + +import org.apache.iotdb.commons.pipe.agent.task.connection.UnboundedBlockingPendingQueue; +import org.apache.iotdb.db.pipe.agent.task.subtask.connector.PipeConnectorSubtaskLifeCycle; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; +import org.apache.iotdb.pipe.api.event.Event; + +import java.io.File; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.CopyOnWriteArraySet; +import java.util.stream.Collectors; + +public class PipeCompactionManager { + + private final Set pipeConnectorSubtaskLifeCycles = + new CopyOnWriteArraySet<>(); + + public void registerPipeConnectorSubtaskLifeCycle( + final PipeConnectorSubtaskLifeCycle pipeConnectorSubtaskLifeCycle) { + pipeConnectorSubtaskLifeCycles.add(pipeConnectorSubtaskLifeCycle); + } + + public void deregisterPipeConnectorSubtaskLifeCycle( + final PipeConnectorSubtaskLifeCycle pipeConnectorSubtaskLifeCycle) { + pipeConnectorSubtaskLifeCycles.remove(pipeConnectorSubtaskLifeCycle); + } + + public void emitResult( + final String storageGroupName, + final String dataRegionId, + final long timePartition, + final List seqFileResources, + final List unseqFileResources, + final List targetFileResources) { + final Set sourceFiles = new HashSet<>(); + seqFileResources.forEach(tsFileResource -> sourceFiles.add(tsFileResource.getTsFile())); + unseqFileResources.forEach(tsFileResource -> sourceFiles.add(tsFileResource.getTsFile())); + final Set targetFiles = + targetFileResources.stream().map(TsFileResource::getTsFile).collect(Collectors.toSet()); + + for (final PipeConnectorSubtaskLifeCycle lifeCycle : pipeConnectorSubtaskLifeCycles) { + final UnboundedBlockingPendingQueue pendingQueue = lifeCycle.getPendingQueue(); + if (pendingQueue != null) { + pendingQueue.replace(sourceFiles, targetFiles); + } + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java index 734c1449dfa7..a9a9878627fb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.storageengine.dataregion.tsfile; import org.apache.iotdb.commons.utils.TimePartitionUtils; +import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; import org.apache.iotdb.db.storageengine.dataregion.tsfile.timeindex.FileTimeIndexCacheRecorder; import org.apache.iotdb.db.storageengine.rescon.memory.TsFileResourceManager; @@ -276,6 +277,15 @@ public void replace( } finally { writeUnlock(); } + + PipeDataNodeResourceManager.compaction() + .emitResult( + storageGroupName, + dataRegionId, + timePartition, + seqFileResources, + unseqFileResources, + targetFileResources); } public boolean contains(TsFileResource tsFileResource, boolean sequence) { diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/connection/UnboundedBlockingPendingQueue.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/connection/UnboundedBlockingPendingQueue.java index 785e89cfb9a8..2702d3b182fa 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/connection/UnboundedBlockingPendingQueue.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/connection/UnboundedBlockingPendingQueue.java @@ -22,6 +22,8 @@ import org.apache.iotdb.commons.pipe.metric.PipeEventCounter; import org.apache.iotdb.pipe.api.event.Event; +import java.io.File; +import java.util.Set; import java.util.concurrent.BlockingDeque; import java.util.concurrent.LinkedBlockingDeque; @@ -37,4 +39,6 @@ public UnboundedBlockingPendingQueue(final PipeEventCounter eventCounter) { public E peekLast() { return pendingDeque.peekLast(); } + + public void replace(Set sourceFiles, Set targetFiles) {} } From 3f56e3bc8ca71021737a60810472490c231ae8d7 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 1 Jul 2025 16:13:18 +0800 Subject: [PATCH 064/185] Help receiver GC --- .../payload/evolvable/request/PipeTransferPlanNodeReq.java | 1 - .../payload/evolvable/request/PipeTransferTabletBatchReq.java | 1 - .../payload/evolvable/request/PipeTransferTabletBinaryReq.java | 1 - .../evolvable/request/PipeTransferTabletInsertNodeReq.java | 1 - .../payload/evolvable/request/PipeTransferTabletRawReq.java | 1 - .../pipe/processor/twostage/exchange/payload/CombineRequest.java | 1 - .../twostage/exchange/payload/FetchCombineResultRequest.java | 1 - .../payload/thrift/request/PipeTransferFilePieceReq.java | 1 - .../payload/thrift/request/PipeTransferFileSealReqV1.java | 1 - .../payload/thrift/request/PipeTransferFileSealReqV2.java | 1 - .../payload/thrift/request/PipeTransferHandshakeV1Req.java | 1 - .../payload/thrift/request/PipeTransferHandshakeV2Req.java | 1 - .../connector/payload/thrift/request/PipeTransferSliceReq.java | 1 - 13 files changed, 13 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferPlanNodeReq.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferPlanNodeReq.java index 580dbe2ed4ca..553a04cbfcd6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferPlanNodeReq.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferPlanNodeReq.java @@ -66,7 +66,6 @@ public static PipeTransferPlanNodeReq fromTPipeTransferReq(TPipeTransferReq tran planNodeReq.version = transferReq.version; planNodeReq.type = transferReq.type; - planNodeReq.body = transferReq.body; return planNodeReq; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletBatchReq.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletBatchReq.java index 8090f6504897..27664cdbac71 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletBatchReq.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletBatchReq.java @@ -182,7 +182,6 @@ public static PipeTransferTabletBatchReq fromTPipeTransferReq( batchReq.version = transferReq.version; batchReq.type = transferReq.type; - batchReq.body = transferReq.body; return batchReq; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletBinaryReq.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletBinaryReq.java index 5e9e0a39103f..4deac8a5cf69 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletBinaryReq.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletBinaryReq.java @@ -89,7 +89,6 @@ public static PipeTransferTabletBinaryReq fromTPipeTransferReq( binaryReq.version = transferReq.version; binaryReq.type = transferReq.type; - binaryReq.body = transferReq.body; return binaryReq; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletInsertNodeReq.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletInsertNodeReq.java index c45417ba99da..376ca121c80c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletInsertNodeReq.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletInsertNodeReq.java @@ -96,7 +96,6 @@ public static PipeTransferTabletInsertNodeReq fromTPipeTransferReq( insertNodeReq.version = transferReq.version; insertNodeReq.type = transferReq.type; - insertNodeReq.body = transferReq.body; return insertNodeReq; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReq.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReq.java index 61790f883ae5..f1e32785df77 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReq.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/request/PipeTransferTabletRawReq.java @@ -132,7 +132,6 @@ public static PipeTransferTabletRawReq fromTPipeTransferReq(final TPipeTransferR tabletReq.version = transferReq.version; tabletReq.type = transferReq.type; - tabletReq.body = transferReq.body; return tabletReq; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/twostage/exchange/payload/CombineRequest.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/twostage/exchange/payload/CombineRequest.java index cb1ba0b9ad9c..0e1f0f2b0334 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/twostage/exchange/payload/CombineRequest.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/twostage/exchange/payload/CombineRequest.java @@ -114,7 +114,6 @@ private CombineRequest translateFromTPipeTransferReq(TPipeTransferReq transferRe version = transferReq.version; type = transferReq.type; - body = transferReq.body; return this; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/twostage/exchange/payload/FetchCombineResultRequest.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/twostage/exchange/payload/FetchCombineResultRequest.java index b20904a0e2b2..752be403008c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/twostage/exchange/payload/FetchCombineResultRequest.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/processor/twostage/exchange/payload/FetchCombineResultRequest.java @@ -99,7 +99,6 @@ private FetchCombineResultRequest translateFromTPipeTransferReq(TPipeTransferReq version = transferReq.version; type = transferReq.type; - body = transferReq.body; return this; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeTransferFilePieceReq.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeTransferFilePieceReq.java index 909d43be46e5..c61abbec2895 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeTransferFilePieceReq.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeTransferFilePieceReq.java @@ -82,7 +82,6 @@ protected final PipeTransferFilePieceReq translateFromTPipeTransferReq( version = transferReq.version; type = transferReq.type; - body = transferReq.body; return this; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeTransferFileSealReqV1.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeTransferFileSealReqV1.java index 0bc0342cc998..157d73e06153 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeTransferFileSealReqV1.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeTransferFileSealReqV1.java @@ -71,7 +71,6 @@ public PipeTransferFileSealReqV1 translateFromTPipeTransferReq(TPipeTransferReq version = req.version; type = req.type; - body = req.body; return this; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeTransferFileSealReqV2.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeTransferFileSealReqV2.java index 3ed9999a4e81..86cce0245ec9 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeTransferFileSealReqV2.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeTransferFileSealReqV2.java @@ -109,7 +109,6 @@ public PipeTransferFileSealReqV2 translateFromTPipeTransferReq(TPipeTransferReq version = req.version; type = req.type; - body = req.body; return this; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeTransferHandshakeV1Req.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeTransferHandshakeV1Req.java index 7c0330a7a834..d4fb192c0358 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeTransferHandshakeV1Req.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeTransferHandshakeV1Req.java @@ -62,7 +62,6 @@ protected final PipeTransferHandshakeV1Req translateFromTPipeTransferReq( version = transferReq.version; type = transferReq.type; - body = transferReq.body; return this; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeTransferHandshakeV2Req.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeTransferHandshakeV2Req.java index 8ed63cda5465..c6fd0f977a10 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeTransferHandshakeV2Req.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeTransferHandshakeV2Req.java @@ -74,7 +74,6 @@ protected final PipeTransferHandshakeV2Req translateFromTPipeTransferReq( version = transferReq.version; type = transferReq.type; - body = transferReq.body; return this; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeTransferSliceReq.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeTransferSliceReq.java index 4df6008400d7..1041cb283c52 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeTransferSliceReq.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/thrift/request/PipeTransferSliceReq.java @@ -128,7 +128,6 @@ public static PipeTransferSliceReq fromTPipeTransferReq(final TPipeTransferReq t sliceReq.version = transferReq.version; sliceReq.type = transferReq.type; - sliceReq.body = transferReq.body; return sliceReq; } From e3ed158d65e44851cd1fe6bedabaa1df8d33d162 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Tue, 1 Jul 2025 16:43:50 +0800 Subject: [PATCH 065/185] Add memory limit for creating Pipe --- .../agent/task/PipeDataNodeTaskAgent.java | 209 ++++++++++++++++++ .../thrift/IoTDBDataNodeReceiver.java | 8 + .../resource/memory/PipeMemoryManager.java | 4 + .../iotdb/commons/conf/CommonConfig.java | 124 +++++++++++ .../pipe/agent/task/PipeTaskAgent.java | 12 + .../iotdb/commons/pipe/config/PipeConfig.java | 42 ++++ .../commons/pipe/config/PipeDescriptor.java | 39 ++++ 7 files changed, 438 insertions(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java index a05abaf236db..9ba7469e7c67 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java @@ -38,6 +38,7 @@ import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant; import org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant; +import org.apache.iotdb.commons.pipe.config.constant.SystemConstant; import org.apache.iotdb.consensus.exception.ConsensusException; import org.apache.iotdb.consensus.pipe.consensuspipe.ConsensusPipeName; import org.apache.iotdb.db.conf.IoTDBConfig; @@ -52,6 +53,7 @@ import org.apache.iotdb.db.pipe.metric.overview.PipeDataNodeSinglePipeMetrics; import org.apache.iotdb.db.pipe.metric.overview.PipeTsFileToTabletsMetrics; import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; +import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryManager; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.pipe.PipeOperateSchemaQueueNode; import org.apache.iotdb.db.schemaengine.SchemaEngine; @@ -93,6 +95,25 @@ import java.util.stream.Collectors; import java.util.stream.StreamSupport; +import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.EXTRACTOR_END_TIME_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.EXTRACTOR_HISTORY_ENABLE_DEFAULT_VALUE; +import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.EXTRACTOR_HISTORY_ENABLE_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.EXTRACTOR_HISTORY_END_TIME_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.EXTRACTOR_HISTORY_START_TIME_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.EXTRACTOR_PATH_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.EXTRACTOR_PATTERN_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.EXTRACTOR_REALTIME_ENABLE_DEFAULT_VALUE; +import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.EXTRACTOR_REALTIME_ENABLE_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.EXTRACTOR_START_TIME_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.SOURCE_END_TIME_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.SOURCE_HISTORY_ENABLE_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.SOURCE_HISTORY_END_TIME_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.SOURCE_HISTORY_START_TIME_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.SOURCE_PATH_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.SOURCE_PATTERN_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.SOURCE_REALTIME_ENABLE_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.SOURCE_START_TIME_KEY; + public class PipeDataNodeTaskAgent extends PipeTaskAgent { private static final Logger LOGGER = LoggerFactory.getLogger(PipeDataNodeTaskAgent.class); @@ -640,4 +661,192 @@ public Map getAllConsensusPipe() { releaseReadLock(); } } + + @Override + protected void calculateMemoryUsage( + final PipeParameters extractorParameters, + final PipeParameters processorParameters, + final PipeParameters connectorParameters) { + if (!PipeConfig.getInstance().isPipeEnableMemoryCheck()) { + return; + } + + long needMemory = 0; + needMemory += + calculateInsertNodeQueueMemory( + extractorParameters, processorParameters, connectorParameters); + needMemory += + calculateTsFileParserMemory(extractorParameters, processorParameters, connectorParameters); + needMemory += + calculateSinkBatchMemory(extractorParameters, processorParameters, connectorParameters); + needMemory += + calculateSendTsFileReadBufferMemory( + extractorParameters, processorParameters, connectorParameters); + + PipeMemoryManager pipeMemoryManager = PipeDataNodeResourceManager.memory(); + if (pipeMemoryManager.getFreeMemorySizeInBytes() + < needMemory + + PipeMemoryManager.getTotalMemorySizeInBytes() + * PipeConfig.getInstance().getReservedMemoryPercentage()) { + final String e = + String.format( + "Not enough memory for pipe. Need memory: %d bytes, free memory: %d bytes, reserved memory: %d bytes, total memory: %d bytes", + needMemory, + pipeMemoryManager.getFreeMemorySizeInBytes(), + (long) + (PipeMemoryManager.getTotalMemorySizeInBytes() + * PipeConfig.getInstance().getReservedMemoryPercentage()), + PipeMemoryManager.getTotalMemorySizeInBytes()); + throw new PipeException(e); + } + } + + private long calculateInsertNodeQueueMemory( + final PipeParameters extractorParameters, + final PipeParameters processorParameters, + final PipeParameters connectorParameters) { + + // Realtime extractor is enabled by default, so we only need to check the source realtime + if (!extractorParameters.getBooleanOrDefault( + Arrays.asList(EXTRACTOR_REALTIME_ENABLE_KEY, SOURCE_REALTIME_ENABLE_KEY), + EXTRACTOR_REALTIME_ENABLE_DEFAULT_VALUE)) { + return 0; + } + + // If the realtime mode is batch or file, we do not need to allocate memory + final String realtimeMode = + extractorParameters.getStringByKeys( + PipeExtractorConstant.EXTRACTOR_REALTIME_MODE_KEY, + PipeExtractorConstant.SOURCE_REALTIME_MODE_KEY); + if (PipeExtractorConstant.EXTRACTOR_REALTIME_MODE_BATCH_MODE_VALUE.equals(realtimeMode) + || PipeExtractorConstant.EXTRACTOR_REALTIME_MODE_FILE_VALUE.equals(realtimeMode)) { + return 0; + } + + return PipeConfig.getInstance().getPipeInodeMemory(); + } + + private long calculateTsFileParserMemory( + final PipeParameters extractorParameters, + final PipeParameters processorParameters, + final PipeParameters connectorParameters) { + + // If the extractor is not history, we do not need to allocate memory + boolean isExtractorHistory = + extractorParameters.getBooleanOrDefault( + SystemConstant.RESTART_KEY, SystemConstant.RESTART_DEFAULT_VALUE) + || extractorParameters.getBooleanOrDefault( + Arrays.asList(EXTRACTOR_HISTORY_ENABLE_KEY, SOURCE_HISTORY_ENABLE_KEY), + EXTRACTOR_HISTORY_ENABLE_DEFAULT_VALUE); + + // If the extractor is history, and has start/end time, we need to allocate memory + boolean isTSFileParser = + isExtractorHistory + && extractorParameters.hasAnyAttributes( + EXTRACTOR_HISTORY_START_TIME_KEY, SOURCE_HISTORY_START_TIME_KEY); + + isTSFileParser = + isTSFileParser + || (isExtractorHistory + && extractorParameters.hasAnyAttributes( + EXTRACTOR_HISTORY_END_TIME_KEY, SOURCE_HISTORY_END_TIME_KEY)); + + // if the extractor has start/end time, we need to allocate memory + isTSFileParser = + isTSFileParser + || extractorParameters.hasAnyAttributes( + SOURCE_START_TIME_KEY, EXTRACTOR_START_TIME_KEY); + + isTSFileParser = + isTSFileParser + || extractorParameters.hasAnyAttributes(SOURCE_END_TIME_KEY, EXTRACTOR_END_TIME_KEY); + + // If the extractor has pattern or path, we need to allocate memory + isTSFileParser = + isTSFileParser + || extractorParameters.hasAnyAttributes(EXTRACTOR_PATTERN_KEY, SOURCE_PATTERN_KEY); + + isTSFileParser = + isTSFileParser || extractorParameters.hasAnyAttributes(EXTRACTOR_PATH_KEY, SOURCE_PATH_KEY); + + // If the extractor is not hybrid, we do need to allocate memory + isTSFileParser = + isTSFileParser + || !PipeConnectorConstant.CONNECTOR_FORMAT_HYBRID_VALUE.equals( + connectorParameters.getStringOrDefault( + Arrays.asList( + PipeConnectorConstant.CONNECTOR_FORMAT_KEY, + PipeConnectorConstant.SINK_FORMAT_KEY), + PipeConnectorConstant.CONNECTOR_FORMAT_HYBRID_VALUE)); + + if (!isTSFileParser) { + return 0; + } + + return PipeConfig.getInstance().getTsFileParserMemory(); + } + + private long calculateSinkBatchMemory( + final PipeParameters extractorParameters, + final PipeParameters processorParameters, + final PipeParameters connectorParameters) { + + // If the connector format is tsfile , we need to use batch + boolean needUseBatch = + PipeConnectorConstant.CONNECTOR_FORMAT_TS_FILE_VALUE.equals( + connectorParameters.getStringOrDefault( + Arrays.asList( + PipeConnectorConstant.CONNECTOR_FORMAT_KEY, + PipeConnectorConstant.SINK_FORMAT_KEY), + PipeConnectorConstant.CONNECTOR_FORMAT_HYBRID_VALUE)); + + if (needUseBatch) { + return PipeConfig.getInstance().getSinkBatchMemoryTsFile(); + } + + // If the connector is batch mode, we need to use batch + needUseBatch = + connectorParameters.getBooleanOrDefault( + Arrays.asList( + PipeConnectorConstant.CONNECTOR_IOTDB_BATCH_MODE_ENABLE_KEY, + PipeConnectorConstant.SINK_IOTDB_BATCH_MODE_ENABLE_KEY), + PipeConnectorConstant.CONNECTOR_IOTDB_BATCH_MODE_ENABLE_DEFAULT_VALUE); + + if (!needUseBatch) { + return 0; + } + + return PipeConfig.getInstance().getSinkBatchMemoryInsertNode(); + } + + private long calculateSendTsFileReadBufferMemory( + final PipeParameters extractorParameters, + final PipeParameters processorParameters, + final PipeParameters connectorParameters) { + // If the extractor is history enable, we need to transfer tsfile + boolean needTransferTsFile = + extractorParameters.getBooleanOrDefault( + SystemConstant.RESTART_KEY, SystemConstant.RESTART_DEFAULT_VALUE) + || extractorParameters.getBooleanOrDefault( + Arrays.asList(EXTRACTOR_HISTORY_ENABLE_KEY, SOURCE_HISTORY_ENABLE_KEY), + EXTRACTOR_HISTORY_ENABLE_DEFAULT_VALUE); + + String format = + connectorParameters.getStringOrDefault( + Arrays.asList( + PipeConnectorConstant.CONNECTOR_FORMAT_KEY, PipeConnectorConstant.SINK_FORMAT_KEY), + PipeConnectorConstant.CONNECTOR_FORMAT_HYBRID_VALUE); + + // If the connector format is tsfile and hybrid, we need to transfer tsfile + needTransferTsFile = + needTransferTsFile + || PipeConnectorConstant.CONNECTOR_FORMAT_HYBRID_VALUE.equals(format) + || PipeConnectorConstant.CONNECTOR_FORMAT_TS_FILE_VALUE.equals(format); + + if (!needTransferTsFile) { + return 0; + } + + return PipeConfig.getInstance().getSendTsFileReadBuffer(); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java index 00a7e4c46a09..9c23f447a466 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java @@ -86,6 +86,7 @@ import org.apache.iotdb.db.storageengine.rescon.disk.strategy.DirectoryStrategyType; import org.apache.iotdb.db.tools.schema.SRStatementGenerator; import org.apache.iotdb.db.tools.schema.SchemaRegionSnapshotParser; +import org.apache.iotdb.pipe.api.exception.PipeException; import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; import org.apache.iotdb.service.rpc.thrift.TPipeTransferReq; @@ -174,6 +175,13 @@ public synchronized TPipeTransferResp receive(final TPipeTransferReq req) { case HANDSHAKE_DATANODE_V1: { try { + if (PipeConfig.getInstance().isPipeEnableMemoryCheck()) { + if (PipeDataNodeResourceManager.memory().getFreeMemorySizeInBytes() + < PipeConfig.getInstance().getPipeMinimumReceiverMemory()) { + throw new PipeException( + "The receiver memory is not enough to handle the handshake request from datanode."); + } + } return handleTransferHandshakeV1( PipeTransferDataNodeHandshakeV1Req.fromTPipeTransferReq(req)); } finally { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryManager.java index 1afef9cf9ff4..8a87cf8bad0a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/memory/PipeMemoryManager.java @@ -654,4 +654,8 @@ public static long getTotalFloatingMemorySizeInBytes() { (TOTAL_MEMORY_SIZE_IN_BYTES * PipeConfig.getInstance().getPipeTotalFloatingMemoryProportion()); } + + public static long getTotalMemorySizeInBytes() { + return TOTAL_MEMORY_SIZE_IN_BYTES; + } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java index 476170874bcc..30dcfdcb07c9 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java @@ -220,6 +220,34 @@ public class CommonConfig { private double PipeDataStructureBatchMemoryProportion = 0.2; private volatile double pipeTotalFloatingMemoryProportion = 0.5; + // Check if memory check is enabled for Pipe + private boolean isPipeEnableMemoryCheck = true; + + // Memory for InsertNode queue: 15MB, used to temporarily store data awaiting processing + private long pipeInsertNodeQueueMemory = 15 * MB; + + // Memory for TsFile to Tablet conversion: 17MB, used for further processing after converting + // TSFile format to Tablet format + // Note: Pipes that do not decompose pattern/time do not need this part of memory + private long pipeTsFileParserMemory = 17 * MB; + + // Memory for Sink batch sending (InsertNode/TsFile, choose one) + // 1. InsertNode: 15MB, used for batch sending data to the downstream system + private long pipeSinkBatchMemoryInsertNode = 15 * MB; + + // 2. TsFile: 15MB, used for storing data about to be written to TsFile, similar to memTable + private long pipeSinkBatchMemoryTsFile = 15 * MB; + + // Memory needed for the ReadBuffer during the TsFile sending process: 15MB, buffer for the file + // sending process + private long pipeSendTsFileReadBuffer = 15 * MB; + + // Reserved memory percentage to accommodate memory fluctuations during system operation + private double pipeReservedMemoryPercentage = 0.15; + + // Minimum memory required for the receiver: 38MB + private long pipeMinimumReceiverMemory = 38 * MB; + private int pipeSubtaskExecutorBasicCheckPointIntervalByConsumedEventCount = 10_000; private long pipeSubtaskExecutorBasicCheckPointIntervalByTimeDuration = 10 * 1000L; private long pipeSubtaskExecutorPendingQueueMaxBlockingTimeMs = 50; @@ -847,6 +875,102 @@ public void setPipeDataStructureBatchMemoryProportion( PipeDataStructureBatchMemoryProportion); } + public boolean isPipeEnableMemoryChecked() { + return isPipeEnableMemoryCheck; + } + + public void setIsPipeEnableMemoryChecked(boolean isPipeEnableMemoryChecked) { + if (this.isPipeEnableMemoryCheck == isPipeEnableMemoryChecked) { + return; + } + this.isPipeEnableMemoryCheck = isPipeEnableMemoryChecked; + logger.info("isPipeEnableMemoryChecked is set to {}.", isPipeEnableMemoryChecked); + } + + public long getPipeInsertNodeQueueMemory() { + return pipeInsertNodeQueueMemory; + } + + public void setPipeInsertNodeQueueMemory(long pipeInsertNodeQueueMemory) { + if (this.pipeInsertNodeQueueMemory == pipeInsertNodeQueueMemory) { + return; + } + this.pipeInsertNodeQueueMemory = pipeInsertNodeQueueMemory; + logger.info("pipeInsertNodeQueueMemory is set to {}.", pipeInsertNodeQueueMemory); + } + + public long getPipeTsFileParserMemory() { + return pipeTsFileParserMemory; + } + + public void setPipeTsFileParserMemory(long pipeTsFileParserMemory) { + if (this.pipeTsFileParserMemory == pipeTsFileParserMemory) { + return; + } + this.pipeTsFileParserMemory = pipeTsFileParserMemory; + logger.info("pipeTsFileParserMemory is set to {}.", pipeTsFileParserMemory); + } + + public long getPipeSinkBatchMemoryInsertNode() { + return pipeSinkBatchMemoryInsertNode; + } + + public void setPipeSinkBatchMemoryInsertNode(long pipeSinkBatchMemoryInsertNode) { + if (this.pipeSinkBatchMemoryInsertNode == pipeSinkBatchMemoryInsertNode) { + return; + } + this.pipeSinkBatchMemoryInsertNode = pipeSinkBatchMemoryInsertNode; + logger.info("pipeSinkBatchMemoryInsertNode is set to {}.", pipeSinkBatchMemoryInsertNode); + } + + public long getPipeSinkBatchMemoryTsFile() { + return pipeSinkBatchMemoryTsFile; + } + + public void setPipeSinkBatchMemoryTsFile(long pipeSinkBatchMemoryTsFile) { + if (this.pipeSinkBatchMemoryTsFile == pipeSinkBatchMemoryTsFile) { + return; + } + this.pipeSinkBatchMemoryTsFile = pipeSinkBatchMemoryTsFile; + logger.info("pipeSinkBatchMemoryTsFile is set to {}.", pipeSinkBatchMemoryTsFile); + } + + public long getPipeSendTsFileReadBuffer() { + return pipeSendTsFileReadBuffer; + } + + public void setPipeSendTsFileReadBuffer(long pipeSendTsFileReadBuffer) { + if (this.pipeSendTsFileReadBuffer == pipeSendTsFileReadBuffer) { + return; + } + this.pipeSendTsFileReadBuffer = pipeSendTsFileReadBuffer; + logger.info("pipeSendTsFileReadBuffer is set to {}.", pipeSendTsFileReadBuffer); + } + + public double getPipeReservedMemoryPercentage() { + return pipeReservedMemoryPercentage; + } + + public void setPipeReservedMemoryPercentage(double pipeReservedMemoryPercentage) { + if (this.pipeReservedMemoryPercentage == pipeReservedMemoryPercentage) { + return; + } + this.pipeReservedMemoryPercentage = pipeReservedMemoryPercentage; + logger.info("pipeReservedMemoryPercentage is set to {}.", pipeReservedMemoryPercentage); + } + + public long getPipeMinimumReceiverMemory() { + return pipeMinimumReceiverMemory; + } + + public void setPipeMinimumReceiverMemory(long pipeMinimumReceiverMemory) { + if (this.pipeMinimumReceiverMemory == pipeMinimumReceiverMemory) { + return; + } + this.pipeMinimumReceiverMemory = pipeMinimumReceiverMemory; + logger.info("pipeMinimumReceiverMemory is set to {}.", pipeMinimumReceiverMemory); + } + public double getPipeTotalFloatingMemoryProportion() { return pipeTotalFloatingMemoryProportion; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java index 4ae678ae0754..9942f645ad87 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java @@ -469,6 +469,11 @@ private boolean createPipe(final PipeMeta pipeMetaFromCoordinator) throws Illega final String pipeName = pipeMetaFromCoordinator.getStaticMeta().getPipeName(); final long creationTime = pipeMetaFromCoordinator.getStaticMeta().getCreationTime(); + calculateMemoryUsage( + pipeMetaFromCoordinator.getStaticMeta().getExtractorParameters(), + pipeMetaFromCoordinator.getStaticMeta().getProcessorParameters(), + pipeMetaFromCoordinator.getStaticMeta().getConnectorParameters()); + final PipeMeta existedPipeMeta = pipeMetaKeeper.getPipeMeta(pipeName); if (existedPipeMeta != null) { if (!checkBeforeCreatePipe(existedPipeMeta, pipeName, creationTime)) { @@ -509,6 +514,13 @@ private boolean createPipe(final PipeMeta pipeMetaFromCoordinator) throws Illega return needToStartPipe; } + protected void calculateMemoryUsage( + final PipeParameters extractorParameters, + final PipeParameters processorParameters, + final PipeParameters connectorParameters) { + // do nothing + } + protected abstract Map buildPipeTasks(final PipeMeta pipeMetaFromCoordinator) throws IllegalPathException; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java index 6fedce58f9d7..3a09b31f49e6 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java @@ -95,6 +95,38 @@ public double getPipeDataStructureBatchMemoryProportion() { return COMMON_CONFIG.getPipeDataStructureBatchMemoryProportion(); } + public boolean isPipeEnableMemoryCheck() { + return COMMON_CONFIG.isPipeEnableMemoryChecked(); + } + + public long getPipeInodeMemory() { + return COMMON_CONFIG.getPipeInsertNodeQueueMemory(); + } + + public long getTsFileParserMemory() { + return COMMON_CONFIG.getPipeTsFileParserMemory(); + } + + public long getSinkBatchMemoryInsertNode() { + return COMMON_CONFIG.getPipeSinkBatchMemoryInsertNode(); + } + + public long getSinkBatchMemoryTsFile() { + return COMMON_CONFIG.getPipeSinkBatchMemoryTsFile(); + } + + public long getSendTsFileReadBuffer() { + return COMMON_CONFIG.getPipeSendTsFileReadBuffer(); + } + + public double getReservedMemoryPercentage() { + return COMMON_CONFIG.getPipeReservedMemoryPercentage(); + } + + public long getPipeMinimumReceiverMemory() { + return COMMON_CONFIG.getPipeMinimumReceiverMemory(); + } + /////////////////////////////// Subtask Connector /////////////////////////////// public int getPipeRealTimeQueuePollTsFileThreshold() { @@ -435,6 +467,16 @@ public void printAllConfigs() { getPipeDataStructureTsFileMemoryBlockAllocationRejectThreshold()); LOGGER.info("PipeTotalFloatingMemoryProportion: {}", getPipeTotalFloatingMemoryProportion()); + LOGGER.info( + "PipeDataStructureBatchMemoryProportion: {}", getPipeDataStructureBatchMemoryProportion()); + LOGGER.info("IsPipeEnableMemoryCheck: {}", isPipeEnableMemoryCheck()); + LOGGER.info("PipeTsFileParserMemory: {}", getTsFileParserMemory()); + LOGGER.info("SinkBatchMemoryInsertNode: {}", getSinkBatchMemoryInsertNode()); + LOGGER.info("SinkBatchMemoryTsFile: {}", getSinkBatchMemoryTsFile()); + LOGGER.info("SendTsFileReadBuffer: {}", getSendTsFileReadBuffer()); + LOGGER.info("PipeReservedMemoryPercentage: {}", getReservedMemoryPercentage()); + LOGGER.info("PipeMinimumReceiverMemory: {}", getPipeMinimumReceiverMemory()); + LOGGER.info( "PipeRealTimeQueuePollTsFileThreshold: {}", getPipeRealTimeQueuePollTsFileThreshold()); LOGGER.info( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java index b1937005eff3..d892b16cff89 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java @@ -246,6 +246,45 @@ public static void loadPipeInternalConfig(CommonConfig config, TrimProperties pr "pipe_total_floating_memory_proportion", String.valueOf(config.getPipeTotalFloatingMemoryProportion())))); + config.setIsPipeEnableMemoryChecked( + Boolean.parseBoolean( + properties.getProperty( + "pipe_enable_memory_checked", String.valueOf(config.isPipeEnableMemoryChecked())))); + config.setPipeInsertNodeQueueMemory( + Long.parseLong( + properties.getProperty( + "pipe_insert_node_queue_memory", + String.valueOf(config.getPipeInsertNodeQueueMemory())))); + config.setPipeTsFileParserMemory( + Long.parseLong( + properties.getProperty( + "pipe_tsfile_parser_memory", String.valueOf(config.getPipeTsFileParserMemory())))); + config.setPipeSinkBatchMemoryInsertNode( + Long.parseLong( + properties.getProperty( + "pipe_sink_batch_memory_insert_node", + String.valueOf(config.getPipeSinkBatchMemoryInsertNode())))); + config.setPipeSinkBatchMemoryTsFile( + Long.parseLong( + properties.getProperty( + "pipe_sink_batch_memory_ts_file", + String.valueOf(config.getPipeSinkBatchMemoryTsFile())))); + config.setPipeSendTsFileReadBuffer( + Long.parseLong( + properties.getProperty( + "pipe_send_tsfile_read_buffer", + String.valueOf(config.getPipeSendTsFileReadBuffer())))); + config.setPipeReservedMemoryPercentage( + Double.parseDouble( + properties.getProperty( + "pipe_reserved_memory_percentage", + String.valueOf(config.getPipeReservedMemoryPercentage())))); + config.setPipeMinimumReceiverMemory( + Long.parseLong( + properties.getProperty( + "pipe_minimum_receiver_memory", + String.valueOf(config.getPipeMinimumReceiverMemory())))); + config.setPipeRealTimeQueuePollTsFileThreshold( Integer.parseInt( Optional.ofNullable( From f271c186adbf1bc782b7aa0ea151961520e8df54 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 1 Jul 2025 16:44:35 +0800 Subject: [PATCH 066/185] May fix --- .../tsfile/PipeTsFileResourceManager.java | 46 ++++++++++++----- .../PipeTsFileResourceManagerTest.java | 51 +++++++++---------- 2 files changed, 58 insertions(+), 39 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java index 638f26a1fec1..5254ba862832 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java @@ -109,24 +109,34 @@ public File increaseFileReference( // the hardlink or copied file, and return the hardlink or copied file. getHardlinkOrCopiedFile2TsFileResourceMap(pipeName) .put(resultFile.getPath(), new PipeTsFileResource(resultFile, isTsFile)); + + increaseAssignerReferenceIfExists(file, pipeName); + return resultFile; } finally { segmentLock.unlock(hardlinkOrCopiedFile); } } - private boolean increaseReferenceIfExists(final File file, final String pipeName) { + private boolean increaseReferenceIfExists(final File file, final @Nullable String pipeName) { + final String path = file.getPath(); final PipeTsFileResource resource = - getHardlinkOrCopiedFile2TsFileResourceMap(pipeName).get(file.getPath()); + getHardlinkOrCopiedFile2TsFileResourceMap(pipeName).get(path); if (resource != null) { resource.increaseReferenceCount(); return true; } + increaseAssignerReferenceIfExists(file, pipeName); + return false; + } + + private void increaseAssignerReferenceIfExists(final File file, final @Nullable String pipeName) { // Increase the assigner's file to avoid hard-link or memory cache cleaning - if (Objects.nonNull(pipeName)) { + // Note that it does not exist for historical files + if (Objects.nonNull(pipeName) + && hardlinkOrCopiedFileToPipeTsFileResourceMap.containsKey(file.getPath())) { hardlinkOrCopiedFileToAssignerTsFileResourceMap.get(file.getPath()).increaseReferenceCount(); } - return false; } public static File getHardlinkOrCopiedFileInPipeDir(final File file, final String pipeName) @@ -145,7 +155,8 @@ public static File getHardlinkOrCopiedFileInPipeDir(final File file, final Strin private static String getPipeTsFileDirPath(File file, final String pipeName) throws IOException { while (!file.getName().equals(IoTDBConstant.SEQUENCE_FOLDER_NAME) - && !file.getName().equals(IoTDBConstant.UNSEQUENCE_FOLDER_NAME)) { + && !file.getName().equals(IoTDBConstant.UNSEQUENCE_FOLDER_NAME) + && !file.getName().equals(PipeConfig.getInstance().getPipeHardlinkBaseDirName())) { file = file.getParentFile(); } return file.getParentFile().getCanonicalPath() @@ -159,7 +170,10 @@ private static String getPipeTsFileDirPath(File file, final String pipeName) thr private static String getRelativeFilePath(File file) { StringBuilder builder = new StringBuilder(file.getName()); while (!file.getName().equals(IoTDBConstant.SEQUENCE_FOLDER_NAME) - && !file.getName().equals(IoTDBConstant.UNSEQUENCE_FOLDER_NAME)) { + && !file.getName().equals(IoTDBConstant.UNSEQUENCE_FOLDER_NAME) + && !file.getParentFile() + .getName() + .equals(PipeConfig.getInstance().getPipeHardlinkTsFileDirName())) { file = file.getParentFile(); builder = new StringBuilder(file.getName()) @@ -186,17 +200,25 @@ public void decreaseFileReference( getHardlinkOrCopiedFile2TsFileResourceMap(pipeName).remove(filePath); } // Decrease the assigner's file to clear hard-link and memory cache - if (Objects.nonNull(pipeName) - && hardlinkOrCopiedFileToAssignerTsFileResourceMap - .get(hardlinkOrCopiedFile.getPath()) - .decreaseReferenceCount()) { - hardlinkOrCopiedFileToPipeTsFileResourceMap.remove(filePath); - } + // Note that it does not exist for historical files + decreaseAssignerReferenceIfExists(hardlinkOrCopiedFile, pipeName); } finally { segmentLock.unlock(hardlinkOrCopiedFile); } } + private void decreaseAssignerReferenceIfExists(final File file, final @Nullable String pipeName) { + // Increase the assigner's file to avoid hard-link or memory cache cleaning + // Note that it does not exist for historical files + if (Objects.nonNull(pipeName) + && hardlinkOrCopiedFileToPipeTsFileResourceMap.containsKey(file.getPath()) + && hardlinkOrCopiedFileToAssignerTsFileResourceMap + .get(file.getPath()) + .decreaseReferenceCount()) { + hardlinkOrCopiedFileToPipeTsFileResourceMap.remove(file.getPath()); + } + } + /** * Get the reference count of the file. * diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java index ec00651d30d8..9867a1893142 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java @@ -158,41 +158,38 @@ public void tearDown() throws Exception { @Test public void testIncreaseTsFile() throws IOException { - File originTsfile = new File(TS_FILE_NAME); - File originModFile = new File(MODS_FILE_NAME); - Assert.assertEquals( - 0, pipeTsFileResourceManager.getFileReferenceCount(originTsfile, PIPE_NAME)); - Assert.assertEquals( - 0, pipeTsFileResourceManager.getFileReferenceCount(originModFile, PIPE_NAME)); - - File pipeTsfile = - pipeTsFileResourceManager.increaseFileReference(originTsfile, true, PIPE_NAME); - File pipeModFile = - pipeTsFileResourceManager.increaseFileReference(originModFile, false, PIPE_NAME); - Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, PIPE_NAME)); - Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, PIPE_NAME)); + final File originTsfile = new File(TS_FILE_NAME); + final File originModFile = new File(MODS_FILE_NAME); + Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(originTsfile, null)); + Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(originModFile, null)); + + final File assignerTsfile = + pipeTsFileResourceManager.increaseFileReference(originTsfile, true, null); + final File assignerModFile = + pipeTsFileResourceManager.increaseFileReference(originModFile, false, null); + Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(assignerTsfile, null)); + Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(assignerModFile, null)); Assert.assertTrue(Files.exists(originTsfile.toPath())); Assert.assertTrue(Files.exists(originModFile.toPath())); - Assert.assertTrue(Files.exists(pipeTsfile.toPath())); - Assert.assertTrue(Files.exists(pipeModFile.toPath())); + Assert.assertTrue(Files.exists(assignerTsfile.toPath())); + Assert.assertTrue(Files.exists(assignerModFile.toPath())); - pipeTsFileResourceManager.increaseFileReference(originTsfile, true, PIPE_NAME); - pipeTsFileResourceManager.increaseFileReference(originModFile, false, PIPE_NAME); - Assert.assertEquals(2, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, PIPE_NAME)); - Assert.assertEquals(2, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, PIPE_NAME)); - - // test use hardlinkTsFile to increase reference counts + // test use assigner's hardlinkTsFile to increase reference counts // test null, shall not reuse the pipe's tsFile - pipeTsFileResourceManager.increaseFileReference(pipeTsfile, true, null); - Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, null)); + pipeTsFileResourceManager.increaseFileReference(assignerTsfile, true, PIPE_NAME); + Assert.assertEquals(2, pipeTsFileResourceManager.getFileReferenceCount(assignerTsfile, null)); + Assert.assertEquals( + 1, pipeTsFileResourceManager.getFileReferenceCount(assignerTsfile, PIPE_NAME)); Assert.assertTrue(Files.exists(originTsfile.toPath())); - Assert.assertTrue(Files.exists(pipeTsfile.toPath())); + Assert.assertTrue(Files.exists(assignerTsfile.toPath())); // test use copyFile to increase reference counts - pipeTsFileResourceManager.increaseFileReference(pipeModFile, false, PIPE_NAME); - Assert.assertEquals(3, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, PIPE_NAME)); + pipeTsFileResourceManager.increaseFileReference(assignerModFile, false, PIPE_NAME); + Assert.assertEquals(2, pipeTsFileResourceManager.getFileReferenceCount(assignerModFile, null)); + Assert.assertEquals( + 1, pipeTsFileResourceManager.getFileReferenceCount(assignerModFile, PIPE_NAME)); Assert.assertTrue(Files.exists(originModFile.toPath())); - Assert.assertTrue(Files.exists(pipeModFile.toPath())); + Assert.assertTrue(Files.exists(assignerModFile.toPath())); } @Test From d40a7aa53ad5524b78a1ce92f88f38e6bb3ea1c0 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 1 Jul 2025 17:06:51 +0800 Subject: [PATCH 067/185] May fix --- .../tsfile/PipeTsFileResourceManager.java | 38 +++++++++++++------ 1 file changed, 27 insertions(+), 11 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java index 5254ba862832..12306be59663 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java @@ -133,9 +133,12 @@ private boolean increaseReferenceIfExists(final File file, final @Nullable Strin private void increaseAssignerReferenceIfExists(final File file, final @Nullable String pipeName) { // Increase the assigner's file to avoid hard-link or memory cache cleaning // Note that it does not exist for historical files - if (Objects.nonNull(pipeName) - && hardlinkOrCopiedFileToPipeTsFileResourceMap.containsKey(file.getPath())) { - hardlinkOrCopiedFileToAssignerTsFileResourceMap.get(file.getPath()).increaseReferenceCount(); + if (Objects.isNull(pipeName)) { + return; + } + final String assignerPath = getAssignerFilePath(file); + if (hardlinkOrCopiedFileToAssignerTsFileResourceMap.containsKey(assignerPath)) { + hardlinkOrCopiedFileToAssignerTsFileResourceMap.get(assignerPath).increaseReferenceCount(); } } @@ -210,15 +213,24 @@ public void decreaseFileReference( private void decreaseAssignerReferenceIfExists(final File file, final @Nullable String pipeName) { // Increase the assigner's file to avoid hard-link or memory cache cleaning // Note that it does not exist for historical files - if (Objects.nonNull(pipeName) - && hardlinkOrCopiedFileToPipeTsFileResourceMap.containsKey(file.getPath()) + if (Objects.isNull(pipeName)) { + return; + } + final String assignerPath = getAssignerFilePath(file); + if (hardlinkOrCopiedFileToAssignerTsFileResourceMap.containsKey(assignerPath) && hardlinkOrCopiedFileToAssignerTsFileResourceMap - .get(file.getPath()) + .get(assignerPath) .decreaseReferenceCount()) { - hardlinkOrCopiedFileToPipeTsFileResourceMap.remove(file.getPath()); + hardlinkOrCopiedFileToPipeTsFileResourceMap.remove(assignerPath); } } + // Warning: Shall not be called by the assigner + private String getAssignerFilePath(final File file) { + // Skip the "pipeName" of this file + return file.getParentFile().getParent() + file.getName(); + } + /** * Get the reference count of the file. * @@ -247,7 +259,8 @@ public boolean cacheObjectsIfAbsent(final File hardlinkOrCopiedTsFile) throws IO segmentLock.lock(hardlinkOrCopiedTsFile); try { final PipeTsFileResource resource = - hardlinkOrCopiedFileToAssignerTsFileResourceMap.get(hardlinkOrCopiedTsFile.getPath()); + hardlinkOrCopiedFileToAssignerTsFileResourceMap.get( + getAssignerFilePath(hardlinkOrCopiedTsFile)); return resource != null && resource.cacheObjectsIfAbsent(); } finally { segmentLock.unlock(hardlinkOrCopiedTsFile); @@ -259,7 +272,8 @@ public Map> getDeviceMeasurementsMapFromCache( segmentLock.lock(hardlinkOrCopiedTsFile); try { final PipeTsFileResource resource = - hardlinkOrCopiedFileToAssignerTsFileResourceMap.get(hardlinkOrCopiedTsFile.getPath()); + hardlinkOrCopiedFileToAssignerTsFileResourceMap.get( + getAssignerFilePath(hardlinkOrCopiedTsFile)); return resource == null ? null : resource.tryGetDeviceMeasurementsMap(); } finally { segmentLock.unlock(hardlinkOrCopiedTsFile); @@ -271,7 +285,8 @@ public Map getDeviceIsAlignedMapFromCache( segmentLock.lock(hardlinkOrCopiedTsFile); try { final PipeTsFileResource resource = - hardlinkOrCopiedFileToAssignerTsFileResourceMap.get(hardlinkOrCopiedTsFile.getPath()); + hardlinkOrCopiedFileToAssignerTsFileResourceMap.get( + getAssignerFilePath(hardlinkOrCopiedTsFile)); return resource == null ? null : resource.tryGetDeviceIsAlignedMap(cacheOtherMetadata); } finally { segmentLock.unlock(hardlinkOrCopiedTsFile); @@ -283,7 +298,8 @@ public Map getMeasurementDataTypeMapFromCache( segmentLock.lock(hardlinkOrCopiedTsFile); try { final PipeTsFileResource resource = - hardlinkOrCopiedFileToAssignerTsFileResourceMap.get(hardlinkOrCopiedTsFile.getPath()); + hardlinkOrCopiedFileToAssignerTsFileResourceMap.get( + getAssignerFilePath(hardlinkOrCopiedTsFile)); return resource == null ? null : resource.tryGetMeasurementDataTypeMap(); } finally { segmentLock.unlock(hardlinkOrCopiedTsFile); From 10646179f6134f0db8bce64c6543d647db79d3ae Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Tue, 1 Jul 2025 17:12:45 +0800 Subject: [PATCH 068/185] fix --- .../env/cluster/config/MppCommonConfig.java | 6 +++++ .../cluster/config/MppSharedCommonConfig.java | 7 +++++ .../env/remote/config/RemoteCommonConfig.java | 5 ++++ .../apache/iotdb/itbase/env/CommonConfig.java | 2 ++ .../it/autocreate/AbstractPipeDualAutoIT.java | 6 +++-- .../it/manual/AbstractPipeDualManualIT.java | 6 +++-- .../pipe/it/single/AbstractPipeSingleIT.java | 1 + .../agent/task/PipeDataNodeTaskAgent.java | 26 ++++++++++++++----- .../thrift/IoTDBDataNodeReceiver.java | 22 +++++++++++----- 9 files changed, 63 insertions(+), 18 deletions(-) diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java index fe4190d7708e..a7d8918d1c95 100644 --- a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java +++ b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppCommonConfig.java @@ -429,6 +429,12 @@ public CommonConfig setSchemaRegionPerDataNode(double schemaRegionPerDataNode) { return this; } + @Override + public CommonConfig setIsPipeEnableMemoryCheck(boolean isPipeEnableMemoryCheck) { + setProperty("pipe_enable_memory_checked", String.valueOf(isPipeEnableMemoryCheck)); + return this; + } + @Override public CommonConfig setPipeAirGapReceiverEnabled(boolean isPipeAirGapReceiverEnabled) { setProperty("pipe_air_gap_receiver_enabled", String.valueOf(isPipeAirGapReceiverEnabled)); diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppSharedCommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppSharedCommonConfig.java index c2ade6eace08..e4a4ace7a44c 100644 --- a/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppSharedCommonConfig.java +++ b/integration-test/src/main/java/org/apache/iotdb/it/env/cluster/config/MppSharedCommonConfig.java @@ -438,6 +438,13 @@ public CommonConfig setSchemaRegionPerDataNode(double schemaRegionPerDataNode) { return this; } + @Override + public CommonConfig setIsPipeEnableMemoryCheck(boolean isPipeEnableMemoryCheck) { + dnConfig.setIsPipeEnableMemoryCheck(isPipeEnableMemoryCheck); + cnConfig.setIsPipeEnableMemoryCheck(isPipeEnableMemoryCheck); + return this; + } + @Override public CommonConfig setPipeAirGapReceiverEnabled(boolean isPipeAirGapReceiverEnabled) { dnConfig.setPipeAirGapReceiverEnabled(isPipeAirGapReceiverEnabled); diff --git a/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteCommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteCommonConfig.java index 581c5f4345ed..23a8dad317e8 100644 --- a/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteCommonConfig.java +++ b/integration-test/src/main/java/org/apache/iotdb/it/env/remote/config/RemoteCommonConfig.java @@ -308,6 +308,11 @@ public CommonConfig setSchemaRegionPerDataNode(double schemaRegionPerDataNode) { return this; } + @Override + public CommonConfig setIsPipeEnableMemoryCheck(boolean isPipeEnableMemoryCheck) { + return this; + } + @Override public CommonConfig setPipeAirGapReceiverEnabled(boolean isPipeAirGapReceiverEnabled) { return this; diff --git a/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java b/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java index 16d509542093..178f182624b8 100644 --- a/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java +++ b/integration-test/src/main/java/org/apache/iotdb/itbase/env/CommonConfig.java @@ -138,6 +138,8 @@ CommonConfig setEnableAutoLeaderBalanceForIoTConsensus( CommonConfig setSchemaRegionPerDataNode(double schemaRegionPerDataNode); + CommonConfig setIsPipeEnableMemoryCheck(boolean isPipeEnableMemoryCheck); + CommonConfig setPipeAirGapReceiverEnabled(boolean isPipeAirGapReceiverEnabled); CommonConfig setDriverTaskExecutionTimeSliceInMs(long driverTaskExecutionTimeSliceInMs); diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/AbstractPipeDualAutoIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/AbstractPipeDualAutoIT.java index 1817efdaa1d3..59478fc36e67 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/AbstractPipeDualAutoIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/AbstractPipeDualAutoIT.java @@ -48,13 +48,15 @@ protected void setupConfig() { .getCommonConfig() .setAutoCreateSchemaEnabled(true) .setConfigNodeConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) - .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS); + .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) + .setIsPipeEnableMemoryCheck(false); receiverEnv .getConfig() .getCommonConfig() .setAutoCreateSchemaEnabled(true) .setConfigNodeConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) - .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS); + .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) + .setIsPipeEnableMemoryCheck(false); // 10 min, assert that the operations will not time out senderEnv.getConfig().getCommonConfig().setDnConnectionTimeoutMs(600000); diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/AbstractPipeDualManualIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/AbstractPipeDualManualIT.java index 3e9f51111656..b7091a1db329 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/AbstractPipeDualManualIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/AbstractPipeDualManualIT.java @@ -48,13 +48,15 @@ protected void setupConfig() { .getCommonConfig() .setAutoCreateSchemaEnabled(false) .setConfigNodeConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) - .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS); + .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) + .setIsPipeEnableMemoryCheck(false); receiverEnv .getConfig() .getCommonConfig() .setAutoCreateSchemaEnabled(false) .setConfigNodeConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) - .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS); + .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) + .setIsPipeEnableMemoryCheck(false); // 10 min, assert that the operations will not time out senderEnv.getConfig().getCommonConfig().setDnConnectionTimeoutMs(600000); diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/single/AbstractPipeSingleIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/single/AbstractPipeSingleIT.java index 4c5135764886..9a484426d762 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/single/AbstractPipeSingleIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/single/AbstractPipeSingleIT.java @@ -36,6 +36,7 @@ public void setUp() { env.getConfig().getCommonConfig().setAutoCreateSchemaEnabled(true); // 10 min, assert that the operations will not time out env.getConfig().getCommonConfig().setDnConnectionTimeoutMs(600000); + env.getConfig().getCommonConfig().setIsPipeEnableMemoryCheck(false); env.initClusterEnvironment(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java index 9ba7469e7c67..2a4569c73409 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java @@ -671,10 +671,10 @@ protected void calculateMemoryUsage( return; } + calculateInsertNodeQueueMemory(extractorParameters, processorParameters, connectorParameters); + long needMemory = 0; - needMemory += - calculateInsertNodeQueueMemory( - extractorParameters, processorParameters, connectorParameters); + needMemory += calculateTsFileParserMemory(extractorParameters, processorParameters, connectorParameters); needMemory += @@ -697,11 +697,12 @@ protected void calculateMemoryUsage( (PipeMemoryManager.getTotalMemorySizeInBytes() * PipeConfig.getInstance().getReservedMemoryPercentage()), PipeMemoryManager.getTotalMemorySizeInBytes()); + LOGGER.warn(e); throw new PipeException(e); } } - private long calculateInsertNodeQueueMemory( + private void calculateInsertNodeQueueMemory( final PipeParameters extractorParameters, final PipeParameters processorParameters, final PipeParameters connectorParameters) { @@ -710,7 +711,7 @@ private long calculateInsertNodeQueueMemory( if (!extractorParameters.getBooleanOrDefault( Arrays.asList(EXTRACTOR_REALTIME_ENABLE_KEY, SOURCE_REALTIME_ENABLE_KEY), EXTRACTOR_REALTIME_ENABLE_DEFAULT_VALUE)) { - return 0; + return; } // If the realtime mode is batch or file, we do not need to allocate memory @@ -720,10 +721,21 @@ private long calculateInsertNodeQueueMemory( PipeExtractorConstant.SOURCE_REALTIME_MODE_KEY); if (PipeExtractorConstant.EXTRACTOR_REALTIME_MODE_BATCH_MODE_VALUE.equals(realtimeMode) || PipeExtractorConstant.EXTRACTOR_REALTIME_MODE_FILE_VALUE.equals(realtimeMode)) { - return 0; + return; } - return PipeConfig.getInstance().getPipeInodeMemory(); + if (PipeMemoryManager.getTotalFloatingMemorySizeInBytes() + - this.getAllFloatingMemoryUsageInByte() + < PipeConfig.getInstance().getPipeInodeMemory()) { + final String m = + String.format( + "Not enough memory for pipe. Need Floating memory: %d bytes, free Floating memory: %d bytes", + PipeConfig.getInstance().getPipeInodeMemory(), + (PipeMemoryManager.getTotalFloatingMemorySizeInBytes() + - this.getAllFloatingMemoryUsageInByte())); + LOGGER.warn(m); + throw new PipeException(m); + } } private long calculateTsFileParserMemory( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java index 9c23f447a466..5009cae8a516 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/thrift/IoTDBDataNodeReceiver.java @@ -86,7 +86,6 @@ import org.apache.iotdb.db.storageengine.rescon.disk.strategy.DirectoryStrategyType; import org.apache.iotdb.db.tools.schema.SRStatementGenerator; import org.apache.iotdb.db.tools.schema.SchemaRegionSnapshotParser; -import org.apache.iotdb.pipe.api.exception.PipeException; import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; import org.apache.iotdb.service.rpc.thrift.TPipeTransferReq; @@ -175,12 +174,13 @@ public synchronized TPipeTransferResp receive(final TPipeTransferReq req) { case HANDSHAKE_DATANODE_V1: { try { - if (PipeConfig.getInstance().isPipeEnableMemoryCheck()) { - if (PipeDataNodeResourceManager.memory().getFreeMemorySizeInBytes() - < PipeConfig.getInstance().getPipeMinimumReceiverMemory()) { - throw new PipeException( - "The receiver memory is not enough to handle the handshake request from datanode."); - } + if (PipeConfig.getInstance().isPipeEnableMemoryCheck() + && PipeDataNodeResourceManager.memory().getFreeMemorySizeInBytes() + < PipeConfig.getInstance().getPipeMinimumReceiverMemory()) { + return new TPipeTransferResp( + RpcUtils.getStatus( + TSStatusCode.PIPE_HANDSHAKE_ERROR.getStatusCode(), + "The receiver memory is not enough to handle the handshake request from datanode.")); } return handleTransferHandshakeV1( PipeTransferDataNodeHandshakeV1Req.fromTPipeTransferReq(req)); @@ -192,6 +192,14 @@ public synchronized TPipeTransferResp receive(final TPipeTransferReq req) { case HANDSHAKE_DATANODE_V2: { try { + if (PipeConfig.getInstance().isPipeEnableMemoryCheck() + && PipeDataNodeResourceManager.memory().getFreeMemorySizeInBytes() + < PipeConfig.getInstance().getPipeMinimumReceiverMemory()) { + return new TPipeTransferResp( + RpcUtils.getStatus( + TSStatusCode.PIPE_HANDSHAKE_ERROR.getStatusCode(), + "The receiver memory is not enough to handle the handshake request from datanode.")); + } return handleTransferHandshakeV2( PipeTransferDataNodeHandshakeV2Req.fromTPipeTransferReq(req)); } finally { From 55da5405ac6e21c051d4db9d0222adf38cc9ce20 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 1 Jul 2025 17:18:56 +0800 Subject: [PATCH 069/185] partial fix --- .../resource/tsfile/PipeTsFileResourceManager.java | 8 +++++--- .../pipe/resource/PipeTsFileResourceManagerTest.java | 12 ++++++------ 2 files changed, 11 insertions(+), 9 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java index 12306be59663..79028896e647 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java @@ -22,6 +22,7 @@ import org.apache.iotdb.commons.conf.IoTDBConstant; import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.utils.FileUtils; +import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.db.storageengine.dataregion.modification.ModificationFile; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; @@ -110,7 +111,7 @@ public File increaseFileReference( getHardlinkOrCopiedFile2TsFileResourceMap(pipeName) .put(resultFile.getPath(), new PipeTsFileResource(resultFile, isTsFile)); - increaseAssignerReferenceIfExists(file, pipeName); + increaseAssignerReferenceIfExists(resultFile, pipeName); return resultFile; } finally { @@ -124,9 +125,9 @@ private boolean increaseReferenceIfExists(final File file, final @Nullable Strin getHardlinkOrCopiedFile2TsFileResourceMap(pipeName).get(path); if (resource != null) { resource.increaseReferenceCount(); + increaseAssignerReferenceIfExists(file, pipeName); return true; } - increaseAssignerReferenceIfExists(file, pipeName); return false; } @@ -228,7 +229,7 @@ private void decreaseAssignerReferenceIfExists(final File file, final @Nullable // Warning: Shall not be called by the assigner private String getAssignerFilePath(final File file) { // Skip the "pipeName" of this file - return file.getParentFile().getParent() + file.getName(); + return file.getParentFile().getParent() + File.separator + file.getName(); } /** @@ -237,6 +238,7 @@ private String getAssignerFilePath(final File file) { * @param hardlinkOrCopiedFile the copied or hardlinked file * @return the reference count of the file */ + @TestOnly public int getFileReferenceCount(final File hardlinkOrCopiedFile, final String pipeName) { segmentLock.lock(hardlinkOrCopiedFile); try { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java index 9867a1893142..dbd533ed9740 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java @@ -176,18 +176,18 @@ public void testIncreaseTsFile() throws IOException { // test use assigner's hardlinkTsFile to increase reference counts // test null, shall not reuse the pipe's tsFile - pipeTsFileResourceManager.increaseFileReference(assignerTsfile, true, PIPE_NAME); + final File pipeTsFile = + pipeTsFileResourceManager.increaseFileReference(assignerTsfile, true, PIPE_NAME); Assert.assertEquals(2, pipeTsFileResourceManager.getFileReferenceCount(assignerTsfile, null)); - Assert.assertEquals( - 1, pipeTsFileResourceManager.getFileReferenceCount(assignerTsfile, PIPE_NAME)); + Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsFile, PIPE_NAME)); Assert.assertTrue(Files.exists(originTsfile.toPath())); Assert.assertTrue(Files.exists(assignerTsfile.toPath())); // test use copyFile to increase reference counts - pipeTsFileResourceManager.increaseFileReference(assignerModFile, false, PIPE_NAME); + final File pipeModFile = + pipeTsFileResourceManager.increaseFileReference(assignerModFile, false, PIPE_NAME); Assert.assertEquals(2, pipeTsFileResourceManager.getFileReferenceCount(assignerModFile, null)); - Assert.assertEquals( - 1, pipeTsFileResourceManager.getFileReferenceCount(assignerModFile, PIPE_NAME)); + Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, PIPE_NAME)); Assert.assertTrue(Files.exists(originModFile.toPath())); Assert.assertTrue(Files.exists(assignerModFile.toPath())); } From 20e97a5f8ff86a73533e148d1666f8922785e077 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Tue, 1 Jul 2025 17:47:50 +0800 Subject: [PATCH 070/185] update IT --- .../iotdb/pipe/it/autocreate/IoTDBPipeAutoConflictIT.java | 6 ++++-- .../iotdb/pipe/it/autocreate/IoTDBPipeClusterIT.java | 6 ++++-- .../it/autocreate/IoTDBPipeConnectorCompressionIT.java | 7 +++++-- .../iotdb/pipe/it/autocreate/IoTDBPipeExtractorIT.java | 6 ++++-- .../iotdb/pipe/it/autocreate/IoTDBPipeIdempotentIT.java | 6 ++++-- .../iotdb/pipe/it/autocreate/IoTDBPipeProcessorIT.java | 6 ++++-- .../iotdb/pipe/it/autocreate/IoTDBPipeProtocolIT.java | 6 ++++-- .../iotdb/pipe/it/autocreate/IoTDBPipeWithLoadIT.java | 6 ++++-- .../it/cluster/IoTDBSubscriptionRestartIT.java | 3 ++- .../subscription/it/dual/AbstractSubscriptionDualIT.java | 3 +++ .../it/dual/IoTDBSubscriptionConsumerGroupIT.java | 3 +++ .../it/dual/IoTDBSubscriptionTimePrecisionIT.java | 2 ++ .../subscription/it/dual/IoTDBSubscriptionTopicIT.java | 3 +++ .../subscription/it/local/AbstractSubscriptionLocalIT.java | 1 + .../it/triple/AbstractSubscriptionTripleIT.java | 4 ++++ .../java/org/apache/iotdb/tools/it/ExportTsFileTestIT.java | 1 + .../iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java | 4 ++-- .../org/apache/iotdb/commons/pipe/config/PipeConfig.java | 2 +- 18 files changed, 55 insertions(+), 20 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeAutoConflictIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeAutoConflictIT.java index 65eb2458df77..9da5591bb2f7 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeAutoConflictIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeAutoConflictIT.java @@ -59,14 +59,16 @@ public void setUp() { .setAutoCreateSchemaEnabled(true) .setConfigNodeConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) - .setDataRegionConsensusProtocolClass(ConsensusFactory.IOT_CONSENSUS); + .setDataRegionConsensusProtocolClass(ConsensusFactory.IOT_CONSENSUS) + .setIsPipeEnableMemoryCheck(false); receiverEnv .getConfig() .getCommonConfig() .setAutoCreateSchemaEnabled(true) .setConfigNodeConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) - .setDataRegionConsensusProtocolClass(ConsensusFactory.IOT_CONSENSUS); + .setDataRegionConsensusProtocolClass(ConsensusFactory.IOT_CONSENSUS) + .setIsPipeEnableMemoryCheck(false); // 10 min, assert that the operations will not time out senderEnv.getConfig().getCommonConfig().setDnConnectionTimeoutMs(600000); diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeClusterIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeClusterIT.java index d2ca8ce138c2..c8d5bc0b4717 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeClusterIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeClusterIT.java @@ -74,7 +74,8 @@ public void setUp() { .setAutoCreateSchemaEnabled(true) .setConfigNodeConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) - .setDataRegionConsensusProtocolClass(ConsensusFactory.IOT_CONSENSUS); + .setDataRegionConsensusProtocolClass(ConsensusFactory.IOT_CONSENSUS) + .setIsPipeEnableMemoryCheck(false); receiverEnv .getConfig() @@ -84,7 +85,8 @@ public void setUp() { .setSchemaReplicationFactor(3) .setConfigNodeConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) - .setDataRegionConsensusProtocolClass(ConsensusFactory.IOT_CONSENSUS); + .setDataRegionConsensusProtocolClass(ConsensusFactory.IOT_CONSENSUS) + .setIsPipeEnableMemoryCheck(false); // 10 min, assert that the operations will not time out senderEnv.getConfig().getCommonConfig().setDnConnectionTimeoutMs(600000); diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCompressionIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCompressionIT.java index 972131500693..2114ed37c7f1 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCompressionIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeConnectorCompressionIT.java @@ -66,14 +66,17 @@ public void setUp() { .getCommonConfig() .setAutoCreateSchemaEnabled(true) .setConfigNodeConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) - .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS); + .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) + .setIsPipeEnableMemoryCheck(false); + receiverEnv .getConfig() .getCommonConfig() .setAutoCreateSchemaEnabled(true) .setPipeAirGapReceiverEnabled(true) .setConfigNodeConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) - .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS); + .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) + .setIsPipeEnableMemoryCheck(false); // 10 min, assert that the operations will not time out senderEnv.getConfig().getCommonConfig().setDnConnectionTimeoutMs(600000); diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeExtractorIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeExtractorIT.java index 0b4636c0d137..4b33de0a31c9 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeExtractorIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeExtractorIT.java @@ -71,13 +71,15 @@ public void setUp() { // Disable sender compaction for tsfile determination in loose range test .setEnableSeqSpaceCompaction(false) .setEnableUnseqSpaceCompaction(false) - .setEnableCrossSpaceCompaction(false); + .setEnableCrossSpaceCompaction(false) + .setIsPipeEnableMemoryCheck(false); receiverEnv .getConfig() .getCommonConfig() .setAutoCreateSchemaEnabled(true) .setConfigNodeConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) - .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS); + .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) + .setIsPipeEnableMemoryCheck(false); // 10 min, assert that the operations will not time out senderEnv.getConfig().getCommonConfig().setDnConnectionTimeoutMs(600000); diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeIdempotentIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeIdempotentIT.java index addf0314b400..771d50c97c68 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeIdempotentIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeIdempotentIT.java @@ -65,13 +65,15 @@ public void setUp() { // of the tested idempotent sql. .setDefaultSchemaRegionGroupNumPerDatabase(1) .setConfigNodeConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) - .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS); + .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) + .setIsPipeEnableMemoryCheck(false); receiverEnv .getConfig() .getCommonConfig() .setAutoCreateSchemaEnabled(true) .setConfigNodeConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) - .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS); + .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) + .setIsPipeEnableMemoryCheck(false); // 10 min, assert that the operations will not time out senderEnv.getConfig().getCommonConfig().setDnConnectionTimeoutMs(600000); diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeProcessorIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeProcessorIT.java index 125fd6972b17..13a63a585a8d 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeProcessorIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeProcessorIT.java @@ -59,13 +59,15 @@ public void setUp() { .setAutoCreateSchemaEnabled(true) .setTimestampPrecision("ms") .setConfigNodeConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) - .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS); + .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) + .setIsPipeEnableMemoryCheck(false); receiverEnv .getConfig() .getCommonConfig() .setAutoCreateSchemaEnabled(true) .setConfigNodeConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) - .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS); + .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) + .setIsPipeEnableMemoryCheck(false); // 10 min, assert that the operations will not time out senderEnv.getConfig().getCommonConfig().setDnConnectionTimeoutMs(600000); diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeProtocolIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeProtocolIT.java index c5d41f001cb8..1f6467685257 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeProtocolIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeProtocolIT.java @@ -74,7 +74,8 @@ private void innerSetUp( .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) .setDataRegionConsensusProtocolClass(dataRegionConsensus) .setSchemaReplicationFactor(schemaRegionReplicationFactor) - .setDataReplicationFactor(dataRegionReplicationFactor); + .setDataReplicationFactor(dataRegionReplicationFactor) + .setIsPipeEnableMemoryCheck(false); receiverEnv .getConfig() .getCommonConfig() @@ -83,7 +84,8 @@ private void innerSetUp( .setSchemaRegionConsensusProtocolClass(schemaRegionConsensus) .setDataRegionConsensusProtocolClass(dataRegionConsensus) .setSchemaReplicationFactor(schemaRegionReplicationFactor) - .setDataReplicationFactor(dataRegionReplicationFactor); + .setDataReplicationFactor(dataRegionReplicationFactor) + .setIsPipeEnableMemoryCheck(false); // 10 min, assert that the operations will not time out senderEnv.getConfig().getCommonConfig().setDnConnectionTimeoutMs(600000); diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeWithLoadIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeWithLoadIT.java index 563377c927ea..c71e212b629a 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeWithLoadIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/autocreate/IoTDBPipeWithLoadIT.java @@ -61,13 +61,15 @@ public void setUp() { // Disable sender compaction to test mods .setEnableSeqSpaceCompaction(false) .setEnableUnseqSpaceCompaction(false) - .setEnableCrossSpaceCompaction(false); + .setEnableCrossSpaceCompaction(false) + .setIsPipeEnableMemoryCheck(false); receiverEnv .getConfig() .getCommonConfig() .setAutoCreateSchemaEnabled(true) .setConfigNodeConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) - .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS); + .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) + .setIsPipeEnableMemoryCheck(false); // 10 min, assert that the operations will not time out senderEnv.getConfig().getCommonConfig().setDnConnectionTimeoutMs(600000); diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/cluster/IoTDBSubscriptionRestartIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/cluster/IoTDBSubscriptionRestartIT.java index de17dec1ed42..8156139b84ce 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/cluster/IoTDBSubscriptionRestartIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/cluster/IoTDBSubscriptionRestartIT.java @@ -82,7 +82,8 @@ public void setUp() throws Exception { .setSchemaRegionConsensusProtocolClass(ConsensusFactory.RATIS_CONSENSUS) .setDataRegionConsensusProtocolClass(ConsensusFactory.IOT_CONSENSUS) .setSchemaReplicationFactor(3) - .setDataReplicationFactor(2); + .setDataReplicationFactor(2) + .setIsPipeEnableMemoryCheck(false); EnvFactory.getEnv().initClusterEnvironment(3, 3); } diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/AbstractSubscriptionDualIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/AbstractSubscriptionDualIT.java index 594f9efe6915..5e61607ff32b 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/AbstractSubscriptionDualIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/AbstractSubscriptionDualIT.java @@ -58,6 +58,9 @@ protected void setUpConfig() { // 10 min, assert that the operations will not time out senderEnv.getConfig().getCommonConfig().setDnConnectionTimeoutMs(600000); receiverEnv.getConfig().getCommonConfig().setDnConnectionTimeoutMs(600000); + + senderEnv.getConfig().getCommonConfig().setIsPipeEnableMemoryCheck(false); + receiverEnv.getConfig().getCommonConfig().setIsPipeEnableMemoryCheck(false); } @Override diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java index 6f8cba43e792..5ad46b10977c 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionConsumerGroupIT.java @@ -117,8 +117,11 @@ static final class SubscriptionInfo { protected void setUpConfig() { super.setUpConfig(); + senderEnv.getConfig().getCommonConfig().setIsPipeEnableMemoryCheck(false); + // Enable air gap receiver receiverEnv.getConfig().getCommonConfig().setPipeAirGapReceiverEnabled(true); + receiverEnv.getConfig().getCommonConfig().setIsPipeEnableMemoryCheck(false); } @Override diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTimePrecisionIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTimePrecisionIT.java index 4327b7a00f04..1939a986c68f 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTimePrecisionIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTimePrecisionIT.java @@ -68,7 +68,9 @@ protected void setUpConfig() { // Set timestamp precision to nanosecond senderEnv.getConfig().getCommonConfig().setTimestampPrecision("ns"); + senderEnv.getConfig().getCommonConfig().setIsPipeEnableMemoryCheck(false); receiverEnv.getConfig().getCommonConfig().setTimestampPrecision("ns"); + receiverEnv.getConfig().getCommonConfig().setIsPipeEnableMemoryCheck(false); } @Test diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java index b046f09a6da9..37971833b97a 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/dual/IoTDBSubscriptionTopicIT.java @@ -91,6 +91,9 @@ protected void setUpConfig() { .setPipeHeartbeatIntervalSecondsForCollectingPipeMeta(30); senderEnv.getConfig().getCommonConfig().setPipeMetaSyncerInitialSyncDelayMinutes(1); senderEnv.getConfig().getCommonConfig().setPipeMetaSyncerSyncIntervalMinutes(1); + senderEnv.getConfig().getCommonConfig().setIsPipeEnableMemoryCheck(false); + + receiverEnv.getConfig().getCommonConfig().setIsPipeEnableMemoryCheck(false); } @Test diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/AbstractSubscriptionLocalIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/AbstractSubscriptionLocalIT.java index 3f4150519287..36aea89df2c2 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/AbstractSubscriptionLocalIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/local/AbstractSubscriptionLocalIT.java @@ -34,6 +34,7 @@ public void setUp() throws Exception { // enable subscription EnvFactory.getEnv().getConfig().getCommonConfig().setSubscriptionEnabled(true); + EnvFactory.getEnv().getConfig().getCommonConfig().setIsPipeEnableMemoryCheck(false); EnvFactory.getEnv().initClusterEnvironment(); } diff --git a/integration-test/src/test/java/org/apache/iotdb/subscription/it/triple/AbstractSubscriptionTripleIT.java b/integration-test/src/test/java/org/apache/iotdb/subscription/it/triple/AbstractSubscriptionTripleIT.java index 3b49eb8ed131..7c2a969835b7 100644 --- a/integration-test/src/test/java/org/apache/iotdb/subscription/it/triple/AbstractSubscriptionTripleIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/subscription/it/triple/AbstractSubscriptionTripleIT.java @@ -69,6 +69,10 @@ protected void setUpConfig() { sender.getConfig().getCommonConfig().setDnConnectionTimeoutMs(600000); receiver1.getConfig().getCommonConfig().setDnConnectionTimeoutMs(600000); receiver2.getConfig().getCommonConfig().setDnConnectionTimeoutMs(600000); + + sender.getConfig().getCommonConfig().setIsPipeEnableMemoryCheck(false); + receiver1.getConfig().getCommonConfig().setIsPipeEnableMemoryCheck(false); + receiver2.getConfig().getCommonConfig().setIsPipeEnableMemoryCheck(false); } @Override diff --git a/integration-test/src/test/java/org/apache/iotdb/tools/it/ExportTsFileTestIT.java b/integration-test/src/test/java/org/apache/iotdb/tools/it/ExportTsFileTestIT.java index 6ad0c843e274..71740e408640 100644 --- a/integration-test/src/test/java/org/apache/iotdb/tools/it/ExportTsFileTestIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/tools/it/ExportTsFileTestIT.java @@ -54,6 +54,7 @@ public class ExportTsFileTestIT extends AbstractScriptIT { public static void setUp() throws Exception { // enable subscription EnvFactory.getEnv().getConfig().getCommonConfig().setSubscriptionEnabled(true); + EnvFactory.getEnv().getConfig().getCommonConfig().setIsPipeEnableMemoryCheck(false); EnvFactory.getEnv().initClusterEnvironment(); ip = EnvFactory.getEnv().getIP(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java index 2a4569c73409..45094b97fb2b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java @@ -726,11 +726,11 @@ private void calculateInsertNodeQueueMemory( if (PipeMemoryManager.getTotalFloatingMemorySizeInBytes() - this.getAllFloatingMemoryUsageInByte() - < PipeConfig.getInstance().getPipeInodeMemory()) { + < PipeConfig.getInstance().PipeInsertNodeQueueMemory()) { final String m = String.format( "Not enough memory for pipe. Need Floating memory: %d bytes, free Floating memory: %d bytes", - PipeConfig.getInstance().getPipeInodeMemory(), + PipeConfig.getInstance().PipeInsertNodeQueueMemory(), (PipeMemoryManager.getTotalFloatingMemorySizeInBytes() - this.getAllFloatingMemoryUsageInByte())); LOGGER.warn(m); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java index 3a09b31f49e6..62631749c7b2 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java @@ -99,7 +99,7 @@ public boolean isPipeEnableMemoryCheck() { return COMMON_CONFIG.isPipeEnableMemoryChecked(); } - public long getPipeInodeMemory() { + public long PipeInsertNodeQueueMemory() { return COMMON_CONFIG.getPipeInsertNodeQueueMemory(); } From 4ce201cb92f4f4bb48f10c552603a54e38f54029 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 1 Jul 2025 17:57:09 +0800 Subject: [PATCH 071/185] Update PipeTsFileResourceManagerTest.java --- .../tsfile/PipeTsFileInsertionEvent.java | 17 +----- .../realtime/PipeRealtimeEventFactory.java | 2 +- .../PipeTsFileResourceManagerTest.java | 56 +++++++++---------- 3 files changed, 30 insertions(+), 45 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java index 12f716463516..a0f036635635 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java @@ -87,23 +87,10 @@ public class PipeTsFileInsertionEvent extends EnrichedEvent private volatile ProgressIndex overridingProgressIndex; - public PipeTsFileInsertionEvent( - final TsFileResource resource, - final boolean isLoaded, - final boolean isGeneratedByHistoricalExtractor) { + public PipeTsFileInsertionEvent(final TsFileResource resource, final boolean isLoaded) { // The modFile must be copied before the event is assigned to the listening pipes this( - resource, - null, - true, - isLoaded, - isGeneratedByHistoricalExtractor, - null, - 0, - null, - null, - Long.MIN_VALUE, - Long.MAX_VALUE); + resource, null, true, isLoaded, false, null, 0, null, null, Long.MIN_VALUE, Long.MAX_VALUE); } public PipeTsFileInsertionEvent( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java index ce4179f18a66..0c3bce5c399e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java @@ -36,7 +36,7 @@ public class PipeRealtimeEventFactory { public static PipeRealtimeEvent createRealtimeEvent( final TsFileResource resource, final boolean isLoaded) { return TS_FILE_EPOCH_MANAGER.bindPipeTsFileInsertionEvent( - new PipeTsFileInsertionEvent(resource, isLoaded, false), resource); + new PipeTsFileInsertionEvent(resource, isLoaded), resource); } public static PipeRealtimeEvent createRealtimeEvent( diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java index dbd533ed9740..9db31eb3759f 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java @@ -24,7 +24,6 @@ import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.utils.FileUtils; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; -import org.apache.iotdb.db.pipe.resource.tsfile.PipeTsFileResource; import org.apache.iotdb.db.pipe.resource.tsfile.PipeTsFileResourceManager; import org.apache.iotdb.db.storageengine.dataregion.modification.Deletion; import org.apache.iotdb.db.storageengine.dataregion.modification.Modification; @@ -47,9 +46,7 @@ import java.io.File; import java.io.IOException; import java.nio.file.Files; -import java.util.concurrent.TimeUnit; -import static org.awaitility.Awaitility.await; import static org.junit.Assert.fail; public class PipeTsFileResourceManagerTest { @@ -194,48 +191,49 @@ public void testIncreaseTsFile() throws IOException { @Test public void testDecreaseTsFile() throws IOException { - File originFile = new File(TS_FILE_NAME); - File originModFile = new File(MODS_FILE_NAME); + final File originFile = new File(TS_FILE_NAME); + final File originModFile = new File(MODS_FILE_NAME); pipeTsFileResourceManager.decreaseFileReference(originFile, null); pipeTsFileResourceManager.decreaseFileReference(originModFile, null); Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(originFile, null)); Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(originModFile, null)); - File pipeTsfile = pipeTsFileResourceManager.increaseFileReference(originFile, true, null); - File pipeModFile = pipeTsFileResourceManager.increaseFileReference(originModFile, false, null); - Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, null)); - Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, null)); - Assert.assertTrue(Files.exists(pipeTsfile.toPath())); - Assert.assertTrue(Files.exists(pipeModFile.toPath())); - Assert.assertTrue(Files.exists(pipeTsfile.toPath())); - Assert.assertTrue(Files.exists(pipeModFile.toPath())); + final File assignerTsfile = + pipeTsFileResourceManager.increaseFileReference(originFile, true, null); + final File assignerModFile = + pipeTsFileResourceManager.increaseFileReference(originModFile, false, null); + Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(assignerTsfile, null)); + Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(assignerModFile, null)); + Assert.assertTrue(Files.exists(assignerTsfile.toPath())); + Assert.assertTrue(Files.exists(assignerModFile.toPath())); + Assert.assertTrue(Files.exists(assignerTsfile.toPath())); + Assert.assertTrue(Files.exists(assignerModFile.toPath())); Assert.assertTrue(originFile.delete()); Assert.assertTrue(originModFile.delete()); Assert.assertFalse(Files.exists(originFile.toPath())); Assert.assertFalse(Files.exists(originModFile.toPath())); - Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, null)); - Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, null)); + final File pipeTsFile = + pipeTsFileResourceManager.increaseFileReference(assignerTsfile, true, PIPE_NAME); + final File pipeModFile = + pipeTsFileResourceManager.increaseFileReference(assignerModFile, false, PIPE_NAME); + pipeTsFileResourceManager.decreaseFileReference(pipeTsFile, PIPE_NAME); + pipeTsFileResourceManager.decreaseFileReference(pipeModFile, PIPE_NAME); + + Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(assignerTsfile, null)); + Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(assignerModFile, null)); Assert.assertFalse(Files.exists(originFile.toPath())); Assert.assertFalse(Files.exists(originModFile.toPath())); - Assert.assertTrue(Files.exists(pipeTsfile.toPath())); - Assert.assertTrue(Files.exists(pipeModFile.toPath())); + Assert.assertTrue(Files.exists(assignerTsfile.toPath())); + Assert.assertTrue(Files.exists(assignerModFile.toPath())); - pipeTsFileResourceManager.decreaseFileReference(pipeTsfile, null); - pipeTsFileResourceManager.decreaseFileReference(pipeModFile, null); - Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, null)); - Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, null)); + pipeTsFileResourceManager.decreaseFileReference(assignerTsfile, null); + pipeTsFileResourceManager.decreaseFileReference(assignerModFile, null); + Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(assignerTsfile, null)); + Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(assignerModFile, null)); Assert.assertFalse(Files.exists(originFile.toPath())); Assert.assertFalse(Files.exists(originModFile.toPath())); - // Pipe TsFile will be cleaned by a timed thread, so we wait some time here. - await() - .atMost(3 * PipeTsFileResource.TSFILE_MIN_TIME_TO_LIVE_IN_MS, TimeUnit.MILLISECONDS) - .untilAsserted( - () -> { - Assert.assertFalse(Files.exists(pipeTsfile.toPath())); - Assert.assertFalse(Files.exists(pipeModFile.toPath())); - }); } } From 5d16d6581a4bab58f31f4a1c78f989717d021fca Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 1 Jul 2025 18:08:50 +0800 Subject: [PATCH 072/185] Fix 2/3 of the floating memory --- .../realtime/PipeRealtimeDataRegionHybridExtractor.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java index 07eefabfb85d..78522a000277 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java @@ -216,7 +216,7 @@ private boolean mayInsertNodeMemoryReachDangerousThreshold(final PipeRealtimeEve final long totalFloatingMemorySizeInBytes = PipeMemoryManager.getTotalFloatingMemorySizeInBytes(); final boolean mayInsertNodeMemoryReachDangerousThreshold = - 3 * floatingMemoryUsageInByte * pipeCount >= 2 * totalFloatingMemorySizeInBytes; + floatingMemoryUsageInByte * pipeCount >= totalFloatingMemorySizeInBytes; if (mayInsertNodeMemoryReachDangerousThreshold && event.mayExtractorUseTablets(this)) { logByLogManager( l -> @@ -225,7 +225,7 @@ private boolean mayInsertNodeMemoryReachDangerousThreshold(final PipeRealtimeEve pipeName, dataRegionId, floatingMemoryUsageInByte * pipeCount, - 2 * totalFloatingMemorySizeInBytes / 3.0d)); + totalFloatingMemorySizeInBytes)); } return mayInsertNodeMemoryReachDangerousThreshold; } From 6a69e5d595f9870da3248ff785b0d51d1e8337d6 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 1 Jul 2025 18:42:13 +0800 Subject: [PATCH 073/185] Fixed UT --- .../connector/PipeConfigNodeThriftRequestTest.java | 4 ---- .../connector/PipeDataNodeThriftRequestTest.java | 12 ------------ 2 files changed, 16 deletions(-) diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/connector/PipeConfigNodeThriftRequestTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/connector/PipeConfigNodeThriftRequestTest.java index c2c04f945299..8854ee647f99 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/connector/PipeConfigNodeThriftRequestTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/manager/pipe/connector/PipeConfigNodeThriftRequestTest.java @@ -44,7 +44,6 @@ public void testPipeTransferConfigHandshakeReq() throws IOException { Assert.assertEquals(req.getVersion(), deserializeReq.getVersion()); Assert.assertEquals(req.getType(), deserializeReq.getType()); - Assert.assertArrayEquals(req.getBody(), deserializeReq.getBody()); Assert.assertEquals(req.getTimestampPrecision(), deserializeReq.getTimestampPrecision()); } @@ -57,7 +56,6 @@ public void testPipeTransferConfigPlanReq() { Assert.assertEquals(req.getVersion(), deserializeReq.getVersion()); Assert.assertEquals(req.getType(), deserializeReq.getType()); - Assert.assertArrayEquals(req.getBody(), deserializeReq.getBody()); } @Test @@ -72,7 +70,6 @@ public void testPipeTransferConfigSnapshotPieceReq() throws IOException { Assert.assertEquals(req.getVersion(), deserializeReq.getVersion()); Assert.assertEquals(req.getType(), deserializeReq.getType()); - Assert.assertArrayEquals(req.getBody(), deserializeReq.getBody()); Assert.assertEquals(req.getFileName(), deserializeReq.getFileName()); Assert.assertEquals(req.getStartWritingOffset(), deserializeReq.getStartWritingOffset()); @@ -95,7 +92,6 @@ public void testPipeTransferConfigSnapshotSealReq() throws IOException { Assert.assertEquals(req.getVersion(), deserializeReq.getVersion()); Assert.assertEquals(req.getType(), deserializeReq.getType()); - Assert.assertArrayEquals(req.getBody(), deserializeReq.getBody()); Assert.assertEquals(req.getFileNames(), deserializeReq.getFileNames()); Assert.assertEquals(req.getFileLengths(), deserializeReq.getFileLengths()); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeDataNodeThriftRequestTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeDataNodeThriftRequestTest.java index 8e69473b95c6..a581d111dc6b 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeDataNodeThriftRequestTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/connector/PipeDataNodeThriftRequestTest.java @@ -71,7 +71,6 @@ public void testPipeTransferDataNodeHandshakeReq() throws IOException { Assert.assertEquals(req.getVersion(), deserializeReq.getVersion()); Assert.assertEquals(req.getType(), deserializeReq.getType()); - Assert.assertArrayEquals(req.getBody(), deserializeReq.getBody()); Assert.assertEquals(req.getTimestampPrecision(), deserializeReq.getTimestampPrecision()); } @@ -94,7 +93,6 @@ public void testPipeTransferInsertNodeReq() { Assert.assertEquals(req.getVersion(), deserializeReq.getVersion()); Assert.assertEquals(req.getType(), deserializeReq.getType()); - Assert.assertArrayEquals(req.getBody(), deserializeReq.getBody()); Assert.assertEquals(req.getInsertNode(), deserializeReq.getInsertNode()); @@ -114,7 +112,6 @@ public void testPipeTransferTabletBinaryReq() { Assert.assertEquals(req.getVersion(), deserializeReq.getVersion()); Assert.assertEquals(req.getType(), deserializeReq.getType()); - Assert.assertArrayEquals(req.getBody(), deserializeReq.getBody()); } @Test @@ -137,7 +134,6 @@ public void testPipeTransferSchemaPlanReq() { Assert.assertEquals(req.getVersion(), deserializeReq.getVersion()); Assert.assertEquals(req.getType(), deserializeReq.getType()); - Assert.assertArrayEquals(req.getBody(), deserializeReq.getBody()); Assert.assertEquals(req.getPlanNode(), deserializeReq.getPlanNode()); } @@ -176,7 +172,6 @@ public void testPipeTransferTabletReq() { Assert.assertEquals(req.getVersion(), deserializeReq.getVersion()); Assert.assertEquals(req.getType(), deserializeReq.getType()); - Assert.assertArrayEquals(req.getBody(), deserializeReq.getBody()); final Statement statement = req.constructStatement(); // will call PipeTransferTabletRawReq.sortTablet() here @@ -264,8 +259,6 @@ public void testPipeTransferTabletBatchReq() throws IOException { final PipeTransferTabletBatchReq deserializedReq = PipeTransferTabletBatchReq.fromTPipeTransferReq(req); - Assert.assertArrayEquals( - new byte[] {'a', 'b'}, deserializedReq.getBinaryReqs().get(0).getBody()); Assert.assertEquals(node, deserializedReq.getInsertNodeReqs().get(0).getInsertNode()); Assert.assertEquals(t, deserializedReq.getTabletReqs().get(0).getTablet()); Assert.assertFalse(deserializedReq.getTabletReqs().get(0).getIsAligned()); @@ -283,7 +276,6 @@ public void testPipeTransferFilePieceReq() throws IOException { Assert.assertEquals(req.getVersion(), deserializeReq.getVersion()); Assert.assertEquals(req.getType(), deserializeReq.getType()); - Assert.assertArrayEquals(req.getBody(), deserializeReq.getBody()); Assert.assertEquals(req.getFileName(), deserializeReq.getFileName()); Assert.assertEquals(req.getStartWritingOffset(), deserializeReq.getStartWritingOffset()); @@ -302,7 +294,6 @@ public void testPipeTransferFilePieceWithModReq() throws IOException { Assert.assertEquals(req.getVersion(), deserializeReq.getVersion()); Assert.assertEquals(req.getType(), deserializeReq.getType()); - Assert.assertArrayEquals(req.getBody(), deserializeReq.getBody()); Assert.assertEquals(req.getFileName(), deserializeReq.getFileName()); Assert.assertEquals(req.getStartWritingOffset(), deserializeReq.getStartWritingOffset()); @@ -321,7 +312,6 @@ public void testPipeTransferSchemaSnapshotPieceReq() throws IOException { Assert.assertEquals(req.getVersion(), deserializeReq.getVersion()); Assert.assertEquals(req.getType(), deserializeReq.getType()); - Assert.assertArrayEquals(req.getBody(), deserializeReq.getBody()); Assert.assertEquals(req.getFileName(), deserializeReq.getFileName()); Assert.assertEquals(req.getStartWritingOffset(), deserializeReq.getStartWritingOffset()); @@ -339,7 +329,6 @@ public void testPipeTransferTsFileSealReq() throws IOException { Assert.assertEquals(req.getVersion(), deserializeReq.getVersion()); Assert.assertEquals(req.getType(), deserializeReq.getType()); - Assert.assertArrayEquals(req.getBody(), deserializeReq.getBody()); Assert.assertEquals(req.getFileName(), deserializeReq.getFileName()); Assert.assertEquals(req.getFileLength(), deserializeReq.getFileLength()); @@ -361,7 +350,6 @@ public void testPipeTransferSchemaSnapshotSealReq() throws IOException { Assert.assertEquals(req.getVersion(), deserializeReq.getVersion()); Assert.assertEquals(req.getType(), deserializeReq.getType()); - Assert.assertArrayEquals(req.getBody(), deserializeReq.getBody()); Assert.assertEquals(req.getFileNames(), deserializeReq.getFileNames()); Assert.assertEquals(req.getFileLengths(), deserializeReq.getFileLengths()); From 751ac9de7764438bab1a1928e026b471e05c9bf2 Mon Sep 17 00:00:00 2001 From: Steve Yurong Su Date: Tue, 1 Jul 2025 18:46:02 +0800 Subject: [PATCH 074/185] impl replace in PipeRealtimePriorityBlockingQueue.java --- .../dml/insertion/TsFileInsertionEvent.java | 10 +++ .../PipeRealtimePriorityBlockingQueue.java | 62 +++++++++++++++---- .../tsfile/PipeTsFileInsertionEvent.java | 1 + .../tsfile/PipeCompactionManager.java | 8 ++- .../task/connection/BlockingPendingQueue.java | 20 +++--- .../UnboundedBlockingPendingQueue.java | 6 +- 6 files changed, 79 insertions(+), 28 deletions(-) diff --git a/iotdb-api/pipe-api/src/main/java/org/apache/iotdb/pipe/api/event/dml/insertion/TsFileInsertionEvent.java b/iotdb-api/pipe-api/src/main/java/org/apache/iotdb/pipe/api/event/dml/insertion/TsFileInsertionEvent.java index 3d0418672427..4c7fffcfba51 100644 --- a/iotdb-api/pipe-api/src/main/java/org/apache/iotdb/pipe/api/event/dml/insertion/TsFileInsertionEvent.java +++ b/iotdb-api/pipe-api/src/main/java/org/apache/iotdb/pipe/api/event/dml/insertion/TsFileInsertionEvent.java @@ -21,6 +21,8 @@ import org.apache.iotdb.pipe.api.event.Event; +import java.io.File; + /** * {@link TsFileInsertionEvent} is used to define the event of writing TsFile. Event data stores in * disks, which is compressed and encoded, and requires IO cost for computational processing. @@ -34,4 +36,12 @@ public interface TsFileInsertionEvent extends Event, AutoCloseable { * @return {@code Iterable} the list of {@link TabletInsertionEvent} */ Iterable toTabletInsertionEvents(); + + /** + * Get the file that stores the data of this {@link TsFileInsertionEvent}. The file is compressed + * and encoded, and requires IO cost for computational processing. + * + * @return the file that stores the data of this {@link TsFileInsertionEvent} + */ + File getTsFile(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java index a4f05447eae2..1b4d36541155 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java @@ -21,23 +21,37 @@ import org.apache.iotdb.commons.pipe.agent.task.connection.BlockingPendingQueue; import org.apache.iotdb.commons.pipe.agent.task.connection.UnboundedBlockingPendingQueue; +import org.apache.iotdb.commons.pipe.agent.task.progress.CommitterKey; import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.db.pipe.agent.task.connection.PipeEventCollector; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; +import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.pipe.metric.source.PipeDataRegionEventCounter; import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.pipe.api.event.dml.insertion.TsFileInsertionEvent; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; import java.util.Objects; +import java.util.Set; import java.util.concurrent.BlockingDeque; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; +import java.util.stream.Collectors; public class PipeRealtimePriorityBlockingQueue extends UnboundedBlockingPendingQueue { + private static final Logger LOGGER = + LoggerFactory.getLogger(PipeRealtimePriorityBlockingQueue.class); + private static final PipeConfig PIPE_CONFIG = PipeConfig.getInstance(); private final BlockingDeque tsfileInsertEventDeque = @@ -55,7 +69,7 @@ public PipeRealtimePriorityBlockingQueue() { } @Override - public boolean directOffer(final Event event) { + public synchronized boolean directOffer(final Event event) { checkBeforeOffer(event); if (event instanceof TsFileInsertionEvent) { @@ -73,18 +87,18 @@ public boolean directOffer(final Event event) { } @Override - public boolean waitedOffer(final Event event) { + public synchronized boolean waitedOffer(final Event event) { return directOffer(event); } @Override - public boolean put(final Event event) { + public synchronized boolean put(final Event event) { directOffer(event); return true; } @Override - public Event directPoll() { + public synchronized Event directPoll() { Event event = null; final int pollHistoricalTsFileThreshold = PIPE_CONFIG.getPipeRealTimeQueuePollHistoricalTsFileThreshold(); @@ -129,7 +143,7 @@ public Event directPoll() { * available. */ @Override - public Event waitedPoll() { + public synchronized Event waitedPoll() { Event event = null; final int pollHistoricalTsFileThreshold = PIPE_CONFIG.getPipeRealTimeQueuePollHistoricalTsFileThreshold(); @@ -176,7 +190,7 @@ public Event waitedPoll() { } @Override - public Event peek() { + public synchronized Event peek() { final Event event = pendingQueue.peek(); if (Objects.nonNull(event)) { return event; @@ -184,20 +198,46 @@ public Event peek() { return tsfileInsertEventDeque.peek(); } + public synchronized void replace( + String dataRegionId, Set sourceFiles, Set targetFiles) { + final Map> eventsToBeRemovedGroupByCommitterKey = + tsfileInsertEventDeque.stream() + .filter(event -> event instanceof PipeTsFileInsertionEvent) + .map(event -> (PipeTsFileInsertionEvent) event) + .collect( + Collectors.groupingBy( + PipeTsFileInsertionEvent::getCommitterKey, Collectors.toSet())) + .entrySet() + .stream() + .filter(entry -> entry.getValue().size() == sourceFiles.size()) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + final Map eventToBeAddedGroupByCommitterKey = + new HashMap<>(); + // TBD + + final Set eventsToRemove = new HashSet<>(); + for (Set pipeTsFileInsertionEvents : + eventsToBeRemovedGroupByCommitterKey.values()) { + eventsToRemove.addAll(pipeTsFileInsertionEvents); + } + tsfileInsertEventDeque.removeIf(eventsToRemove::contains); + } + @Override - public void clear() { + public synchronized void clear() { super.clear(); tsfileInsertEventDeque.clear(); } @Override - public void forEach(final Consumer action) { + public synchronized void forEach(final Consumer action) { super.forEach(action); tsfileInsertEventDeque.forEach(action); } @Override - public void discardAllEvents() { + public synchronized void discardAllEvents() { super.discardAllEvents(); tsfileInsertEventDeque.removeIf( event -> { @@ -212,7 +252,7 @@ public void discardAllEvents() { } @Override - public void discardEventsOfPipe(final String pipeNameToDrop, final int regionId) { + public synchronized void discardEventsOfPipe(final String pipeNameToDrop, final int regionId) { super.discardEventsOfPipe(pipeNameToDrop, regionId); tsfileInsertEventDeque.removeIf( event -> { @@ -244,7 +284,7 @@ public int getTsFileInsertionEventCount() { return tsfileInsertEventDeque.size(); } - public void setOfferTsFileCounter(AtomicInteger offerTsFileCounter) { + public synchronized void setOfferTsFileCounter(AtomicInteger offerTsFileCounter) { this.offerTsFileCounter = offerTsFileCounter; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java index 12f716463516..aa9f492a2d61 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java @@ -211,6 +211,7 @@ public boolean waitForTsFileClose() throws InterruptedException { return !resource.isEmpty(); } + @Override public File getTsFile() { return tsFile; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeCompactionManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeCompactionManager.java index 970e819ea91e..2aa612235765 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeCompactionManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeCompactionManager.java @@ -21,6 +21,7 @@ import org.apache.iotdb.commons.pipe.agent.task.connection.UnboundedBlockingPendingQueue; import org.apache.iotdb.db.pipe.agent.task.subtask.connector.PipeConnectorSubtaskLifeCycle; +import org.apache.iotdb.db.pipe.agent.task.subtask.connector.PipeRealtimePriorityBlockingQueue; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.iotdb.pipe.api.event.Event; @@ -61,8 +62,11 @@ public void emitResult( for (final PipeConnectorSubtaskLifeCycle lifeCycle : pipeConnectorSubtaskLifeCycles) { final UnboundedBlockingPendingQueue pendingQueue = lifeCycle.getPendingQueue(); - if (pendingQueue != null) { - pendingQueue.replace(sourceFiles, targetFiles); + // TODO: support non realtime priority blocking queue + if (pendingQueue instanceof PipeRealtimePriorityBlockingQueue) { + final PipeRealtimePriorityBlockingQueue realtimePriorityBlockingQueue = + (PipeRealtimePriorityBlockingQueue) pendingQueue; + realtimePriorityBlockingQueue.replace(dataRegionId, sourceFiles, targetFiles); } } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/connection/BlockingPendingQueue.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/connection/BlockingPendingQueue.java index aa93b0932545..2fde7ffef1db 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/connection/BlockingPendingQueue.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/connection/BlockingPendingQueue.java @@ -50,7 +50,7 @@ protected BlockingPendingQueue( this.eventCounter = eventCounter; } - public boolean waitedOffer(final E event) { + public synchronized boolean waitedOffer(final E event) { checkBeforeOffer(event); try { final boolean offered = @@ -69,7 +69,7 @@ public boolean waitedOffer(final E event) { } } - public boolean directOffer(final E event) { + public synchronized boolean directOffer(final E event) { checkBeforeOffer(event); final boolean offered = pendingQueue.offer(event); if (offered) { @@ -78,7 +78,7 @@ public boolean directOffer(final E event) { return offered; } - public boolean put(final E event) { + public synchronized boolean put(final E event) { checkBeforeOffer(event); try { pendingQueue.put(event); @@ -91,13 +91,13 @@ public boolean put(final E event) { } } - public E directPoll() { + public synchronized E directPoll() { final E event = pendingQueue.poll(); eventCounter.decreaseEventCount(event); return event; } - public E waitedPoll() { + public synchronized E waitedPoll() { E event = null; try { event = @@ -112,22 +112,22 @@ public E waitedPoll() { return event; } - public E peek() { + public synchronized E peek() { return pendingQueue.peek(); } - public void clear() { + public synchronized void clear() { isClosed.set(true); pendingQueue.clear(); eventCounter.reset(); } /** DO NOT FORGET to set eventCounter to new value after invoking this method. */ - public void forEach(final Consumer action) { + public synchronized void forEach(final Consumer action) { pendingQueue.forEach(action); } - public void discardAllEvents() { + public synchronized void discardAllEvents() { isClosed.set(true); pendingQueue.removeIf( event -> { @@ -141,7 +141,7 @@ public void discardAllEvents() { eventCounter.reset(); } - public void discardEventsOfPipe(final String pipeNameToDrop, final int regionId) { + public synchronized void discardEventsOfPipe(final String pipeNameToDrop, final int regionId) { pendingQueue.removeIf( event -> { if (event instanceof EnrichedEvent diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/connection/UnboundedBlockingPendingQueue.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/connection/UnboundedBlockingPendingQueue.java index 2702d3b182fa..1172ae19a8f2 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/connection/UnboundedBlockingPendingQueue.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/connection/UnboundedBlockingPendingQueue.java @@ -22,8 +22,6 @@ import org.apache.iotdb.commons.pipe.metric.PipeEventCounter; import org.apache.iotdb.pipe.api.event.Event; -import java.io.File; -import java.util.Set; import java.util.concurrent.BlockingDeque; import java.util.concurrent.LinkedBlockingDeque; @@ -36,9 +34,7 @@ public UnboundedBlockingPendingQueue(final PipeEventCounter eventCounter) { pendingDeque = (BlockingDeque) pendingQueue; } - public E peekLast() { + public synchronized E peekLast() { return pendingDeque.peekLast(); } - - public void replace(Set sourceFiles, Set targetFiles) {} } From 1d8246e290b34891b3b5547e5b016b48e81d5562 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Wed, 2 Jul 2025 10:00:20 +0800 Subject: [PATCH 075/185] fix --- .../agent/task/PipeDataNodeTaskAgent.java | 39 +++++++++---------- 1 file changed, 19 insertions(+), 20 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java index 45094b97fb2b..83854f2e4edd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java @@ -684,21 +684,21 @@ protected void calculateMemoryUsage( extractorParameters, processorParameters, connectorParameters); PipeMemoryManager pipeMemoryManager = PipeDataNodeResourceManager.memory(); - if (pipeMemoryManager.getFreeMemorySizeInBytes() - < needMemory - + PipeMemoryManager.getTotalMemorySizeInBytes() - * PipeConfig.getInstance().getReservedMemoryPercentage()) { - final String e = + final long freeMemorySizeInBytes = pipeMemoryManager.getFreeMemorySizeInBytes(); + final long reservedMemorySizeInBytes = + (long) + (PipeMemoryManager.getTotalMemorySizeInBytes() + * PipeConfig.getInstance().getReservedMemoryPercentage()); + if (freeMemorySizeInBytes < needMemory + reservedMemorySizeInBytes) { + final String message = String.format( "Not enough memory for pipe. Need memory: %d bytes, free memory: %d bytes, reserved memory: %d bytes, total memory: %d bytes", needMemory, - pipeMemoryManager.getFreeMemorySizeInBytes(), - (long) - (PipeMemoryManager.getTotalMemorySizeInBytes() - * PipeConfig.getInstance().getReservedMemoryPercentage()), + freeMemorySizeInBytes, + freeMemorySizeInBytes, PipeMemoryManager.getTotalMemorySizeInBytes()); - LOGGER.warn(e); - throw new PipeException(e); + LOGGER.warn(message); + throw new PipeException(message); } } @@ -724,17 +724,16 @@ private void calculateInsertNodeQueueMemory( return; } - if (PipeMemoryManager.getTotalFloatingMemorySizeInBytes() - - this.getAllFloatingMemoryUsageInByte() - < PipeConfig.getInstance().PipeInsertNodeQueueMemory()) { - final String m = + final long allocatedMemorySizeInBytes = this.getAllFloatingMemoryUsageInByte(); + final long remainingMemory = + PipeMemoryManager.getTotalFloatingMemorySizeInBytes() - allocatedMemorySizeInBytes; + if (remainingMemory < PipeConfig.getInstance().PipeInsertNodeQueueMemory()) { + final String message = String.format( "Not enough memory for pipe. Need Floating memory: %d bytes, free Floating memory: %d bytes", - PipeConfig.getInstance().PipeInsertNodeQueueMemory(), - (PipeMemoryManager.getTotalFloatingMemorySizeInBytes() - - this.getAllFloatingMemoryUsageInByte())); - LOGGER.warn(m); - throw new PipeException(m); + PipeConfig.getInstance().PipeInsertNodeQueueMemory(), remainingMemory); + LOGGER.warn(message); + throw new PipeException(message); } } From 7b4f4cb6edcfc20ca08552169c53dc93a8849961 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 2 Jul 2025 10:57:37 +0800 Subject: [PATCH 076/185] first delete assigner --- .../PipeRealtimeDataRegionExtractor.java | 27 +++ ...eRealtimeDataRegionHeartbeatExtractor.java | 28 +-- ...PipeRealtimeDataRegionHybridExtractor.java | 21 ++ .../PipeRealtimeDataRegionLogExtractor.java | 61 ++--- ...PipeRealtimeDataRegionTsFileExtractor.java | 35 +++ .../assigner/PipeDataRegionAssigner.java | 215 ++++++----------- .../PipeInsertionDataNodeListener.java | 9 +- .../matcher/CachedSchemaPatternMatcher.java | 202 ++++------------ .../matcher/PipeDataRegionMatcher.java | 52 ----- .../db/pipe/metric/PipeDataNodeMetrics.java | 3 - .../metric/source/PipeAssignerMetrics.java | 165 ------------- .../resource/tsfile/PipeTsFileResource.java | 1 - .../TsFileInsertionDataContainerTest.java | 3 +- .../CachedSchemaPatternMatcherTest.java | 218 ------------------ .../iotdb/commons/conf/CommonConfig.java | 13 -- .../iotdb/commons/pipe/config/PipeConfig.java | 5 - 16 files changed, 231 insertions(+), 827 deletions(-) delete mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/matcher/PipeDataRegionMatcher.java delete mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/source/PipeAssignerMetrics.java delete mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/pattern/CachedSchemaPatternMatcherTest.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java index c7a778357a44..970fc84241a3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java @@ -35,6 +35,7 @@ import org.apache.iotdb.db.pipe.extractor.dataregion.DataRegionListeningFilter; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.listener.PipeInsertionDataNodeListener; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.listener.PipeTimePartitionListener; +import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.matcher.CachedSchemaPatternMatcher; import org.apache.iotdb.db.pipe.metric.source.PipeDataRegionEventCounter; import org.apache.iotdb.db.storageengine.StorageEngine; import org.apache.iotdb.db.storageengine.dataregion.DataRegion; @@ -395,6 +396,32 @@ protected void extractDirectly(final PipeRealtimeEvent event) { } } + @Override + public Event supply() { + PipeRealtimeEvent realtimeEvent = (PipeRealtimeEvent) pendingQueue.directPoll(); + + while (realtimeEvent != null) { + while (!CachedSchemaPatternMatcher.match(realtimeEvent, this)) { + realtimeEvent.decreaseReferenceCount( + PipeRealtimeDataRegionTsFileExtractor.class.getName(), false); + realtimeEvent = (PipeRealtimeEvent) pendingQueue.directPoll(); + } + + final Event suppliedEvent = doSupply(realtimeEvent); + + realtimeEvent.decreaseReferenceCount(PipeRealtimeDataRegionExtractor.class.getName(), false); + + if (suppliedEvent != null) { + return suppliedEvent; + } + } + + // means the pending queue is empty. + return null; + } + + protected abstract Event doSupply(final PipeRealtimeEvent realtimeEvent); + protected Event supplyHeartbeat(final PipeRealtimeEvent event) { if (event.increaseReferenceCount(PipeRealtimeDataRegionExtractor.class.getName())) { return event.getEvent(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHeartbeatExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHeartbeatExtractor.java index 1df62eecc924..7ebaa673d5d3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHeartbeatExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHeartbeatExtractor.java @@ -27,29 +27,13 @@ public class PipeRealtimeDataRegionHeartbeatExtractor extends PipeRealtimeDataRegionExtractor { @Override - public Event supply() { - PipeRealtimeEvent realtimeEvent = (PipeRealtimeEvent) pendingQueue.directPoll(); - - while (realtimeEvent != null) { - Event suppliedEvent = null; - - // only supply PipeHeartbeatEvent - if (realtimeEvent.getEvent() instanceof PipeHeartbeatEvent) { - suppliedEvent = supplyHeartbeat(realtimeEvent); - } else if (realtimeEvent.getEvent() instanceof ProgressReportEvent) { - suppliedEvent = supplyDirectly(realtimeEvent); - } - - realtimeEvent.decreaseReferenceCount( - PipeRealtimeDataRegionHeartbeatExtractor.class.getName(), false); - - if (suppliedEvent != null) { - return suppliedEvent; - } - - realtimeEvent = (PipeRealtimeEvent) pendingQueue.directPoll(); + protected Event doSupply(PipeRealtimeEvent realtimeEvent) { + // only supply PipeHeartbeatEvent + if (realtimeEvent.getEvent() instanceof PipeHeartbeatEvent) { + return supplyHeartbeat(realtimeEvent); + } else if (realtimeEvent.getEvent() instanceof ProgressReportEvent) { + return supplyDirectly(realtimeEvent); } - return null; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java index 78522a000277..6b354e151a03 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java @@ -350,6 +350,27 @@ public Event supply() { return null; } + @Override + protected Event doSupply(final PipeRealtimeEvent realtimeEvent) { + // Used to judge the type of the event, not directly for supplying. + final Event eventToSupply = realtimeEvent.getEvent(); + if (eventToSupply instanceof TabletInsertionEvent) { + return supplyTabletInsertion(realtimeEvent); + } else if (eventToSupply instanceof TsFileInsertionEvent) { + return supplyTsFileInsertion(realtimeEvent); + } else if (eventToSupply instanceof PipeHeartbeatEvent) { + return supplyHeartbeat(realtimeEvent); + } else if (eventToSupply instanceof PipeSchemaRegionWritePlanEvent + || eventToSupply instanceof ProgressReportEvent) { + return supplyDirectly(realtimeEvent); + } else { + throw new UnsupportedOperationException( + String.format( + "Unsupported event type %s for hybrid realtime extractor %s to supply.", + eventToSupply.getClass(), this)); + } + } + private Event supplyTabletInsertion(final PipeRealtimeEvent event) { event .getTsFileEpoch() diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionLogExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionLogExtractor.java index 937d8aa53890..74d5c371dcee 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionLogExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionLogExtractor.java @@ -120,46 +120,29 @@ public boolean isNeedListenToInsertNode() { } @Override - public Event supply() { - PipeRealtimeEvent realtimeEvent = (PipeRealtimeEvent) pendingQueue.directPoll(); - - while (realtimeEvent != null) { - Event suppliedEvent = null; - - if (realtimeEvent.getEvent() instanceof PipeHeartbeatEvent) { - suppliedEvent = supplyHeartbeat(realtimeEvent); - } else if (realtimeEvent.getEvent() instanceof PipeSchemaRegionWritePlanEvent - || realtimeEvent.getEvent() instanceof ProgressReportEvent) { - suppliedEvent = supplyDirectly(realtimeEvent); - } else if (realtimeEvent.increaseReferenceCount( - PipeRealtimeDataRegionLogExtractor.class.getName())) { - suppliedEvent = realtimeEvent.getEvent(); - } else { - // if the event's reference count can not be increased, it means the data represented by - // this event is not reliable anymore. the data has been lost. we simply discard this event - // and report the exception to PipeRuntimeAgent. - final String errorMessage = - String.format( - "Event %s can not be supplied because " - + "the reference count can not be increased, " - + "the data represented by this event is lost", - realtimeEvent.getEvent()); - LOGGER.error(errorMessage); - PipeDataNodeAgent.runtime() - .report(pipeTaskMeta, new PipeRuntimeNonCriticalException(errorMessage)); - } - - realtimeEvent.decreaseReferenceCount( - PipeRealtimeDataRegionLogExtractor.class.getName(), false); - - if (suppliedEvent != null) { - return suppliedEvent; - } - - realtimeEvent = (PipeRealtimeEvent) pendingQueue.directPoll(); + protected Event doSupply(final PipeRealtimeEvent realtimeEvent) { + if (realtimeEvent.getEvent() instanceof PipeHeartbeatEvent) { + return supplyHeartbeat(realtimeEvent); + } else if (realtimeEvent.getEvent() instanceof PipeSchemaRegionWritePlanEvent + || realtimeEvent.getEvent() instanceof ProgressReportEvent) { + return supplyDirectly(realtimeEvent); + } else if (realtimeEvent.increaseReferenceCount( + PipeRealtimeDataRegionLogExtractor.class.getName())) { + return realtimeEvent.getEvent(); + } else { + // if the event's reference count can not be increased, it means the data represented by + // this event is not reliable anymore. the data has been lost. we simply discard this event + // and report the exception to PipeRuntimeAgent. + final String errorMessage = + String.format( + "Event %s can not be supplied because " + + "the reference count can not be increased, " + + "the data represented by this event is lost", + realtimeEvent.getEvent()); + LOGGER.error(errorMessage); + PipeDataNodeAgent.runtime() + .report(pipeTaskMeta, new PipeRuntimeNonCriticalException(errorMessage)); } - - // means the pending queue is empty. return null; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionTsFileExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionTsFileExtractor.java index 8072499b3daf..855e67b28a4d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionTsFileExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionTsFileExtractor.java @@ -26,6 +26,7 @@ import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.epoch.TsFileEpoch; +import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.matcher.CachedSchemaPatternMatcher; import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.pipe.api.event.dml.insertion.TsFileInsertionEvent; @@ -88,6 +89,12 @@ public Event supply() { PipeRealtimeEvent realtimeEvent = (PipeRealtimeEvent) pendingQueue.directPoll(); while (realtimeEvent != null) { + while (!CachedSchemaPatternMatcher.match(realtimeEvent, this)) { + realtimeEvent.decreaseReferenceCount( + PipeRealtimeDataRegionTsFileExtractor.class.getName(), false); + realtimeEvent = (PipeRealtimeEvent) pendingQueue.directPoll(); + } + Event suppliedEvent = null; if (realtimeEvent.getEvent() instanceof PipeHeartbeatEvent) { @@ -126,4 +133,32 @@ public Event supply() { // means the pending queue is empty. return null; } + + @Override + protected Event doSupply(final PipeRealtimeEvent realtimeEvent) { + if (realtimeEvent.getEvent() instanceof PipeHeartbeatEvent) { + return supplyHeartbeat(realtimeEvent); + } else if (realtimeEvent.getEvent() instanceof PipeSchemaRegionWritePlanEvent + || realtimeEvent.getEvent() instanceof ProgressReportEvent) { + return supplyDirectly(realtimeEvent); + } else if (realtimeEvent.increaseReferenceCount( + PipeRealtimeDataRegionTsFileExtractor.class.getName())) { + return realtimeEvent.getEvent(); + } else { + // if the event's reference count can not be increased, it means the data represented by + // this event is not reliable anymore. the data has been lost. we simply discard this event + // and report the exception to PipeRuntimeAgent. + final String errorMessage = + String.format( + "Event %s can not be supplied because " + + "the reference count can not be increased, " + + "the data represented by this event is lost", + realtimeEvent.getEvent()); + LOGGER.error(errorMessage); + PipeDataNodeAgent.runtime() + .report(pipeTaskMeta, new PipeRuntimeNonCriticalException(errorMessage)); + } + + return null; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java index ed91f636ac1a..27bb46fefe24 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java @@ -24,23 +24,19 @@ import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.event.ProgressReportEvent; -import org.apache.iotdb.commons.pipe.metric.PipeEventCounter; -import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEventFactory; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionExtractor; -import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.matcher.CachedSchemaPatternMatcher; -import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.matcher.PipeDataRegionMatcher; -import org.apache.iotdb.db.pipe.metric.source.PipeAssignerMetrics; -import org.apache.iotdb.db.pipe.metric.source.PipeDataRegionEventCounter; import org.apache.iotdb.pipe.api.event.dml.insertion.TsFileInsertionEvent; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.Closeable; +import java.util.Set; +import java.util.concurrent.CopyOnWriteArraySet; import java.util.concurrent.atomic.AtomicReference; public class PipeDataRegionAssigner implements Closeable { @@ -49,143 +45,83 @@ public class PipeDataRegionAssigner implements Closeable { private static final PipeConfig PIPE_CONFIG = PipeConfig.getInstance(); - /** - * The {@link PipeDataRegionMatcher} is used to match the event with the extractor based on the - * pattern. - */ - private final PipeDataRegionMatcher matcher; - - /** The {@link DisruptorQueue} is used to assign the event to the extractor. */ - private final DisruptorQueue disruptor; - private final String dataRegionId; + protected final Set extractors = new CopyOnWriteArraySet<>(); private int counter = 0; private final AtomicReference maxProgressIndexForRealtimeEvent = new AtomicReference<>(MinimumProgressIndex.INSTANCE); - private final PipeEventCounter eventCounter = new PipeDataRegionEventCounter(); - public String getDataRegionId() { return dataRegionId; } public PipeDataRegionAssigner(final String dataRegionId) { - this.matcher = new CachedSchemaPatternMatcher(); - this.disruptor = new DisruptorQueue(this::assignToExtractor, this::onAssignedHook); this.dataRegionId = dataRegionId; - PipeAssignerMetrics.getInstance().register(this); - } - - public void publishToAssign(final PipeRealtimeEvent event) { - if (!event.increaseReferenceCount(PipeDataRegionAssigner.class.getName())) { - LOGGER.warn( - "The reference count of the realtime event {} cannot be increased, skipping it.", event); - return; - } - - final EnrichedEvent innerEvent = event.getEvent(); - eventCounter.increaseEventCount(innerEvent); - if (innerEvent instanceof PipeHeartbeatEvent) { - ((PipeHeartbeatEvent) innerEvent).onPublished(); - } - - // use synchronized here for completely preventing reference count leaks under extreme thread - // scheduling when closing - synchronized (this) { - if (!disruptor.isClosed()) { - disruptor.publish(event); - } else { - onAssignedHook(event); - } - } } - private void onAssignedHook(final PipeRealtimeEvent realtimeEvent) { - realtimeEvent.gcSchemaInfo(); - realtimeEvent.decreaseReferenceCount(PipeDataRegionAssigner.class.getName(), false); - - final EnrichedEvent innerEvent = realtimeEvent.getEvent(); - eventCounter.decreaseEventCount(innerEvent); - if (innerEvent instanceof PipeHeartbeatEvent) { - ((PipeHeartbeatEvent) innerEvent).onAssigned(); - } - } - - private void assignToExtractor( - final PipeRealtimeEvent event, final long sequence, final boolean endOfBatch) { - if (disruptor.isClosed()) { - return; - } - - matcher - .match(event) - .forEach( - extractor -> { - if (disruptor.isClosed()) { + public void assignToExtractor(final PipeRealtimeEvent event) { + extractors.forEach( + extractor -> { + if (event.getEvent().isGeneratedByPipe() && !extractor.isForwardingPipeRequests()) { + // The frequency of progress reports is limited by the counter, while progress + // reports to TsFileInsertionEvent are not limited. + if (!(event.getEvent() instanceof TsFileInsertionEvent)) { + if (counter < PIPE_CONFIG.getPipeNonForwardingEventsProgressReportInterval()) { + counter++; return; } - - if (event.getEvent().isGeneratedByPipe() && !extractor.isForwardingPipeRequests()) { - // The frequency of progress reports is limited by the counter, while progress - // reports to TsFileInsertionEvent are not limited. - if (!(event.getEvent() instanceof TsFileInsertionEvent)) { - if (counter < PIPE_CONFIG.getPipeNonForwardingEventsProgressReportInterval()) { - counter++; - return; - } - counter = 0; - } - - final ProgressReportEvent reportEvent = - new ProgressReportEvent( - extractor.getPipeName(), - extractor.getCreationTime(), - extractor.getPipeTaskMeta(), - extractor.getPipePattern(), - extractor.getRealtimeDataExtractionStartTime(), - extractor.getRealtimeDataExtractionEndTime()); - reportEvent.bindProgressIndex(event.getProgressIndex()); - if (!reportEvent.increaseReferenceCount(PipeDataRegionAssigner.class.getName())) { - LOGGER.warn( - "The reference count of the event {} cannot be increased, skipping it.", - reportEvent); - return; - } - extractor.extract(PipeRealtimeEventFactory.createRealtimeEvent(reportEvent)); - return; - } - - final PipeRealtimeEvent copiedEvent = - event.shallowCopySelfAndBindPipeTaskMetaForProgressReport( - extractor.getPipeName(), - extractor.getCreationTime(), - extractor.getPipeTaskMeta(), - extractor.getPipePattern(), - extractor.getRealtimeDataExtractionStartTime(), - extractor.getRealtimeDataExtractionEndTime()); - final EnrichedEvent innerEvent = copiedEvent.getEvent(); - if (innerEvent instanceof PipeTsFileInsertionEvent) { - final PipeTsFileInsertionEvent tsFileInsertionEvent = - (PipeTsFileInsertionEvent) innerEvent; - tsFileInsertionEvent.disableMod4NonTransferPipes( - extractor.isShouldTransferModFile()); - } - - if (innerEvent instanceof PipeTsFileInsertionEvent - || innerEvent instanceof PipeInsertNodeTabletInsertionEvent) { - bindOrUpdateProgressIndexForRealtimeEvent(copiedEvent); - } - - if (!copiedEvent.increaseReferenceCount(PipeDataRegionAssigner.class.getName())) { - LOGGER.warn( - "The reference count of the event {} cannot be increased, skipping it.", - copiedEvent); - return; - } - extractor.extract(copiedEvent); - }); + counter = 0; + } + + final ProgressReportEvent reportEvent = + new ProgressReportEvent( + extractor.getPipeName(), + extractor.getCreationTime(), + extractor.getPipeTaskMeta(), + extractor.getPipePattern(), + extractor.getRealtimeDataExtractionStartTime(), + extractor.getRealtimeDataExtractionEndTime()); + reportEvent.bindProgressIndex(event.getProgressIndex()); + if (!reportEvent.increaseReferenceCount(PipeDataRegionAssigner.class.getName())) { + LOGGER.warn( + "The reference count of the event {} cannot be increased, skipping it.", + reportEvent); + return; + } + extractor.extract(PipeRealtimeEventFactory.createRealtimeEvent(reportEvent)); + return; + } + + final PipeRealtimeEvent copiedEvent = + event.shallowCopySelfAndBindPipeTaskMetaForProgressReport( + extractor.getPipeName(), + extractor.getCreationTime(), + extractor.getPipeTaskMeta(), + extractor.getPipePattern(), + extractor.getRealtimeDataExtractionStartTime(), + extractor.getRealtimeDataExtractionEndTime()); + final EnrichedEvent innerEvent = copiedEvent.getEvent(); + if (innerEvent instanceof PipeTsFileInsertionEvent) { + final PipeTsFileInsertionEvent tsFileInsertionEvent = + (PipeTsFileInsertionEvent) innerEvent; + tsFileInsertionEvent.disableMod4NonTransferPipes(extractor.isShouldTransferModFile()); + } + + if (innerEvent instanceof PipeTsFileInsertionEvent + || innerEvent instanceof PipeInsertNodeTabletInsertionEvent) { + bindOrUpdateProgressIndexForRealtimeEvent(copiedEvent); + } + + if (!copiedEvent.increaseReferenceCount(PipeDataRegionAssigner.class.getName())) { + LOGGER.warn( + "The reference count of the event {} cannot be increased, skipping it.", + copiedEvent); + return; + } + extractor.extract(copiedEvent); + }); } private void bindOrUpdateProgressIndexForRealtimeEvent(final PipeRealtimeEvent event) { @@ -217,45 +153,26 @@ private ProgressIndex getProgressIndex4RealtimeEvent(final PipeRealtimeEvent eve } public void startAssignTo(final PipeRealtimeDataRegionExtractor extractor) { - matcher.register(extractor); + extractors.add(extractor); } public void stopAssignTo(final PipeRealtimeDataRegionExtractor extractor) { - matcher.deregister(extractor); + extractors.remove(extractor); } public boolean notMoreExtractorNeededToBeAssigned() { - return matcher.getRegisterCount() == 0; + return extractors.isEmpty(); } - /** - * Clear the matcher and disruptor. The method {@link PipeDataRegionAssigner#publishToAssign} - * should not be used after calling this method. - */ @Override // use synchronized here for completely preventing reference count leaks under extreme thread // scheduling when closing public synchronized void close() { - PipeAssignerMetrics.getInstance().deregister(dataRegionId); - final long startTime = System.currentTimeMillis(); - disruptor.shutdown(); - matcher.clear(); + extractors.clear(); LOGGER.info( "Pipe: Assigner on data region {} shutdown internal disruptor within {} ms", dataRegionId, System.currentTimeMillis() - startTime); } - - public int getTabletInsertionEventCount() { - return eventCounter.getTabletInsertionEventCount(); - } - - public int getTsFileInsertionEventCount() { - return eventCounter.getTsFileInsertionEventCount(); - } - - public int getPipeHeartbeatEventCount() { - return eventCounter.getPipeHeartbeatEventCount(); - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java index 0769a54bbbab..69a276b9d0a9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java @@ -110,7 +110,7 @@ public void listenToTsFile( return; } - assigner.publishToAssign( + assigner.assignToExtractor( PipeRealtimeEventFactory.createRealtimeEvent(tsFileResource, isLoaded)); } @@ -127,20 +127,21 @@ public void listenToInsertNode( return; } - assigner.publishToAssign( + assigner.assignToExtractor( PipeRealtimeEventFactory.createRealtimeEvent(insertNode, tsFileResource)); } public void listenToHeartbeat(boolean shouldPrintMessage) { dataRegionId2Assigner.forEach( (key, value) -> - value.publishToAssign( + value.assignToExtractor( PipeRealtimeEventFactory.createRealtimeEvent(key, shouldPrintMessage))); } public void listenToDeleteData(DeleteDataNode node) { dataRegionId2Assigner.forEach( - (key, value) -> value.publishToAssign(PipeRealtimeEventFactory.createRealtimeEvent(node))); + (key, value) -> + value.assignToExtractor(PipeRealtimeEventFactory.createRealtimeEvent(node))); } /////////////////////////////// singleton /////////////////////////////// diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/matcher/CachedSchemaPatternMatcher.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/matcher/CachedSchemaPatternMatcher.java index 015ece38f8bc..1e4e8d4e232d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/matcher/CachedSchemaPatternMatcher.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/matcher/CachedSchemaPatternMatcher.java @@ -19,190 +19,84 @@ package org.apache.iotdb.db.pipe.extractor.dataregion.realtime.matcher; -import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.datastructure.pattern.PipePattern; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionExtractor; -import com.github.benmanes.caffeine.cache.Cache; -import com.github.benmanes.caffeine.cache.Caffeine; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.HashSet; import java.util.Map; import java.util.Objects; -import java.util.Set; -import java.util.concurrent.CopyOnWriteArraySet; -import java.util.concurrent.locks.ReentrantReadWriteLock; -import java.util.stream.Collectors; -public class CachedSchemaPatternMatcher implements PipeDataRegionMatcher { +public class CachedSchemaPatternMatcher { protected static final Logger LOGGER = LoggerFactory.getLogger(CachedSchemaPatternMatcher.class); - protected final ReentrantReadWriteLock lock; - - protected final Set extractors; - protected final Cache> deviceToExtractorsCache; - - public CachedSchemaPatternMatcher() { - this.lock = new ReentrantReadWriteLock(); - // Should be thread-safe because the extractors will be returned by {@link #match} and - // iterated by {@link #assignToExtractor}, at the same time the extractors may be added or - // removed by {@link #register} and {@link #deregister}. - this.extractors = new CopyOnWriteArraySet<>(); - this.deviceToExtractorsCache = - Caffeine.newBuilder() - .maximumSize(PipeConfig.getInstance().getPipeExtractorMatcherCacheSize()) - .build(); - } - - @Override - public void register(final PipeRealtimeDataRegionExtractor extractor) { - lock.writeLock().lock(); - try { - extractors.add(extractor); - deviceToExtractorsCache.invalidateAll(); - } finally { - lock.writeLock().unlock(); + public static boolean match( + final PipeRealtimeEvent event, final PipeRealtimeDataRegionExtractor extractor) { + // HeartbeatEvent will be assigned to all extractors + if (event.getEvent() instanceof PipeHeartbeatEvent) { + return true; } - } - @Override - public void deregister(final PipeRealtimeDataRegionExtractor extractor) { - lock.writeLock().lock(); - try { - extractors.remove(extractor); - deviceToExtractorsCache.invalidateAll(); - } finally { - lock.writeLock().unlock(); + // Deletion event will be assigned to extractors listened to it + if (event.getEvent() instanceof PipeSchemaRegionWritePlanEvent) { + return extractor.shouldExtractDeletion(); } - } - @Override - public int getRegisterCount() { - lock.readLock().lock(); - try { - return extractors.size(); - } finally { - lock.readLock().unlock(); - } - } - - @Override - public Set match(final PipeRealtimeEvent event) { - final Set matchedExtractors = new HashSet<>(); - - lock.readLock().lock(); - try { - if (extractors.isEmpty()) { - return matchedExtractors; - } - - // HeartbeatEvent will be assigned to all extractors - if (event.getEvent() instanceof PipeHeartbeatEvent) { - return extractors; - } + for (final Map.Entry entry : event.getSchemaInfo().entrySet()) { + final String device = entry.getKey(); + final String[] measurements = entry.getValue(); - // Deletion event will be assigned to extractors listened to it - if (event.getEvent() instanceof PipeSchemaRegionWritePlanEvent) { - return extractors.stream() - .filter(PipeRealtimeDataRegionExtractor::shouldExtractDeletion) - .collect(Collectors.toSet()); + if (!filterByDevice(device, extractor)) { + return false; } - for (final Map.Entry entry : event.getSchemaInfo().entrySet()) { - final String device = entry.getKey(); - final String[] measurements = entry.getValue(); - - // 1. try to get matched extractors from cache, if not success, match them by device - final Set extractorsFilteredByDevice = - deviceToExtractorsCache.get(device, this::filterExtractorsByDevice); - // this would not happen - if (extractorsFilteredByDevice == null) { - LOGGER.warn("Match result NPE when handle device {}", device); - continue; - } - - // 2. filter matched candidate extractors by measurements - if (measurements.length == 0) { - // `measurements` is empty (only in case of tsfile event). match all extractors. - // - // case 1: the pattern can match all measurements of the device. - // in this case, the extractor can be matched without checking the measurements. - // - // case 2: the pattern may match some measurements of the device. - // in this case, we can't get all measurements efficiently here, - // so we just ASSUME the extractor matches and do more checks later. - matchedExtractors.addAll(extractorsFilteredByDevice); + // 2. filter matched candidate extractors by measurements + if (measurements.length == 0) { + // `measurements` is empty (only in case of tsfile event). match all extractors. + // + // case 1: the pattern can match all measurements of the device. + // in this case, the extractor can be matched without checking the measurements. + // + // case 2: the pattern may match some measurements of the device. + // in this case, we can't get all measurements efficiently here, + // so we just ASSUME the extractor matches and do more checks later. + return true; + } else { + final PipePattern pattern = extractor.getPipePattern(); + if (Objects.isNull(pattern) || pattern.isRoot() || pattern.coversDevice(device)) { + // The pattern can match all measurements of the device. + return true; } else { - // `measurements` is not empty (only in case of tablet event). - // Match extractors by measurements. - extractorsFilteredByDevice.forEach( - extractor -> { - final PipePattern pattern = extractor.getPipePattern(); - if (Objects.isNull(pattern) || pattern.isRoot() || pattern.coversDevice(device)) { - // The pattern can match all measurements of the device. - matchedExtractors.add(extractor); - } else { - for (final String measurement : measurements) { - // Ignore null measurement for partial insert - if (measurement == null) { - continue; - } - - if (pattern.matchesMeasurement(device, measurement)) { - matchedExtractors.add(extractor); - // There would be no more matched extractors because the measurements are - // unique - break; - } - } - } - }); - } - - if (matchedExtractors.size() == extractors.size()) { - break; + for (final String measurement : measurements) { + // Ignore null measurement for partial insert + if (measurement == null) { + continue; + } + + if (pattern.matchesMeasurement(device, measurement)) { + return true; + } + } } } - } finally { - lock.readLock().unlock(); } - return matchedExtractors; + return false; } - protected Set filterExtractorsByDevice(final String device) { - final Set filteredExtractors = new HashSet<>(); - - for (final PipeRealtimeDataRegionExtractor extractor : extractors) { - // Return if the extractor only extract deletion - if (!extractor.shouldExtractInsertion()) { - continue; - } - - final PipePattern pipePattern = extractor.getPipePattern(); - if (Objects.isNull(pipePattern) || pipePattern.mayOverlapWithDevice(device)) { - filteredExtractors.add(extractor); - } - } - - return filteredExtractors; + private static boolean filterByDevice( + final String device, final PipeRealtimeDataRegionExtractor extractor) { + return extractor.shouldExtractInsertion() + && (Objects.isNull(extractor.getPipePattern()) + || extractor.getPipePattern().mayOverlapWithDevice(device)); } - @Override - public void clear() { - lock.writeLock().lock(); - try { - extractors.clear(); - deviceToExtractorsCache.invalidateAll(); - deviceToExtractorsCache.cleanUp(); - } finally { - lock.writeLock().unlock(); - } + private CachedSchemaPatternMatcher() { + // Utility class } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/matcher/PipeDataRegionMatcher.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/matcher/PipeDataRegionMatcher.java deleted file mode 100644 index 4e102a1f7cf6..000000000000 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/matcher/PipeDataRegionMatcher.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * 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.pipe.extractor.dataregion.realtime.matcher; - -import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; -import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionExtractor; - -import java.util.Set; - -public interface PipeDataRegionMatcher { - - /** - * Register a extractor. If the extractor's pattern matches the event's schema info, the event - * will be assigned to the extractor. - */ - void register(PipeRealtimeDataRegionExtractor extractor); - - /** Deregister a extractor. */ - void deregister(PipeRealtimeDataRegionExtractor extractor); - - /** Get the number of registered extractors in this matcher. */ - int getRegisterCount(); - - /** - * Match the event's schema info with the registered extractors' patterns. If the event's schema - * info matches the pattern of a extractor, the extractor will be returned. - * - * @param event the event to be matched - * @return the matched extractors - */ - Set match(PipeRealtimeEvent event); - - /** Clear all the registered extractors and internal data structures. */ - void clear(); -} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeMetrics.java index d7f1577a85c6..f5a48b3f6b69 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeMetrics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeMetrics.java @@ -30,7 +30,6 @@ import org.apache.iotdb.db.pipe.metric.schema.PipeSchemaRegionExtractorMetrics; import org.apache.iotdb.db.pipe.metric.schema.PipeSchemaRegionListenerMetrics; import org.apache.iotdb.db.pipe.metric.sink.PipeDataRegionConnectorMetrics; -import org.apache.iotdb.db.pipe.metric.source.PipeAssignerMetrics; import org.apache.iotdb.db.pipe.metric.source.PipeDataRegionExtractorMetrics; import org.apache.iotdb.metrics.AbstractMetricService; import org.apache.iotdb.metrics.metricsets.IMetricSet; @@ -41,7 +40,6 @@ public class PipeDataNodeMetrics implements IMetricSet { @Override public void bindTo(final AbstractMetricService metricService) { - PipeAssignerMetrics.getInstance().bindTo(metricService); PipeDataRegionExtractorMetrics.getInstance().bindTo(metricService); PipeProcessorMetrics.getInstance().bindTo(metricService); PipeDataRegionConnectorMetrics.getInstance().bindTo(metricService); @@ -58,7 +56,6 @@ public void bindTo(final AbstractMetricService metricService) { @Override public void unbindFrom(final AbstractMetricService metricService) { - PipeAssignerMetrics.getInstance().unbindFrom(metricService); PipeDataRegionExtractorMetrics.getInstance().unbindFrom(metricService); PipeProcessorMetrics.getInstance().unbindFrom(metricService); PipeDataRegionConnectorMetrics.getInstance().unbindFrom(metricService); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/source/PipeAssignerMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/source/PipeAssignerMetrics.java deleted file mode 100644 index 3a35305584f6..000000000000 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/source/PipeAssignerMetrics.java +++ /dev/null @@ -1,165 +0,0 @@ -/* - * 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.pipe.metric.source; - -import org.apache.iotdb.commons.service.metric.enums.Metric; -import org.apache.iotdb.commons.service.metric.enums.Tag; -import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.assigner.PipeDataRegionAssigner; -import org.apache.iotdb.metrics.AbstractMetricService; -import org.apache.iotdb.metrics.metricsets.IMetricSet; -import org.apache.iotdb.metrics.utils.MetricLevel; -import org.apache.iotdb.metrics.utils.MetricType; - -import com.google.common.collect.ImmutableSet; -import org.checkerframework.checker.nullness.qual.NonNull; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.HashMap; -import java.util.Map; -import java.util.Objects; - -public class PipeAssignerMetrics implements IMetricSet { - - private static final Logger LOGGER = LoggerFactory.getLogger(PipeAssignerMetrics.class); - - private AbstractMetricService metricService; - - private final Map assignerMap = new HashMap<>(); - - //////////////////////////// bindTo & unbindFrom (metric framework) //////////////////////////// - - @Override - public void bindTo(AbstractMetricService metricService) { - this.metricService = metricService; - synchronized (this) { - for (String dataRegionId : assignerMap.keySet()) { - createMetrics(dataRegionId); - } - } - } - - private void createMetrics(String dataRegionId) { - createAutoGauge(dataRegionId); - } - - private void createAutoGauge(String dataRegionId) { - metricService.createAutoGauge( - Metric.UNASSIGNED_HEARTBEAT_COUNT.toString(), - MetricLevel.IMPORTANT, - assignerMap.get(dataRegionId), - PipeDataRegionAssigner::getPipeHeartbeatEventCount, - Tag.REGION.toString(), - dataRegionId); - metricService.createAutoGauge( - Metric.UNASSIGNED_TABLET_COUNT.toString(), - MetricLevel.IMPORTANT, - assignerMap.get(dataRegionId), - PipeDataRegionAssigner::getTabletInsertionEventCount, - Tag.REGION.toString(), - dataRegionId); - metricService.createAutoGauge( - Metric.UNASSIGNED_TSFILE_COUNT.toString(), - MetricLevel.IMPORTANT, - assignerMap.get(dataRegionId), - PipeDataRegionAssigner::getTsFileInsertionEventCount, - Tag.REGION.toString(), - dataRegionId); - } - - @Override - public void unbindFrom(AbstractMetricService metricService) { - ImmutableSet dataRegionIds = ImmutableSet.copyOf(assignerMap.keySet()); - for (String dataRegionId : dataRegionIds) { - deregister(dataRegionId); - } - if (!assignerMap.isEmpty()) { - LOGGER.warn("Failed to unbind from pipe assigner metrics, assigner map not empty"); - } - } - - private void removeMetrics(String dataRegionId) { - removeAutoGauge(dataRegionId); - } - - private void removeAutoGauge(String dataRegionId) { - metricService.remove( - MetricType.AUTO_GAUGE, - Metric.UNASSIGNED_HEARTBEAT_COUNT.toString(), - Tag.REGION.toString(), - dataRegionId); - metricService.remove( - MetricType.AUTO_GAUGE, - Metric.UNASSIGNED_TABLET_COUNT.toString(), - Tag.REGION.toString(), - dataRegionId); - metricService.remove( - MetricType.AUTO_GAUGE, - Metric.UNASSIGNED_TSFILE_COUNT.toString(), - Tag.REGION.toString(), - dataRegionId); - } - - //////////////////////////// register & deregister (pipe integration) //////////////////////////// - - public void register(@NonNull PipeDataRegionAssigner pipeDataRegionAssigner) { - String dataRegionId = pipeDataRegionAssigner.getDataRegionId(); - synchronized (this) { - assignerMap.putIfAbsent(dataRegionId, pipeDataRegionAssigner); - if (Objects.nonNull(metricService)) { - createMetrics(dataRegionId); - } - } - } - - public void deregister(String dataRegionId) { - synchronized (this) { - if (!assignerMap.containsKey(dataRegionId)) { - LOGGER.warn( - "Failed to deregister pipe assigner metrics, PipeDataRegionAssigner({}) does not exist", - dataRegionId); - return; - } - if (Objects.nonNull(metricService)) { - removeMetrics(dataRegionId); - } - assignerMap.remove(dataRegionId); - } - } - - //////////////////////////// singleton //////////////////////////// - - private static class PipeAssignerMetricsHolder { - - private static final PipeAssignerMetrics INSTANCE = new PipeAssignerMetrics(); - - private PipeAssignerMetricsHolder() { - // empty constructor - } - } - - public static PipeAssignerMetrics getInstance() { - return PipeAssignerMetrics.PipeAssignerMetricsHolder.INSTANCE; - } - - private PipeAssignerMetrics() { - // empty constructor - } -} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResource.java index 4e05d1c80160..ebc2acb91ff7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResource.java @@ -45,7 +45,6 @@ public class PipeTsFileResource implements AutoCloseable { private static final Logger LOGGER = LoggerFactory.getLogger(PipeTsFileResource.class); - public static final long TSFILE_MIN_TIME_TO_LIVE_IN_MS = 1000L * 20; public static final float MEMORY_SUFFICIENT_THRESHOLD = 0.7f; private final File hardlinkOrCopiedFile; diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/TsFileInsertionDataContainerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/TsFileInsertionDataContainerTest.java index 50816ebf9688..f59038982ea1 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/TsFileInsertionDataContainerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/TsFileInsertionDataContainerTest.java @@ -19,7 +19,6 @@ package org.apache.iotdb.db.pipe.event; -import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.pipe.datastructure.pattern.IoTDBPipePattern; import org.apache.iotdb.commons.pipe.datastructure.pattern.PipePattern; import org.apache.iotdb.commons.pipe.datastructure.pattern.PrefixPipePattern; @@ -523,7 +522,7 @@ private void testMixedTsFileWithEmptyChunk(final boolean isQuery) throws IOExcep } private void testPartialNullValue(final boolean isQuery) - throws IOException, WriteProcessException, IllegalPathException { + throws IOException, WriteProcessException { alignedTsFile = new File("0-0-2-0.tsfile"); final List schemaList = new ArrayList<>(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/pattern/CachedSchemaPatternMatcherTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/pattern/CachedSchemaPatternMatcherTest.java deleted file mode 100644 index 2e3e57cd49e3..000000000000 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/pattern/CachedSchemaPatternMatcherTest.java +++ /dev/null @@ -1,218 +0,0 @@ -/* - * 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.pipe.pattern; - -import org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant; -import org.apache.iotdb.commons.pipe.config.plugin.configuraion.PipeTaskRuntimeConfiguration; -import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskExtractorRuntimeEnvironment; -import org.apache.iotdb.commons.pipe.datastructure.pattern.PipePattern; -import org.apache.iotdb.commons.pipe.datastructure.pattern.PrefixPipePattern; -import org.apache.iotdb.commons.pipe.event.EnrichedEvent; -import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; -import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionExtractor; -import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.epoch.TsFileEpoch; -import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.matcher.CachedSchemaPatternMatcher; -import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; -import org.apache.iotdb.pipe.api.event.Event; - -import org.apache.tsfile.common.constant.TsFileConstant; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.stream.Collectors; -import java.util.stream.IntStream; - -public class CachedSchemaPatternMatcherTest { - - private static class MockedPipeRealtimeEvent extends PipeRealtimeEvent { - - public MockedPipeRealtimeEvent( - EnrichedEvent event, - TsFileEpoch tsFileEpoch, - Map device2Measurements, - PipePattern pattern) { - super(event, tsFileEpoch, device2Measurements, pattern); - } - - @Override - public boolean shouldParseTime() { - return false; - } - - @Override - public boolean shouldParsePattern() { - return false; - } - } - - private CachedSchemaPatternMatcher matcher; - private ExecutorService executorService; - private List extractors; - - @Before - public void setUp() { - matcher = new CachedSchemaPatternMatcher(); - executorService = Executors.newSingleThreadExecutor(); - extractors = new ArrayList<>(); - } - - @After - public void tearDown() { - executorService.shutdownNow(); - } - - @Test - public void testCachedMatcher() throws Exception { - PipeRealtimeDataRegionExtractor dataRegionExtractor = new PipeRealtimeDataRegionFakeExtractor(); - dataRegionExtractor.customize( - new PipeParameters( - new HashMap() { - { - put(PipeExtractorConstant.EXTRACTOR_PATTERN_KEY, "root"); - } - }), - new PipeTaskRuntimeConfiguration(new PipeTaskExtractorRuntimeEnvironment("1", 1, 1, null))); - extractors.add(dataRegionExtractor); - - int deviceExtractorNum = 10; - int seriesExtractorNum = 10; - for (int i = 0; i < deviceExtractorNum; i++) { - PipeRealtimeDataRegionExtractor deviceExtractor = new PipeRealtimeDataRegionFakeExtractor(); - int finalI1 = i; - deviceExtractor.customize( - new PipeParameters( - new HashMap() { - { - put(PipeExtractorConstant.EXTRACTOR_PATTERN_KEY, "root." + finalI1); - } - }), - new PipeTaskRuntimeConfiguration( - new PipeTaskExtractorRuntimeEnvironment("1", 1, 1, null))); - extractors.add(deviceExtractor); - for (int j = 0; j < seriesExtractorNum; j++) { - PipeRealtimeDataRegionExtractor seriesExtractor = new PipeRealtimeDataRegionFakeExtractor(); - int finalI = i; - int finalJ = j; - seriesExtractor.customize( - new PipeParameters( - new HashMap() { - { - put( - PipeExtractorConstant.EXTRACTOR_PATTERN_KEY, - "root." + finalI + "." + finalJ); - } - }), - new PipeTaskRuntimeConfiguration( - new PipeTaskExtractorRuntimeEnvironment("1", 1, 1, null))); - extractors.add(seriesExtractor); - } - } - - Future future = - executorService.submit(() -> extractors.forEach(extractor -> matcher.register(extractor))); - - int epochNum = 10000; - int deviceNum = 1000; - int seriesNum = 100; - Map deviceMap = - IntStream.range(0, deviceNum) - .mapToObj(String::valueOf) - .collect(Collectors.toMap(s -> "root." + s, s -> new String[0])); - String[] measurements = - IntStream.range(0, seriesNum).mapToObj(String::valueOf).toArray(String[]::new); - long totalTime = 0; - for (int i = 0; i < epochNum; i++) { - for (int j = 0; j < deviceNum; j++) { - MockedPipeRealtimeEvent event = - new MockedPipeRealtimeEvent( - null, null, Collections.singletonMap("root." + i, measurements), null); - long startTime = System.currentTimeMillis(); - matcher.match(event).forEach(extractor -> extractor.extract(event)); - totalTime += (System.currentTimeMillis() - startTime); - } - MockedPipeRealtimeEvent event = new MockedPipeRealtimeEvent(null, null, deviceMap, null); - long startTime = System.currentTimeMillis(); - matcher.match(event).forEach(extractor -> extractor.extract(event)); - totalTime += (System.currentTimeMillis() - startTime); - } - System.out.println("matcher.getRegisterCount() = " + matcher.getRegisterCount()); - System.out.println("totalTime = " + totalTime); - System.out.println( - "device match per second = " - + ((double) (epochNum * (deviceNum + 1)) / (double) (totalTime) * 1000.0)); - - future.get(); - } - - public static class PipeRealtimeDataRegionFakeExtractor extends PipeRealtimeDataRegionExtractor { - - public PipeRealtimeDataRegionFakeExtractor() { - pipePattern = new PrefixPipePattern(null); - } - - @Override - public Event supply() { - return null; - } - - @Override - protected void doExtract(PipeRealtimeEvent event) { - final boolean[] match = {false}; - event - .getSchemaInfo() - .forEach( - (k, v) -> { - if (v.length > 0) { - for (String s : v) { - match[0] = - match[0] - || (k + TsFileConstant.PATH_SEPARATOR + s) - .startsWith(getPatternString()); - } - } else { - match[0] = - match[0] - || (getPatternString().startsWith(k) || k.startsWith(getPatternString())); - } - }); - Assert.assertTrue(match[0]); - } - - @Override - public boolean isNeedListenToTsFile() { - return true; - } - - @Override - public boolean isNeedListenToInsertNode() { - return true; - } - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java index 30dcfdcb07c9..5fedbddd494f 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java @@ -259,7 +259,6 @@ public class CommonConfig { private int pipeExtractorAssignerDisruptorRingBufferSize = 65536; private long pipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes = 50; // 50B - private int pipeExtractorMatcherCacheSize = 1024; private int pipeConnectorHandshakeTimeoutMs = 10 * 1000; // 10 seconds private int pipeConnectorTransferTimeoutMs = 15 * 60 * 1000; // 15 minutes @@ -1018,18 +1017,6 @@ public void setPipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes( pipeExtractorAssignerDisruptorRingBufferEntrySize); } - public int getPipeExtractorMatcherCacheSize() { - return pipeExtractorMatcherCacheSize; - } - - public void setPipeExtractorMatcherCacheSize(int pipeExtractorMatcherCacheSize) { - if (this.pipeExtractorMatcherCacheSize == pipeExtractorMatcherCacheSize) { - return; - } - this.pipeExtractorMatcherCacheSize = pipeExtractorMatcherCacheSize; - logger.info("pipeExtractorMatcherCacheSize is set to {}.", pipeExtractorMatcherCacheSize); - } - public int getPipeConnectorHandshakeTimeoutMs() { return pipeConnectorHandshakeTimeoutMs; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java index 62631749c7b2..e6dc4542ad27 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java @@ -181,10 +181,6 @@ public long getPipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes() { return COMMON_CONFIG.getPipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes(); } - public int getPipeExtractorMatcherCacheSize() { - return COMMON_CONFIG.getPipeExtractorMatcherCacheSize(); - } - /////////////////////////////// Connector /////////////////////////////// public int getPipeConnectorHandshakeTimeoutMs() { @@ -506,7 +502,6 @@ public void printAllConfigs() { LOGGER.info( "PipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes: {}", getPipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes()); - LOGGER.info("PipeExtractorMatcherCacheSize: {}", getPipeExtractorMatcherCacheSize()); LOGGER.info("PipeConnectorHandshakeTimeoutMs: {}", getPipeConnectorHandshakeTimeoutMs()); LOGGER.info("PipeConnectorTransferTimeoutMs: {}", getPipeConnectorTransferTimeoutMs()); From db30ba764da32a64fb6ab01558d46f7447991b86 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 2 Jul 2025 11:59:44 +0800 Subject: [PATCH 077/185] Fix --- .../tsfile/PipeTsFileInsertionEvent.java | 7 +- .../TsFileInsertionDataContainerProvider.java | 7 +- ...peHistoricalDataRegionTsFileExtractor.java | 1 - .../tsfile/PipeTsFileMemResource.java | 229 ++++++++++++++++++ .../resource/tsfile/PipeTsFileResource.java | 206 +--------------- .../tsfile/PipeTsFileResourceManager.java | 131 +++++----- .../PipeTsFileResourceManagerTest.java | 84 +++---- .../commons/pipe/config/PipeDescriptor.java | 7 - 8 files changed, 348 insertions(+), 324 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileMemResource.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java index b425d8a3e937..c50cdb165ea5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java @@ -89,13 +89,11 @@ public class PipeTsFileInsertionEvent extends EnrichedEvent public PipeTsFileInsertionEvent(final TsFileResource resource, final boolean isLoaded) { // The modFile must be copied before the event is assigned to the listening pipes - this( - resource, null, true, isLoaded, false, null, 0, null, null, Long.MIN_VALUE, Long.MAX_VALUE); + this(resource, true, isLoaded, false, null, 0, null, null, Long.MIN_VALUE, Long.MAX_VALUE); } public PipeTsFileInsertionEvent( final TsFileResource resource, - final File tsFile, final boolean isWithMod, final boolean isLoaded, final boolean isGeneratedByHistoricalExtractor, @@ -113,7 +111,7 @@ public PipeTsFileInsertionEvent( // For events created for source, the tsFile is inherited from the assigner, because the // original tsFile may be gone, and we need to get the assigner's hard-linked tsFile to // hard-link it to each pipe dir - this.tsFile = Objects.isNull(tsFile) ? resource.getTsFile() : tsFile; + this.tsFile = resource.getTsFile(); final ModificationFile modFile = resource.getModFile(); this.isWithMod = isWithMod && modFile.exists(); @@ -337,7 +335,6 @@ public PipeTsFileInsertionEvent shallowCopySelfAndBindPipeTaskMetaForProgressRep final long endTime) { return new PipeTsFileInsertionEvent( resource, - tsFile, isWithMod, isLoaded, isGeneratedByHistoricalExtractor, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/TsFileInsertionDataContainerProvider.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/TsFileInsertionDataContainerProvider.java index 21ce698141d8..18ef721b9248 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/TsFileInsertionDataContainerProvider.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/TsFileInsertionDataContainerProvider.java @@ -28,7 +28,8 @@ import org.apache.iotdb.db.pipe.event.common.tsfile.container.scan.TsFileInsertionScanDataContainer; import org.apache.iotdb.db.pipe.metric.overview.PipeTsFileToTabletsMetrics; import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; -import org.apache.iotdb.db.pipe.resource.tsfile.PipeTsFileResource; +import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryManager; +import org.apache.iotdb.db.pipe.resource.tsfile.PipeTsFileMemResource; import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.file.metadata.PlainDeviceID; @@ -79,8 +80,8 @@ public TsFileInsertionDataContainer provide() throws IOException { // Use scan container to save memory if ((double) PipeDataNodeResourceManager.memory().getUsedMemorySizeInBytes() - / PipeDataNodeResourceManager.memory().getTotalNonFloatingMemorySizeInBytes() - > PipeTsFileResource.MEMORY_SUFFICIENT_THRESHOLD) { + / PipeMemoryManager.getTotalNonFloatingMemorySizeInBytes() + > PipeTsFileMemResource.MEMORY_SUFFICIENT_THRESHOLD) { return new TsFileInsertionScanDataContainer( pipeName, creationTime, tsFile, pattern, startTime, endTime, pipeTaskMeta, sourceEvent); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java index 12e76afa4a65..0408e4f3a039 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java @@ -547,7 +547,6 @@ public synchronized Event supply() { final PipeTsFileInsertionEvent event = new PipeTsFileInsertionEvent( resource, - null, shouldTransferModFile, false, true, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileMemResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileMemResource.java new file mode 100644 index 000000000000..c7a4b23205fd --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileMemResource.java @@ -0,0 +1,229 @@ +/* + * 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.pipe.resource.tsfile; + +import org.apache.iotdb.commons.pipe.config.PipeConfig; +import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; +import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryBlock; +import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryWeightUtil; + +import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.read.TsFileDeviceIterator; +import org.apache.tsfile.read.TsFileSequenceReader; +import org.apache.tsfile.utils.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +public class PipeTsFileMemResource extends PipeTsFileResource { + private static final Logger LOGGER = LoggerFactory.getLogger(PipeTsFileMemResource.class); + public static final float MEMORY_SUFFICIENT_THRESHOLD = 0.7f; + private PipeMemoryBlock allocatedMemoryBlock; + private Map> deviceMeasurementsMap = null; + private Map deviceIsAlignedMap = null; + private Map measurementDataTypeMap = null; + + public PipeTsFileMemResource(final File hardlinkOrCopiedFile) { + super(hardlinkOrCopiedFile); + } + + @Override + public void close() { + if (deviceMeasurementsMap != null) { + deviceMeasurementsMap = null; + } + + if (deviceIsAlignedMap != null) { + deviceIsAlignedMap = null; + } + + if (measurementDataTypeMap != null) { + measurementDataTypeMap = null; + } + + if (allocatedMemoryBlock != null) { + allocatedMemoryBlock.close(); + allocatedMemoryBlock = null; + } + } + + //////////////////////////// Cache Getter //////////////////////////// + + public synchronized Map> tryGetDeviceMeasurementsMap() + throws IOException { + if (deviceMeasurementsMap == null) { + cacheObjectsIfAbsent(); + } + return deviceMeasurementsMap; + } + + public synchronized Map tryGetDeviceIsAlignedMap( + final boolean cacheOtherMetadata) throws IOException { + if (deviceIsAlignedMap == null) { + if (cacheOtherMetadata) { + cacheObjectsIfAbsent(); + } else { + cacheDeviceIsAlignedMapIfAbsent(); + } + } + return deviceIsAlignedMap; + } + + public synchronized Map tryGetMeasurementDataTypeMap() throws IOException { + if (measurementDataTypeMap == null) { + cacheObjectsIfAbsent(); + } + return measurementDataTypeMap; + } + + synchronized boolean cacheDeviceIsAlignedMapIfAbsent() throws IOException { + + if (allocatedMemoryBlock != null) { + // This means objects are already cached. + return true; + } + + // See if pipe memory is sufficient to be allocated for TsFileSequenceReader. + // Only allocate when pipe memory used is less than 50%, because memory here + // is hard to shrink and may consume too much memory. + allocatedMemoryBlock = + PipeDataNodeResourceManager.memory() + .forceAllocateIfSufficient( + PipeConfig.getInstance().getPipeMemoryAllocateForTsFileSequenceReaderInBytes(), + MEMORY_SUFFICIENT_THRESHOLD); + if (allocatedMemoryBlock == null) { + LOGGER.info( + "Failed to cacheDeviceIsAlignedMapIfAbsent for tsfile {}, because memory usage is high", + hardlinkOrCopiedFile.getPath()); + return false; + } + + long memoryRequiredInBytes = 0L; + try (TsFileSequenceReader sequenceReader = + new TsFileSequenceReader(hardlinkOrCopiedFile.getPath(), true, false)) { + deviceIsAlignedMap = new HashMap<>(); + final TsFileDeviceIterator deviceIsAlignedIterator = + sequenceReader.getAllDevicesIteratorWithIsAligned(); + while (deviceIsAlignedIterator.hasNext()) { + final Pair deviceIsAlignedPair = deviceIsAlignedIterator.next(); + deviceIsAlignedMap.put(deviceIsAlignedPair.getLeft(), deviceIsAlignedPair.getRight()); + } + memoryRequiredInBytes += PipeMemoryWeightUtil.memoryOfIDeviceId2Bool(deviceIsAlignedMap); + } + // Release memory of TsFileSequenceReader. + allocatedMemoryBlock.close(); + allocatedMemoryBlock = null; + + // Allocate again for the cached objects. + allocatedMemoryBlock = + PipeDataNodeResourceManager.memory() + .forceAllocateIfSufficient(memoryRequiredInBytes, MEMORY_SUFFICIENT_THRESHOLD); + if (allocatedMemoryBlock == null) { + LOGGER.info( + "PipeTsFileResource: Failed to cache objects for tsfile {} in cache, because memory usage is high", + hardlinkOrCopiedFile.getPath()); + deviceIsAlignedMap = null; + return false; + } + + LOGGER.info( + "PipeTsFileResource: Cached deviceIsAlignedMap for tsfile {}.", + hardlinkOrCopiedFile.getPath()); + return true; + } + + synchronized boolean cacheObjectsIfAbsent() throws IOException { + + if (allocatedMemoryBlock != null) { + if (deviceMeasurementsMap != null) { + return true; + } else { + // Recalculate it again because only deviceIsAligned map is cached + allocatedMemoryBlock.close(); + allocatedMemoryBlock = null; + } + } + + // See if pipe memory is sufficient to be allocated for TsFileSequenceReader. + // Only allocate when pipe memory used is less than 50%, because memory here + // is hard to shrink and may consume too much memory. + allocatedMemoryBlock = + PipeDataNodeResourceManager.memory() + .forceAllocateIfSufficient( + PipeConfig.getInstance().getPipeMemoryAllocateForTsFileSequenceReaderInBytes(), + MEMORY_SUFFICIENT_THRESHOLD); + if (allocatedMemoryBlock == null) { + LOGGER.info( + "Failed to cacheObjectsIfAbsent for tsfile {}, because memory usage is high", + hardlinkOrCopiedFile.getPath()); + return false; + } + + long memoryRequiredInBytes = 0L; + try (TsFileSequenceReader sequenceReader = + new TsFileSequenceReader(hardlinkOrCopiedFile.getPath(), true, true)) { + deviceMeasurementsMap = sequenceReader.getDeviceMeasurementsMap(); + memoryRequiredInBytes += + PipeMemoryWeightUtil.memoryOfIDeviceID2StrList(deviceMeasurementsMap); + + if (Objects.isNull(deviceIsAlignedMap)) { + deviceIsAlignedMap = new HashMap<>(); + final TsFileDeviceIterator deviceIsAlignedIterator = + sequenceReader.getAllDevicesIteratorWithIsAligned(); + while (deviceIsAlignedIterator.hasNext()) { + final Pair deviceIsAlignedPair = deviceIsAlignedIterator.next(); + deviceIsAlignedMap.put(deviceIsAlignedPair.getLeft(), deviceIsAlignedPair.getRight()); + } + } + memoryRequiredInBytes += PipeMemoryWeightUtil.memoryOfIDeviceId2Bool(deviceIsAlignedMap); + + measurementDataTypeMap = sequenceReader.getFullPathDataTypeMap(); + memoryRequiredInBytes += PipeMemoryWeightUtil.memoryOfStr2TSDataType(measurementDataTypeMap); + } + // Release memory of TsFileSequenceReader. + allocatedMemoryBlock.close(); + allocatedMemoryBlock = null; + + // Allocate again for the cached objects. + allocatedMemoryBlock = + PipeDataNodeResourceManager.memory() + .forceAllocateIfSufficient(memoryRequiredInBytes, MEMORY_SUFFICIENT_THRESHOLD); + if (allocatedMemoryBlock == null) { + LOGGER.info( + "PipeTsFileResource: Failed to cache objects for tsfile {} in cache, because memory usage is high", + hardlinkOrCopiedFile.getPath()); + deviceIsAlignedMap = null; + deviceMeasurementsMap = null; + measurementDataTypeMap = null; + return false; + } + + LOGGER.info( + "PipeTsFileResource: Cached objects for tsfile {}.", hardlinkOrCopiedFile.getPath()); + return true; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResource.java index ebc2acb91ff7..2b0f480c4b2a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResource.java @@ -19,48 +19,25 @@ package org.apache.iotdb.db.pipe.resource.tsfile; -import org.apache.iotdb.commons.pipe.config.PipeConfig; -import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; -import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryBlock; -import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryWeightUtil; - -import org.apache.tsfile.enums.TSDataType; -import org.apache.tsfile.file.metadata.IDeviceID; -import org.apache.tsfile.read.TsFileDeviceIterator; -import org.apache.tsfile.read.TsFileSequenceReader; -import org.apache.tsfile.utils.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.File; -import java.io.IOException; import java.nio.file.Files; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; import java.util.concurrent.atomic.AtomicInteger; public class PipeTsFileResource implements AutoCloseable { private static final Logger LOGGER = LoggerFactory.getLogger(PipeTsFileResource.class); - public static final float MEMORY_SUFFICIENT_THRESHOLD = 0.7f; - - private final File hardlinkOrCopiedFile; - private final boolean isTsFile; + protected final File hardlinkOrCopiedFile; private volatile long fileSize = -1L; private final AtomicInteger referenceCount; - private PipeMemoryBlock allocatedMemoryBlock; - private Map> deviceMeasurementsMap = null; - private Map deviceIsAlignedMap = null; - private Map measurementDataTypeMap = null; - public PipeTsFileResource(final File hardlinkOrCopiedFile, final boolean isTsFile) { + public PipeTsFileResource(final File hardlinkOrCopiedFile) { this.hardlinkOrCopiedFile = hardlinkOrCopiedFile; - this.isTsFile = isTsFile; referenceCount = new AtomicInteger(1); } @@ -104,23 +81,6 @@ public boolean decreaseReferenceCount() { @Override public synchronized void close() { - if (deviceMeasurementsMap != null) { - deviceMeasurementsMap = null; - } - - if (deviceIsAlignedMap != null) { - deviceIsAlignedMap = null; - } - - if (measurementDataTypeMap != null) { - measurementDataTypeMap = null; - } - - if (allocatedMemoryBlock != null) { - allocatedMemoryBlock.close(); - allocatedMemoryBlock = null; - } - try { Files.deleteIfExists(hardlinkOrCopiedFile.toPath()); } catch (final Exception e) { @@ -133,166 +93,4 @@ public synchronized void close() { LOGGER.info("PipeTsFileResource: Closed tsfile {} and cleaned up.", hardlinkOrCopiedFile); } - - //////////////////////////// Cache Getter //////////////////////////// - - public synchronized Map> tryGetDeviceMeasurementsMap() - throws IOException { - if (deviceMeasurementsMap == null && isTsFile) { - cacheObjectsIfAbsent(); - } - return deviceMeasurementsMap; - } - - public synchronized Map tryGetDeviceIsAlignedMap( - final boolean cacheOtherMetadata) throws IOException { - if (deviceIsAlignedMap == null && isTsFile) { - if (cacheOtherMetadata) { - cacheObjectsIfAbsent(); - } else { - cacheDeviceIsAlignedMapIfAbsent(); - } - } - return deviceIsAlignedMap; - } - - public synchronized Map tryGetMeasurementDataTypeMap() throws IOException { - if (measurementDataTypeMap == null && isTsFile) { - cacheObjectsIfAbsent(); - } - return measurementDataTypeMap; - } - - synchronized boolean cacheDeviceIsAlignedMapIfAbsent() throws IOException { - if (!isTsFile) { - return false; - } - - if (allocatedMemoryBlock != null) { - // This means objects are already cached. - return true; - } - - // See if pipe memory is sufficient to be allocated for TsFileSequenceReader. - // Only allocate when pipe memory used is less than 50%, because memory here - // is hard to shrink and may consume too much memory. - allocatedMemoryBlock = - PipeDataNodeResourceManager.memory() - .forceAllocateIfSufficient( - PipeConfig.getInstance().getPipeMemoryAllocateForTsFileSequenceReaderInBytes(), - MEMORY_SUFFICIENT_THRESHOLD); - if (allocatedMemoryBlock == null) { - LOGGER.info( - "Failed to cacheDeviceIsAlignedMapIfAbsent for tsfile {}, because memory usage is high", - hardlinkOrCopiedFile.getPath()); - return false; - } - - long memoryRequiredInBytes = 0L; - try (TsFileSequenceReader sequenceReader = - new TsFileSequenceReader(hardlinkOrCopiedFile.getPath(), true, false)) { - deviceIsAlignedMap = new HashMap<>(); - final TsFileDeviceIterator deviceIsAlignedIterator = - sequenceReader.getAllDevicesIteratorWithIsAligned(); - while (deviceIsAlignedIterator.hasNext()) { - final Pair deviceIsAlignedPair = deviceIsAlignedIterator.next(); - deviceIsAlignedMap.put(deviceIsAlignedPair.getLeft(), deviceIsAlignedPair.getRight()); - } - memoryRequiredInBytes += PipeMemoryWeightUtil.memoryOfIDeviceId2Bool(deviceIsAlignedMap); - } - // Release memory of TsFileSequenceReader. - allocatedMemoryBlock.close(); - allocatedMemoryBlock = null; - - // Allocate again for the cached objects. - allocatedMemoryBlock = - PipeDataNodeResourceManager.memory() - .forceAllocateIfSufficient(memoryRequiredInBytes, MEMORY_SUFFICIENT_THRESHOLD); - if (allocatedMemoryBlock == null) { - LOGGER.info( - "PipeTsFileResource: Failed to cache objects for tsfile {} in cache, because memory usage is high", - hardlinkOrCopiedFile.getPath()); - deviceIsAlignedMap = null; - return false; - } - - LOGGER.info( - "PipeTsFileResource: Cached deviceIsAlignedMap for tsfile {}.", - hardlinkOrCopiedFile.getPath()); - return true; - } - - synchronized boolean cacheObjectsIfAbsent() throws IOException { - if (!isTsFile) { - return false; - } - - if (allocatedMemoryBlock != null) { - if (deviceMeasurementsMap != null) { - return true; - } else { - // Recalculate it again because only deviceIsAligned map is cached - allocatedMemoryBlock.close(); - allocatedMemoryBlock = null; - } - } - - // See if pipe memory is sufficient to be allocated for TsFileSequenceReader. - // Only allocate when pipe memory used is less than 50%, because memory here - // is hard to shrink and may consume too much memory. - allocatedMemoryBlock = - PipeDataNodeResourceManager.memory() - .forceAllocateIfSufficient( - PipeConfig.getInstance().getPipeMemoryAllocateForTsFileSequenceReaderInBytes(), - MEMORY_SUFFICIENT_THRESHOLD); - if (allocatedMemoryBlock == null) { - LOGGER.info( - "Failed to cacheObjectsIfAbsent for tsfile {}, because memory usage is high", - hardlinkOrCopiedFile.getPath()); - return false; - } - - long memoryRequiredInBytes = 0L; - try (TsFileSequenceReader sequenceReader = - new TsFileSequenceReader(hardlinkOrCopiedFile.getPath(), true, true)) { - deviceMeasurementsMap = sequenceReader.getDeviceMeasurementsMap(); - memoryRequiredInBytes += - PipeMemoryWeightUtil.memoryOfIDeviceID2StrList(deviceMeasurementsMap); - - if (Objects.isNull(deviceIsAlignedMap)) { - deviceIsAlignedMap = new HashMap<>(); - final TsFileDeviceIterator deviceIsAlignedIterator = - sequenceReader.getAllDevicesIteratorWithIsAligned(); - while (deviceIsAlignedIterator.hasNext()) { - final Pair deviceIsAlignedPair = deviceIsAlignedIterator.next(); - deviceIsAlignedMap.put(deviceIsAlignedPair.getLeft(), deviceIsAlignedPair.getRight()); - } - } - memoryRequiredInBytes += PipeMemoryWeightUtil.memoryOfIDeviceId2Bool(deviceIsAlignedMap); - - measurementDataTypeMap = sequenceReader.getFullPathDataTypeMap(); - memoryRequiredInBytes += PipeMemoryWeightUtil.memoryOfStr2TSDataType(measurementDataTypeMap); - } - // Release memory of TsFileSequenceReader. - allocatedMemoryBlock.close(); - allocatedMemoryBlock = null; - - // Allocate again for the cached objects. - allocatedMemoryBlock = - PipeDataNodeResourceManager.memory() - .forceAllocateIfSufficient(memoryRequiredInBytes, MEMORY_SUFFICIENT_THRESHOLD); - if (allocatedMemoryBlock == null) { - LOGGER.info( - "PipeTsFileResource: Failed to cache objects for tsfile {} in cache, because memory usage is high", - hardlinkOrCopiedFile.getPath()); - deviceIsAlignedMap = null; - deviceMeasurementsMap = null; - measurementDataTypeMap = null; - return false; - } - - LOGGER.info( - "PipeTsFileResource: Cached objects for tsfile {}.", hardlinkOrCopiedFile.getPath()); - return true; - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java index 79028896e647..85c8a65750dd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java @@ -31,6 +31,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.io.File; @@ -47,7 +48,7 @@ public class PipeTsFileResourceManager { // This is used to hold the assigner pinned tsFiles. // Also, it is used to provide metadata cache of the tsFile, and is shared by all the pipe's // tsFiles. - private final Map hardlinkOrCopiedFileToAssignerTsFileResourceMap = + private final Map hardlinkOrCopiedFileToTsFileMemResourceMap = new ConcurrentHashMap<>(); // PipeName -> TsFilePath -> PipeTsFileResource @@ -75,12 +76,12 @@ public class PipeTsFileResourceManager { * @throws IOException when create hardlink or copy file failed */ public File increaseFileReference( - final File file, final boolean isTsFile, final @Nullable String pipeName) throws IOException { + final File file, final boolean isTsFile, final @Nonnull String pipeName) throws IOException { // If the file is already a hardlink or copied file, // just increase reference count and return it segmentLock.lock(file); try { - if (increaseReferenceIfExists(file, pipeName)) { + if (increaseReferenceIfExists(file, isTsFile, pipeName)) { return file; } } finally { @@ -92,8 +93,9 @@ public File increaseFileReference( final File hardlinkOrCopiedFile = getHardlinkOrCopiedFileInPipeDir(file, pipeName); segmentLock.lock(hardlinkOrCopiedFile); try { - if (increaseReferenceIfExists(hardlinkOrCopiedFile, pipeName)) { - return getHardlinkOrCopiedFile2TsFileResourceMap(pipeName) + if (increaseReferenceIfExists(hardlinkOrCopiedFile, isTsFile, pipeName)) { + return hardlinkOrCopiedFileToPipeTsFileResourceMap + .computeIfAbsent(pipeName, pipe -> new ConcurrentHashMap<>()) .get(hardlinkOrCopiedFile.getPath()) .getFile(); } @@ -108,10 +110,11 @@ public File increaseFileReference( // If the file is not a hardlink or copied file, and there is no related hardlink or copied // file in pipe dir, create a hardlink or copy it to pipe dir, maintain a reference count for // the hardlink or copied file, and return the hardlink or copied file. - getHardlinkOrCopiedFile2TsFileResourceMap(pipeName) - .put(resultFile.getPath(), new PipeTsFileResource(resultFile, isTsFile)); + hardlinkOrCopiedFileToPipeTsFileResourceMap + .computeIfAbsent(pipeName, pipe -> new ConcurrentHashMap<>()) + .put(resultFile.getPath(), new PipeTsFileResource(resultFile)); - increaseAssignerReferenceIfExists(resultFile, pipeName); + increaseMemReference(resultFile, isTsFile); return resultFile; } finally { @@ -119,28 +122,37 @@ public File increaseFileReference( } } - private boolean increaseReferenceIfExists(final File file, final @Nullable String pipeName) { + private boolean increaseReferenceIfExists( + final File file, final boolean isTsFile, final @Nonnull String pipeName) { final String path = file.getPath(); final PipeTsFileResource resource = - getHardlinkOrCopiedFile2TsFileResourceMap(pipeName).get(path); + hardlinkOrCopiedFileToPipeTsFileResourceMap + .computeIfAbsent(pipeName, pipe -> new ConcurrentHashMap<>()) + .get(path); if (resource != null) { resource.increaseReferenceCount(); - increaseAssignerReferenceIfExists(file, pipeName); + increaseMemReference(file, isTsFile); return true; } return false; } - private void increaseAssignerReferenceIfExists(final File file, final @Nullable String pipeName) { - // Increase the assigner's file to avoid hard-link or memory cache cleaning - // Note that it does not exist for historical files - if (Objects.isNull(pipeName)) { + private void increaseMemReference(final File file, final boolean isTsFile) { + if (!isTsFile) { return; } - final String assignerPath = getAssignerFilePath(file); - if (hardlinkOrCopiedFileToAssignerTsFileResourceMap.containsKey(assignerPath)) { - hardlinkOrCopiedFileToAssignerTsFileResourceMap.get(assignerPath).increaseReferenceCount(); - } + // Increase the assigner's file to avoid hard-link or memory cache cleaning + // Note that it does not exist for historical files + hardlinkOrCopiedFileToTsFileMemResourceMap.compute( + getCommonFilePath(file), + (k, v) -> { + if (Objects.isNull(v)) { + return new PipeTsFileMemResource(file); + } else { + v.increaseReferenceCount(); + return v; + } + }); } public static File getHardlinkOrCopiedFileInPipeDir(final File file, final String pipeName) @@ -194,40 +206,41 @@ private static String getRelativeFilePath(File file) { * @param hardlinkOrCopiedFile the copied or hard-linked file */ public void decreaseFileReference( - final File hardlinkOrCopiedFile, final @Nullable String pipeName) { + final File hardlinkOrCopiedFile, final @Nonnull String pipeName) { segmentLock.lock(hardlinkOrCopiedFile); try { final String filePath = hardlinkOrCopiedFile.getPath(); final PipeTsFileResource resource = - getHardlinkOrCopiedFile2TsFileResourceMap(pipeName).get(filePath); + hardlinkOrCopiedFileToPipeTsFileResourceMap + .computeIfAbsent(pipeName, pipe -> new ConcurrentHashMap<>()) + .get(filePath); if (resource != null && resource.decreaseReferenceCount()) { - getHardlinkOrCopiedFile2TsFileResourceMap(pipeName).remove(filePath); + hardlinkOrCopiedFileToPipeTsFileResourceMap + .computeIfAbsent(pipeName, pipe -> new ConcurrentHashMap<>()) + .remove(filePath); } // Decrease the assigner's file to clear hard-link and memory cache // Note that it does not exist for historical files - decreaseAssignerReferenceIfExists(hardlinkOrCopiedFile, pipeName); + decreaseMemReferenceIfExists(hardlinkOrCopiedFile); } finally { segmentLock.unlock(hardlinkOrCopiedFile); } } - private void decreaseAssignerReferenceIfExists(final File file, final @Nullable String pipeName) { + private void decreaseMemReferenceIfExists(final File file) { // Increase the assigner's file to avoid hard-link or memory cache cleaning // Note that it does not exist for historical files - if (Objects.isNull(pipeName)) { - return; - } - final String assignerPath = getAssignerFilePath(file); - if (hardlinkOrCopiedFileToAssignerTsFileResourceMap.containsKey(assignerPath) - && hardlinkOrCopiedFileToAssignerTsFileResourceMap - .get(assignerPath) + final String commonFilePath = getCommonFilePath(file); + if (hardlinkOrCopiedFileToTsFileMemResourceMap.containsKey(commonFilePath) + && hardlinkOrCopiedFileToTsFileMemResourceMap + .get(commonFilePath) .decreaseReferenceCount()) { - hardlinkOrCopiedFileToPipeTsFileResourceMap.remove(assignerPath); + hardlinkOrCopiedFileToPipeTsFileResourceMap.remove(commonFilePath); } } // Warning: Shall not be called by the assigner - private String getAssignerFilePath(final File file) { + private String getCommonFilePath(final File file) { // Skip the "pipeName" of this file return file.getParentFile().getParent() + File.separator + file.getName(); } @@ -239,12 +252,17 @@ private String getAssignerFilePath(final File file) { * @return the reference count of the file */ @TestOnly - public int getFileReferenceCount(final File hardlinkOrCopiedFile, final String pipeName) { + public int getFileReferenceCount( + final File hardlinkOrCopiedFile, final @Nullable String pipeName) { segmentLock.lock(hardlinkOrCopiedFile); try { - final String filePath = hardlinkOrCopiedFile.getPath(); final PipeTsFileResource resource = - getHardlinkOrCopiedFile2TsFileResourceMap(pipeName).get(filePath); + Objects.nonNull(pipeName) + ? hardlinkOrCopiedFileToPipeTsFileResourceMap + .computeIfAbsent(pipeName, pipe -> new ConcurrentHashMap<>()) + .get(hardlinkOrCopiedFile.getPath()) + : hardlinkOrCopiedFileToTsFileMemResourceMap.get( + getCommonFilePath(hardlinkOrCopiedFile)); return resource != null ? resource.getReferenceCount() : 0; } finally { segmentLock.unlock(hardlinkOrCopiedFile); @@ -260,9 +278,8 @@ public int getFileReferenceCount(final File hardlinkOrCopiedFile, final String p public boolean cacheObjectsIfAbsent(final File hardlinkOrCopiedTsFile) throws IOException { segmentLock.lock(hardlinkOrCopiedTsFile); try { - final PipeTsFileResource resource = - hardlinkOrCopiedFileToAssignerTsFileResourceMap.get( - getAssignerFilePath(hardlinkOrCopiedTsFile)); + final PipeTsFileMemResource resource = + hardlinkOrCopiedFileToTsFileMemResourceMap.get(getCommonFilePath(hardlinkOrCopiedTsFile)); return resource != null && resource.cacheObjectsIfAbsent(); } finally { segmentLock.unlock(hardlinkOrCopiedTsFile); @@ -273,9 +290,8 @@ public Map> getDeviceMeasurementsMapFromCache( final File hardlinkOrCopiedTsFile) throws IOException { segmentLock.lock(hardlinkOrCopiedTsFile); try { - final PipeTsFileResource resource = - hardlinkOrCopiedFileToAssignerTsFileResourceMap.get( - getAssignerFilePath(hardlinkOrCopiedTsFile)); + final PipeTsFileMemResource resource = + hardlinkOrCopiedFileToTsFileMemResourceMap.get(getCommonFilePath(hardlinkOrCopiedTsFile)); return resource == null ? null : resource.tryGetDeviceMeasurementsMap(); } finally { segmentLock.unlock(hardlinkOrCopiedTsFile); @@ -286,9 +302,8 @@ public Map getDeviceIsAlignedMapFromCache( final File hardlinkOrCopiedTsFile, final boolean cacheOtherMetadata) throws IOException { segmentLock.lock(hardlinkOrCopiedTsFile); try { - final PipeTsFileResource resource = - hardlinkOrCopiedFileToAssignerTsFileResourceMap.get( - getAssignerFilePath(hardlinkOrCopiedTsFile)); + final PipeTsFileMemResource resource = + hardlinkOrCopiedFileToTsFileMemResourceMap.get(getCommonFilePath(hardlinkOrCopiedTsFile)); return resource == null ? null : resource.tryGetDeviceIsAlignedMap(cacheOtherMetadata); } finally { segmentLock.unlock(hardlinkOrCopiedTsFile); @@ -299,9 +314,8 @@ public Map getMeasurementDataTypeMapFromCache( final File hardlinkOrCopiedTsFile) throws IOException { segmentLock.lock(hardlinkOrCopiedTsFile); try { - final PipeTsFileResource resource = - hardlinkOrCopiedFileToAssignerTsFileResourceMap.get( - getAssignerFilePath(hardlinkOrCopiedTsFile)); + final PipeTsFileMemResource resource = + hardlinkOrCopiedFileToTsFileMemResourceMap.get(getCommonFilePath(hardlinkOrCopiedTsFile)); return resource == null ? null : resource.tryGetMeasurementDataTypeMap(); } finally { segmentLock.unlock(hardlinkOrCopiedTsFile); @@ -328,12 +342,17 @@ public void unpinTsFileResource(final TsFileResource resource, final String pipe } } - public int getLinkedTsFileCount(final String pipeName) { - return getHardlinkOrCopiedFile2TsFileResourceMap(pipeName).size(); + public int getLinkedTsFileCount(final @Nonnull String pipeName) { + return hardlinkOrCopiedFileToPipeTsFileResourceMap + .computeIfAbsent(pipeName, pipe -> new ConcurrentHashMap<>()) + .size(); } - public long getTotalLinkedTsFileSize(final String pipeName) { - return getHardlinkOrCopiedFile2TsFileResourceMap(pipeName).values().stream() + public long getTotalLinkedTsFileSize(final @Nonnull String pipeName) { + return hardlinkOrCopiedFileToPipeTsFileResourceMap + .computeIfAbsent(pipeName, pipe -> new ConcurrentHashMap<>()) + .values() + .stream() .mapToLong( resource -> { try { @@ -345,12 +364,4 @@ public long getTotalLinkedTsFileSize(final String pipeName) { }) .sum(); } - - private Map getHardlinkOrCopiedFile2TsFileResourceMap( - final String pipeName) { - return Objects.nonNull(pipeName) - ? hardlinkOrCopiedFileToPipeTsFileResourceMap.computeIfAbsent( - pipeName, pipe -> new ConcurrentHashMap<>()) - : hardlinkOrCopiedFileToAssignerTsFileResourceMap; - } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java index 9db31eb3759f..111c092befe4 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java @@ -160,33 +160,31 @@ public void testIncreaseTsFile() throws IOException { Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(originTsfile, null)); Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(originModFile, null)); - final File assignerTsfile = - pipeTsFileResourceManager.increaseFileReference(originTsfile, true, null); - final File assignerModFile = - pipeTsFileResourceManager.increaseFileReference(originModFile, false, null); - Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(assignerTsfile, null)); - Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(assignerModFile, null)); + final File pipeTsfile = + pipeTsFileResourceManager.increaseFileReference(originTsfile, true, PIPE_NAME); + final File pipeModFile = + pipeTsFileResourceManager.increaseFileReference(originModFile, false, PIPE_NAME); + Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, null)); + Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, null)); Assert.assertTrue(Files.exists(originTsfile.toPath())); Assert.assertTrue(Files.exists(originModFile.toPath())); - Assert.assertTrue(Files.exists(assignerTsfile.toPath())); - Assert.assertTrue(Files.exists(assignerModFile.toPath())); + Assert.assertTrue(Files.exists(pipeTsfile.toPath())); + Assert.assertTrue(Files.exists(pipeModFile.toPath())); // test use assigner's hardlinkTsFile to increase reference counts // test null, shall not reuse the pipe's tsFile - final File pipeTsFile = - pipeTsFileResourceManager.increaseFileReference(assignerTsfile, true, PIPE_NAME); - Assert.assertEquals(2, pipeTsFileResourceManager.getFileReferenceCount(assignerTsfile, null)); - Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsFile, PIPE_NAME)); + pipeTsFileResourceManager.increaseFileReference(pipeTsfile, true, PIPE_NAME); + Assert.assertEquals(2, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, null)); + Assert.assertEquals(2, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, PIPE_NAME)); Assert.assertTrue(Files.exists(originTsfile.toPath())); - Assert.assertTrue(Files.exists(assignerTsfile.toPath())); + Assert.assertTrue(Files.exists(pipeTsfile.toPath())); // test use copyFile to increase reference counts - final File pipeModFile = - pipeTsFileResourceManager.increaseFileReference(assignerModFile, false, PIPE_NAME); - Assert.assertEquals(2, pipeTsFileResourceManager.getFileReferenceCount(assignerModFile, null)); - Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, PIPE_NAME)); + pipeTsFileResourceManager.increaseFileReference(pipeModFile, false, PIPE_NAME); + Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, null)); + Assert.assertEquals(2, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, PIPE_NAME)); Assert.assertTrue(Files.exists(originModFile.toPath())); - Assert.assertTrue(Files.exists(assignerModFile.toPath())); + Assert.assertTrue(Files.exists(pipeModFile.toPath())); } @Test @@ -194,45 +192,43 @@ public void testDecreaseTsFile() throws IOException { final File originFile = new File(TS_FILE_NAME); final File originModFile = new File(MODS_FILE_NAME); - pipeTsFileResourceManager.decreaseFileReference(originFile, null); - pipeTsFileResourceManager.decreaseFileReference(originModFile, null); + pipeTsFileResourceManager.decreaseFileReference(originFile, PIPE_NAME); + pipeTsFileResourceManager.decreaseFileReference(originModFile, PIPE_NAME); Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(originFile, null)); Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(originModFile, null)); - final File assignerTsfile = - pipeTsFileResourceManager.increaseFileReference(originFile, true, null); - final File assignerModFile = - pipeTsFileResourceManager.increaseFileReference(originModFile, false, null); - Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(assignerTsfile, null)); - Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(assignerModFile, null)); - Assert.assertTrue(Files.exists(assignerTsfile.toPath())); - Assert.assertTrue(Files.exists(assignerModFile.toPath())); - Assert.assertTrue(Files.exists(assignerTsfile.toPath())); - Assert.assertTrue(Files.exists(assignerModFile.toPath())); + final File pipeTsfile = + pipeTsFileResourceManager.increaseFileReference(originFile, true, PIPE_NAME); + final File pipeModFile = + pipeTsFileResourceManager.increaseFileReference(originModFile, false, PIPE_NAME); + Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, null)); + Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, null)); + Assert.assertTrue(Files.exists(pipeTsfile.toPath())); + Assert.assertTrue(Files.exists(pipeModFile.toPath())); + Assert.assertTrue(Files.exists(pipeTsfile.toPath())); + Assert.assertTrue(Files.exists(pipeModFile.toPath())); Assert.assertTrue(originFile.delete()); Assert.assertTrue(originModFile.delete()); Assert.assertFalse(Files.exists(originFile.toPath())); Assert.assertFalse(Files.exists(originModFile.toPath())); - final File pipeTsFile = - pipeTsFileResourceManager.increaseFileReference(assignerTsfile, true, PIPE_NAME); - final File pipeModFile = - pipeTsFileResourceManager.increaseFileReference(assignerModFile, false, PIPE_NAME); - pipeTsFileResourceManager.decreaseFileReference(pipeTsFile, PIPE_NAME); - pipeTsFileResourceManager.decreaseFileReference(pipeModFile, PIPE_NAME); - Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(assignerTsfile, null)); - Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(assignerModFile, null)); + Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, PIPE_NAME)); + Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, null)); + Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, PIPE_NAME)); + Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, null)); Assert.assertFalse(Files.exists(originFile.toPath())); Assert.assertFalse(Files.exists(originModFile.toPath())); - Assert.assertTrue(Files.exists(assignerTsfile.toPath())); - Assert.assertTrue(Files.exists(assignerModFile.toPath())); + Assert.assertTrue(Files.exists(pipeTsfile.toPath())); + Assert.assertTrue(Files.exists(pipeModFile.toPath())); - pipeTsFileResourceManager.decreaseFileReference(assignerTsfile, null); - pipeTsFileResourceManager.decreaseFileReference(assignerModFile, null); - Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(assignerTsfile, null)); - Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(assignerModFile, null)); + pipeTsFileResourceManager.decreaseFileReference(pipeTsfile, PIPE_NAME); + pipeTsFileResourceManager.decreaseFileReference(pipeModFile, PIPE_NAME); + Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, PIPE_NAME)); + Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, null)); + Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, PIPE_NAME)); + Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, null)); Assert.assertFalse(Files.exists(originFile.toPath())); Assert.assertFalse(Files.exists(originModFile.toPath())); } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java index d892b16cff89..cbea25ff50cc 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java @@ -346,13 +346,6 @@ public static void loadPipeInternalConfig(CommonConfig config, TrimProperties pr String.valueOf( config .getPipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes()))))); - config.setPipeExtractorMatcherCacheSize( - Integer.parseInt( - Optional.ofNullable(properties.getProperty("pipe_source_matcher_cache_size")) - .orElse( - properties.getProperty( - "pipe_extractor_matcher_cache_size", - String.valueOf(config.getPipeExtractorMatcherCacheSize()))))); config.setPipeConnectorHandshakeTimeoutMs( Long.parseLong( From 42bca9e7e4e21c2a6f0578b11fe9db5de4aaa90d Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 2 Jul 2025 12:13:16 +0800 Subject: [PATCH 078/185] Update PipeTsFileResourceManagerTest.java --- .../iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java index 111c092befe4..5c5dc8402dfb 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java @@ -213,7 +213,6 @@ public void testDecreaseTsFile() throws IOException { Assert.assertFalse(Files.exists(originFile.toPath())); Assert.assertFalse(Files.exists(originModFile.toPath())); - Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, PIPE_NAME)); Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, null)); Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, PIPE_NAME)); From 948bcf50986e998f60327f81cc0fc3eedbcf9371 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 2 Jul 2025 12:20:14 +0800 Subject: [PATCH 079/185] Update PipeDataNodeHardlinkOrCopiedFileDirStartupCleaner.java --- .../PipeDataNodeHardlinkOrCopiedFileDirStartupCleaner.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/PipeDataNodeHardlinkOrCopiedFileDirStartupCleaner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/PipeDataNodeHardlinkOrCopiedFileDirStartupCleaner.java index ec3f9f55ccc6..3afe5eb62e1a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/PipeDataNodeHardlinkOrCopiedFileDirStartupCleaner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/PipeDataNodeHardlinkOrCopiedFileDirStartupCleaner.java @@ -53,8 +53,7 @@ private static void cleanTsFileDir() { "Pipe hardlink dir found, deleting it: {}, result: {}", pipeHardLinkDir, // For child directories, we need them to recover each pipe's progress - // Hence we do not delete them here, only delete the child files (from older version || - // assigner pinned tsFiles) + // Hence we do not delete them here, only delete the child files (older version) FileUtils.deleteFileChildrenQuietly(pipeHardLinkDir)); } } From 3206928d28c844dbc29698ddae11d36b38cce79a Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 2 Jul 2025 12:31:22 +0800 Subject: [PATCH 080/185] Move --- .../iotdb/db/storageengine/dataregion/DataRegion.java | 8 -------- .../dataregion/memtable/TsFileProcessor.java | 10 ++++++++++ 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java index 6883743e6537..0d2737ab721b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java @@ -2682,14 +2682,6 @@ private void closeUnsealedTsFileProcessorCallBack(TsFileProcessor tsFileProcesso } if (isEmptyFile || isValidateTsFileFailed) { tsFileManager.remove(tsFileProcessor.getTsFileResource(), tsFileProcessor.isSequence()); - } else { - PipeInsertionDataNodeListener.getInstance() - .listenToTsFile( - dataRegionInfo.getDataRegion().getDataRegionId(), - tsFileProcessor.getTsFileResource(), - false, - tsFileProcessor.getWorkMemTable() != null - && tsFileProcessor.getWorkMemTable().isTotallyGeneratedByPipe()); } // closingSequenceTsFileProcessor is a thread safety class. diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java index 40f9c4112c7a..e4f095b52c78 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java @@ -1570,6 +1570,16 @@ private void endFile() throws IOException, TsFileProcessorException { logger.debug("Start to end file {}", tsFileResource); } writer.endFile(); + + // Listen after "endFile" to avoid unnecessary waiting for tsFile close + // before resource serialization to avoid missing hardlink after restart + PipeInsertionDataNodeListener.getInstance() + .listenToTsFile( + dataRegionInfo.getDataRegion().getDataRegionId(), + tsFileResource, + false, + workMemTable != null && workMemTable.isTotallyGeneratedByPipe()); + tsFileResource.serialize(); FileTimeIndexCacheRecorder.getInstance().logFileTimeIndex(tsFileResource); if (logger.isDebugEnabled()) { From 0d804a726cf6d0aef8e8d6d6bfed4a8de6923b60 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 2 Jul 2025 14:11:59 +0800 Subject: [PATCH 081/185] partial --- ...PipeRealtimeDataRegionHybridExtractor.java | 39 --------------- ...PipeRealtimeDataRegionTsFileExtractor.java | 50 ------------------- .../extractor/PipeRealtimeExtractTest.java | 1 - 3 files changed, 90 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java index 6b354e151a03..5e086f7ffd5e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java @@ -311,45 +311,6 @@ private boolean mayTsFileLinkedCountReachDangerousThreshold(final PipeRealtimeEv return mayTsFileLinkedCountReachDangerousThreshold; } - @Override - public Event supply() { - PipeRealtimeEvent realtimeEvent = (PipeRealtimeEvent) pendingQueue.directPoll(); - - while (realtimeEvent != null) { - final Event suppliedEvent; - - // Used to judge the type of the event, not directly for supplying. - final Event eventToSupply = realtimeEvent.getEvent(); - if (eventToSupply instanceof TabletInsertionEvent) { - suppliedEvent = supplyTabletInsertion(realtimeEvent); - } else if (eventToSupply instanceof TsFileInsertionEvent) { - suppliedEvent = supplyTsFileInsertion(realtimeEvent); - } else if (eventToSupply instanceof PipeHeartbeatEvent) { - suppliedEvent = supplyHeartbeat(realtimeEvent); - } else if (eventToSupply instanceof PipeSchemaRegionWritePlanEvent - || eventToSupply instanceof ProgressReportEvent) { - suppliedEvent = supplyDirectly(realtimeEvent); - } else { - throw new UnsupportedOperationException( - String.format( - "Unsupported event type %s for hybrid realtime extractor %s to supply.", - eventToSupply.getClass(), this)); - } - - realtimeEvent.decreaseReferenceCount( - PipeRealtimeDataRegionHybridExtractor.class.getName(), false); - - if (suppliedEvent != null) { - return suppliedEvent; - } - - realtimeEvent = (PipeRealtimeEvent) pendingQueue.directPoll(); - } - - // Means the pending queue is empty. - return null; - } - @Override protected Event doSupply(final PipeRealtimeEvent realtimeEvent) { // Used to judge the type of the event, not directly for supplying. diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionTsFileExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionTsFileExtractor.java index 855e67b28a4d..2bdcc10c7b8d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionTsFileExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionTsFileExtractor.java @@ -84,56 +84,6 @@ public boolean isNeedListenToInsertNode() { return false; } - @Override - public Event supply() { - PipeRealtimeEvent realtimeEvent = (PipeRealtimeEvent) pendingQueue.directPoll(); - - while (realtimeEvent != null) { - while (!CachedSchemaPatternMatcher.match(realtimeEvent, this)) { - realtimeEvent.decreaseReferenceCount( - PipeRealtimeDataRegionTsFileExtractor.class.getName(), false); - realtimeEvent = (PipeRealtimeEvent) pendingQueue.directPoll(); - } - - Event suppliedEvent = null; - - if (realtimeEvent.getEvent() instanceof PipeHeartbeatEvent) { - suppliedEvent = supplyHeartbeat(realtimeEvent); - } else if (realtimeEvent.getEvent() instanceof PipeSchemaRegionWritePlanEvent - || realtimeEvent.getEvent() instanceof ProgressReportEvent) { - suppliedEvent = supplyDirectly(realtimeEvent); - } else if (realtimeEvent.increaseReferenceCount( - PipeRealtimeDataRegionTsFileExtractor.class.getName())) { - suppliedEvent = realtimeEvent.getEvent(); - } else { - // if the event's reference count can not be increased, it means the data represented by - // this event is not reliable anymore. the data has been lost. we simply discard this event - // and report the exception to PipeRuntimeAgent. - final String errorMessage = - String.format( - "Event %s can not be supplied because " - + "the reference count can not be increased, " - + "the data represented by this event is lost", - realtimeEvent.getEvent()); - LOGGER.error(errorMessage); - PipeDataNodeAgent.runtime() - .report(pipeTaskMeta, new PipeRuntimeNonCriticalException(errorMessage)); - } - - realtimeEvent.decreaseReferenceCount( - PipeRealtimeDataRegionTsFileExtractor.class.getName(), false); - - if (suppliedEvent != null) { - return suppliedEvent; - } - - realtimeEvent = (PipeRealtimeEvent) pendingQueue.directPoll(); - } - - // means the pending queue is empty. - return null; - } - @Override protected Event doSupply(final PipeRealtimeEvent realtimeEvent) { if (realtimeEvent.getEvent() instanceof PipeHeartbeatEvent) { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipeRealtimeExtractTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipeRealtimeExtractTest.java index edaa11c49439..3c0581777fee 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipeRealtimeExtractTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipeRealtimeExtractTest.java @@ -250,7 +250,6 @@ public void testRealtimeExtractProcess() { } private Future write2DataRegion(int writeNum, String dataRegionId, int startNum) { - File dataRegionDir = new File(tsFileDir.getPath() + File.separator + dataRegionId + File.separator + "0"); boolean ignored = dataRegionDir.mkdirs(); From f71e98c29660b4767df07b57e6f74fb429d0f6b0 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 2 Jul 2025 15:24:42 +0800 Subject: [PATCH 082/185] partially --- .../common/heartbeat/PipeHeartbeatEvent.java | 33 +++--------- .../event/realtime/PipeRealtimeEvent.java | 8 +-- .../PipeRealtimeDataRegionExtractor.java | 52 ++++++++++--------- ...PipeRealtimeDataRegionTsFileExtractor.java | 1 - .../assigner/PipeDataRegionAssigner.java | 5 ++ .../tsfile/PipeTsFileMemResource.java | 43 +++++++-------- .../resource/tsfile/PipeTsFileResource.java | 2 +- .../tsfile/PipeTsFileResourceManager.java | 26 +++++++--- .../extractor/PipeRealtimeExtractTest.java | 9 ++-- 9 files changed, 87 insertions(+), 92 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/heartbeat/PipeHeartbeatEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/heartbeat/PipeHeartbeatEvent.java index ad29b2854424..6860847baf1a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/heartbeat/PipeHeartbeatEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/heartbeat/PipeHeartbeatEvent.java @@ -43,7 +43,6 @@ public class PipeHeartbeatEvent extends EnrichedEvent { private final String dataRegionId; private long timePublished; - private long timeAssigned; private long timeProcessed; private long timeTransferred; @@ -151,22 +150,12 @@ public void onPublished() { } } - public void onAssigned() { - if (shouldPrintMessage) { - timeAssigned = System.currentTimeMillis(); - if (timePublished != 0) { - PipeHeartbeatEventMetrics.getInstance() - .recordPublishedToAssignedTime(timeAssigned - timePublished); - } - } - } - public void onProcessed() { if (shouldPrintMessage) { timeProcessed = System.currentTimeMillis(); - if (timeAssigned != 0) { + if (timePublished != 0) { PipeHeartbeatEventMetrics.getInstance() - .recordAssignedToProcessedTime(timeProcessed - timeAssigned); + .recordAssignedToProcessedTime(timeProcessed - timePublished); } } } @@ -223,13 +212,9 @@ public String toString() { (timePublished != 0) ? DateTimeUtils.convertLongToDate(timePublished, "ms") : unknownMessage; - final String publishedToAssignedMessage = - (timeAssigned != 0 && timePublished != 0) - ? (timeAssigned - timePublished) + "ms" - : unknownMessage; final String assignedToProcessedMessage = - (timeProcessed != 0 && timeAssigned != 0) - ? (timeProcessed - timeAssigned) + "ms" + (timeProcessed != 0 && timePublished != 0) + ? (timeProcessed - timePublished) + "ms" : unknownMessage; final String processedToTransferredMessage = (timeTransferred != 0 && timeProcessed != 0) @@ -243,11 +228,11 @@ public String toString() { final String disruptorSizeMessage = Integer.toString(disruptorSize); final String extractorQueueTabletSizeMessage = - timeAssigned != 0 ? Integer.toString(extractorQueueTabletSize) : unknownMessage; + timePublished != 0 ? Integer.toString(extractorQueueTabletSize) : unknownMessage; final String extractorQueueTsFileSizeMessage = - timeAssigned != 0 ? Integer.toString(extractorQueueTsFileSize) : unknownMessage; + timePublished != 0 ? Integer.toString(extractorQueueTsFileSize) : unknownMessage; final String extractorQueueSizeMessage = - timeAssigned != 0 ? Integer.toString(extractorQueueSize) : unknownMessage; + timePublished != 0 ? Integer.toString(extractorQueueSize) : unknownMessage; final String connectorQueueTabletSizeMessage = timeProcessed != 0 ? Integer.toString(connectorQueueTabletSize) : unknownMessage; @@ -263,9 +248,7 @@ public String toString() { + dataRegionId + ", startTime=" + startTimeMessage - + ", publishedToAssigned=" - + publishedToAssignedMessage - + ", assignedToProcessed=" + + ", publishedToProcessed=" + assignedToProcessedMessage + ", processedToTransferred=" + processedToTransferredMessage diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEvent.java index d4a39193467a..65525ffe76ac 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEvent.java @@ -84,10 +84,6 @@ public Map getSchemaInfo() { return device2Measurements; } - public void gcSchemaInfo() { - device2Measurements = null; - } - public boolean mayExtractorUseTablets(final PipeRealtimeDataRegionExtractor extractor) { final TsFileEpoch.State state = tsFileEpoch.getState(extractor); return state.equals(TsFileEpoch.State.EMPTY) || state.equals(TsFileEpoch.State.USING_TABLET); @@ -182,9 +178,7 @@ public PipeRealtimeEvent shallowCopySelfAndBindPipeTaskMetaForProgressReport( event.shallowCopySelfAndBindPipeTaskMetaForProgressReport( pipeName, creationTime, pipeTaskMeta, pattern, startTime, endTime), this.tsFileEpoch, - // device2Measurements is not used anymore, so it is not copied. - // If null is not passed, the field will not be GCed and may cause OOM. - null, + this.device2Measurements, pipeTaskMeta, pattern, startTime, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java index 970fc84241a3..192c7bde1e2d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java @@ -313,29 +313,7 @@ public final void extract(final PipeRealtimeEvent event) { } } - // 1. Check if time parsing is necessary. If not, it means that the timestamps of the data - // contained in this event are definitely within the time range [start time, end time]. - // 2. Check if the event's data timestamps may intersect with the time range. If not, it means - // that the data timestamps of this event are definitely not within the time range. - // 3. Check if pattern parsing is necessary. If not, it means that the paths of the data - // contained in this event are definitely covered by the pattern. - // 4. Check if the event's data paths may intersect with the pattern. If not, it means that the - // data of this event is definitely not overlapped with the pattern. - if ((!event.shouldParseTime() || event.getEvent().mayEventTimeOverlappedWithTimeRange()) - && (!event.shouldParsePattern() || event.getEvent().mayEventPathsOverlappedWithPattern())) { - if (sloppyTimeRange) { - // only skip parsing time for events whose data timestamps may intersect with the time range - event.skipParsingTime(); - } - if (sloppyPattern) { - // only skip parsing pattern for events whose data paths may intersect with the pattern - event.skipParsingPattern(); - } - - doExtract(event); - } else { - event.decreaseReferenceCount(PipeRealtimeDataRegionExtractor.class.getName(), false); - } + doExtract(event); synchronized (isClosed) { if (isClosed.get()) { @@ -401,7 +379,8 @@ public Event supply() { PipeRealtimeEvent realtimeEvent = (PipeRealtimeEvent) pendingQueue.directPoll(); while (realtimeEvent != null) { - while (!CachedSchemaPatternMatcher.match(realtimeEvent, this)) { + while (!CachedSchemaPatternMatcher.match(realtimeEvent, this) + || !coarseFilterEvent(realtimeEvent)) { realtimeEvent.decreaseReferenceCount( PipeRealtimeDataRegionTsFileExtractor.class.getName(), false); realtimeEvent = (PipeRealtimeEvent) pendingQueue.directPoll(); @@ -420,6 +399,31 @@ public Event supply() { return null; } + // This may require some time thus we leave it for processor thread instead of writing thread + private boolean coarseFilterEvent(final PipeRealtimeEvent event) { + // 1. Check if time parsing is necessary. If not, it means that the timestamps of the data + // contained in this event are definitely within the time range [start time, end time]. + // 2. Check if the event's data timestamps may intersect with the time range. If not, it means + // that the data timestamps of this event are definitely not within the time range. + // 3. Check if pattern parsing is necessary. If not, it means that the paths of the data + // contained in this event are definitely covered by the pattern. + // 4. Check if the event's data paths may intersect with the pattern. If not, it means that the + // data of this event is definitely not overlapped with the pattern. + if ((!event.shouldParseTime() || event.getEvent().mayEventTimeOverlappedWithTimeRange()) + && (!event.shouldParsePattern() || event.getEvent().mayEventPathsOverlappedWithPattern())) { + if (sloppyTimeRange) { + // only skip parsing time for events whose data timestamps may intersect with the time range + event.skipParsingTime(); + } + if (sloppyPattern) { + // only skip parsing pattern for events whose data paths may intersect with the pattern + event.skipParsingPattern(); + } + return true; + } + return false; + } + protected abstract Event doSupply(final PipeRealtimeEvent realtimeEvent); protected Event supplyHeartbeat(final PipeRealtimeEvent event) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionTsFileExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionTsFileExtractor.java index 2bdcc10c7b8d..d881ec49943e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionTsFileExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionTsFileExtractor.java @@ -26,7 +26,6 @@ import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.epoch.TsFileEpoch; -import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.matcher.CachedSchemaPatternMatcher; import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.pipe.api.event.dml.insertion.TsFileInsertionEvent; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java index 27bb46fefe24..d5ddad181232 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java @@ -24,6 +24,7 @@ import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.event.ProgressReportEvent; +import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; @@ -62,6 +63,10 @@ public PipeDataRegionAssigner(final String dataRegionId) { } public void assignToExtractor(final PipeRealtimeEvent event) { + if (event.getEvent() instanceof PipeHeartbeatEvent) { + ((PipeHeartbeatEvent) event.getEvent()).onPublished(); + } + extractors.forEach( extractor -> { if (event.getEvent().isGeneratedByPipe() && !extractor.isForwardingPipeRequests()) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileMemResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileMemResource.java index c7a4b23205fd..3673afb92e64 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileMemResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileMemResource.java @@ -47,8 +47,8 @@ public class PipeTsFileMemResource extends PipeTsFileResource { private Map deviceIsAlignedMap = null; private Map measurementDataTypeMap = null; - public PipeTsFileMemResource(final File hardlinkOrCopiedFile) { - super(hardlinkOrCopiedFile); + public PipeTsFileMemResource() { + super(null); } @Override @@ -73,34 +73,35 @@ public void close() { //////////////////////////// Cache Getter //////////////////////////// - public synchronized Map> tryGetDeviceMeasurementsMap() + public synchronized Map> tryGetDeviceMeasurementsMap(final File tsFile) throws IOException { if (deviceMeasurementsMap == null) { - cacheObjectsIfAbsent(); + cacheObjectsIfAbsent(tsFile); } return deviceMeasurementsMap; } public synchronized Map tryGetDeviceIsAlignedMap( - final boolean cacheOtherMetadata) throws IOException { + final boolean cacheOtherMetadata, final File tsFile) throws IOException { if (deviceIsAlignedMap == null) { if (cacheOtherMetadata) { - cacheObjectsIfAbsent(); + cacheObjectsIfAbsent(tsFile); } else { - cacheDeviceIsAlignedMapIfAbsent(); + cacheDeviceIsAlignedMapIfAbsent(tsFile); } } return deviceIsAlignedMap; } - public synchronized Map tryGetMeasurementDataTypeMap() throws IOException { + public synchronized Map tryGetMeasurementDataTypeMap(final File tsFile) + throws IOException { if (measurementDataTypeMap == null) { - cacheObjectsIfAbsent(); + cacheObjectsIfAbsent(tsFile); } return measurementDataTypeMap; } - synchronized boolean cacheDeviceIsAlignedMapIfAbsent() throws IOException { + synchronized boolean cacheDeviceIsAlignedMapIfAbsent(final File tsFile) throws IOException { if (allocatedMemoryBlock != null) { // This means objects are already cached. @@ -118,13 +119,13 @@ synchronized boolean cacheDeviceIsAlignedMapIfAbsent() throws IOException { if (allocatedMemoryBlock == null) { LOGGER.info( "Failed to cacheDeviceIsAlignedMapIfAbsent for tsfile {}, because memory usage is high", - hardlinkOrCopiedFile.getPath()); + tsFile.getPath()); return false; } long memoryRequiredInBytes = 0L; try (TsFileSequenceReader sequenceReader = - new TsFileSequenceReader(hardlinkOrCopiedFile.getPath(), true, false)) { + new TsFileSequenceReader(tsFile.getPath(), true, false)) { deviceIsAlignedMap = new HashMap<>(); final TsFileDeviceIterator deviceIsAlignedIterator = sequenceReader.getAllDevicesIteratorWithIsAligned(); @@ -145,19 +146,16 @@ synchronized boolean cacheDeviceIsAlignedMapIfAbsent() throws IOException { if (allocatedMemoryBlock == null) { LOGGER.info( "PipeTsFileResource: Failed to cache objects for tsfile {} in cache, because memory usage is high", - hardlinkOrCopiedFile.getPath()); + tsFile.getPath()); deviceIsAlignedMap = null; return false; } - LOGGER.info( - "PipeTsFileResource: Cached deviceIsAlignedMap for tsfile {}.", - hardlinkOrCopiedFile.getPath()); + LOGGER.info("PipeTsFileResource: Cached deviceIsAlignedMap for tsfile {}.", tsFile.getPath()); return true; } - synchronized boolean cacheObjectsIfAbsent() throws IOException { - + synchronized boolean cacheObjectsIfAbsent(final File tsFile) throws IOException { if (allocatedMemoryBlock != null) { if (deviceMeasurementsMap != null) { return true; @@ -179,13 +177,13 @@ synchronized boolean cacheObjectsIfAbsent() throws IOException { if (allocatedMemoryBlock == null) { LOGGER.info( "Failed to cacheObjectsIfAbsent for tsfile {}, because memory usage is high", - hardlinkOrCopiedFile.getPath()); + tsFile.getPath()); return false; } long memoryRequiredInBytes = 0L; try (TsFileSequenceReader sequenceReader = - new TsFileSequenceReader(hardlinkOrCopiedFile.getPath(), true, true)) { + new TsFileSequenceReader(tsFile.getPath(), true, true)) { deviceMeasurementsMap = sequenceReader.getDeviceMeasurementsMap(); memoryRequiredInBytes += PipeMemoryWeightUtil.memoryOfIDeviceID2StrList(deviceMeasurementsMap); @@ -215,15 +213,14 @@ synchronized boolean cacheObjectsIfAbsent() throws IOException { if (allocatedMemoryBlock == null) { LOGGER.info( "PipeTsFileResource: Failed to cache objects for tsfile {} in cache, because memory usage is high", - hardlinkOrCopiedFile.getPath()); + tsFile.getPath()); deviceIsAlignedMap = null; deviceMeasurementsMap = null; measurementDataTypeMap = null; return false; } - LOGGER.info( - "PipeTsFileResource: Cached objects for tsfile {}.", hardlinkOrCopiedFile.getPath()); + LOGGER.info("PipeTsFileResource: Cached objects for tsfile {}.", tsFile.getPath()); return true; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResource.java index 2b0f480c4b2a..5065821ebe4f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResource.java @@ -30,7 +30,7 @@ public class PipeTsFileResource implements AutoCloseable { private static final Logger LOGGER = LoggerFactory.getLogger(PipeTsFileResource.class); - protected final File hardlinkOrCopiedFile; + private final File hardlinkOrCopiedFile; private volatile long fileSize = -1L; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java index 85c8a65750dd..37b3014a499b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java @@ -147,7 +147,7 @@ private void increaseMemReference(final File file, final boolean isTsFile) { getCommonFilePath(file), (k, v) -> { if (Objects.isNull(v)) { - return new PipeTsFileMemResource(file); + return new PipeTsFileMemResource(); } else { v.increaseReferenceCount(); return v; @@ -240,9 +240,13 @@ private void decreaseMemReferenceIfExists(final File file) { } // Warning: Shall not be called by the assigner - private String getCommonFilePath(final File file) { + private String getCommonFilePath(final @Nonnull File file) { + // If the parent or grandparent is null then this is testing scenario // Skip the "pipeName" of this file - return file.getParentFile().getParent() + File.separator + file.getName(); + return Objects.isNull(file.getParentFile()) + || Objects.isNull(file.getParentFile().getParentFile()) + ? file.getPath() + : file.getParentFile().getParent() + File.separator + file.getName(); } /** @@ -278,9 +282,13 @@ public int getFileReferenceCount( public boolean cacheObjectsIfAbsent(final File hardlinkOrCopiedTsFile) throws IOException { segmentLock.lock(hardlinkOrCopiedTsFile); try { + if (hardlinkOrCopiedTsFile.getParentFile() == null + || hardlinkOrCopiedTsFile.getParentFile().getParentFile() == null) { + return false; + } final PipeTsFileMemResource resource = hardlinkOrCopiedFileToTsFileMemResourceMap.get(getCommonFilePath(hardlinkOrCopiedTsFile)); - return resource != null && resource.cacheObjectsIfAbsent(); + return resource != null && resource.cacheObjectsIfAbsent(hardlinkOrCopiedTsFile); } finally { segmentLock.unlock(hardlinkOrCopiedTsFile); } @@ -292,7 +300,7 @@ public Map> getDeviceMeasurementsMapFromCache( try { final PipeTsFileMemResource resource = hardlinkOrCopiedFileToTsFileMemResourceMap.get(getCommonFilePath(hardlinkOrCopiedTsFile)); - return resource == null ? null : resource.tryGetDeviceMeasurementsMap(); + return resource == null ? null : resource.tryGetDeviceMeasurementsMap(hardlinkOrCopiedTsFile); } finally { segmentLock.unlock(hardlinkOrCopiedTsFile); } @@ -304,7 +312,9 @@ public Map getDeviceIsAlignedMapFromCache( try { final PipeTsFileMemResource resource = hardlinkOrCopiedFileToTsFileMemResourceMap.get(getCommonFilePath(hardlinkOrCopiedTsFile)); - return resource == null ? null : resource.tryGetDeviceIsAlignedMap(cacheOtherMetadata); + return resource == null + ? null + : resource.tryGetDeviceIsAlignedMap(cacheOtherMetadata, hardlinkOrCopiedTsFile); } finally { segmentLock.unlock(hardlinkOrCopiedTsFile); } @@ -316,7 +326,9 @@ public Map getMeasurementDataTypeMapFromCache( try { final PipeTsFileMemResource resource = hardlinkOrCopiedFileToTsFileMemResourceMap.get(getCommonFilePath(hardlinkOrCopiedTsFile)); - return resource == null ? null : resource.tryGetMeasurementDataTypeMap(); + return resource == null + ? null + : resource.tryGetMeasurementDataTypeMap(hardlinkOrCopiedTsFile); } finally { segmentLock.unlock(hardlinkOrCopiedTsFile); } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipeRealtimeExtractTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipeRealtimeExtractTest.java index 3c0581777fee..3918b83b5756 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipeRealtimeExtractTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipeRealtimeExtractTest.java @@ -39,6 +39,7 @@ import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; import org.apache.tsfile.common.constant.TsFileConstant; +import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.file.metadata.PlainDeviceID; import org.junit.After; import org.junit.Assert; @@ -283,9 +284,9 @@ private Future write2DataRegion(int writeNum, String dataRegionId, int startN new PartialPath(device), false, new String[] {"a"}, - null, + new TSDataType[] {TSDataType.INT32}, 0, - null, + new Integer[] {1}, false), resource); PipeInsertionDataNodeListener.getInstance() @@ -296,9 +297,9 @@ private Future write2DataRegion(int writeNum, String dataRegionId, int startN new PartialPath(device), false, new String[] {"b"}, - null, + new TSDataType[] {TSDataType.INT32}, 0, - null, + new Integer[] {1}, false), resource); PipeInsertionDataNodeListener.getInstance() From 12f49fe9b63e0227f2ad2c07991547582794cf48 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 2 Jul 2025 15:26:49 +0800 Subject: [PATCH 083/185] Update PipeRealtimeExtractTest.java --- .../extractor/PipeRealtimeExtractTest.java | 58 ++++++++++--------- 1 file changed, 31 insertions(+), 27 deletions(-) diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipeRealtimeExtractTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipeRealtimeExtractTest.java index 3918b83b5756..37c956783721 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipeRealtimeExtractTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipeRealtimeExtractTest.java @@ -104,36 +104,37 @@ public void tearDown() { public void testRealtimeExtractProcess() { // set up realtime extractor - try (PipeRealtimeDataRegionLogExtractor extractor0 = new PipeRealtimeDataRegionLogExtractor(); - PipeRealtimeDataRegionHybridExtractor extractor1 = + try (final PipeRealtimeDataRegionLogExtractor extractor0 = + new PipeRealtimeDataRegionLogExtractor(); + final PipeRealtimeDataRegionHybridExtractor extractor1 = new PipeRealtimeDataRegionHybridExtractor(); - PipeRealtimeDataRegionTsFileExtractor extractor2 = + final PipeRealtimeDataRegionTsFileExtractor extractor2 = new PipeRealtimeDataRegionTsFileExtractor(); - PipeRealtimeDataRegionHybridExtractor extractor3 = + final PipeRealtimeDataRegionHybridExtractor extractor3 = new PipeRealtimeDataRegionHybridExtractor()) { - PipeParameters parameters0 = + final PipeParameters parameters0 = new PipeParameters( new HashMap() { { put(PipeExtractorConstant.EXTRACTOR_PATTERN_KEY, pattern1); } }); - PipeParameters parameters1 = + final PipeParameters parameters1 = new PipeParameters( new HashMap() { { put(PipeExtractorConstant.EXTRACTOR_PATTERN_KEY, pattern2); } }); - PipeParameters parameters2 = + final PipeParameters parameters2 = new PipeParameters( new HashMap() { { put(PipeExtractorConstant.EXTRACTOR_PATTERN_KEY, pattern1); } }); - PipeParameters parameters3 = + final PipeParameters parameters3 = new PipeParameters( new HashMap() { { @@ -141,16 +142,16 @@ public void testRealtimeExtractProcess() { } }); - PipeTaskRuntimeConfiguration configuration0 = + final PipeTaskRuntimeConfiguration configuration0 = new PipeTaskRuntimeConfiguration( new PipeTaskExtractorRuntimeEnvironment("1", 1, Integer.parseInt(dataRegion1), null)); - PipeTaskRuntimeConfiguration configuration1 = + final PipeTaskRuntimeConfiguration configuration1 = new PipeTaskRuntimeConfiguration( new PipeTaskExtractorRuntimeEnvironment("1", 1, Integer.parseInt(dataRegion1), null)); - PipeTaskRuntimeConfiguration configuration2 = + final PipeTaskRuntimeConfiguration configuration2 = new PipeTaskRuntimeConfiguration( new PipeTaskExtractorRuntimeEnvironment("1", 1, Integer.parseInt(dataRegion2), null)); - PipeTaskRuntimeConfiguration configuration3 = + final PipeTaskRuntimeConfiguration configuration3 = new PipeTaskRuntimeConfiguration( new PipeTaskExtractorRuntimeEnvironment("1", 1, Integer.parseInt(dataRegion2), null)); @@ -164,7 +165,7 @@ public void testRealtimeExtractProcess() { extractor3.validate(new PipeParameterValidator(parameters3)); extractor3.customize(parameters3, configuration3); - PipeRealtimeDataRegionExtractor[] extractors = + final PipeRealtimeDataRegionExtractor[] extractors = new PipeRealtimeDataRegionExtractor[] {extractor0, extractor1, extractor2, extractor3}; // start extractor 0, 1 @@ -172,7 +173,7 @@ public void testRealtimeExtractProcess() { extractors[1].start(); // test result of extractor 0, 1 - int writeNum = 10; + final int writeNum = 10; List> writeFutures = Arrays.asList( write2DataRegion(writeNum, dataRegion1, 0), @@ -190,7 +191,7 @@ public void testRealtimeExtractProcess() { try { listenFutures.get(0).get(10, TimeUnit.MINUTES); listenFutures.get(1).get(10, TimeUnit.MINUTES); - } catch (TimeoutException e) { + } catch (final TimeoutException e) { LOGGER.warn("Time out when listening extractor", e); alive.set(false); Assert.fail(); @@ -232,7 +233,7 @@ public void testRealtimeExtractProcess() { listenFutures.get(1).get(10, TimeUnit.MINUTES); listenFutures.get(2).get(10, TimeUnit.MINUTES); listenFutures.get(3).get(10, TimeUnit.MINUTES); - } catch (TimeoutException e) { + } catch (final TimeoutException e) { LOGGER.warn("Time out when listening extractor", e); alive.set(false); Assert.fail(); @@ -245,33 +246,34 @@ public void testRealtimeExtractProcess() { throw new RuntimeException(e); } }); - } catch (Exception e) { + } catch (final Exception e) { throw new RuntimeException(e); } } - private Future write2DataRegion(int writeNum, String dataRegionId, int startNum) { - File dataRegionDir = + private Future write2DataRegion( + final int writeNum, final String dataRegionId, final int startNum) { + final File dataRegionDir = new File(tsFileDir.getPath() + File.separator + dataRegionId + File.separator + "0"); - boolean ignored = dataRegionDir.mkdirs(); + final boolean ignored = dataRegionDir.mkdirs(); return writeService.submit( () -> { for (int i = startNum; i < startNum + writeNum; ++i) { - File tsFile = new File(dataRegionDir, String.format("%s-%s-0-0.tsfile", i, i)); + final File tsFile = new File(dataRegionDir, String.format("%s-%s-0-0.tsfile", i, i)); try { - boolean ignored1 = tsFile.createNewFile(); - } catch (IOException e) { + final boolean ignored1 = tsFile.createNewFile(); + } catch (final IOException e) { e.printStackTrace(); throw new RuntimeException(e); } - TsFileResource resource = new TsFileResource(tsFile); + final TsFileResource resource = new TsFileResource(tsFile); resource.updateStartTime( new PlainDeviceID(String.join(TsFileConstant.PATH_SEPARATOR, device)), 0); try { resource.close(); - } catch (IOException e) { + } catch (final IOException e) { e.printStackTrace(); throw new RuntimeException(e); } @@ -309,7 +311,9 @@ private Future write2DataRegion(int writeNum, String dataRegionId, int startN } private Future listen( - PipeRealtimeDataRegionExtractor extractor, Function weight, int expectNum) { + final PipeRealtimeDataRegionExtractor extractor, + final Function weight, + final int expectNum) { return listenerService.submit( () -> { int eventNum = 0; @@ -318,7 +322,7 @@ private Future listen( Event event; try { event = extractor.supply(); - } catch (Exception e) { + } catch (final Exception e) { throw new RuntimeException(e); } if (event != null) { From a37bf451acc10745ce919e35842485759dba7639 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 2 Jul 2025 15:38:50 +0800 Subject: [PATCH 084/185] bug fix --- .../PipeRealtimeDataRegionExtractor.java | 25 +++++++++++++------ 1 file changed, 17 insertions(+), 8 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java index 192c7bde1e2d..724d9876348e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java @@ -376,16 +376,9 @@ protected void extractDirectly(final PipeRealtimeEvent event) { @Override public Event supply() { - PipeRealtimeEvent realtimeEvent = (PipeRealtimeEvent) pendingQueue.directPoll(); + PipeRealtimeEvent realtimeEvent = getNextRealtimeEvent(); while (realtimeEvent != null) { - while (!CachedSchemaPatternMatcher.match(realtimeEvent, this) - || !coarseFilterEvent(realtimeEvent)) { - realtimeEvent.decreaseReferenceCount( - PipeRealtimeDataRegionTsFileExtractor.class.getName(), false); - realtimeEvent = (PipeRealtimeEvent) pendingQueue.directPoll(); - } - final Event suppliedEvent = doSupply(realtimeEvent); realtimeEvent.decreaseReferenceCount(PipeRealtimeDataRegionExtractor.class.getName(), false); @@ -393,12 +386,28 @@ public Event supply() { if (suppliedEvent != null) { return suppliedEvent; } + + realtimeEvent = getNextRealtimeEvent(); } // means the pending queue is empty. return null; } + private PipeRealtimeEvent getNextRealtimeEvent() { + PipeRealtimeEvent realtimeEvent = (PipeRealtimeEvent) pendingQueue.directPoll(); + + while (realtimeEvent != null + && (!CachedSchemaPatternMatcher.match(realtimeEvent, this) + || !coarseFilterEvent(realtimeEvent))) { + realtimeEvent.decreaseReferenceCount( + PipeRealtimeDataRegionTsFileExtractor.class.getName(), false); + realtimeEvent = (PipeRealtimeEvent) pendingQueue.directPoll(); + } + + return realtimeEvent; + } + // This may require some time thus we leave it for processor thread instead of writing thread private boolean coarseFilterEvent(final PipeRealtimeEvent event) { // 1. Check if time parsing is necessary. If not, it means that the timestamps of the data From 90937d6a36479f4ae4dc308e733ea26cf2ac2601 Mon Sep 17 00:00:00 2001 From: Steve Yurong Su Date: Wed, 2 Jul 2025 16:10:46 +0800 Subject: [PATCH 085/185] ready for PipeCompactedTsFileInsertionEvent constructor --- .../PipeRealtimePriorityBlockingQueue.java | 95 ++++++++++++++++++- .../PipeCompactedTsFileInsertionEvent.java | 36 +++++++ .../tsfile/PipeTsFileInsertionEvent.java | 26 ++--- .../tsfile/PipeCompactionManager.java | 12 +-- 4 files changed, 143 insertions(+), 26 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java index 1b4d36541155..f2c8b4bbbef1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java @@ -26,22 +26,25 @@ import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.db.pipe.agent.task.connection.PipeEventCollector; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; +import org.apache.iotdb.db.pipe.event.common.tsfile.PipeCompactedTsFileInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.pipe.metric.source.PipeDataRegionEventCounter; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.pipe.api.event.dml.insertion.TsFileInsertionEvent; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.File; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Set; import java.util.concurrent.BlockingDeque; import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; @@ -199,23 +202,105 @@ public synchronized Event peek() { } public synchronized void replace( - String dataRegionId, Set sourceFiles, Set targetFiles) { + String dataRegionId, Set sourceFiles, List targetFiles) { + final int regionId = Integer.parseInt(dataRegionId); final Map> eventsToBeRemovedGroupByCommitterKey = tsfileInsertEventDeque.stream() - .filter(event -> event instanceof PipeTsFileInsertionEvent) + .filter( + event -> + event instanceof PipeTsFileInsertionEvent + && ((PipeTsFileInsertionEvent) event).getRegionId() == regionId) .map(event -> (PipeTsFileInsertionEvent) event) .collect( Collectors.groupingBy( PipeTsFileInsertionEvent::getCommitterKey, Collectors.toSet())) .entrySet() .stream() + // Replace if all source files are present in the queue .filter(entry -> entry.getValue().size() == sourceFiles.size()) .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + if (eventsToBeRemovedGroupByCommitterKey.isEmpty()) { + return; + } - final Map eventToBeAddedGroupByCommitterKey = + final Map> eventsToBeAddedGroupByCommitterKey = new HashMap<>(); - // TBD + for (final Map.Entry> entry : + eventsToBeRemovedGroupByCommitterKey.entrySet()) { + final CommitterKey committerKey = entry.getKey(); + final Set newEvents = new HashSet<>(); + for (int i = 0; i < targetFiles.size(); i++) { + newEvents.add( + new PipeCompactedTsFileInsertionEvent( + committerKey, entry.getValue(), targetFiles.get(i), i == targetFiles.size() - 1)); + } + eventsToBeAddedGroupByCommitterKey.put(committerKey, newEvents); + } + // Handling new events + final Set successfullyReferenceIncreasedEvents = new HashSet<>(); + final AtomicBoolean + allSuccess = // To track if all events successfully increased the reference count + new AtomicBoolean(true); + outerLoop: + for (final Map.Entry> committerKeySetEntry : + eventsToBeAddedGroupByCommitterKey.entrySet()) { + for (final PipeTsFileInsertionEvent event : committerKeySetEntry.getValue()) { + if (event != null) { + try { + if (!event.increaseReferenceCount(PipeRealtimePriorityBlockingQueue.class.getName())) { + allSuccess.set(false); + break outerLoop; + } else { + successfullyReferenceIncreasedEvents.add(event); + } + } catch (final Exception e) { + allSuccess.set(false); + break outerLoop; + } + } + } + } + if (!allSuccess.get()) { + // If any event failed to increase the reference count, + // we need to decrease the reference count for all successfully increased events + for (final PipeTsFileInsertionEvent event : successfullyReferenceIncreasedEvents) { + try { + event.decreaseReferenceCount(PipeRealtimePriorityBlockingQueue.class.getName(), false); + } catch (final Exception e) { + LOGGER.warn( + "Failed to decrease reference count for event {} in PipeRealtimePriorityBlockingQueue", + event, + e); + } + } + return; // Exit early if any event failed to increase the reference count + } else { + // If all events successfully increased reference count, + // we can proceed to add them to the deque + for (final PipeTsFileInsertionEvent event : successfullyReferenceIncreasedEvents) { + tsfileInsertEventDeque.add(event); + eventCounter.increaseEventCount(event); + } + } + + // Handling old events + for (final Map.Entry> entry : + eventsToBeRemovedGroupByCommitterKey.entrySet()) { + for (final PipeTsFileInsertionEvent event : entry.getValue()) { + if (event != null) { + try { + event.decreaseReferenceCount(PipeRealtimePriorityBlockingQueue.class.getName(), false); + } catch (final Exception e) { + LOGGER.warn( + "Failed to decrease reference count for event {} in PipeRealtimePriorityBlockingQueue", + event, + e); + } + eventCounter.decreaseEventCount(event); + } + } + } final Set eventsToRemove = new HashSet<>(); for (Set pipeTsFileInsertionEvents : eventsToBeRemovedGroupByCommitterKey.values()) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java new file mode 100644 index 000000000000..049ed49fb3a2 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java @@ -0,0 +1,36 @@ +/* + * 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.pipe.event.common.tsfile; + +import org.apache.iotdb.commons.pipe.agent.task.progress.CommitterKey; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; + +import java.util.Set; + +public class PipeCompactedTsFileInsertionEvent extends PipeTsFileInsertionEvent { + + public PipeCompactedTsFileInsertionEvent( + final CommitterKey committerKey, + final Set value, + final TsFileResource tsFileResource, + final boolean shouldReportProgress) { + super(null, false); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java index b425d8a3e937..b470a64c2a74 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java @@ -65,27 +65,27 @@ public class PipeTsFileInsertionEvent extends EnrichedEvent private static final Logger LOGGER = LoggerFactory.getLogger(PipeTsFileInsertionEvent.class); - private final TsFileResource resource; - private File tsFile; - private long extractTime = 0; + protected final TsFileResource resource; + protected File tsFile; + protected long extractTime = 0; // This is true iff the modFile exists and should be transferred - private boolean isWithMod; - private File modFile; + protected boolean isWithMod; + protected File modFile; - private final boolean isLoaded; - private final boolean isGeneratedByPipe; - private final boolean isGeneratedByPipeConsensus; - private final boolean isGeneratedByHistoricalExtractor; + protected final boolean isLoaded; + protected final boolean isGeneratedByPipe; + protected final boolean isGeneratedByPipeConsensus; + protected final boolean isGeneratedByHistoricalExtractor; - private final AtomicBoolean isClosed; - private final AtomicReference dataContainer; + protected final AtomicBoolean isClosed; + protected final AtomicReference dataContainer; // The point count of the TsFile. Used for metrics on PipeConsensus' receiver side. // May be updated after it is flushed. Should be negative if not set. - private long flushPointCount = TsFileProcessor.FLUSH_POINT_COUNT_NOT_SET; + protected long flushPointCount = TsFileProcessor.FLUSH_POINT_COUNT_NOT_SET; - private volatile ProgressIndex overridingProgressIndex; + protected volatile ProgressIndex overridingProgressIndex; public PipeTsFileInsertionEvent(final TsFileResource resource, final boolean isLoaded) { // The modFile must be copied before the event is assigned to the listening pipes diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeCompactionManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeCompactionManager.java index 2aa612235765..29d880c720d7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeCompactionManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeCompactionManager.java @@ -25,12 +25,10 @@ import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.iotdb.pipe.api.event.Event; -import java.io.File; import java.util.HashSet; import java.util.List; import java.util.Set; import java.util.concurrent.CopyOnWriteArraySet; -import java.util.stream.Collectors; public class PipeCompactionManager { @@ -54,11 +52,8 @@ public void emitResult( final List seqFileResources, final List unseqFileResources, final List targetFileResources) { - final Set sourceFiles = new HashSet<>(); - seqFileResources.forEach(tsFileResource -> sourceFiles.add(tsFileResource.getTsFile())); - unseqFileResources.forEach(tsFileResource -> sourceFiles.add(tsFileResource.getTsFile())); - final Set targetFiles = - targetFileResources.stream().map(TsFileResource::getTsFile).collect(Collectors.toSet()); + final Set sourceFileResources = new HashSet<>(seqFileResources); + sourceFileResources.addAll(unseqFileResources); for (final PipeConnectorSubtaskLifeCycle lifeCycle : pipeConnectorSubtaskLifeCycles) { final UnboundedBlockingPendingQueue pendingQueue = lifeCycle.getPendingQueue(); @@ -66,7 +61,8 @@ public void emitResult( if (pendingQueue instanceof PipeRealtimePriorityBlockingQueue) { final PipeRealtimePriorityBlockingQueue realtimePriorityBlockingQueue = (PipeRealtimePriorityBlockingQueue) pendingQueue; - realtimePriorityBlockingQueue.replace(dataRegionId, sourceFiles, targetFiles); + realtimePriorityBlockingQueue.replace( + dataRegionId, sourceFileResources, targetFileResources); } } } From 009db93b60818a4b4430f92e20a54258e2dc827d Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 2 Jul 2025 17:25:00 +0800 Subject: [PATCH 086/185] made resource nullable --- .../tsfile/PipeTsFileInsertionEvent.java | 47 +++++++++++++------ ...peHistoricalDataRegionTsFileExtractor.java | 1 + 2 files changed, 34 insertions(+), 14 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java index c50cdb165ea5..eb2d8fbb0885 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java @@ -48,6 +48,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.io.File; import java.io.IOException; import java.util.Collections; @@ -65,7 +67,7 @@ public class PipeTsFileInsertionEvent extends EnrichedEvent private static final Logger LOGGER = LoggerFactory.getLogger(PipeTsFileInsertionEvent.class); - private final TsFileResource resource; + private @Nullable final TsFileResource resource; private File tsFile; private long extractTime = 0; @@ -89,11 +91,13 @@ public class PipeTsFileInsertionEvent extends EnrichedEvent public PipeTsFileInsertionEvent(final TsFileResource resource, final boolean isLoaded) { // The modFile must be copied before the event is assigned to the listening pipes - this(resource, true, isLoaded, false, null, 0, null, null, Long.MIN_VALUE, Long.MAX_VALUE); + this( + resource, null, true, isLoaded, false, null, 0, null, null, Long.MIN_VALUE, Long.MAX_VALUE); } public PipeTsFileInsertionEvent( - final TsFileResource resource, + final @Nullable TsFileResource resource, + final @Nullable File tsFile, final boolean isWithMod, final boolean isLoaded, final boolean isGeneratedByHistoricalExtractor, @@ -104,24 +108,35 @@ public PipeTsFileInsertionEvent( final long startTime, final long endTime) { super(pipeName, creationTime, pipeTaskMeta, pattern, startTime, endTime); - + assert Objects.nonNull(tsFile) || Objects.nonNull(resource); this.resource = resource; // For events created at assigner or historical extractor, the tsFile is get from the resource // For events created for source, the tsFile is inherited from the assigner, because the // original tsFile may be gone, and we need to get the assigner's hard-linked tsFile to // hard-link it to each pipe dir - this.tsFile = resource.getTsFile(); + this.tsFile = Objects.nonNull(resource) ? resource.getTsFile() : tsFile; - final ModificationFile modFile = resource.getModFile(); + final ModificationFile modFile = + Objects.nonNull(resource) + ? resource.getModFile() + : new ModificationFile(tsFile.getAbsolutePath() + ModificationFile.FILE_SUFFIX); this.isWithMod = isWithMod && modFile.exists(); this.modFile = this.isWithMod ? new File(modFile.getFilePath()) : null; this.isLoaded = isLoaded; - this.isGeneratedByPipe = resource.isGeneratedByPipe(); - this.isGeneratedByPipeConsensus = resource.isGeneratedByPipeConsensus(); + this.isGeneratedByPipe = Objects.nonNull(resource) && resource.isGeneratedByPipe(); + this.isGeneratedByPipeConsensus = + Objects.nonNull(resource) && resource.isGeneratedByPipeConsensus(); this.isGeneratedByHistoricalExtractor = isGeneratedByHistoricalExtractor; + this.dataContainer = new AtomicReference<>(null); + + if (Objects.isNull(resource)) { + isClosed = new AtomicBoolean(true); + return; + } + isClosed = new AtomicBoolean(resource.isClosed()); // Register close listener if TsFile is not closed if (!isClosed.get()) { @@ -157,8 +172,6 @@ public PipeTsFileInsertionEvent( // If the status is "closed", then the resource status is "closed", the tsFile won't be altered // and can be sent. isClosed.set(resource.isClosed()); - - this.dataContainer = new AtomicReference<>(null); } /** @@ -166,6 +179,10 @@ public PipeTsFileInsertionEvent( * otherwise. */ public boolean waitForTsFileClose() throws InterruptedException { + if (Objects.isNull(resource)) { + return true; + } + if (!isClosed.get()) { isClosed.set(resource.isClosed()); @@ -319,7 +336,7 @@ protected void reportProgress() { } public void eliminateProgressIndex() { - if (Objects.isNull(overridingProgressIndex)) { + if (Objects.isNull(overridingProgressIndex) && Objects.nonNull(resource)) { PipeTsFileEpochProgressIndexKeeper.getInstance() .eliminateProgressIndex(resource.getDataRegionId(), resource.getTsFilePath()); } @@ -335,6 +352,7 @@ public PipeTsFileInsertionEvent shallowCopySelfAndBindPipeTaskMetaForProgressRep final long endTime) { return new PipeTsFileInsertionEvent( resource, + tsFile, isWithMod, isLoaded, isGeneratedByHistoricalExtractor, @@ -355,12 +373,13 @@ public boolean isGeneratedByPipe() { public boolean mayEventTimeOverlappedWithTimeRange() { // Notice that this is only called at realtime extraction, and the tsFile is always closed // Thus we can use the end time to judge the overlap - return startTime <= resource.getFileEndTime() && resource.getFileStartTime() <= endTime; + return Objects.isNull(resource) + || startTime <= resource.getFileEndTime() && resource.getFileStartTime() <= endTime; } @Override public boolean mayEventPathsOverlappedWithPattern() { - if (!resource.isClosed()) { + if (Objects.isNull(resource) || !resource.isClosed()) { return true; } @@ -541,7 +560,7 @@ private TsFileInsertionDataContainer initDataContainer() { } catch (final IOException e) { close(); - final String errorMsg = String.format("Read TsFile %s error.", resource.getTsFilePath()); + final String errorMsg = String.format("Read TsFile %s error.", tsFile.getPath()); LOGGER.warn(errorMsg, e); throw new PipeException(errorMsg); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java index 0408e4f3a039..12e76afa4a65 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java @@ -547,6 +547,7 @@ public synchronized Event supply() { final PipeTsFileInsertionEvent event = new PipeTsFileInsertionEvent( resource, + null, shouldTransferModFile, false, true, From eab0c4005711294ad43d296445b403d8413055bb Mon Sep 17 00:00:00 2001 From: Steve Yurong Su Date: Wed, 2 Jul 2025 17:26:08 +0800 Subject: [PATCH 087/185] handle binding progress of PipeTsFileInsertionEvent --- .../PipeRealtimePriorityBlockingQueue.java | 7 +- .../PipeCompactedTsFileInsertionEvent.java | 87 ++++++++++++++++++- 2 files changed, 91 insertions(+), 3 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java index f2c8b4bbbef1..4d960773b0a4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java @@ -228,11 +228,16 @@ public synchronized void replace( for (final Map.Entry> entry : eventsToBeRemovedGroupByCommitterKey.entrySet()) { final CommitterKey committerKey = entry.getKey(); + final PipeTsFileInsertionEvent anyEvent = entry.getValue().stream().findFirst().orElse(null); final Set newEvents = new HashSet<>(); for (int i = 0; i < targetFiles.size(); i++) { newEvents.add( new PipeCompactedTsFileInsertionEvent( - committerKey, entry.getValue(), targetFiles.get(i), i == targetFiles.size() - 1)); + committerKey, + entry.getValue(), + anyEvent, + targetFiles.get(i), + i == targetFiles.size() - 1)); } eventsToBeAddedGroupByCommitterKey.put(committerKey, newEvents); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java index 049ed49fb3a2..00a2493660e3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java @@ -19,18 +19,101 @@ package org.apache.iotdb.db.pipe.event.common.tsfile; +import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; import org.apache.iotdb.commons.pipe.agent.task.progress.CommitterKey; +import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.assigner.PipeTsFileEpochProgressIndexKeeper; +import org.apache.iotdb.db.storageengine.dataregion.memtable.TsFileProcessor; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; +import java.io.File; +import java.util.Objects; import java.util.Set; +import java.util.stream.Collectors; public class PipeCompactedTsFileInsertionEvent extends PipeTsFileInsertionEvent { + private final String dataRegionId; + private final Set originFilePaths; + public PipeCompactedTsFileInsertionEvent( final CommitterKey committerKey, - final Set value, + final Set originalEvents, + final PipeTsFileInsertionEvent anyOfOriginalEvents, final TsFileResource tsFileResource, final boolean shouldReportProgress) { - super(null, false); + super( + tsFileResource, + bindIsWithMod(originalEvents), + bindIsLoaded(originalEvents), + bindIsGeneratedByHistoricalExtractor(originalEvents), + committerKey.getPipeName(), + committerKey.getCreationTime(), + anyOfOriginalEvents.getPipeTaskMeta(), + anyOfOriginalEvents.getPipePattern(), + anyOfOriginalEvents.getStartTime(), + anyOfOriginalEvents.getEndTime()); + + this.dataRegionId = String.valueOf(committerKey.getRegionId()); + this.originFilePaths = + originalEvents.stream() + .map(PipeTsFileInsertionEvent::getTsFile) + .map(File::getPath) + .collect(Collectors.toSet()); + + // init fields of EnrichedEvent + + // init fields of PipeTsFileInsertionEvent + flushPointCount = bindFlushPointCount(originalEvents); + overridingProgressIndex = bindOverridingProgressIndex(originalEvents); + } + + private static boolean bindIsWithMod(Set originalEvents) { + return originalEvents.stream().anyMatch(PipeTsFileInsertionEvent::isWithMod); + } + + private static boolean bindIsLoaded(Set originalEvents) { + return originalEvents.stream().anyMatch(PipeTsFileInsertionEvent::isLoaded); + } + + private static boolean bindIsGeneratedByHistoricalExtractor( + Set originalEvents) { + return originalEvents.stream() + .anyMatch(PipeTsFileInsertionEvent::isGeneratedByHistoricalExtractor); + } + + private static long bindFlushPointCount(Set originalEvents) { + return originalEvents.stream() + .mapToLong( + e -> + e.getFlushPointCount() == TsFileProcessor.FLUSH_POINT_COUNT_NOT_SET + ? 0 + : e.getFlushPointCount()) + .sum(); + } + + private ProgressIndex bindOverridingProgressIndex(Set originalEvents) { + ProgressIndex overridingProgressIndex = MinimumProgressIndex.INSTANCE; + for (PipeTsFileInsertionEvent originalEvent : originalEvents) { + if (originalEvent.overridingProgressIndex != null) { + overridingProgressIndex = + overridingProgressIndex.updateToMinimumEqualOrIsAfterProgressIndex( + originalEvent.overridingProgressIndex); + } + } + return overridingProgressIndex != null + && !overridingProgressIndex.equals(MinimumProgressIndex.INSTANCE) + ? overridingProgressIndex + : null; + } + + @Override + public void eliminateProgressIndex() { + if (Objects.isNull(overridingProgressIndex)) { + for (final String originFilePath : originFilePaths) { + PipeTsFileEpochProgressIndexKeeper.getInstance() + .eliminateProgressIndex(dataRegionId, originFilePath); + } + } } } From ee19849a622bc7b92fd825d1f87046678ea36543 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 2 Jul 2025 17:55:15 +0800 Subject: [PATCH 088/185] partially --- ...peHistoricalDataRegionTsFileExtractor.java | 8 +++++ .../tsfile/PipeTsFileResourceManager.java | 35 +++++++++++++++++++ 2 files changed, 43 insertions(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java index 12e76afa4a65..a9a1573d3ae4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java @@ -53,6 +53,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.File; import java.io.IOException; import java.util.ArrayDeque; import java.util.ArrayList; @@ -120,6 +121,7 @@ public class PipeHistoricalDataRegionTsFileExtractor implements PipeHistoricalDa private volatile boolean hasBeenStarted = false; private Queue pendingQueue; + private List tsFileList; @Override public void validate(final PipeParameterValidator validator) { @@ -323,6 +325,12 @@ public synchronized void start() { } hasBeenStarted = true; + // Recover + tsFileList = PipeDataNodeResourceManager.tsfile().recoverTsFile(pipeName); + if (Objects.nonNull(tsFileList)) { + return; + } + final DataRegion dataRegion = StorageEngine.getInstance().getDataRegion(new DataRegionId(dataRegionId)); if (Objects.isNull(dataRegion)) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java index 37b3014a499b..68b0fdbb60cd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java @@ -20,12 +20,16 @@ package org.apache.iotdb.db.pipe.resource.tsfile; import org.apache.iotdb.commons.conf.IoTDBConstant; +import org.apache.iotdb.commons.file.SystemFileFactory; import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.utils.FileUtils; import org.apache.iotdb.commons.utils.TestOnly; +import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.storageengine.dataregion.modification.ModificationFile; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; +import org.apache.iotdb.pipe.api.exception.PipeException; +import org.apache.tsfile.common.constant.TsFileConstant; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.file.metadata.IDeviceID; import org.slf4j.Logger; @@ -36,6 +40,8 @@ import java.io.File; import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Objects; @@ -354,6 +360,35 @@ public void unpinTsFileResource(final TsFileResource resource, final String pipe } } + public List recoverTsFile(final @Nonnull String pipeName) { + final List result = new ArrayList<>(); + final String suffix = + File.separator + + PipeConfig.getInstance().getPipeHardlinkBaseDirName() + + File.separator + + PipeConfig.getInstance().getPipeHardlinkTsFileDirName() + + File.separator + + pipeName; + for (final String dataDir : IoTDBDescriptor.getInstance().getConfig().getDataDirs()) { + final File pipeDir = SystemFileFactory.INSTANCE.getFile(dataDir + suffix); + if (pipeDir.exists() && pipeDir.isDirectory()) { + final File[] files = pipeDir.listFiles(); + for (final File file : files) { + try { + increaseFileReference( + file, file.getName().endsWith(TsFileConstant.TSFILE_SUFFIX), pipeName); + } catch (final IOException e) { + throw new PipeException(e.getMessage()); + } + } + result.addAll(Arrays.asList(files)); + } else { + return null; + } + } + return result; + } + public int getLinkedTsFileCount(final @Nonnull String pipeName) { return hardlinkOrCopiedFileToPipeTsFileResourceMap .computeIfAbsent(pipeName, pipe -> new ConcurrentHashMap<>()) From d0e5230a5a55f547dfdda1425838643cdc6914bc Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 2 Jul 2025 17:57:52 +0800 Subject: [PATCH 089/185] Fix --- .../pipe/resource/tsfile/PipeTsFileResourceManager.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java index 68b0fdbb60cd..ad7e2ae5935f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java @@ -41,7 +41,6 @@ import java.io.File; import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Objects; @@ -375,13 +374,15 @@ public List recoverTsFile(final @Nonnull String pipeName) { final File[] files = pipeDir.listFiles(); for (final File file : files) { try { - increaseFileReference( - file, file.getName().endsWith(TsFileConstant.TSFILE_SUFFIX), pipeName); + final boolean isTsFile = file.getName().endsWith(TsFileConstant.TSFILE_SUFFIX); + increaseFileReference(file, isTsFile, pipeName); + if (isTsFile) { + result.add(file); + } } catch (final IOException e) { throw new PipeException(e.getMessage()); } } - result.addAll(Arrays.asList(files)); } else { return null; } From 8cb16781f3b4e750c544ec0c15c903a7f1d89d85 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 2 Jul 2025 18:01:28 +0800 Subject: [PATCH 090/185] partial --- ...PipeHistoricalDataRegionTsFileExtractor.java | 17 +++++++++++++---- .../tsfile/PipeTsFileResourceManager.java | 7 ++++--- 2 files changed, 17 insertions(+), 7 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java index a9a1573d3ae4..a2e183b01fea 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java @@ -121,7 +121,7 @@ public class PipeHistoricalDataRegionTsFileExtractor implements PipeHistoricalDa private volatile boolean hasBeenStarted = false; private Queue pendingQueue; - private List tsFileList; + private Queue tsFileList; @Override public void validate(final PipeParameterValidator validator) { @@ -532,12 +532,21 @@ public synchronized Event supply() { start(); } - if (Objects.isNull(pendingQueue)) { + if (Objects.isNull(pendingQueue) && Objects.isNull(tsFileList)) { return null; } - final TsFileResource resource = pendingQueue.poll(); - if (resource == null) { + TsFileResource resource = null; + if (Objects.nonNull(pendingQueue)) { + resource = pendingQueue.poll(); + } + + File tsFile = null; + if (Objects.nonNull(tsFileList)) { + tsFile = tsFileList.poll(); + } + + if (resource == null && tsFile == null) { final PipeTerminateEvent terminateEvent = new PipeTerminateEvent(pipeName, creationTime, pipeTaskMeta, dataRegionId); if (!terminateEvent.increaseReferenceCount( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java index ad7e2ae5935f..ae53f4749f9c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java @@ -40,10 +40,11 @@ import java.io.File; import java.io.IOException; -import java.util.ArrayList; +import java.util.ArrayDeque; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Queue; import java.util.concurrent.ConcurrentHashMap; public class PipeTsFileResourceManager { @@ -359,8 +360,8 @@ public void unpinTsFileResource(final TsFileResource resource, final String pipe } } - public List recoverTsFile(final @Nonnull String pipeName) { - final List result = new ArrayList<>(); + public Queue recoverTsFile(final @Nonnull String pipeName) { + final Queue result = new ArrayDeque<>(); final String suffix = File.separator + PipeConfig.getInstance().getPipeHardlinkBaseDirName() From ad5fda52464446a9491933a9d5c2a734313e134c Mon Sep 17 00:00:00 2001 From: Steve Yurong Su Date: Wed, 2 Jul 2025 18:01:32 +0800 Subject: [PATCH 091/185] PipeCompactedTsFileInsertionEvent done --- .../PipeRealtimePriorityBlockingQueue.java | 2 + .../PipeCompactedTsFileInsertionEvent.java | 49 +++++++++++++++++++ .../commons/pipe/event/EnrichedEvent.java | 9 ++++ 3 files changed, 60 insertions(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java index 4d960773b0a4..5dc0c4b6ddcd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java @@ -203,6 +203,8 @@ public synchronized Event peek() { public synchronized void replace( String dataRegionId, Set sourceFiles, List targetFiles) { + // TODO: return if pipe sink is for pipe consensus + final int regionId = Integer.parseInt(dataRegionId); final Map> eventsToBeRemovedGroupByCommitterKey = tsfileInsertEventDeque.stream() diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java index 00a2493660e3..02e6a5a7a3a5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java @@ -22,11 +22,13 @@ import org.apache.iotdb.commons.consensus.index.ProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; import org.apache.iotdb.commons.pipe.agent.task.progress.CommitterKey; +import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.assigner.PipeTsFileEpochProgressIndexKeeper; import org.apache.iotdb.db.storageengine.dataregion.memtable.TsFileProcessor; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import java.io.File; +import java.util.List; import java.util.Objects; import java.util.Set; import java.util.stream.Collectors; @@ -35,6 +37,7 @@ public class PipeCompactedTsFileInsertionEvent extends PipeTsFileInsertionEvent private final String dataRegionId; private final Set originFilePaths; + private final List commitIds; public PipeCompactedTsFileInsertionEvent( final CommitterKey committerKey, @@ -60,8 +63,18 @@ public PipeCompactedTsFileInsertionEvent( .map(PipeTsFileInsertionEvent::getTsFile) .map(File::getPath) .collect(Collectors.toSet()); + this.commitIds = + originalEvents.stream() + .map(PipeTsFileInsertionEvent::getCommitId) + .distinct() + .collect(Collectors.toList()); // init fields of EnrichedEvent + this.committerKey = committerKey; + // TODO pipe consensus: handle rebootTimes + isPatternParsed = bindIsPatternParsed(originalEvents); + isTimeParsed = bindIsTimeParsed(originalEvents); + this.shouldReportOnCommit = shouldReportProgress; // init fields of PipeTsFileInsertionEvent flushPointCount = bindFlushPointCount(originalEvents); @@ -82,6 +95,14 @@ private static boolean bindIsGeneratedByHistoricalExtractor( .anyMatch(PipeTsFileInsertionEvent::isGeneratedByHistoricalExtractor); } + private static boolean bindIsTimeParsed(Set originalEvents) { + return originalEvents.stream().noneMatch(EnrichedEvent::shouldParseTime); + } + + private static boolean bindIsPatternParsed(Set originalEvents) { + return originalEvents.stream().noneMatch(EnrichedEvent::shouldParsePattern); + } + private static long bindFlushPointCount(Set originalEvents) { return originalEvents.stream() .mapToLong( @@ -107,6 +128,34 @@ private ProgressIndex bindOverridingProgressIndex(Set : null; } + @Override + public int getRebootTimes() { + throw new UnsupportedOperationException( + "PipeCompactedTsFileInsertionEvent does not support getRebootTimes."); + } + + @Override + public boolean hasMultipleCommitIds() { + return true; + } + + @Override + public long getCommitId() { + throw new UnsupportedOperationException( + "PipeCompactedTsFileInsertionEvent does not support getCommitId."); + } + + @Override + public List getCommitIds() { + return commitIds; + } + + @Override + public boolean equalsInPipeConsensus(final Object o) { + throw new UnsupportedOperationException( + "PipeCompactedTsFileInsertionEvent does not support equalsInPipeConsensus."); + } + @Override public void eliminateProgressIndex() { if (Objects.isNull(overridingProgressIndex)) { diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/EnrichedEvent.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/EnrichedEvent.java index a9dae7719672..4ef36c29c7ef 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/EnrichedEvent.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/EnrichedEvent.java @@ -32,6 +32,7 @@ import org.slf4j.LoggerFactory; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; @@ -395,10 +396,18 @@ public CommitterKey getCommitterKey() { return committerKey; } + public boolean hasMultipleCommitIds() { + return false; + } + public long getCommitId() { return commitId; } + public List getCommitIds() { + return Collections.singletonList(commitId); + } + public void onCommitted() { onCommittedHooks.forEach(Supplier::get); } From fcd09951b9cbbcaba4502f070b64e4578e405e6e Mon Sep 17 00:00:00 2001 From: Steve Yurong Su Date: Wed, 2 Jul 2025 18:04:59 +0800 Subject: [PATCH 092/185] Update PipeCompactedTsFileInsertionEvent.java --- .../event/common/tsfile/PipeCompactedTsFileInsertionEvent.java | 1 + 1 file changed, 1 insertion(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java index 02e6a5a7a3a5..5bca69bd5d84 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java @@ -47,6 +47,7 @@ public PipeCompactedTsFileInsertionEvent( final boolean shouldReportProgress) { super( tsFileResource, + tsFileResource.getTsFile(), bindIsWithMod(originalEvents), bindIsLoaded(originalEvents), bindIsGeneratedByHistoricalExtractor(originalEvents), From f4d1bd11fecf2e8bd0758432c9a15ba3ac8237f7 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 2 Jul 2025 18:09:04 +0800 Subject: [PATCH 093/185] may complete historical --- ...peHistoricalDataRegionTsFileExtractor.java | 49 ++++++++++++++----- .../tsfile/PipeTsFileResourceManager.java | 5 +- 2 files changed, 38 insertions(+), 16 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java index a2e183b01fea..909290cda067 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java @@ -121,7 +121,7 @@ public class PipeHistoricalDataRegionTsFileExtractor implements PipeHistoricalDa private volatile boolean hasBeenStarted = false; private Queue pendingQueue; - private Queue tsFileList; + private Queue tsFileQueue; @Override public void validate(final PipeParameterValidator validator) { @@ -326,8 +326,8 @@ public synchronized void start() { hasBeenStarted = true; // Recover - tsFileList = PipeDataNodeResourceManager.tsfile().recoverTsFile(pipeName); - if (Objects.nonNull(tsFileList)) { + tsFileQueue = PipeDataNodeResourceManager.tsfile().recoverTsFile(pipeName); + if (Objects.nonNull(tsFileQueue)) { return; } @@ -532,7 +532,7 @@ public synchronized Event supply() { start(); } - if (Objects.isNull(pendingQueue) && Objects.isNull(tsFileList)) { + if (Objects.isNull(pendingQueue) && Objects.isNull(tsFileQueue)) { return null; } @@ -542,8 +542,8 @@ public synchronized Event supply() { } File tsFile = null; - if (Objects.nonNull(tsFileList)) { - tsFile = tsFileList.poll(); + if (Objects.nonNull(tsFileQueue)) { + tsFile = tsFileQueue.poll(); } if (resource == null && tsFile == null) { @@ -564,7 +564,7 @@ public synchronized Event supply() { final PipeTsFileInsertionEvent event = new PipeTsFileInsertionEvent( resource, - null, + tsFile, shouldTransferModFile, false, true, @@ -578,7 +578,8 @@ public synchronized Event supply() { event.skipParsingPattern(); } - if (sloppyTimeRange || isTsFileResourceCoveredByTimeRange(resource)) { + if (sloppyTimeRange + || Objects.nonNull(resource) && isTsFileResourceCoveredByTimeRange(resource)) { event.skipParsingTime(); } @@ -595,13 +596,14 @@ public synchronized Event supply() { return isReferenceCountIncreased ? event : null; } finally { try { - PipeDataNodeResourceManager.tsfile().unpinTsFileResource(resource, pipeName); + PipeDataNodeResourceManager.tsfile() + .unpinTsFile(Objects.nonNull(resource) ? resource.getTsFile() : tsFile, pipeName); } catch (final IOException e) { LOGGER.warn( "Pipe {}@{}: failed to unpin TsFileResource after creating event, original path: {}", pipeName, dataRegionId, - resource.getTsFilePath()); + Objects.nonNull(resource) ? resource.getTsFilePath() : tsFile.getPath()); } } } @@ -610,13 +612,17 @@ public synchronized Event supply() { public synchronized boolean hasConsumedAll() { // If the pendingQueue is null when the function is called, it implies that the extractor only // extracts deletion thus the historical event has nothing to consume. + final Queue dataQueue = Objects.nonNull(pendingQueue) ? pendingQueue : tsFileQueue; return hasBeenStarted - && (Objects.isNull(pendingQueue) || pendingQueue.isEmpty() && isTerminateSignalSent); + && (Objects.isNull(dataQueue) || dataQueue.isEmpty() && isTerminateSignalSent); } @Override public int getPendingQueueSize() { - return Objects.nonNull(pendingQueue) ? pendingQueue.size() : 0; + if (Objects.nonNull(pendingQueue)) { + return pendingQueue.size(); + } + return Objects.nonNull(tsFileQueue) ? tsFileQueue.size() : 0; } @Override @@ -625,7 +631,7 @@ public synchronized void close() { pendingQueue.forEach( resource -> { try { - PipeDataNodeResourceManager.tsfile().unpinTsFileResource(resource, pipeName); + PipeDataNodeResourceManager.tsfile().unpinTsFile(resource.getTsFile(), pipeName); } catch (final IOException e) { LOGGER.warn( "Pipe {}@{}: failed to unpin TsFileResource after dropping pipe, original path: {}", @@ -637,5 +643,22 @@ public synchronized void close() { pendingQueue.clear(); pendingQueue = null; } + + if (Objects.nonNull(tsFileQueue)) { + tsFileQueue.forEach( + tsFile -> { + try { + PipeDataNodeResourceManager.tsfile().unpinTsFile(tsFile, pipeName); + } catch (final IOException e) { + LOGGER.warn( + "Pipe {}@{}: failed to unpin TsFile after dropping pipe, original path: {}", + pipeName, + dataRegionId, + tsFile.getPath()); + } + }); + tsFileQueue.clear(); + tsFileQueue = null; + } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java index ae53f4749f9c..334b7829b74f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java @@ -349,9 +349,8 @@ public void pinTsFileResource( } } - public void unpinTsFileResource(final TsFileResource resource, final String pipeName) - throws IOException { - final File pinnedFile = getHardlinkOrCopiedFileInPipeDir(resource.getTsFile(), pipeName); + public void unpinTsFile(final File tsFile, final String pipeName) throws IOException { + final File pinnedFile = getHardlinkOrCopiedFileInPipeDir(tsFile, pipeName); decreaseFileReference(pinnedFile, pipeName); final File modFile = new File(pinnedFile + ModificationFile.FILE_SUFFIX); From c6fc727815409883d0041034e001c4172b87d66f Mon Sep 17 00:00:00 2001 From: Steve Yurong Su Date: Wed, 2 Jul 2025 18:48:20 +0800 Subject: [PATCH 094/185] intro dummy events --- ...peTransferTabletInsertionEventHandler.java | 2 +- .../handler/PipeTransferTsFileHandler.java | 4 +- .../PipeCompactedTsFileInsertionEvent.java | 88 ++++++++++++++++++- .../task/progress/PipeEventCommitManager.java | 25 +++++- .../task/progress/PipeEventCommitter.java | 5 ++ .../commons/pipe/event/EnrichedEvent.java | 4 + 6 files changed, 121 insertions(+), 7 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTabletInsertionEventHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTabletInsertionEventHandler.java index f77d41b6662d..b3eb223e58e2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTabletInsertionEventHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTabletInsertionEventHandler.java @@ -104,7 +104,7 @@ protected void onErrorInternal(final Exception exception) { ? ((EnrichedEvent) event).coreReportMessage() : event.toString(), event instanceof EnrichedEvent ? ((EnrichedEvent) event).getCommitterKey() : null, - event instanceof EnrichedEvent ? ((EnrichedEvent) event).getCommitId() : null, + event instanceof EnrichedEvent ? ((EnrichedEvent) event).getCommitIds() : null, exception); } finally { connector.addFailureEventToRetryQueue(event); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTsFileHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTsFileHandler.java index ccce6ba0aa2a..1c586ce04ec1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTsFileHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTsFileHandler.java @@ -285,7 +285,7 @@ protected boolean onCompleteInternal(final TPipeTransferResp response) { "Successfully transferred file {} (committer key={}, commit id={}, reference count={}).", tsFile, events.stream().map(EnrichedEvent::getCommitterKey).collect(Collectors.toList()), - events.stream().map(EnrichedEvent::getCommitId).collect(Collectors.toList()), + events.stream().map(EnrichedEvent::getCommitIds).collect(Collectors.toList()), referenceCount); } else { LOGGER.info( @@ -350,7 +350,7 @@ protected void onErrorInternal(final Exception exception) { "Failed to transfer TsFileInsertionEvent {} (committer key {}, commit id {}).", tsFile, events.stream().map(EnrichedEvent::getCommitterKey).collect(Collectors.toList()), - events.stream().map(EnrichedEvent::getCommitId).collect(Collectors.toList()), + events.stream().map(EnrichedEvent::getCommitIds).collect(Collectors.toList()), exception); } else { LOGGER.warn( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java index 5bca69bd5d84..b1069c7dfe16 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java @@ -21,7 +21,9 @@ import org.apache.iotdb.commons.consensus.index.ProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; +import org.apache.iotdb.commons.pipe.agent.task.meta.PipeTaskMeta; import org.apache.iotdb.commons.pipe.agent.task.progress.CommitterKey; +import org.apache.iotdb.commons.pipe.datastructure.pattern.PipePattern; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.assigner.PipeTsFileEpochProgressIndexKeeper; import org.apache.iotdb.db.storageengine.dataregion.memtable.TsFileProcessor; @@ -142,8 +144,22 @@ public boolean hasMultipleCommitIds() { @Override public long getCommitId() { - throw new UnsupportedOperationException( - "PipeCompactedTsFileInsertionEvent does not support getCommitId."); + // max of commitIds is used as the commit id for this event + return commitIds.stream() + .max(Long::compareTo) + .orElseThrow( + () -> + new IllegalStateException( + "No commit IDs found in PipeCompactedTsFileInsertionEvent.")); + } + + // return dummy events for each commit ID (except the max one) + @Override + public List getDummyEventsForCommitIds() { + return commitIds.stream() + .filter(commitId -> commitId != getCommitId()) + .map(PipeCompactedTsFileInsertionDummyEvent::new) + .collect(Collectors.toList()); } @Override @@ -166,4 +182,72 @@ public void eliminateProgressIndex() { } } } + + public class PipeCompactedTsFileInsertionDummyEvent extends EnrichedEvent { + + private final long commitId; + + public PipeCompactedTsFileInsertionDummyEvent(final long commitId) { + super( + PipeCompactedTsFileInsertionEvent.this.pipeName, + PipeCompactedTsFileInsertionEvent.this.creationTime, + PipeCompactedTsFileInsertionEvent.this.pipeTaskMeta, + null, // PipePattern is not needed for dummy event + Long.MIN_VALUE, + Long.MAX_VALUE); + this.commitId = commitId; // Use the commitId passed in + this.shouldReportOnCommit = false; // Dummy events do not report progress + } + + @Override + public long getCommitId() { + return commitId; + } + + @Override + public boolean internallyIncreaseResourceReferenceCount(String holderMessage) { + return true; + } + + @Override + public boolean internallyDecreaseResourceReferenceCount(String holderMessage) { + return true; + } + + @Override + public ProgressIndex getProgressIndex() { + return MinimumProgressIndex.INSTANCE; + } + + @Override + public EnrichedEvent shallowCopySelfAndBindPipeTaskMetaForProgressReport( + String pipeName, + long creationTime, + PipeTaskMeta pipeTaskMeta, + PipePattern pattern, + long startTime, + long endTime) { + return null; + } + + @Override + public boolean isGeneratedByPipe() { + return false; + } + + @Override + public boolean mayEventTimeOverlappedWithTimeRange() { + return false; + } + + @Override + public boolean mayEventPathsOverlappedWithPattern() { + return false; + } + + @Override + public String coreReportMessage() { + return "PipeCompactedTsFileInsertionDummyEvent"; + } + } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/progress/PipeEventCommitManager.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/progress/PipeEventCommitManager.java index b37bd07d1d9b..7056b052a3ee 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/progress/PipeEventCommitManager.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/progress/PipeEventCommitManager.java @@ -118,9 +118,29 @@ public void commit(final EnrichedEvent event, final CommitterKey committerKey) { } } } - if (committerKey == null || event.getCommitId() <= EnrichedEvent.NO_COMMIT_ID) { + if (committerKey == null) { return; } + if (event.hasMultipleCommitIds()) { + commitMultipleIds(committerKey, event); + } else { + commitSingleId(committerKey, event.getCommitId(), event); + } + } + + private void commitMultipleIds(final CommitterKey committerKey, final EnrichedEvent event) { + for (final long commitId : event.getCommitIds()) { + if (commitSingleId(committerKey, commitId, event)) { + return; + } + } + } + + private boolean commitSingleId( + final CommitterKey committerKey, final long commitId, final EnrichedEvent event) { + if (commitId <= EnrichedEvent.NO_COMMIT_ID) { + return false; + } final PipeEventCommitter committer = eventCommitterMap.get(committerKey); if (committer == null) { @@ -142,10 +162,11 @@ public void commit(final EnrichedEvent event, final CommitterKey committerKey) { Thread.currentThread().getStackTrace()); } } - return; + return false; } committer.commit(event); + return true; } private CommitterKey generateCommitterKey( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/progress/PipeEventCommitter.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/progress/PipeEventCommitter.java index 9479c7a752ac..0e5921262abe 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/progress/PipeEventCommitter.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/progress/PipeEventCommitter.java @@ -58,6 +58,11 @@ public synchronized long generateCommitId() { @SuppressWarnings("java:S899") public synchronized void commit(final EnrichedEvent event) { + if (event.hasMultipleCommitIds()) { + for (final EnrichedEvent dummyEvent : event.getDummyEventsForCommitIds()) { + commitQueue.offer(dummyEvent); + } + } commitQueue.offer(event); final int commitQueueSizeBeforeCommit = commitQueue.size(); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/EnrichedEvent.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/EnrichedEvent.java index 4ef36c29c7ef..374bc5446091 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/EnrichedEvent.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/EnrichedEvent.java @@ -404,6 +404,10 @@ public long getCommitId() { return commitId; } + public List getDummyEventsForCommitIds() { + return Collections.emptyList(); + } + public List getCommitIds() { return Collections.singletonList(commitId); } From 3bedc365c5e12d688d33b20403c970ccbb171792 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 2 Jul 2025 19:17:30 +0800 Subject: [PATCH 095/185] Seq send by default --- .../main/java/org/apache/iotdb/commons/conf/CommonConfig.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java index 5fedbddd494f..a3c8aa8842d4 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java @@ -204,7 +204,9 @@ public class CommonConfig { private boolean pipeFileReceiverFsyncEnabled = true; private int pipeRealTimeQueuePollTsFileThreshold = 10; - private int pipeRealTimeQueuePollHistoricalTsFileThreshold = 3; + + // Sequentially poll the tsFile by default + private int pipeRealTimeQueuePollHistoricalTsFileThreshold = 1; private int pipeRealTimeQueueMaxWaitingTsFileSize = 1; /** The maximum number of threads that can be used to execute subtasks in PipeSubtaskExecutor. */ From 766abb0e0a6b1514056fe8be776e006442ebed46 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 2 Jul 2025 20:28:00 +0800 Subject: [PATCH 096/185] Added the report frequency --- .../assigner/PipeDataRegionAssigner.java | 16 ---------------- .../iotdb/commons/conf/CommonConfig.java | 19 ------------------- .../iotdb/commons/pipe/config/PipeConfig.java | 10 ---------- .../commons/pipe/config/PipeDescriptor.java | 9 +-------- 4 files changed, 1 insertion(+), 53 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java index d5ddad181232..0ba9906fc7b7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java @@ -21,7 +21,6 @@ import org.apache.iotdb.commons.consensus.index.ProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; -import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.event.ProgressReportEvent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; @@ -30,7 +29,6 @@ import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEventFactory; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionExtractor; -import org.apache.iotdb.pipe.api.event.dml.insertion.TsFileInsertionEvent; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -44,13 +42,9 @@ public class PipeDataRegionAssigner implements Closeable { private static final Logger LOGGER = LoggerFactory.getLogger(PipeDataRegionAssigner.class); - private static final PipeConfig PIPE_CONFIG = PipeConfig.getInstance(); - private final String dataRegionId; protected final Set extractors = new CopyOnWriteArraySet<>(); - private int counter = 0; - private final AtomicReference maxProgressIndexForRealtimeEvent = new AtomicReference<>(MinimumProgressIndex.INSTANCE); @@ -70,16 +64,6 @@ public void assignToExtractor(final PipeRealtimeEvent event) { extractors.forEach( extractor -> { if (event.getEvent().isGeneratedByPipe() && !extractor.isForwardingPipeRequests()) { - // The frequency of progress reports is limited by the counter, while progress - // reports to TsFileInsertionEvent are not limited. - if (!(event.getEvent() instanceof TsFileInsertionEvent)) { - if (counter < PIPE_CONFIG.getPipeNonForwardingEventsProgressReportInterval()) { - counter++; - return; - } - counter = 0; - } - final ProgressReportEvent reportEvent = new ProgressReportEvent( extractor.getPipeName(), diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java index a3c8aa8842d4..53980d23e0b2 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java @@ -213,8 +213,6 @@ public class CommonConfig { private int pipeSubtaskExecutorMaxThreadNum = Math.max(5, Runtime.getRuntime().availableProcessors() / 2); - private int pipeNonForwardingEventsProgressReportInterval = 100; - private int pipeDataStructureTabletRowSize = 2048; private int pipeDataStructureTabletSizeInBytes = 2097152; private double pipeDataStructureTabletMemoryBlockAllocationRejectThreshold = 0.3; @@ -737,23 +735,6 @@ public boolean isTimestampPrecisionCheckEnabled() { return timestampPrecisionCheckEnabled; } - public int getPipeNonForwardingEventsProgressReportInterval() { - return pipeNonForwardingEventsProgressReportInterval; - } - - public void setPipeNonForwardingEventsProgressReportInterval( - int pipeNonForwardingEventsProgressReportInterval) { - if (this.pipeNonForwardingEventsProgressReportInterval - == pipeNonForwardingEventsProgressReportInterval) { - return; - } - this.pipeNonForwardingEventsProgressReportInterval = - pipeNonForwardingEventsProgressReportInterval; - logger.info( - "pipeNonForwardingEventsProgressReportInterval is set to {}.", - pipeNonForwardingEventsProgressReportInterval); - } - public String getPipeHardlinkBaseDirName() { return pipeHardlinkBaseDirName; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java index e6dc4542ad27..7c372adf983e 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java @@ -30,12 +30,6 @@ public class PipeConfig { private static final CommonConfig COMMON_CONFIG = CommonDescriptor.getInstance().getConfig(); - /////////////////////////////// Data Synchronization /////////////////////////////// - - public int getPipeNonForwardingEventsProgressReportInterval() { - return COMMON_CONFIG.getPipeNonForwardingEventsProgressReportInterval(); - } - /////////////////////////////// File /////////////////////////////// public String getPipeHardlinkBaseDirName() { @@ -440,10 +434,6 @@ public long getPipeEventReferenceEliminateIntervalSeconds() { private static final Logger LOGGER = LoggerFactory.getLogger(PipeConfig.class); public void printAllConfigs() { - LOGGER.info( - "PipeNonForwardingEventsProgressReportInterval: {}", - getPipeNonForwardingEventsProgressReportInterval()); - LOGGER.info("PipeHardlinkBaseDirName: {}", getPipeHardlinkBaseDirName()); LOGGER.info("PipeHardlinkTsFileDirName: {}", getPipeHardlinkTsFileDirName()); LOGGER.info("PipeProgressIndexPersistDirName: {}", getPipeProgressIndexPersistDirName()); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java index cbea25ff50cc..293878312c32 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java @@ -199,14 +199,7 @@ public static void loadPipeStaticConfig(CommonConfig config, TrimProperties prop .trim())); } - public static void loadPipeInternalConfig(CommonConfig config, TrimProperties properties) - throws IOException { - config.setPipeNonForwardingEventsProgressReportInterval( - Integer.parseInt( - properties.getProperty( - "pipe_non_forwarding_events_progress_report_interval", - Integer.toString(config.getPipeNonForwardingEventsProgressReportInterval())))); - + public static void loadPipeInternalConfig(CommonConfig config, TrimProperties properties) { config.setPipeFileReceiverFsyncEnabled( Boolean.parseBoolean( properties.getProperty( From aee6335a8e8f7319181054f35c6d615b22d6d514 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 2 Jul 2025 20:34:46 +0800 Subject: [PATCH 097/185] revert tsFile hardlink recover --- .../tsfile/PipeTsFileInsertionEvent.java | 29 +++------- ...peHistoricalDataRegionTsFileExtractor.java | 58 +++---------------- .../tsfile/PipeTsFileResourceManager.java | 42 +------------- 3 files changed, 19 insertions(+), 110 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java index e01281854a3a..d2c443e787e1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java @@ -48,8 +48,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.annotation.Nullable; - import java.io.File; import java.io.IOException; import java.util.Collections; @@ -67,7 +65,7 @@ public class PipeTsFileInsertionEvent extends EnrichedEvent private static final Logger LOGGER = LoggerFactory.getLogger(PipeTsFileInsertionEvent.class); - protected @Nullable final TsFileResource resource; + protected final TsFileResource resource; protected File tsFile; protected long extractTime = 0; @@ -91,13 +89,11 @@ public class PipeTsFileInsertionEvent extends EnrichedEvent public PipeTsFileInsertionEvent(final TsFileResource resource, final boolean isLoaded) { // The modFile must be copied before the event is assigned to the listening pipes - this( - resource, null, true, isLoaded, false, null, 0, null, null, Long.MIN_VALUE, Long.MAX_VALUE); + this(resource, true, isLoaded, false, null, 0, null, null, Long.MIN_VALUE, Long.MAX_VALUE); } public PipeTsFileInsertionEvent( - final @Nullable TsFileResource resource, - final @Nullable File tsFile, + final TsFileResource resource, final boolean isWithMod, final boolean isLoaded, final boolean isGeneratedByHistoricalExtractor, @@ -108,35 +104,25 @@ public PipeTsFileInsertionEvent( final long startTime, final long endTime) { super(pipeName, creationTime, pipeTaskMeta, pattern, startTime, endTime); - assert Objects.nonNull(tsFile) || Objects.nonNull(resource); this.resource = resource; // For events created at assigner or historical extractor, the tsFile is get from the resource // For events created for source, the tsFile is inherited from the assigner, because the // original tsFile may be gone, and we need to get the assigner's hard-linked tsFile to // hard-link it to each pipe dir - this.tsFile = Objects.nonNull(resource) ? resource.getTsFile() : tsFile; + this.tsFile = resource.getTsFile(); - final ModificationFile modFile = - Objects.nonNull(resource) - ? resource.getModFile() - : new ModificationFile(tsFile.getAbsolutePath() + ModificationFile.FILE_SUFFIX); + final ModificationFile modFile = resource.getModFile(); this.isWithMod = isWithMod && modFile.exists(); this.modFile = this.isWithMod ? new File(modFile.getFilePath()) : null; this.isLoaded = isLoaded; - this.isGeneratedByPipe = Objects.nonNull(resource) && resource.isGeneratedByPipe(); - this.isGeneratedByPipeConsensus = - Objects.nonNull(resource) && resource.isGeneratedByPipeConsensus(); + this.isGeneratedByPipe = resource.isGeneratedByPipe(); + this.isGeneratedByPipeConsensus = resource.isGeneratedByPipeConsensus(); this.isGeneratedByHistoricalExtractor = isGeneratedByHistoricalExtractor; this.dataContainer = new AtomicReference<>(null); - if (Objects.isNull(resource)) { - isClosed = new AtomicBoolean(true); - return; - } - isClosed = new AtomicBoolean(resource.isClosed()); // Register close listener if TsFile is not closed if (!isClosed.get()) { @@ -352,7 +338,6 @@ public PipeTsFileInsertionEvent shallowCopySelfAndBindPipeTaskMetaForProgressRep final long endTime) { return new PipeTsFileInsertionEvent( resource, - tsFile, isWithMod, isLoaded, isGeneratedByHistoricalExtractor, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java index 909290cda067..8c8a3a314120 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java @@ -53,7 +53,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.File; import java.io.IOException; import java.util.ArrayDeque; import java.util.ArrayList; @@ -121,7 +120,6 @@ public class PipeHistoricalDataRegionTsFileExtractor implements PipeHistoricalDa private volatile boolean hasBeenStarted = false; private Queue pendingQueue; - private Queue tsFileQueue; @Override public void validate(final PipeParameterValidator validator) { @@ -325,12 +323,6 @@ public synchronized void start() { } hasBeenStarted = true; - // Recover - tsFileQueue = PipeDataNodeResourceManager.tsfile().recoverTsFile(pipeName); - if (Objects.nonNull(tsFileQueue)) { - return; - } - final DataRegion dataRegion = StorageEngine.getInstance().getDataRegion(new DataRegionId(dataRegionId)); if (Objects.isNull(dataRegion)) { @@ -532,21 +524,13 @@ public synchronized Event supply() { start(); } - if (Objects.isNull(pendingQueue) && Objects.isNull(tsFileQueue)) { + if (Objects.isNull(pendingQueue)) { return null; } - TsFileResource resource = null; - if (Objects.nonNull(pendingQueue)) { - resource = pendingQueue.poll(); - } + final TsFileResource resource = pendingQueue.poll(); - File tsFile = null; - if (Objects.nonNull(tsFileQueue)) { - tsFile = tsFileQueue.poll(); - } - - if (resource == null && tsFile == null) { + if (resource == null) { final PipeTerminateEvent terminateEvent = new PipeTerminateEvent(pipeName, creationTime, pipeTaskMeta, dataRegionId); if (!terminateEvent.increaseReferenceCount( @@ -564,7 +548,6 @@ public synchronized Event supply() { final PipeTsFileInsertionEvent event = new PipeTsFileInsertionEvent( resource, - tsFile, shouldTransferModFile, false, true, @@ -578,8 +561,7 @@ public synchronized Event supply() { event.skipParsingPattern(); } - if (sloppyTimeRange - || Objects.nonNull(resource) && isTsFileResourceCoveredByTimeRange(resource)) { + if (sloppyTimeRange || isTsFileResourceCoveredByTimeRange(resource)) { event.skipParsingTime(); } @@ -596,14 +578,13 @@ public synchronized Event supply() { return isReferenceCountIncreased ? event : null; } finally { try { - PipeDataNodeResourceManager.tsfile() - .unpinTsFile(Objects.nonNull(resource) ? resource.getTsFile() : tsFile, pipeName); + PipeDataNodeResourceManager.tsfile().unpinTsFileResource(resource, pipeName); } catch (final IOException e) { LOGGER.warn( "Pipe {}@{}: failed to unpin TsFileResource after creating event, original path: {}", pipeName, dataRegionId, - Objects.nonNull(resource) ? resource.getTsFilePath() : tsFile.getPath()); + resource.getTsFilePath()); } } } @@ -612,17 +593,13 @@ public synchronized Event supply() { public synchronized boolean hasConsumedAll() { // If the pendingQueue is null when the function is called, it implies that the extractor only // extracts deletion thus the historical event has nothing to consume. - final Queue dataQueue = Objects.nonNull(pendingQueue) ? pendingQueue : tsFileQueue; return hasBeenStarted - && (Objects.isNull(dataQueue) || dataQueue.isEmpty() && isTerminateSignalSent); + && (Objects.isNull(pendingQueue) || pendingQueue.isEmpty() && isTerminateSignalSent); } @Override public int getPendingQueueSize() { - if (Objects.nonNull(pendingQueue)) { - return pendingQueue.size(); - } - return Objects.nonNull(tsFileQueue) ? tsFileQueue.size() : 0; + return Objects.nonNull(pendingQueue) ? pendingQueue.size() : 0; } @Override @@ -631,7 +608,7 @@ public synchronized void close() { pendingQueue.forEach( resource -> { try { - PipeDataNodeResourceManager.tsfile().unpinTsFile(resource.getTsFile(), pipeName); + PipeDataNodeResourceManager.tsfile().unpinTsFileResource(resource, pipeName); } catch (final IOException e) { LOGGER.warn( "Pipe {}@{}: failed to unpin TsFileResource after dropping pipe, original path: {}", @@ -643,22 +620,5 @@ public synchronized void close() { pendingQueue.clear(); pendingQueue = null; } - - if (Objects.nonNull(tsFileQueue)) { - tsFileQueue.forEach( - tsFile -> { - try { - PipeDataNodeResourceManager.tsfile().unpinTsFile(tsFile, pipeName); - } catch (final IOException e) { - LOGGER.warn( - "Pipe {}@{}: failed to unpin TsFile after dropping pipe, original path: {}", - pipeName, - dataRegionId, - tsFile.getPath()); - } - }); - tsFileQueue.clear(); - tsFileQueue = null; - } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java index 334b7829b74f..37b3014a499b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java @@ -20,16 +20,12 @@ package org.apache.iotdb.db.pipe.resource.tsfile; import org.apache.iotdb.commons.conf.IoTDBConstant; -import org.apache.iotdb.commons.file.SystemFileFactory; import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.utils.FileUtils; import org.apache.iotdb.commons.utils.TestOnly; -import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.storageengine.dataregion.modification.ModificationFile; import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; -import org.apache.iotdb.pipe.api.exception.PipeException; -import org.apache.tsfile.common.constant.TsFileConstant; import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.file.metadata.IDeviceID; import org.slf4j.Logger; @@ -40,11 +36,9 @@ import java.io.File; import java.io.IOException; -import java.util.ArrayDeque; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Queue; import java.util.concurrent.ConcurrentHashMap; public class PipeTsFileResourceManager { @@ -349,8 +343,9 @@ public void pinTsFileResource( } } - public void unpinTsFile(final File tsFile, final String pipeName) throws IOException { - final File pinnedFile = getHardlinkOrCopiedFileInPipeDir(tsFile, pipeName); + public void unpinTsFileResource(final TsFileResource resource, final String pipeName) + throws IOException { + final File pinnedFile = getHardlinkOrCopiedFileInPipeDir(resource.getTsFile(), pipeName); decreaseFileReference(pinnedFile, pipeName); final File modFile = new File(pinnedFile + ModificationFile.FILE_SUFFIX); @@ -359,37 +354,6 @@ public void unpinTsFile(final File tsFile, final String pipeName) throws IOExcep } } - public Queue recoverTsFile(final @Nonnull String pipeName) { - final Queue result = new ArrayDeque<>(); - final String suffix = - File.separator - + PipeConfig.getInstance().getPipeHardlinkBaseDirName() - + File.separator - + PipeConfig.getInstance().getPipeHardlinkTsFileDirName() - + File.separator - + pipeName; - for (final String dataDir : IoTDBDescriptor.getInstance().getConfig().getDataDirs()) { - final File pipeDir = SystemFileFactory.INSTANCE.getFile(dataDir + suffix); - if (pipeDir.exists() && pipeDir.isDirectory()) { - final File[] files = pipeDir.listFiles(); - for (final File file : files) { - try { - final boolean isTsFile = file.getName().endsWith(TsFileConstant.TSFILE_SUFFIX); - increaseFileReference(file, isTsFile, pipeName); - if (isTsFile) { - result.add(file); - } - } catch (final IOException e) { - throw new PipeException(e.getMessage()); - } - } - } else { - return null; - } - } - return result; - } - public int getLinkedTsFileCount(final @Nonnull String pipeName) { return hardlinkOrCopiedFileToPipeTsFileResourceMap .computeIfAbsent(pipeName, pipe -> new ConcurrentHashMap<>()) From 93baf53b1d52fc643fa02da4cb5474416f861079 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 2 Jul 2025 21:28:03 +0800 Subject: [PATCH 098/185] introduce segment index --- .../PipeRealtimeDataRegionExtractor.java | 6 + .../consensus/index/ProgressIndex.java | 2 +- .../consensus/index/ProgressIndexType.java | 8 +- .../index/impl/IoTProgressIndex.java | 9 - .../index/impl/SegmentProgressIndex.java | 189 ++++++++++++++++++ 5 files changed, 203 insertions(+), 11 deletions(-) create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SegmentProgressIndex.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java index 724d9876348e..ad8b6c76846c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java @@ -20,6 +20,8 @@ package org.apache.iotdb.db.pipe.extractor.dataregion.realtime; import org.apache.iotdb.commons.consensus.DataRegionId; +import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; +import org.apache.iotdb.commons.consensus.index.impl.SegmentProgressIndex; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeNonCriticalException; import org.apache.iotdb.commons.pipe.agent.task.connection.UnboundedBlockingPendingQueue; import org.apache.iotdb.commons.pipe.agent.task.meta.PipeTaskMeta; @@ -201,6 +203,10 @@ public void customize( dataRegionId = String.valueOf(environment.getRegionId()); pipeTaskMeta = environment.getPipeTaskMeta(); + if (pipeTaskMeta.getProgressIndex() instanceof MinimumProgressIndex) { + pipeTaskMeta.updateProgressIndex(new SegmentProgressIndex()); + } + // Metrics related to TsFileEpoch are managed in PipeExtractorMetrics. These metrics are // indexed by the taskID of IoTDBDataRegionExtractor. To avoid PipeRealtimeDataRegionExtractor // holding a reference to IoTDBDataRegionExtractor, the taskID should be constructed to diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/ProgressIndex.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/ProgressIndex.java index b54d6db4dab4..3c8d13bab54d 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/ProgressIndex.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/ProgressIndex.java @@ -215,7 +215,7 @@ protected static ProgressIndex blendProgressIndex( *

Notice:TotalOrderSumTuple is an ordered tuple, the larger the subscript the higher the * weight of the element when comparing sizes, e.g. (1, 2) is larger than (2, 1). */ - protected static class TotalOrderSumTuple implements Comparable { + public static class TotalOrderSumTuple implements Comparable { private final ImmutableList tuple; /** diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/ProgressIndexType.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/ProgressIndexType.java index 58548e18c4b6..d4b0f4ff18ba 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/ProgressIndexType.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/ProgressIndexType.java @@ -24,6 +24,7 @@ import org.apache.iotdb.commons.consensus.index.impl.MetaProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.RecoverProgressIndex; +import org.apache.iotdb.commons.consensus.index.impl.SegmentProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.SimpleProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.StateProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.TimeWindowStateProgressIndex; @@ -43,7 +44,8 @@ public enum ProgressIndexType { HYBRID_PROGRESS_INDEX((short) 5), META_PROGRESS_INDEX((short) 6), TIME_WINDOW_STATE_PROGRESS_INDEX((short) 7), - STATE_PROGRESS_INDEX((short) 8); + STATE_PROGRESS_INDEX((short) 8), + SEGMENT_PROGRESS_INDEX((short) 9); private final short type; @@ -82,6 +84,8 @@ public static ProgressIndex deserializeFrom(ByteBuffer byteBuffer) { return TimeWindowStateProgressIndex.deserializeFrom(byteBuffer); case 8: return StateProgressIndex.deserializeFrom(byteBuffer); + case 9: + return SegmentProgressIndex.deserializeFrom(byteBuffer); default: throw new UnsupportedOperationException( String.format("Unsupported progress index type %s.", indexType)); @@ -107,6 +111,8 @@ public static ProgressIndex deserializeFrom(InputStream stream) throws IOExcepti return TimeWindowStateProgressIndex.deserializeFrom(stream); case 8: return StateProgressIndex.deserializeFrom(stream); + case 9: + return SegmentProgressIndex.deserializeFrom(stream); default: throw new UnsupportedOperationException( String.format("Unsupported progress index type %s.", indexType)); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/IoTProgressIndex.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/IoTProgressIndex.java index 8b02d85da5ae..a1275452737b 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/IoTProgressIndex.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/IoTProgressIndex.java @@ -204,15 +204,6 @@ public TotalOrderSumTuple getTotalOrderSumTuple() { } } - public int getPeerId2SearchIndexSize() { - lock.readLock().lock(); - try { - return peerId2SearchIndex.size(); - } finally { - lock.readLock().unlock(); - } - } - public static IoTProgressIndex deserializeFrom(ByteBuffer byteBuffer) { final IoTProgressIndex ioTProgressIndex = new IoTProgressIndex(); final int size = ReadWriteIOUtils.readInt(byteBuffer); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SegmentProgressIndex.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SegmentProgressIndex.java new file mode 100644 index 000000000000..b912f1289bd7 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SegmentProgressIndex.java @@ -0,0 +1,189 @@ +/* + * 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.consensus.index.impl; + +import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.commons.consensus.index.ProgressIndexType; + +import org.apache.tsfile.utils.Pair; +import org.apache.tsfile.utils.RamUsageEstimator; +import org.apache.tsfile.utils.ReadWriteIOUtils; + +import javax.annotation.Nonnull; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Objects; + +/** + * {@link SegmentProgressIndex} is a usual {@link ProgressIndex} with broken segments allowed. An + * {@link org.apache.iotdb.pipe.api.event.Event} is sent if its {@link ProgressIndex} <= the {@link + * #latestProgressIndex} and none of the {@link #brokenProgressIndexes}es has its {@link Pair#left} + * <= its {@link ProgressIndex} <= {@link Pair#right}. If the {@link #brokenProgressIndexes} {@link + * List#isEmpty()}, the progress Index behave just like the {@link #latestProgressIndex}. It is only + * used in the realtime data region extractor to handle downgrading. + */ +public class SegmentProgressIndex extends ProgressIndex { + private static final long INSTANCE_SIZE = + RamUsageEstimator.shallowSizeOfInstance(SegmentProgressIndex.class); + public static final long LIST_SIZE = RamUsageEstimator.shallowSizeOfInstance(ArrayList.class); + public static final long PAIR_SIZE = RamUsageEstimator.shallowSizeOfInstance(Pair.class); + private ProgressIndex latestProgressIndex = MinimumProgressIndex.INSTANCE; + + // of the downgraded segments + private final List> brokenProgressIndexes = new LinkedList<>(); + + public void recordStart(final ProgressIndex index) { + brokenProgressIndexes.add(new Pair<>(index, null)); + } + + public void recordEnd(final ProgressIndex index) { + brokenProgressIndexes.get(brokenProgressIndexes.size() - 1).setRight(index); + } + + public void eliminate(final ProgressIndex index) { + final Iterator> iterator = brokenProgressIndexes.iterator(); + while (iterator.hasNext()) { + if (index.equals(iterator.next().getRight())) { + iterator.remove(); + return; + } + } + } + + @Override + public void serialize(final ByteBuffer byteBuffer) { + ProgressIndexType.SEGMENT_PROGRESS_INDEX.serialize(byteBuffer); + + latestProgressIndex.serialize(byteBuffer); + ReadWriteIOUtils.write(brokenProgressIndexes.size(), byteBuffer); + for (final Pair index : brokenProgressIndexes) { + index.getLeft().serialize(byteBuffer); + index.getRight().serialize(byteBuffer); + } + } + + @Override + public void serialize(final OutputStream stream) throws IOException { + ProgressIndexType.SEGMENT_PROGRESS_INDEX.serialize(stream); + + latestProgressIndex.serialize(stream); + ReadWriteIOUtils.write(brokenProgressIndexes.size(), stream); + for (final Pair index : brokenProgressIndexes) { + index.getLeft().serialize(stream); + index.getRight().serialize(stream); + } + } + + @Override + public boolean isAfter(final @Nonnull ProgressIndex progressIndex) { + return latestProgressIndex.isAfter(progressIndex) + && brokenProgressIndexes.stream() + .noneMatch( + pair -> + pair.getRight().isAfter(progressIndex) + && progressIndex.isAfter(pair.getLeft())); + } + + @Override + public boolean equals(final ProgressIndex progressIndex) { + if (progressIndex == null) { + return false; + } + if (this == progressIndex) { + return true; + } + if (progressIndex instanceof SegmentProgressIndex) { + final SegmentProgressIndex that = (SegmentProgressIndex) progressIndex; + return this.latestProgressIndex.equals(that.latestProgressIndex) + && this.brokenProgressIndexes.equals(that.brokenProgressIndexes); + } + return this.latestProgressIndex.equals(progressIndex); + } + + @Override + public ProgressIndex updateToMinimumEqualOrIsAfterProgressIndex( + final ProgressIndex progressIndex) { + return latestProgressIndex.updateToMinimumEqualOrIsAfterProgressIndex(progressIndex); + } + + @Override + public ProgressIndexType getType() { + return ProgressIndexType.SEGMENT_PROGRESS_INDEX; + } + + @Override + public TotalOrderSumTuple getTotalOrderSumTuple() { + throw new UnsupportedOperationException( + "This progressIndex is not for tsFile and shall never be used to sort resources"); + } + + @Override + public long ramBytesUsed() { + return INSTANCE_SIZE + + latestProgressIndex.ramBytesUsed() + + shallowSizeOfList(brokenProgressIndexes) + + PAIR_SIZE * brokenProgressIndexes.size() + + brokenProgressIndexes.stream() + .mapToLong(index -> index.getLeft().ramBytesUsed() + index.getRight().ramBytesUsed()) + .reduce(0L, Long::sum); + } + + public static SegmentProgressIndex deserializeFrom(final ByteBuffer byteBuffer) { + final SegmentProgressIndex segmentProgressIndex = new SegmentProgressIndex(); + segmentProgressIndex.latestProgressIndex = ProgressIndexType.deserializeFrom(byteBuffer); + final int size = ReadWriteIOUtils.readInt(byteBuffer); + for (int i = 0; i < size; i++) { + segmentProgressIndex.brokenProgressIndexes.add( + new Pair<>( + ProgressIndexType.deserializeFrom(byteBuffer), + ProgressIndexType.deserializeFrom(byteBuffer))); + } + return segmentProgressIndex; + } + + public static SegmentProgressIndex deserializeFrom(final InputStream stream) throws IOException { + final SegmentProgressIndex segmentProgressIndex = new SegmentProgressIndex(); + segmentProgressIndex.latestProgressIndex = ProgressIndexType.deserializeFrom(stream); + final int size = ReadWriteIOUtils.readInt(stream); + for (int i = 0; i < size; i++) { + segmentProgressIndex.brokenProgressIndexes.add( + new Pair<>( + ProgressIndexType.deserializeFrom(stream), + ProgressIndexType.deserializeFrom(stream))); + } + return segmentProgressIndex; + } + + private long shallowSizeOfList(final List list) { + return Objects.nonNull(list) + ? SegmentProgressIndex.LIST_SIZE + + RamUsageEstimator.alignObjectSize( + RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + + (long) RamUsageEstimator.NUM_BYTES_OBJECT_REF * list.size()) + : 0L; + } +} From 16e2019b3267988c6a851ec54364e6940f40adf6 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 2 Jul 2025 21:30:34 +0800 Subject: [PATCH 099/185] Update SegmentProgressIndex.java --- .../commons/consensus/index/impl/SegmentProgressIndex.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SegmentProgressIndex.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SegmentProgressIndex.java index b912f1289bd7..14ab7c6ad74d 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SegmentProgressIndex.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SegmentProgressIndex.java @@ -54,14 +54,15 @@ public class SegmentProgressIndex extends ProgressIndex { private ProgressIndex latestProgressIndex = MinimumProgressIndex.INSTANCE; // of the downgraded segments - private final List> brokenProgressIndexes = new LinkedList<>(); + private final LinkedList> brokenProgressIndexes = + new LinkedList<>(); public void recordStart(final ProgressIndex index) { brokenProgressIndexes.add(new Pair<>(index, null)); } public void recordEnd(final ProgressIndex index) { - brokenProgressIndexes.get(brokenProgressIndexes.size() - 1).setRight(index); + brokenProgressIndexes.getLast().setRight(index); } public void eliminate(final ProgressIndex index) { From 88faaa2d5e8298dc416edab619d2d74688990d18 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 2 Jul 2025 21:33:32 +0800 Subject: [PATCH 100/185] polishing --- .../commons/consensus/index/impl/SegmentProgressIndex.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SegmentProgressIndex.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SegmentProgressIndex.java index 14ab7c6ad74d..2930d888cfce 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SegmentProgressIndex.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SegmentProgressIndex.java @@ -44,7 +44,9 @@ * #latestProgressIndex} and none of the {@link #brokenProgressIndexes}es has its {@link Pair#left} * <= its {@link ProgressIndex} <= {@link Pair#right}. If the {@link #brokenProgressIndexes} {@link * List#isEmpty()}, the progress Index behave just like the {@link #latestProgressIndex}. It is only - * used in the realtime data region extractor to handle downgrading. + * used in the realtime data region extractor's {@link + * org.apache.iotdb.commons.pipe.agent.task.meta.PipeTaskMeta} to handle downgrading, and will never + * be in the insertNodes or tsFiles. */ public class SegmentProgressIndex extends ProgressIndex { private static final long INSTANCE_SIZE = @@ -106,7 +108,8 @@ public boolean isAfter(final @Nonnull ProgressIndex progressIndex) { .noneMatch( pair -> pair.getRight().isAfter(progressIndex) - && progressIndex.isAfter(pair.getLeft())); + && (progressIndex.isAfter(pair.getLeft()) + || progressIndex.equals(pair.getLeft()))); } @Override From 6856dcfaabf283bf5e86761b5cd6c3bc11238550 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 2 Jul 2025 21:34:33 +0800 Subject: [PATCH 101/185] Update SegmentProgressIndex.java --- .../commons/consensus/index/impl/SegmentProgressIndex.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SegmentProgressIndex.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SegmentProgressIndex.java index 2930d888cfce..5309e5439a85 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SegmentProgressIndex.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SegmentProgressIndex.java @@ -42,9 +42,9 @@ * {@link SegmentProgressIndex} is a usual {@link ProgressIndex} with broken segments allowed. An * {@link org.apache.iotdb.pipe.api.event.Event} is sent if its {@link ProgressIndex} <= the {@link * #latestProgressIndex} and none of the {@link #brokenProgressIndexes}es has its {@link Pair#left} - * <= its {@link ProgressIndex} <= {@link Pair#right}. If the {@link #brokenProgressIndexes} {@link - * List#isEmpty()}, the progress Index behave just like the {@link #latestProgressIndex}. It is only - * used in the realtime data region extractor's {@link + * <= its {@link ProgressIndex} < {@link Pair#right}. If the {@link #brokenProgressIndexes} {@link + * List#isEmpty()}, the {@link ProgressIndex} behave just like the {@link #latestProgressIndex}. It + * is only used in the realtime data region extractor's {@link * org.apache.iotdb.commons.pipe.agent.task.meta.PipeTaskMeta} to handle downgrading, and will never * be in the insertNodes or tsFiles. */ From 2aafa46a9fb9a3841abba83f8c0eaa5f18bc42a1 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 3 Jul 2025 10:00:53 +0800 Subject: [PATCH 102/185] continue reverting --- .../event/common/tsfile/PipeCompactedTsFileInsertionEvent.java | 1 - 1 file changed, 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java index b1069c7dfe16..0f034dbdb96d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java @@ -49,7 +49,6 @@ public PipeCompactedTsFileInsertionEvent( final boolean shouldReportProgress) { super( tsFileResource, - tsFileResource.getTsFile(), bindIsWithMod(originalEvents), bindIsLoaded(originalEvents), bindIsGeneratedByHistoricalExtractor(originalEvents), From 7b42852ee23930b85c1e1d94d7fe77180bf5bb87 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 3 Jul 2025 10:35:48 +0800 Subject: [PATCH 103/185] test fix --- .../extractor/PipeRealtimeExtractTest.java | 30 ++++++++++++++++--- .../index/impl/IoTProgressIndex.java | 2 +- 2 files changed, 27 insertions(+), 5 deletions(-) diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipeRealtimeExtractTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipeRealtimeExtractTest.java index 37c956783721..8eb725f18aa5 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipeRealtimeExtractTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipeRealtimeExtractTest.java @@ -20,7 +20,9 @@ package org.apache.iotdb.db.pipe.extractor; import org.apache.iotdb.commons.conf.IoTDBConstant; +import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.pipe.agent.task.meta.PipeTaskMeta; import org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant; import org.apache.iotdb.commons.pipe.config.plugin.configuraion.PipeTaskRuntimeConfiguration; import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskExtractorRuntimeEnvironment; @@ -144,16 +146,36 @@ public void testRealtimeExtractProcess() { final PipeTaskRuntimeConfiguration configuration0 = new PipeTaskRuntimeConfiguration( - new PipeTaskExtractorRuntimeEnvironment("1", 1, Integer.parseInt(dataRegion1), null)); + new PipeTaskExtractorRuntimeEnvironment( + "1", + 1, + Integer.parseInt(dataRegion1), + new PipeTaskMeta( + MinimumProgressIndex.INSTANCE, 1, Integer.parseInt(dataRegion1), false))); final PipeTaskRuntimeConfiguration configuration1 = new PipeTaskRuntimeConfiguration( - new PipeTaskExtractorRuntimeEnvironment("1", 1, Integer.parseInt(dataRegion1), null)); + new PipeTaskExtractorRuntimeEnvironment( + "1", + 1, + Integer.parseInt(dataRegion1), + new PipeTaskMeta( + MinimumProgressIndex.INSTANCE, 1, Integer.parseInt(dataRegion1), false))); final PipeTaskRuntimeConfiguration configuration2 = new PipeTaskRuntimeConfiguration( - new PipeTaskExtractorRuntimeEnvironment("1", 1, Integer.parseInt(dataRegion2), null)); + new PipeTaskExtractorRuntimeEnvironment( + "1", + 1, + Integer.parseInt(dataRegion2), + new PipeTaskMeta( + MinimumProgressIndex.INSTANCE, 1, Integer.parseInt(dataRegion2), false))); final PipeTaskRuntimeConfiguration configuration3 = new PipeTaskRuntimeConfiguration( - new PipeTaskExtractorRuntimeEnvironment("1", 1, Integer.parseInt(dataRegion2), null)); + new PipeTaskExtractorRuntimeEnvironment( + "1", + 1, + Integer.parseInt(dataRegion2), + new PipeTaskMeta( + MinimumProgressIndex.INSTANCE, 1, Integer.parseInt(dataRegion2), false))); // Some parameters of extractor are validated and initialized during the validation process. extractor0.validate(new PipeParameterValidator(parameters0)); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/IoTProgressIndex.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/IoTProgressIndex.java index a1275452737b..a07f750c42ba 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/IoTProgressIndex.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/IoTProgressIndex.java @@ -116,7 +116,7 @@ public boolean isAfter(@Nonnull ProgressIndex progressIndex) { entry -> !thisIoTProgressIndex.peerId2SearchIndex.containsKey(entry.getKey()) || thisIoTProgressIndex.peerId2SearchIndex.get(entry.getKey()) - <= entry.getValue()); + < entry.getValue()); } finally { lock.readLock().unlock(); } From 68fd8e7589bf7d25cb88e91befaf7feef2e1cca7 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 3 Jul 2025 10:47:34 +0800 Subject: [PATCH 104/185] Optimize IoTProgressIndex --- .../consensus/index/impl/IoTProgressIndex.java | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/IoTProgressIndex.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/IoTProgressIndex.java index a07f750c42ba..8f6a24845aa5 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/IoTProgressIndex.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/IoTProgressIndex.java @@ -109,14 +109,18 @@ public boolean isAfter(@Nonnull ProgressIndex progressIndex) { return false; } - final IoTProgressIndex thisIoTProgressIndex = this; final IoTProgressIndex thatIoTProgressIndex = (IoTProgressIndex) progressIndex; - return thatIoTProgressIndex.peerId2SearchIndex.entrySet().stream() - .noneMatch( - entry -> - !thisIoTProgressIndex.peerId2SearchIndex.containsKey(entry.getKey()) - || thisIoTProgressIndex.peerId2SearchIndex.get(entry.getKey()) - < entry.getValue()); + boolean isEquals = true; + for (final Map.Entry entry : + thatIoTProgressIndex.peerId2SearchIndex.entrySet()) { + if (!peerId2SearchIndex.containsKey(entry.getKey()) + || peerId2SearchIndex.get(entry.getKey()) < entry.getValue()) { + return false; + } else if (peerId2SearchIndex.get(entry.getKey()) > entry.getValue()) { + isEquals = false; + } + } + return !isEquals; } finally { lock.readLock().unlock(); } From 8d6abd1306a646ae26855eae98eac7acd202462e Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 3 Jul 2025 14:35:15 +0800 Subject: [PATCH 105/185] Fix NPE --- .../overview/PipeDataNodeSinglePipeMetrics.java | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeSinglePipeMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeSinglePipeMetrics.java index 534958e90112..255ec5876202 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeSinglePipeMetrics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeSinglePipeMetrics.java @@ -42,6 +42,7 @@ import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; public class PipeDataNodeSinglePipeMetrics implements IMetricSet { @@ -228,31 +229,35 @@ private void removeAutoGauge(final String pipeID) { public void register(final IoTDBDataRegionExtractor extractor) { // The metric is global thus the regionId is omitted final String pipeID = extractor.getPipeName() + "_" + extractor.getCreationTime(); + final AtomicBoolean first = new AtomicBoolean(false); remainingEventAndTimeOperatorMap.computeIfAbsent( pipeID, k -> { - if (Objects.nonNull(metricService)) { - createMetrics(pipeID); - } + first.set(true); return new PipeDataNodeRemainingEventAndTimeOperator( extractor.getPipeName(), extractor.getCreationTime()); }); + if (Objects.nonNull(metricService) && first.get()) { + createMetrics(pipeID); + } } public void register(final IoTDBSchemaRegionExtractor extractor) { // The metric is global thus the regionId is omitted final String pipeID = extractor.getPipeName() + "_" + extractor.getCreationTime(); + final AtomicBoolean first = new AtomicBoolean(false); remainingEventAndTimeOperatorMap .computeIfAbsent( pipeID, k -> { - if (Objects.nonNull(metricService)) { - createMetrics(pipeID); - } + first.set(true); return new PipeDataNodeRemainingEventAndTimeOperator( extractor.getPipeName(), extractor.getCreationTime()); }) .register(extractor); + if (Objects.nonNull(metricService) && first.get()) { + createMetrics(pipeID); + } } public void increaseInsertNodeEventCount(final String pipeName, final long creationTime) { From 739daf389140c1d2c02bb686d3920b2b4111c3be Mon Sep 17 00:00:00 2001 From: Steve Yurong Su Date: Thu, 3 Jul 2025 15:33:27 +0800 Subject: [PATCH 106/185] add logs --- .../connector/PipeRealtimePriorityBlockingQueue.java | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java index 5dc0c4b6ddcd..c66b6e86feea 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java @@ -314,6 +314,18 @@ public synchronized void replace( eventsToRemove.addAll(pipeTsFileInsertionEvents); } tsfileInsertEventDeque.removeIf(eventsToRemove::contains); + + LOGGER.info( + "Region {}: Replaced TsFileInsertionEvents {} with {}", + regionId, + eventsToBeRemovedGroupByCommitterKey.values().stream() + .flatMap(Set::stream) + .map(PipeTsFileInsertionEvent::coreReportMessage) + .collect(Collectors.joining(", ")), + eventsToBeAddedGroupByCommitterKey.values().stream() + .flatMap(Set::stream) + .map(PipeTsFileInsertionEvent::coreReportMessage) + .collect(Collectors.joining(", "))); } @Override From 39021f178799f687b743add69ab30cc31fc2feec Mon Sep 17 00:00:00 2001 From: Steve Yurong Su Date: Thu, 3 Jul 2025 15:35:25 +0800 Subject: [PATCH 107/185] Add log for missing TsFileInsertionEvents replacement Introduced an info-level log message when no TsFileInsertionEvents are found to replace for the given source files, aiding in debugging and monitoring. --- .../connector/PipeRealtimePriorityBlockingQueue.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java index c66b6e86feea..719f7ad4ea21 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java @@ -222,6 +222,12 @@ public synchronized void replace( .filter(entry -> entry.getValue().size() == sourceFiles.size()) .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); if (eventsToBeRemovedGroupByCommitterKey.isEmpty()) { + LOGGER.info( + "Region {}: No TsFileInsertionEvents to replace for source files {}", + regionId, + sourceFiles.stream() + .map(TsFileResource::getTsFilePath) + .collect(Collectors.joining(", "))); return; } From 334136fe1c65d0d9986c95d291e05a6c78ac676f Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 3 Jul 2025 17:45:42 +0800 Subject: [PATCH 108/185] NPE fix & heartbeat resp 2 common --- .../handlers/rpc/DataNodeAsyncRequestRPCHandler.java | 2 +- .../async/handlers/rpc/PipeHeartbeatRPCHandler.java | 2 +- .../manager/pipe/agent/task/PipeConfigNodeTaskAgent.java | 2 +- .../runtime/heartbeat/PipeHeartbeatScheduler.java | 2 +- .../iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java | 2 +- .../realtime/matcher/CachedSchemaPatternMatcher.java | 4 +++- .../thrift/impl/DataNodeInternalRPCServiceImpl.java | 2 +- .../iotdb/commons/pipe/agent/task/PipeTaskAgent.java | 2 +- .../thrift-commons/src/main/thrift/common.thrift | 7 +++++++ .../thrift-confignode/src/main/thrift/confignode.thrift | 3 +++ .../thrift-datanode/src/main/thrift/datanode.thrift | 9 +-------- 11 files changed, 21 insertions(+), 16 deletions(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/DataNodeAsyncRequestRPCHandler.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/DataNodeAsyncRequestRPCHandler.java index 6e2a9dc97ccd..69021eaec1ae 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/DataNodeAsyncRequestRPCHandler.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/DataNodeAsyncRequestRPCHandler.java @@ -20,6 +20,7 @@ package org.apache.iotdb.confignode.client.async.handlers.rpc; import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; +import org.apache.iotdb.common.rpc.thrift.TPipeHeartbeatResp; import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.common.rpc.thrift.TTestConnectionResp; import org.apache.iotdb.commons.client.request.AsyncRequestContext; @@ -32,7 +33,6 @@ import org.apache.iotdb.mpp.rpc.thrift.TCheckTimeSeriesExistenceResp; import org.apache.iotdb.mpp.rpc.thrift.TCountPathsUsingTemplateResp; import org.apache.iotdb.mpp.rpc.thrift.TFetchSchemaBlackListResp; -import org.apache.iotdb.mpp.rpc.thrift.TPipeHeartbeatResp; import org.apache.iotdb.mpp.rpc.thrift.TPushConsumerGroupMetaResp; import org.apache.iotdb.mpp.rpc.thrift.TPushPipeMetaResp; import org.apache.iotdb.mpp.rpc.thrift.TPushTopicMetaResp; diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/PipeHeartbeatRPCHandler.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/PipeHeartbeatRPCHandler.java index 569424afbff4..e5fa157961d7 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/PipeHeartbeatRPCHandler.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/client/async/handlers/rpc/PipeHeartbeatRPCHandler.java @@ -20,8 +20,8 @@ package org.apache.iotdb.confignode.client.async.handlers.rpc; import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; +import org.apache.iotdb.common.rpc.thrift.TPipeHeartbeatResp; import org.apache.iotdb.confignode.client.async.CnToDnAsyncRequestType; -import org.apache.iotdb.mpp.rpc.thrift.TPipeHeartbeatResp; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/agent/task/PipeConfigNodeTaskAgent.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/agent/task/PipeConfigNodeTaskAgent.java index bf145085c134..911909d104ce 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/agent/task/PipeConfigNodeTaskAgent.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/agent/task/PipeConfigNodeTaskAgent.java @@ -19,6 +19,7 @@ package org.apache.iotdb.confignode.manager.pipe.agent.task; +import org.apache.iotdb.common.rpc.thrift.TPipeHeartbeatResp; import org.apache.iotdb.commons.consensus.index.ProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.MetaProgressIndex; import org.apache.iotdb.commons.exception.IllegalPathException; @@ -35,7 +36,6 @@ import org.apache.iotdb.confignode.manager.pipe.metric.source.PipeConfigRegionExtractorMetrics; import org.apache.iotdb.confignode.manager.pipe.resource.PipeConfigNodeResourceManager; import org.apache.iotdb.mpp.rpc.thrift.TPipeHeartbeatReq; -import org.apache.iotdb.mpp.rpc.thrift.TPipeHeartbeatResp; import org.apache.iotdb.mpp.rpc.thrift.TPushPipeMetaRespExceptionMessage; import org.apache.iotdb.pipe.api.exception.PipeException; diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/coordinator/runtime/heartbeat/PipeHeartbeatScheduler.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/coordinator/runtime/heartbeat/PipeHeartbeatScheduler.java index 3533b40158d0..120ddb65f862 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/coordinator/runtime/heartbeat/PipeHeartbeatScheduler.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/coordinator/runtime/heartbeat/PipeHeartbeatScheduler.java @@ -20,6 +20,7 @@ package org.apache.iotdb.confignode.manager.pipe.coordinator.runtime.heartbeat; import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; +import org.apache.iotdb.common.rpc.thrift.TPipeHeartbeatResp; import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory; import org.apache.iotdb.commons.concurrent.ThreadName; import org.apache.iotdb.commons.concurrent.threadpool.ScheduledExecutorUtil; @@ -31,7 +32,6 @@ import org.apache.iotdb.confignode.manager.ConfigManager; import org.apache.iotdb.confignode.manager.pipe.agent.PipeConfigNodeAgent; import org.apache.iotdb.mpp.rpc.thrift.TPipeHeartbeatReq; -import org.apache.iotdb.mpp.rpc.thrift.TPipeHeartbeatResp; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java index 83854f2e4edd..99ef87207dc8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.pipe.agent.task; +import org.apache.iotdb.common.rpc.thrift.TPipeHeartbeatResp; import org.apache.iotdb.commons.concurrent.IoTThreadFactory; import org.apache.iotdb.commons.concurrent.ThreadName; import org.apache.iotdb.commons.concurrent.threadpool.WrappedThreadPoolExecutor; @@ -61,7 +62,6 @@ import org.apache.iotdb.db.subscription.agent.SubscriptionAgent; import org.apache.iotdb.mpp.rpc.thrift.TDataNodeHeartbeatResp; import org.apache.iotdb.mpp.rpc.thrift.TPipeHeartbeatReq; -import org.apache.iotdb.mpp.rpc.thrift.TPipeHeartbeatResp; import org.apache.iotdb.mpp.rpc.thrift.TPushPipeMetaRespExceptionMessage; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; import org.apache.iotdb.pipe.api.exception.PipeException; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/matcher/CachedSchemaPatternMatcher.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/matcher/CachedSchemaPatternMatcher.java index 1e4e8d4e232d..30795c68dcff 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/matcher/CachedSchemaPatternMatcher.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/matcher/CachedSchemaPatternMatcher.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.pipe.extractor.dataregion.realtime.matcher; import org.apache.iotdb.commons.pipe.datastructure.pattern.PipePattern; +import org.apache.iotdb.commons.pipe.event.ProgressReportEvent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; @@ -38,7 +39,8 @@ public class CachedSchemaPatternMatcher { public static boolean match( final PipeRealtimeEvent event, final PipeRealtimeDataRegionExtractor extractor) { // HeartbeatEvent will be assigned to all extractors - if (event.getEvent() instanceof PipeHeartbeatEvent) { + if (event.getEvent() instanceof PipeHeartbeatEvent + || event.getEvent() instanceof ProgressReportEvent) { return true; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java index 0be487aa7fd1..5d7fa891530c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/thrift/impl/DataNodeInternalRPCServiceImpl.java @@ -26,6 +26,7 @@ import org.apache.iotdb.common.rpc.thrift.TFlushReq; import org.apache.iotdb.common.rpc.thrift.TLoadSample; import org.apache.iotdb.common.rpc.thrift.TNodeLocations; +import org.apache.iotdb.common.rpc.thrift.TPipeHeartbeatResp; import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.common.rpc.thrift.TSender; import org.apache.iotdb.common.rpc.thrift.TServiceType; @@ -214,7 +215,6 @@ import org.apache.iotdb.mpp.rpc.thrift.TMaintainPeerReq; import org.apache.iotdb.mpp.rpc.thrift.TNotifyRegionMigrationReq; import org.apache.iotdb.mpp.rpc.thrift.TPipeHeartbeatReq; -import org.apache.iotdb.mpp.rpc.thrift.TPipeHeartbeatResp; import org.apache.iotdb.mpp.rpc.thrift.TPushConsumerGroupMetaReq; import org.apache.iotdb.mpp.rpc.thrift.TPushConsumerGroupMetaResp; import org.apache.iotdb.mpp.rpc.thrift.TPushConsumerGroupMetaRespExceptionMessage; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java index 9942f645ad87..9d18744d0bdf 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java @@ -19,6 +19,7 @@ package org.apache.iotdb.commons.pipe.agent.task; +import org.apache.iotdb.common.rpc.thrift.TPipeHeartbeatResp; import org.apache.iotdb.commons.conf.CommonDescriptor; import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeConnectorCriticalException; @@ -36,7 +37,6 @@ import org.apache.iotdb.commons.pipe.connector.limiter.PipeEndPointRateLimiter; import org.apache.iotdb.commons.subscription.config.SubscriptionConfig; import org.apache.iotdb.mpp.rpc.thrift.TPipeHeartbeatReq; -import org.apache.iotdb.mpp.rpc.thrift.TPipeHeartbeatResp; import org.apache.iotdb.mpp.rpc.thrift.TPushPipeMetaRespExceptionMessage; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; diff --git a/iotdb-protocol/thrift-commons/src/main/thrift/common.thrift b/iotdb-protocol/thrift-commons/src/main/thrift/common.thrift index aed1b5f88f56..73aef5c4c594 100644 --- a/iotdb-protocol/thrift-commons/src/main/thrift/common.thrift +++ b/iotdb-protocol/thrift-commons/src/main/thrift/common.thrift @@ -196,6 +196,13 @@ struct TSetThrottleQuotaReq { 2: required TThrottleQuota throttleQuota } +struct TPipeHeartbeatResp { + 1: required list pipeMetaList + 2: optional list pipeCompletedList + 3: optional list pipeRemainingEventCountList + 4: optional list pipeRemainingTimeList +} + struct TLicense { 1: required i64 licenseIssueTimestamp 2: required i64 expireTimestamp diff --git a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift index 4642f3e20af5..3231499388d7 100644 --- a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift +++ b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift @@ -1771,5 +1771,8 @@ service IConfigNodeRPCService { /** Get throttle quota information */ TThrottleQuotaResp getThrottleQuota() + + /** Push heartbeat in shutdown */ + common.TSStatus pushHeartbeat(common.TPipeHeartbeatResp resp) } diff --git a/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift b/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift index d58e1eb1591d..f7a9c4c38549 100644 --- a/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift +++ b/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift @@ -313,13 +313,6 @@ struct TPipeHeartbeatReq { 1: required i64 heartbeatId } -struct TPipeHeartbeatResp { - 1: required list pipeMetaList - 2: optional list pipeCompletedList - 3: optional list pipeRemainingEventCountList - 4: optional list pipeRemainingTimeList -} - enum TSchemaLimitLevel{ DEVICE, TIMESERIES @@ -1041,7 +1034,7 @@ service IDataNodeRPCService { /** * ConfigNode will ask DataNode for pipe meta in every few seconds **/ - TPipeHeartbeatResp pipeHeartbeat(TPipeHeartbeatReq req) + common.TPipeHeartbeatResp pipeHeartbeat(TPipeHeartbeatReq req) /** * Execute CQ on DataNode From bf1adc3752351aa65de7f5b2f2481f505fef2f63 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 3 Jul 2025 17:47:14 +0800 Subject: [PATCH 109/185] skip the commit of the later events when there are tsFiles --- .../realtime/assigner/PipeDataRegionAssigner.java | 15 ++------------- 1 file changed, 2 insertions(+), 13 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java index 0ba9906fc7b7..bfe8a987df9b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java @@ -20,7 +20,6 @@ package org.apache.iotdb.db.pipe.extractor.dataregion.realtime.assigner; import org.apache.iotdb.commons.consensus.index.ProgressIndex; -import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.event.ProgressReportEvent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; @@ -36,7 +35,6 @@ import java.io.Closeable; import java.util.Set; import java.util.concurrent.CopyOnWriteArraySet; -import java.util.concurrent.atomic.AtomicReference; public class PipeDataRegionAssigner implements Closeable { @@ -45,9 +43,6 @@ public class PipeDataRegionAssigner implements Closeable { private final String dataRegionId; protected final Set extractors = new CopyOnWriteArraySet<>(); - private final AtomicReference maxProgressIndexForRealtimeEvent = - new AtomicReference<>(MinimumProgressIndex.INSTANCE); - public String getDataRegionId() { return dataRegionId; } @@ -119,19 +114,13 @@ private void bindOrUpdateProgressIndexForRealtimeEvent(final PipeRealtimeEvent e dataRegionId, event.getTsFileEpoch().getFilePath(), getProgressIndex4RealtimeEvent(event))) { - event.bindProgressIndex(maxProgressIndexForRealtimeEvent.get()); + event.skipReportOnCommit(); if (LOGGER.isDebugEnabled()) { LOGGER.debug( - "Data region {} bind {} to event {} because it was flushed prematurely.", + "Data region {} skip commit of event {} because it was flushed prematurely.", dataRegionId, - maxProgressIndexForRealtimeEvent, event.coreReportMessage()); } - } else { - maxProgressIndexForRealtimeEvent.updateAndGet( - index -> - index.updateToMinimumEqualOrIsAfterProgressIndex( - getProgressIndex4RealtimeEvent(event))); } } From fd4923258efa779b82d06d2fe0ffad3f247e86be Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 3 Jul 2025 18:14:42 +0800 Subject: [PATCH 110/185] report heartbeat --- .../confignode/manager/ConfigManager.java | 18 +++++++++++ .../iotdb/confignode/manager/IManager.java | 3 ++ .../thrift/ConfigNodeRPCServiceProcessor.java | 6 ++++ .../agent/task/PipeDataNodeTaskAgent.java | 30 ++++++++++++++++++- .../db/protocol/client/ConfigNodeClient.java | 8 +++++ .../db/service/DataNodeShutdownHook.java | 2 +- .../src/main/thrift/confignode.thrift | 2 +- 7 files changed, 66 insertions(+), 3 deletions(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java index 0a207ab72385..8bb92f9d2bbd 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java @@ -26,6 +26,7 @@ import org.apache.iotdb.common.rpc.thrift.TDataNodeConfiguration; import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; import org.apache.iotdb.common.rpc.thrift.TFlushReq; +import org.apache.iotdb.common.rpc.thrift.TPipeHeartbeatResp; import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet; import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.common.rpc.thrift.TSchemaNode; @@ -2562,6 +2563,23 @@ public TThrottleQuotaResp getThrottleQuota() { : new TThrottleQuotaResp(status); } + @Override + public TSStatus pushHeartbeat(final int dataNodeId, final TPipeHeartbeatResp resp) { + final TSStatus status = confirmLeader(); + if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + return status; + } + pipeManager + .getPipeRuntimeCoordinator() + .parseHeartbeat( + dataNodeId, + resp.getPipeMetaList(), + resp.getPipeCompletedList(), + resp.getPipeRemainingEventCountList(), + resp.getPipeRemainingTimeList()); + return StatusUtils.OK; + } + @Override public DataSet registerAINode(TAINodeRegisterReq req) { TSStatus status = confirmLeader(); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java index 657f068e674a..88731538be29 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java @@ -22,6 +22,7 @@ import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation; import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; import org.apache.iotdb.common.rpc.thrift.TFlushReq; +import org.apache.iotdb.common.rpc.thrift.TPipeHeartbeatResp; import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.common.rpc.thrift.TSetConfigurationReq; import org.apache.iotdb.common.rpc.thrift.TSetSpaceQuotaReq; @@ -823,4 +824,6 @@ TDataPartitionTableResp getOrCreateDataPartition( /** Set space quota. */ TSStatus setSpaceQuota(TSetSpaceQuotaReq req); + + TSStatus pushHeartbeat(final int dataNodeId, final TPipeHeartbeatResp resp); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java index b8ec6f773bd7..21dbccd07a75 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java @@ -23,6 +23,7 @@ import org.apache.iotdb.common.rpc.thrift.TDataNodeLocation; import org.apache.iotdb.common.rpc.thrift.TFlushReq; import org.apache.iotdb.common.rpc.thrift.TNodeLocations; +import org.apache.iotdb.common.rpc.thrift.TPipeHeartbeatResp; import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.common.rpc.thrift.TSetConfigurationReq; import org.apache.iotdb.common.rpc.thrift.TSetSpaceQuotaReq; @@ -1298,4 +1299,9 @@ public TThrottleQuotaResp showThrottleQuota(TShowThrottleReq req) { public TThrottleQuotaResp getThrottleQuota() { return configManager.getThrottleQuota(); } + + @Override + public TSStatus pushHeartbeat(final int dataNodeId, final TPipeHeartbeatResp resp) { + return configManager.pushHeartbeat(dataNodeId, resp); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java index 99ef87207dc8..d81cc861a66c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.pipe.agent.task; import org.apache.iotdb.common.rpc.thrift.TPipeHeartbeatResp; +import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.concurrent.IoTThreadFactory; import org.apache.iotdb.commons.concurrent.ThreadName; import org.apache.iotdb.commons.concurrent.threadpool.WrappedThreadPoolExecutor; @@ -55,6 +56,9 @@ import org.apache.iotdb.db.pipe.metric.overview.PipeTsFileToTabletsMetrics; import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryManager; +import org.apache.iotdb.db.protocol.client.ConfigNodeClient; +import org.apache.iotdb.db.protocol.client.ConfigNodeClientManager; +import org.apache.iotdb.db.protocol.client.ConfigNodeInfo; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.pipe.PipeOperateSchemaQueueNode; import org.apache.iotdb.db.schemaengine.SchemaEngine; @@ -65,6 +69,7 @@ import org.apache.iotdb.mpp.rpc.thrift.TPushPipeMetaRespExceptionMessage; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; import org.apache.iotdb.pipe.api.exception.PipeException; +import org.apache.iotdb.rpc.TSStatusCode; import com.google.common.collect.ImmutableMap; import org.apache.thrift.TException; @@ -455,7 +460,8 @@ private void collectPipeMetaListInternal(final TDataNodeHeartbeatResp resp) thro protected void collectPipeMetaListInternal( final TPipeHeartbeatReq req, final TPipeHeartbeatResp resp) throws TException { // Do nothing if data node is removing or removed, or request does not need pipe meta list - if (PipeDataNodeAgent.runtime().isShutdown()) { + // If the heartbeat + if (PipeDataNodeAgent.runtime().isShutdown() && req.heartbeatId != Long.MIN_VALUE) { return; } LOGGER.info("Received pipe heartbeat request {} from config node.", req.heartbeatId); @@ -597,6 +603,28 @@ public void runPipeTasks( ///////////////////////// Shutdown Logic ///////////////////////// + public void persistAllProgressIndex() { + persistAllProgressIndexLocally(); + persistAllProgressIndex2ConfigNode(); + } + + public void persistAllProgressIndex2ConfigNode() { + try (final ConfigNodeClient configNodeClient = + ConfigNodeClientManager.getInstance().borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { + // Send request to some API server + final TPipeHeartbeatResp resp = new TPipeHeartbeatResp(); + collectPipeMetaList(new TPipeHeartbeatReq(Long.MIN_VALUE), resp); + final TSStatus result = + configNodeClient.pushHeartbeat( + IoTDBDescriptor.getInstance().getConfig().getDataNodeId(), resp); + if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != result.getCode()) { + LOGGER.warn("Failed to persist progress index to configNode, status: {}", result); + } + } catch (final Exception e) { + LOGGER.warn(e.getMessage()); + } + } + public void persistAllProgressIndexLocally() { if (!PipeConfig.getInstance().isPipeProgressIndexPersistEnabled()) { LOGGER.info( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java index 979c18320ca8..2825eefaf4c4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java @@ -24,6 +24,7 @@ import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.common.rpc.thrift.TFlushReq; import org.apache.iotdb.common.rpc.thrift.TNodeLocations; +import org.apache.iotdb.common.rpc.thrift.TPipeHeartbeatResp; import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.common.rpc.thrift.TSetConfigurationReq; import org.apache.iotdb.common.rpc.thrift.TSetSpaceQuotaReq; @@ -1271,6 +1272,13 @@ public TThrottleQuotaResp getThrottleQuota() throws TException { () -> client.getThrottleQuota(), resp -> !updateConfigNodeLeader(resp.status)); } + @Override + public TSStatus pushHeartbeat(final int dataNodeId, final TPipeHeartbeatResp resp) + throws TException { + return executeRemoteCallWithRetry( + () -> client.pushHeartbeat(dataNodeId, resp), status -> !updateConfigNodeLeader(status)); + } + public static class Factory extends ThriftClientFactory { public Factory( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java index 3c4cc1b7f3d5..d795d8a5fbce 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java @@ -111,7 +111,7 @@ public void run() { } } // Persist progress index before shutdown to accurate recovery after restart - PipeDataNodeAgent.task().persistAllProgressIndexLocally(); + PipeDataNodeAgent.task().persistAllProgressIndex(); // Shutdown pipe progressIndex background service PipePeriodicalJobExecutor.shutdownBackgroundService(); diff --git a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift index 3231499388d7..19e8e2a7d1f2 100644 --- a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift +++ b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift @@ -1773,6 +1773,6 @@ service IConfigNodeRPCService { TThrottleQuotaResp getThrottleQuota() /** Push heartbeat in shutdown */ - common.TSStatus pushHeartbeat(common.TPipeHeartbeatResp resp) + common.TSStatus pushHeartbeat(i32 dataNodeId, common.TPipeHeartbeatResp resp) } From 18aef844941e717fb30af56de8d69d2e24ef789c Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 3 Jul 2025 19:27:03 +0800 Subject: [PATCH 111/185] try fix --- .../PipeCompactedTsFileInsertionEvent.java | 2 +- .../tsfile/PipeTsFileInsertionEvent.java | 2 +- .../PipeRealtimeDataRegionExtractor.java | 34 +++++++++++++++++++ ...PipeRealtimeDataRegionHybridExtractor.java | 11 ++++-- .../assigner/PipeDataRegionAssigner.java | 29 ---------------- .../PipeTsFileEpochProgressIndexKeeper.java | 29 ++++++++++++---- 6 files changed, 67 insertions(+), 40 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java index 0f034dbdb96d..fb0e808a0642 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java @@ -177,7 +177,7 @@ public void eliminateProgressIndex() { if (Objects.isNull(overridingProgressIndex)) { for (final String originFilePath : originFilePaths) { PipeTsFileEpochProgressIndexKeeper.getInstance() - .eliminateProgressIndex(dataRegionId, originFilePath); + .eliminateProgressIndex(dataRegionId, pipeName, originFilePath); } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java index d2c443e787e1..a05a7be54359 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java @@ -324,7 +324,7 @@ protected void reportProgress() { public void eliminateProgressIndex() { if (Objects.isNull(overridingProgressIndex) && Objects.nonNull(resource)) { PipeTsFileEpochProgressIndexKeeper.getInstance() - .eliminateProgressIndex(resource.getDataRegionId(), resource.getTsFilePath()); + .eliminateProgressIndex(resource.getDataRegionId(), pipeName, resource.getTsFilePath()); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java index ad8b6c76846c..1924c116801c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.pipe.extractor.dataregion.realtime; import org.apache.iotdb.commons.consensus.DataRegionId; +import org.apache.iotdb.commons.consensus.index.ProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.SegmentProgressIndex; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeNonCriticalException; @@ -33,8 +34,11 @@ import org.apache.iotdb.commons.utils.TimePartitionUtils; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; +import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; +import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; import org.apache.iotdb.db.pipe.extractor.dataregion.DataRegionListeningFilter; +import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.assigner.PipeTsFileEpochProgressIndexKeeper; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.listener.PipeInsertionDataNodeListener; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.listener.PipeTimePartitionListener; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.matcher.CachedSchemaPatternMatcher; @@ -384,7 +388,13 @@ protected void extractDirectly(final PipeRealtimeEvent event) { public Event supply() { PipeRealtimeEvent realtimeEvent = getNextRealtimeEvent(); + final EnrichedEvent innerEvent = realtimeEvent.getEvent(); while (realtimeEvent != null) { + if (innerEvent instanceof PipeTsFileInsertionEvent + || innerEvent instanceof PipeInsertNodeTabletInsertionEvent) { + bindOrUpdateProgressIndexForRealtimeEvent(realtimeEvent); + } + final Event suppliedEvent = doSupply(realtimeEvent); realtimeEvent.decreaseReferenceCount(PipeRealtimeDataRegionExtractor.class.getName(), false); @@ -542,6 +552,30 @@ public final boolean isShouldTransferModFile() { return shouldTransferModFile; } + private void bindOrUpdateProgressIndexForRealtimeEvent(final PipeRealtimeEvent event) { + if (PipeTsFileEpochProgressIndexKeeper.getInstance() + .isProgressIndexAfterOrEquals( + dataRegionId, + pipeName, + event.getTsFileEpoch().getFilePath(), + getProgressIndex4RealtimeEvent(event))) { + event.skipReportOnCommit(); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug( + "Pipe {} on data region {} skip commit of event {} because it was flushed prematurely.", + pipeName, + dataRegionId, + event.coreReportMessage()); + } + } + } + + private ProgressIndex getProgressIndex4RealtimeEvent(final PipeRealtimeEvent event) { + return event.getEvent() instanceof PipeTsFileInsertionEvent + ? ((PipeTsFileInsertionEvent) event.getEvent()).forceGetProgressIndex() + : event.getProgressIndex(); + } + @Override public String toString() { return "PipeRealtimeDataRegionExtractor{" diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java index 5e086f7ffd5e..5bc7f1b47c71 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java @@ -83,6 +83,8 @@ private void extractTabletInsertion(final PipeRealtimeEvent event) { if (canNotUseTabletAnyMore(event)) { event.getTsFileEpoch().migrateState(this, curState -> TsFileEpoch.State.USING_TSFILE); + PipeTsFileEpochProgressIndexKeeper.getInstance() + .registerProgressIndex(dataRegionId, pipeName, event.getTsFileEpoch().getFilePath()); } else { event .getTsFileEpoch() @@ -355,11 +357,14 @@ private Event supplyTabletInsertion(final PipeRealtimeEvent event) { }); final TsFileEpoch.State state = event.getTsFileEpoch().getState(this); + if (state == TsFileEpoch.State.USING_TSFILE) { + PipeTsFileEpochProgressIndexKeeper.getInstance() + .registerProgressIndex(dataRegionId, pipeName, event.getTsFileEpoch().getFilePath()); + } + switch (state) { case USING_TSFILE: // If the state is USING_TSFILE, discard the event and poll the next one. - PipeTsFileEpochProgressIndexKeeper.getInstance() - .eliminateProgressIndex(dataRegionId, event.getTsFileEpoch().getFilePath()); return null; case EMPTY: case USING_TABLET: @@ -400,6 +405,8 @@ private Event supplyTsFileInsertion(final PipeRealtimeEvent event) { switch (state) { case USING_TABLET: // If the state is USING_TABLET, discard the event and poll the next one. + PipeTsFileEpochProgressIndexKeeper.getInstance() + .eliminateProgressIndex(dataRegionId, pipeName, event.getTsFileEpoch().getFilePath()); return null; case EMPTY: case USING_TSFILE: diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java index bfe8a987df9b..a955e9f25ac3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java @@ -19,11 +19,9 @@ package org.apache.iotdb.db.pipe.extractor.dataregion.realtime.assigner; -import org.apache.iotdb.commons.consensus.index.ProgressIndex; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.event.ProgressReportEvent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; -import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEventFactory; @@ -93,11 +91,6 @@ public void assignToExtractor(final PipeRealtimeEvent event) { tsFileInsertionEvent.disableMod4NonTransferPipes(extractor.isShouldTransferModFile()); } - if (innerEvent instanceof PipeTsFileInsertionEvent - || innerEvent instanceof PipeInsertNodeTabletInsertionEvent) { - bindOrUpdateProgressIndexForRealtimeEvent(copiedEvent); - } - if (!copiedEvent.increaseReferenceCount(PipeDataRegionAssigner.class.getName())) { LOGGER.warn( "The reference count of the event {} cannot be increased, skipping it.", @@ -108,28 +101,6 @@ public void assignToExtractor(final PipeRealtimeEvent event) { }); } - private void bindOrUpdateProgressIndexForRealtimeEvent(final PipeRealtimeEvent event) { - if (PipeTsFileEpochProgressIndexKeeper.getInstance() - .isProgressIndexAfterOrEquals( - dataRegionId, - event.getTsFileEpoch().getFilePath(), - getProgressIndex4RealtimeEvent(event))) { - event.skipReportOnCommit(); - if (LOGGER.isDebugEnabled()) { - LOGGER.debug( - "Data region {} skip commit of event {} because it was flushed prematurely.", - dataRegionId, - event.coreReportMessage()); - } - } - } - - private ProgressIndex getProgressIndex4RealtimeEvent(final PipeRealtimeEvent event) { - return event.getEvent() instanceof PipeTsFileInsertionEvent - ? ((PipeTsFileInsertionEvent) event.getEvent()).forceGetProgressIndex() - : event.getProgressIndex(); - } - public void startAssignTo(final PipeRealtimeDataRegionExtractor extractor) { extractors.add(extractor); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeTsFileEpochProgressIndexKeeper.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeTsFileEpochProgressIndexKeeper.java index da2cde90667e..17807fd3918a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeTsFileEpochProgressIndexKeeper.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeTsFileEpochProgressIndexKeeper.java @@ -20,6 +20,9 @@ package org.apache.iotdb.db.pipe.extractor.dataregion.realtime.assigner; import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; + +import javax.annotation.Nonnull; import java.util.Map; import java.util.Map.Entry; @@ -28,33 +31,45 @@ public class PipeTsFileEpochProgressIndexKeeper { - // data region id -> (tsFile path, max progress index) - private final Map> progressIndexKeeper = + // data region id -> pipeName -> tsFile path -> max progress index + private final Map>> progressIndexKeeper = new ConcurrentHashMap<>(); + public synchronized void registerProgressIndex( + final String dataRegionId, final String pipeName, final String tsFileName) { + progressIndexKeeper + .computeIfAbsent(dataRegionId, k -> new ConcurrentHashMap<>()) + .computeIfAbsent(pipeName, k -> new ConcurrentHashMap<>()) + .put(tsFileName, MinimumProgressIndex.INSTANCE); + } + public synchronized void updateProgressIndex( final String dataRegionId, final String tsFileName, final ProgressIndex progressIndex) { progressIndexKeeper + .computeIfAbsent(dataRegionId, k -> new ConcurrentHashMap<>()) .computeIfAbsent(dataRegionId, k -> new ConcurrentHashMap<>()) .compute( tsFileName, (k, v) -> - v == null - ? progressIndex - : v.updateToMinimumEqualOrIsAfterProgressIndex(progressIndex)); + v == null ? null : v.updateToMinimumEqualOrIsAfterProgressIndex(progressIndex)); } public synchronized void eliminateProgressIndex( - final String dataRegionId, final String filePath) { + final String dataRegionId, final @Nonnull String pipeName, final String filePath) { progressIndexKeeper .computeIfAbsent(dataRegionId, k -> new ConcurrentHashMap<>()) + .computeIfAbsent(pipeName, k -> new ConcurrentHashMap<>()) .remove(filePath); } public synchronized boolean isProgressIndexAfterOrEquals( - final String dataRegionId, final String tsFilePath, final ProgressIndex progressIndex) { + final String dataRegionId, + final String pipeName, + final String tsFilePath, + final ProgressIndex progressIndex) { return progressIndexKeeper .computeIfAbsent(dataRegionId, k -> new ConcurrentHashMap<>()) + .computeIfAbsent(pipeName, k -> new ConcurrentHashMap<>()) .entrySet() .stream() .filter(entry -> !Objects.equals(entry.getKey(), tsFilePath)) From c4e04a59c97b9da034bfe8563f73e001619da15e Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 3 Jul 2025 19:51:15 +0800 Subject: [PATCH 112/185] next fix --- ...PipeRealtimeDataRegionHybridExtractor.java | 4 ++-- .../PipeTsFileEpochProgressIndexKeeper.java | 21 +++++-------------- .../realtime/epoch/TsFileEpoch.java | 17 ++++++++------- .../realtime/epoch/TsFileEpochManager.java | 4 ++-- .../dataregion/tsfile/TsFileResource.java | 7 ------- 5 files changed, 18 insertions(+), 35 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java index 5bc7f1b47c71..3a098ca282c0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java @@ -84,7 +84,7 @@ private void extractTabletInsertion(final PipeRealtimeEvent event) { if (canNotUseTabletAnyMore(event)) { event.getTsFileEpoch().migrateState(this, curState -> TsFileEpoch.State.USING_TSFILE); PipeTsFileEpochProgressIndexKeeper.getInstance() - .registerProgressIndex(dataRegionId, pipeName, event.getTsFileEpoch().getFilePath()); + .registerProgressIndex(dataRegionId, pipeName, event.getTsFileEpoch().getResource()); } else { event .getTsFileEpoch() @@ -359,7 +359,7 @@ private Event supplyTabletInsertion(final PipeRealtimeEvent event) { final TsFileEpoch.State state = event.getTsFileEpoch().getState(this); if (state == TsFileEpoch.State.USING_TSFILE) { PipeTsFileEpochProgressIndexKeeper.getInstance() - .registerProgressIndex(dataRegionId, pipeName, event.getTsFileEpoch().getFilePath()); + .registerProgressIndex(dataRegionId, pipeName, event.getTsFileEpoch().getResource()); } switch (state) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeTsFileEpochProgressIndexKeeper.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeTsFileEpochProgressIndexKeeper.java index 17807fd3918a..0d980cb221a6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeTsFileEpochProgressIndexKeeper.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeTsFileEpochProgressIndexKeeper.java @@ -20,7 +20,7 @@ package org.apache.iotdb.db.pipe.extractor.dataregion.realtime.assigner; import org.apache.iotdb.commons.consensus.index.ProgressIndex; -import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import javax.annotation.Nonnull; @@ -32,26 +32,15 @@ public class PipeTsFileEpochProgressIndexKeeper { // data region id -> pipeName -> tsFile path -> max progress index - private final Map>> progressIndexKeeper = + private final Map>> progressIndexKeeper = new ConcurrentHashMap<>(); public synchronized void registerProgressIndex( - final String dataRegionId, final String pipeName, final String tsFileName) { + final String dataRegionId, final String pipeName, final TsFileResource resource) { progressIndexKeeper .computeIfAbsent(dataRegionId, k -> new ConcurrentHashMap<>()) .computeIfAbsent(pipeName, k -> new ConcurrentHashMap<>()) - .put(tsFileName, MinimumProgressIndex.INSTANCE); - } - - public synchronized void updateProgressIndex( - final String dataRegionId, final String tsFileName, final ProgressIndex progressIndex) { - progressIndexKeeper - .computeIfAbsent(dataRegionId, k -> new ConcurrentHashMap<>()) - .computeIfAbsent(dataRegionId, k -> new ConcurrentHashMap<>()) - .compute( - tsFileName, - (k, v) -> - v == null ? null : v.updateToMinimumEqualOrIsAfterProgressIndex(progressIndex)); + .put(resource.getTsFilePath(), resource); } public synchronized void eliminateProgressIndex( @@ -75,7 +64,7 @@ public synchronized boolean isProgressIndexAfterOrEquals( .filter(entry -> !Objects.equals(entry.getKey(), tsFilePath)) .map(Entry::getValue) .filter(Objects::nonNull) - .anyMatch(index -> !index.isAfter(progressIndex)); + .anyMatch(resource -> !resource.getMaxProgressIndex().isAfter(progressIndex)); } //////////////////////////// singleton //////////////////////////// diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/epoch/TsFileEpoch.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/epoch/TsFileEpoch.java index c2db4c77c86d..83b69ea60ec5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/epoch/TsFileEpoch.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/epoch/TsFileEpoch.java @@ -21,6 +21,7 @@ import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionExtractor; import org.apache.iotdb.db.pipe.metric.source.PipeDataRegionExtractorMetrics; +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -29,13 +30,13 @@ public class TsFileEpoch { - private final String filePath; + private final TsFileResource resource; private final ConcurrentMap> dataRegionExtractor2State; private final AtomicLong insertNodeMinTime; - public TsFileEpoch(final String filePath) { - this.filePath = filePath; + public TsFileEpoch(final TsFileResource resource) { + this.resource = resource; this.dataRegionExtractor2State = new ConcurrentHashMap<>(); this.insertNodeMinTime = new AtomicLong(Long.MAX_VALUE); } @@ -64,19 +65,19 @@ public void updateInsertNodeMinTime(final long newComingMinTime) { insertNodeMinTime.updateAndGet(recordedMinTime -> Math.min(recordedMinTime, newComingMinTime)); } - public long getInsertNodeMinTime() { - return insertNodeMinTime.get(); + public TsFileResource getResource() { + return resource; } public String getFilePath() { - return filePath; + return resource.getTsFilePath(); } @Override public String toString() { return "TsFileEpoch{" - + "filePath='" - + filePath + + "resource='" + + resource + '\'' + ", dataRegionExtractor2State=" + dataRegionExtractor2State diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/epoch/TsFileEpochManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/epoch/TsFileEpochManager.java index 6ee0a7384526..9e18258dd17d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/epoch/TsFileEpochManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/epoch/TsFileEpochManager.java @@ -56,7 +56,7 @@ public PipeRealtimeEvent bindPipeTsFileInsertionEvent( filePath, path -> { LOGGER.info("TsFileEpoch not found for TsFile {}, creating a new one", path); - return new TsFileEpoch(path); + return new TsFileEpoch(resource); }); final TsFileEpoch epoch = filePath2Epoch.remove(filePath); @@ -79,7 +79,7 @@ public PipeRealtimeEvent bindPipeTsFileInsertionEvent( public PipeRealtimeEvent bindPipeInsertNodeTabletInsertionEvent( PipeInsertNodeTabletInsertionEvent event, InsertNode node, TsFileResource resource) { final TsFileEpoch epoch = - filePath2Epoch.computeIfAbsent(resource.getTsFilePath(), TsFileEpoch::new); + filePath2Epoch.computeIfAbsent(resource.getTsFilePath(), k -> new TsFileEpoch(resource)); epoch.updateInsertNodeMinTime(node.getMinTime()); return new PipeRealtimeEvent( event, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java index ee05d2f1c74d..2b8baeb5112b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java @@ -28,7 +28,6 @@ import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.exception.load.PartitionViolationException; -import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.assigner.PipeTsFileEpochProgressIndexKeeper; import org.apache.iotdb.db.schemaengine.schemaregion.utils.ResourceByPathUtils; import org.apache.iotdb.db.storageengine.dataregion.DataRegion; import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.utils.InsertionCompactionCandidateStatus; @@ -1202,9 +1201,6 @@ public void updateProgressIndex(ProgressIndex progressIndex) { (maxProgressIndex == null ? progressIndex : maxProgressIndex.updateToMinimumEqualOrIsAfterProgressIndex(progressIndex)); - - PipeTsFileEpochProgressIndexKeeper.getInstance() - .updateProgressIndex(getDataRegionId(), getTsFilePath(), maxProgressIndex); } public void setProgressIndex(ProgressIndex progressIndex) { @@ -1213,9 +1209,6 @@ public void setProgressIndex(ProgressIndex progressIndex) { } maxProgressIndex = progressIndex; - - PipeTsFileEpochProgressIndexKeeper.getInstance() - .updateProgressIndex(getDataRegionId(), getTsFilePath(), maxProgressIndex); } public ProgressIndex getMaxProgressIndex() { From 7c85c7fbd6695e207a0275759295859c377a8f88 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 4 Jul 2025 09:54:29 +0800 Subject: [PATCH 113/185] Historical NPE fix --- .../pipe/extractor/dataregion/IoTDBDataRegionExtractor.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java index 3d5b6160dacf..8158c22d1a68 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java @@ -393,7 +393,9 @@ public void start() throws Exception { "Pipe {}@{}: Started historical extractor {} and realtime extractor {} successfully within {} ms.", pipeName, regionId, - historicalExtractor.getClass().getSimpleName(), + Objects.nonNull(historicalExtractor) + ? historicalExtractor.getClass().getSimpleName() + : null, realtimeExtractor.getClass().getSimpleName(), System.currentTimeMillis() - startTime); return; From 75995066114c746f063cdd9928c2d99a44856b10 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 4 Jul 2025 10:51:05 +0800 Subject: [PATCH 114/185] realtime NPE fix & reduce log --- .../pipe/agent/task/PipeDataNodeTaskAgent.java | 16 ++++++++-------- .../dataregion/DataRegionWatermarkInjector.java | 4 ---- .../PipeRealtimeDataRegionExtractor.java | 2 +- .../PipeRealtimeDataRegionHybridExtractor.java | 8 ++++---- .../apache/iotdb/commons/conf/CommonConfig.java | 8 ++++---- .../iotdb/commons/pipe/config/PipeConfig.java | 2 +- .../commons/pipe/config/PipeDescriptor.java | 2 +- .../pipe/resource/log/PipeLogManager.java | 2 +- .../commons/pipe/resource/log/PipeLogStatus.java | 6 +++--- 9 files changed, 23 insertions(+), 27 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java index d81cc861a66c..4fc6bb7266c1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java @@ -464,6 +464,13 @@ protected void collectPipeMetaListInternal( if (PipeDataNodeAgent.runtime().isShutdown() && req.heartbeatId != Long.MIN_VALUE) { return; } + final Optional logger = + PipeDataNodeResourceManager.log() + .schedule( + PipeDataNodeTaskAgent.class, + PipeConfig.getInstance().getPipeMetaReportMaxLogNumPerRound(), + PipeConfig.getInstance().getPipeMetaReportMaxLogIntervalRounds(), + pipeMetaKeeper.getPipeMetaCount()); LOGGER.info("Received pipe heartbeat request {} from config node.", req.heartbeatId); final Set dataRegionIds = @@ -476,13 +483,6 @@ protected void collectPipeMetaListInternal( final List pipeRemainingEventCountList = new ArrayList<>(); final List pipeRemainingTimeList = new ArrayList<>(); try { - final Optional logger = - PipeDataNodeResourceManager.log() - .schedule( - PipeDataNodeTaskAgent.class, - PipeConfig.getInstance().getPipeMetaReportMaxLogNumPerRound(), - PipeConfig.getInstance().getPipeMetaReportMaxLogIntervalRounds(), - pipeMetaKeeper.getPipeMetaCount()); for (final PipeMeta pipeMeta : pipeMetaKeeper.getPipeMetaList()) { pipeMetaBinaryList.add(pipeMeta.serialize()); @@ -529,7 +529,7 @@ protected void collectPipeMetaListInternal( remainingEventAndTime.getLeft(), remainingEventAndTime.getRight())); } - LOGGER.info("Reported {} pipe metas.", pipeMetaBinaryList.size()); + logger.ifPresent(l -> l.info("Reported {} pipe metas.", pipeMetaBinaryList.size())); } catch (final IOException | IllegalPathException e) { throw new TException(e); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/DataRegionWatermarkInjector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/DataRegionWatermarkInjector.java index a8d95d448103..612b1a9701f2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/DataRegionWatermarkInjector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/DataRegionWatermarkInjector.java @@ -48,10 +48,6 @@ public long getInjectionIntervalInMs() { return injectionIntervalInMs; } - public long getNextInjectionTime() { - return nextInjectionTime; - } - public PipeWatermarkEvent inject() { if (System.currentTimeMillis() < nextInjectionTime) { return null; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java index 1924c116801c..238b9b1ffede 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java @@ -388,8 +388,8 @@ protected void extractDirectly(final PipeRealtimeEvent event) { public Event supply() { PipeRealtimeEvent realtimeEvent = getNextRealtimeEvent(); - final EnrichedEvent innerEvent = realtimeEvent.getEvent(); while (realtimeEvent != null) { + final EnrichedEvent innerEvent = realtimeEvent.getEvent(); if (innerEvent instanceof PipeTsFileInsertionEvent || innerEvent instanceof PipeInsertNodeTabletInsertionEvent) { bindOrUpdateProgressIndexForRealtimeEvent(realtimeEvent); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java index 3a098ca282c0..bdb21cd08114 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java @@ -223,7 +223,7 @@ private boolean mayInsertNodeMemoryReachDangerousThreshold(final PipeRealtimeEve logByLogManager( l -> l.info( - "Pipe task {}@{} canNotUseTabletAnyMore(2): The memory usage of the insert node {} has reached the dangerous threshold {}", + "Pipe task {}@{} canNotUseTabletAnyMore(1): The memory usage of the insert node {} has reached the dangerous threshold {}", pipeName, dataRegionId, floatingMemoryUsageInByte * pipeCount, @@ -267,7 +267,7 @@ private boolean isHistoricalTsFileEventCountExceededLimit(final PipeRealtimeEven >= PipeConfig.getInstance().getPipeMaxAllowedHistoricalTsFilePerDataRegion(); if (isHistoricalTsFileEventCountExceededLimit && event.mayExtractorUseTablets(this)) { LOGGER.info( - "Pipe task {}@{} canNotUseTabletAnymoreDeprecated(2): The number of historical tsFile events {} has exceeded the limit {}", + "Pipe task {}@{} canNotUseTabletAnymoreDeprecated(1): The number of historical tsFile events {} has exceeded the limit {}", pipeName, dataRegionId, extractor.getHistoricalTsFileInsertionEventCount(), @@ -286,7 +286,7 @@ private boolean isRealtimeTsFileEventCountExceededLimit(final PipeRealtimeEvent >= PipeConfig.getInstance().getPipeMaxAllowedPendingTsFileEpochPerDataRegion(); if (isRealtimeTsFileEventCountExceededLimit && event.mayExtractorUseTablets(this)) { LOGGER.info( - "Pipe task {}@{} canNotUseTabletAnymoreDeprecated(3): The number of realtime tsFile events {} has exceeded the limit {}", + "Pipe task {}@{} canNotUseTabletAnymoreDeprecated(2): The number of realtime tsFile events {} has exceeded the limit {}", pipeName, dataRegionId, pendingQueue.getTsFileInsertionEventCount(), @@ -304,7 +304,7 @@ private boolean mayTsFileLinkedCountReachDangerousThreshold(final PipeRealtimeEv >= PipeConfig.getInstance().getPipeMaxAllowedLinkedTsFileCount(); if (mayTsFileLinkedCountReachDangerousThreshold && event.mayExtractorUseTablets(this)) { LOGGER.info( - "Pipe task {}@{} canNotUseTabletAnymoreDeprecated(4): The number of linked tsFiles {} has reached the dangerous threshold {}", + "Pipe task {}@{} canNotUseTabletAnymoreDeprecated(3): The number of linked tsFiles {} has reached the dangerous threshold {}", pipeName, dataRegionId, PipeDataNodeResourceManager.tsfile().getLinkedTsFileCount(pipeName), diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java index 6a66c4d54d0d..c21372158d21 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java @@ -306,8 +306,8 @@ public class CommonConfig { private int pipeMaxAllowedPendingTsFileEpochPerDataRegion = Integer.MAX_VALUE; // Deprecated private long pipeMaxAllowedLinkedTsFileCount = Long.MAX_VALUE; // Deprecated - private int pipeMetaReportMaxLogNumPerRound = 10; - private int pipeMetaReportMaxLogIntervalRounds = 36; + private double pipeMetaReportMaxLogNumPerRound = 0.1; + private int pipeMetaReportMaxLogIntervalRounds = 360; private int pipeTsFilePinMaxLogNumPerRound = 10; private int pipeTsFilePinMaxLogIntervalRounds = 90; @@ -1596,11 +1596,11 @@ public void setPipeMaxAllowedLinkedTsFileCount(long pipeMaxAllowedLinkedTsFileCo logger.info("pipeMaxAllowedLinkedTsFileCount is set to {}", pipeMaxAllowedLinkedTsFileCount); } - public int getPipeMetaReportMaxLogNumPerRound() { + public double getPipeMetaReportMaxLogNumPerRound() { return pipeMetaReportMaxLogNumPerRound; } - public void setPipeMetaReportMaxLogNumPerRound(int pipeMetaReportMaxLogNumPerRound) { + public void setPipeMetaReportMaxLogNumPerRound(double pipeMetaReportMaxLogNumPerRound) { if (this.pipeMetaReportMaxLogNumPerRound == pipeMetaReportMaxLogNumPerRound) { return; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java index f25d7235256c..e768c67b371f 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java @@ -363,7 +363,7 @@ public long getPipeMaxAllowedLinkedTsFileCount() { /////////////////////////////// Logger /////////////////////////////// - public int getPipeMetaReportMaxLogNumPerRound() { + public double getPipeMetaReportMaxLogNumPerRound() { return COMMON_CONFIG.getPipeMetaReportMaxLogNumPerRound(); } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java index 8656810c3557..fff467fa395f 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java @@ -121,7 +121,7 @@ public static void loadPipeStaticConfig(CommonConfig config, TrimProperties prop Integer.toString(config.getPipeAirGapReceiverPort())))); config.setPipeMetaReportMaxLogNumPerRound( - Integer.parseInt( + Double.parseDouble( properties.getProperty( "pipe_meta_report_max_log_num_per_round", String.valueOf(config.getPipeMetaReportMaxLogNumPerRound())))); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/resource/log/PipeLogManager.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/resource/log/PipeLogManager.java index 49699fdf8783..69d8b5294db3 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/resource/log/PipeLogManager.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/resource/log/PipeLogManager.java @@ -34,7 +34,7 @@ public class PipeLogManager { public Optional schedule( final Class logClass, - final int maxAverageScale, + final double maxAverageScale, final int maxLogInterval, final int scale) { return logClass2LogStatusMap diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/resource/log/PipeLogStatus.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/resource/log/PipeLogStatus.java index 9348708281fe..a30b12ab5eb0 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/resource/log/PipeLogStatus.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/resource/log/PipeLogStatus.java @@ -29,11 +29,11 @@ class PipeLogStatus { private final Logger logger; - private final int maxAverageScale; + private final double maxAverageScale; private final int maxLogInterval; private final AtomicLong currentRounds = new AtomicLong(0); - PipeLogStatus(final Class logClass, final int maxAverageScale, final int maxLogInterval) { + PipeLogStatus(final Class logClass, final double maxAverageScale, final int maxLogInterval) { logger = LoggerFactory.getLogger(logClass); this.maxAverageScale = maxAverageScale; @@ -42,7 +42,7 @@ class PipeLogStatus { synchronized Optional schedule(final int scale) { if (currentRounds.incrementAndGet() - >= Math.min((int) Math.ceil((double) scale / maxAverageScale), maxLogInterval)) { + >= Math.min((int) Math.ceil(scale / maxAverageScale), maxLogInterval)) { currentRounds.set(0); return Optional.of(logger); } From 07e0136231f08c082dcd298a3cbfc952404e33ba Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 4 Jul 2025 12:19:14 +0800 Subject: [PATCH 115/185] Update PipeEventCommitter.java --- .../pipe/agent/task/progress/PipeEventCommitter.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/progress/PipeEventCommitter.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/progress/PipeEventCommitter.java index 0e5921262abe..61f23907819d 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/progress/PipeEventCommitter.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/progress/PipeEventCommitter.java @@ -88,12 +88,11 @@ public synchronized void commit(final EnrichedEvent event) { final EnrichedEvent e = commitQueue.peek(); if (e.getCommitId() <= lastCommitId.get()) { - LOGGER.warn( - "commit id must be monotonically increasing, current commit id: {}, last commit id: {}, event: {}, stack trace: {}", + LOGGER.info( + "commit id is not monotonically increasing, current commit id: {}, last commit id: {}, event: {}, may be because the tsFile has been compacted", e.getCommitId(), lastCommitId.get(), - e.coreReportMessage(), - Thread.currentThread().getStackTrace()); + e.coreReportMessage()); commitQueue.poll(); continue; } From 0ce9b20b0aed4f7175798235812c1b1d066dd1bc Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 4 Jul 2025 14:13:44 +0800 Subject: [PATCH 116/185] Fix hardlink --- .../historical/PipeHistoricalDataRegionTsFileExtractor.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java index f330980e103f..7909df1ccca6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java @@ -33,7 +33,6 @@ import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.pipe.extractor.dataregion.DataRegionListeningFilter; import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; -import org.apache.iotdb.db.pipe.resource.tsfile.PipeTsFileResourceManager; import org.apache.iotdb.db.storageengine.StorageEngine; import org.apache.iotdb.db.storageengine.dataregion.DataRegion; import org.apache.iotdb.db.storageengine.dataregion.memtable.TsFileProcessor; @@ -486,10 +485,7 @@ private boolean mayTsFileResourceOverlappedWithPattern(final TsFileResource reso try { final Map deviceIsAlignedMap = PipeDataNodeResourceManager.tsfile() - .getDeviceIsAlignedMapFromCache( - PipeTsFileResourceManager.getHardlinkOrCopiedFileInPipeDir( - resource.getTsFile(), pipeName), - false); + .getDeviceIsAlignedMapFromCache(resource.getTsFile(), false); deviceSet = Objects.nonNull(deviceIsAlignedMap) ? deviceIsAlignedMap.keySet() : resource.getDevices(); } catch (final IOException e) { From d9acbaf40d1df7681ba6bbf8eb6809e7e458f94c Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 4 Jul 2025 14:56:07 +0800 Subject: [PATCH 117/185] Update PipeTsFileEpochProgressIndexKeeper.java --- .../realtime/assigner/PipeTsFileEpochProgressIndexKeeper.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeTsFileEpochProgressIndexKeeper.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeTsFileEpochProgressIndexKeeper.java index 0d980cb221a6..50b7d778fc21 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeTsFileEpochProgressIndexKeeper.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeTsFileEpochProgressIndexKeeper.java @@ -40,7 +40,7 @@ public synchronized void registerProgressIndex( progressIndexKeeper .computeIfAbsent(dataRegionId, k -> new ConcurrentHashMap<>()) .computeIfAbsent(pipeName, k -> new ConcurrentHashMap<>()) - .put(resource.getTsFilePath(), resource); + .putIfAbsent(resource.getTsFilePath(), resource); } public synchronized void eliminateProgressIndex( From e9a945d3d542ba2ff3e9c386a580fbbfa25adfeb Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 4 Jul 2025 17:19:11 +0800 Subject: [PATCH 118/185] rollback remaining --- .../PipeDataNodeSinglePipeMetrics.java | 23 +++++++------------ .../overview/PipeTsFileToTabletsMetrics.java | 3 ++- 2 files changed, 10 insertions(+), 16 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeSinglePipeMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeSinglePipeMetrics.java index 255ec5876202..9c19187932fc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeSinglePipeMetrics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeSinglePipeMetrics.java @@ -42,7 +42,6 @@ import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; public class PipeDataNodeSinglePipeMetrics implements IMetricSet { @@ -229,15 +228,12 @@ private void removeAutoGauge(final String pipeID) { public void register(final IoTDBDataRegionExtractor extractor) { // The metric is global thus the regionId is omitted final String pipeID = extractor.getPipeName() + "_" + extractor.getCreationTime(); - final AtomicBoolean first = new AtomicBoolean(false); remainingEventAndTimeOperatorMap.computeIfAbsent( pipeID, - k -> { - first.set(true); - return new PipeDataNodeRemainingEventAndTimeOperator( - extractor.getPipeName(), extractor.getCreationTime()); - }); - if (Objects.nonNull(metricService) && first.get()) { + k -> + new PipeDataNodeRemainingEventAndTimeOperator( + extractor.getPipeName(), extractor.getCreationTime())); + if (Objects.nonNull(metricService)) { createMetrics(pipeID); } } @@ -245,17 +241,14 @@ public void register(final IoTDBDataRegionExtractor extractor) { public void register(final IoTDBSchemaRegionExtractor extractor) { // The metric is global thus the regionId is omitted final String pipeID = extractor.getPipeName() + "_" + extractor.getCreationTime(); - final AtomicBoolean first = new AtomicBoolean(false); remainingEventAndTimeOperatorMap .computeIfAbsent( pipeID, - k -> { - first.set(true); - return new PipeDataNodeRemainingEventAndTimeOperator( - extractor.getPipeName(), extractor.getCreationTime()); - }) + k -> + new PipeDataNodeRemainingEventAndTimeOperator( + extractor.getPipeName(), extractor.getCreationTime())) .register(extractor); - if (Objects.nonNull(metricService) && first.get()) { + if (Objects.nonNull(metricService)) { createMetrics(pipeID); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeTsFileToTabletsMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeTsFileToTabletsMetrics.java index 16108b36c5f4..04015fe6f82d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeTsFileToTabletsMetrics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeTsFileToTabletsMetrics.java @@ -104,7 +104,8 @@ private void removeMetrics(final String pipeID) { public void register(final IoTDBDataRegionExtractor extractor) { final String pipeID = extractor.getPipeName() + "_" + extractor.getCreationTime(); - if (Objects.nonNull(metricService) && pipe.add(pipeID)) { + pipe.add(pipeID); + if (Objects.nonNull(metricService)) { createMetrics(pipeID); } } From a5d4319e8ec034133a11ac1241b795a0e36a0023 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 4 Jul 2025 18:16:00 +0800 Subject: [PATCH 119/185] revert hard-link --- ...PipeDataNodeHardlinkOrCopiedFileDirStartupCleaner.java | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/PipeDataNodeHardlinkOrCopiedFileDirStartupCleaner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/PipeDataNodeHardlinkOrCopiedFileDirStartupCleaner.java index 3afe5eb62e1a..b58d934988f1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/PipeDataNodeHardlinkOrCopiedFileDirStartupCleaner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/PipeDataNodeHardlinkOrCopiedFileDirStartupCleaner.java @@ -21,9 +21,9 @@ import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.resource.snapshot.PipeSnapshotResourceManager; -import org.apache.iotdb.commons.utils.FileUtils; import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.commons.io.FileUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -52,9 +52,7 @@ private static void cleanTsFileDir() { LOGGER.info( "Pipe hardlink dir found, deleting it: {}, result: {}", pipeHardLinkDir, - // For child directories, we need them to recover each pipe's progress - // Hence we do not delete them here, only delete the child files (older version) - FileUtils.deleteFileChildrenQuietly(pipeHardLinkDir)); + FileUtils.deleteQuietly(pipeHardLinkDir)); } } } @@ -67,7 +65,7 @@ private static void cleanSnapshotDir() { + PipeSnapshotResourceManager.PIPE_SNAPSHOT_DIR_NAME); if (pipeConsensusDir.isDirectory()) { LOGGER.info("Pipe snapshot dir found, deleting it: {},", pipeConsensusDir); - FileUtils.deleteFileOrDirectory(pipeConsensusDir); + org.apache.iotdb.commons.utils.FileUtils.deleteFileOrDirectory(pipeConsensusDir); } } From fcde95e781152af458185b3c4f84f135956e8578 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Fri, 4 Jul 2025 19:42:39 +0800 Subject: [PATCH 120/185] fix assigner --- .../PipeRealtimeDataRegionExtractor.java | 37 ++-- .../assigner/PipeDataRegionAssigner.java | 177 ++++++++++----- .../PipeInsertionDataNodeListener.java | 9 +- .../matcher/CachedSchemaPatternMatcher.java | 204 +++++++++++++----- .../matcher/PipeDataRegionMatcher.java | 52 +++++ .../iotdb/commons/conf/CommonConfig.java | 13 ++ .../iotdb/commons/pipe/config/PipeConfig.java | 5 + .../commons/pipe/config/PipeDescriptor.java | 8 + 8 files changed, 382 insertions(+), 123 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/matcher/PipeDataRegionMatcher.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java index 238b9b1ffede..bfafc55763d2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java @@ -41,7 +41,6 @@ import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.assigner.PipeTsFileEpochProgressIndexKeeper; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.listener.PipeInsertionDataNodeListener; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.listener.PipeTimePartitionListener; -import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.matcher.CachedSchemaPatternMatcher; import org.apache.iotdb.db.pipe.metric.source.PipeDataRegionEventCounter; import org.apache.iotdb.db.storageengine.StorageEngine; import org.apache.iotdb.db.storageengine.dataregion.DataRegion; @@ -323,7 +322,29 @@ public final void extract(final PipeRealtimeEvent event) { } } - doExtract(event); + // 1. Check if time parsing is necessary. If not, it means that the timestamps of the data + // contained in this event are definitely within the time range [start time, end time]. + // 2. Check if the event's data timestamps may intersect with the time range. If not, it means + // that the data timestamps of this event are definitely not within the time range. + // 3. Check if pattern parsing is necessary. If not, it means that the paths of the data + // contained in this event are definitely covered by the pattern. + // 4. Check if the event's data paths may intersect with the pattern. If not, it means that the + // data of this event is definitely not overlapped with the pattern. + if ((!event.shouldParseTime() || event.getEvent().mayEventTimeOverlappedWithTimeRange()) + && (!event.shouldParsePattern() || event.getEvent().mayEventPathsOverlappedWithPattern())) { + if (sloppyTimeRange) { + // only skip parsing time for events whose data timestamps may intersect with the time range + event.skipParsingTime(); + } + if (sloppyPattern) { + // only skip parsing pattern for events whose data paths may intersect with the pattern + event.skipParsingPattern(); + } + + doExtract(event); + } else { + event.decreaseReferenceCount(PipeRealtimeDataRegionExtractor.class.getName(), false); + } synchronized (isClosed) { if (isClosed.get()) { @@ -411,17 +432,7 @@ public Event supply() { } private PipeRealtimeEvent getNextRealtimeEvent() { - PipeRealtimeEvent realtimeEvent = (PipeRealtimeEvent) pendingQueue.directPoll(); - - while (realtimeEvent != null - && (!CachedSchemaPatternMatcher.match(realtimeEvent, this) - || !coarseFilterEvent(realtimeEvent))) { - realtimeEvent.decreaseReferenceCount( - PipeRealtimeDataRegionTsFileExtractor.class.getName(), false); - realtimeEvent = (PipeRealtimeEvent) pendingQueue.directPoll(); - } - - return realtimeEvent; + return (PipeRealtimeEvent) pendingQueue.directPoll(); } // This may require some time thus we leave it for processor thread instead of writing thread diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java index a955e9f25ac3..e85e6b7fcfba 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java @@ -19,106 +19,173 @@ package org.apache.iotdb.db.pipe.extractor.dataregion.realtime.assigner; +import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; +import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.event.ProgressReportEvent; +import org.apache.iotdb.commons.pipe.metric.PipeEventCounter; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEventFactory; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionExtractor; +import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.matcher.CachedSchemaPatternMatcher; +import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.matcher.PipeDataRegionMatcher; +import org.apache.iotdb.db.pipe.metric.source.PipeDataRegionEventCounter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.Closeable; -import java.util.Set; -import java.util.concurrent.CopyOnWriteArraySet; +import java.util.concurrent.atomic.AtomicReference; public class PipeDataRegionAssigner implements Closeable { private static final Logger LOGGER = LoggerFactory.getLogger(PipeDataRegionAssigner.class); + private static final PipeConfig PIPE_CONFIG = PipeConfig.getInstance(); + + /** + * The {@link PipeDataRegionMatcher} is used to match the event with the extractor based on the + * pattern. + */ + private final PipeDataRegionMatcher matcher; + + /** The {@link DisruptorQueue} is used to assign the event to the extractor. */ + private final DisruptorQueue disruptor; + private final String dataRegionId; - protected final Set extractors = new CopyOnWriteArraySet<>(); + + private final AtomicReference maxProgressIndexForRealtimeEvent = + new AtomicReference<>(MinimumProgressIndex.INSTANCE); + + private final PipeEventCounter eventCounter = new PipeDataRegionEventCounter(); public String getDataRegionId() { return dataRegionId; } public PipeDataRegionAssigner(final String dataRegionId) { + this.matcher = new CachedSchemaPatternMatcher(); + this.disruptor = new DisruptorQueue(this::assignToExtractor, this::onAssignedHook); this.dataRegionId = dataRegionId; } - public void assignToExtractor(final PipeRealtimeEvent event) { - if (event.getEvent() instanceof PipeHeartbeatEvent) { - ((PipeHeartbeatEvent) event.getEvent()).onPublished(); + public void publishToAssign(final PipeRealtimeEvent event) { + if (!event.increaseReferenceCount(PipeDataRegionAssigner.class.getName())) { + LOGGER.warn( + "The reference count of the realtime event {} cannot be increased, skipping it.", event); + return; + } + + final EnrichedEvent innerEvent = event.getEvent(); + eventCounter.increaseEventCount(innerEvent); + if (innerEvent instanceof PipeHeartbeatEvent) { + ((PipeHeartbeatEvent) innerEvent).onPublished(); + } + + // use synchronized here for completely preventing reference count leaks under extreme thread + // scheduling when closing + synchronized (this) { + if (!disruptor.isClosed()) { + disruptor.publish(event); + } else { + onAssignedHook(event); + } + } + } + + private void onAssignedHook(final PipeRealtimeEvent realtimeEvent) { + realtimeEvent.decreaseReferenceCount(PipeDataRegionAssigner.class.getName(), false); + + final EnrichedEvent innerEvent = realtimeEvent.getEvent(); + eventCounter.decreaseEventCount(innerEvent); + } + + private void assignToExtractor( + final PipeRealtimeEvent event, final long sequence, final boolean endOfBatch) { + if (disruptor.isClosed()) { + return; } - extractors.forEach( - extractor -> { - if (event.getEvent().isGeneratedByPipe() && !extractor.isForwardingPipeRequests()) { - final ProgressReportEvent reportEvent = - new ProgressReportEvent( - extractor.getPipeName(), - extractor.getCreationTime(), - extractor.getPipeTaskMeta(), - extractor.getPipePattern(), - extractor.getRealtimeDataExtractionStartTime(), - extractor.getRealtimeDataExtractionEndTime()); - reportEvent.bindProgressIndex(event.getProgressIndex()); - if (!reportEvent.increaseReferenceCount(PipeDataRegionAssigner.class.getName())) { - LOGGER.warn( - "The reference count of the event {} cannot be increased, skipping it.", - reportEvent); - return; - } - extractor.extract(PipeRealtimeEventFactory.createRealtimeEvent(reportEvent)); - return; - } - - final PipeRealtimeEvent copiedEvent = - event.shallowCopySelfAndBindPipeTaskMetaForProgressReport( - extractor.getPipeName(), - extractor.getCreationTime(), - extractor.getPipeTaskMeta(), - extractor.getPipePattern(), - extractor.getRealtimeDataExtractionStartTime(), - extractor.getRealtimeDataExtractionEndTime()); - final EnrichedEvent innerEvent = copiedEvent.getEvent(); - if (innerEvent instanceof PipeTsFileInsertionEvent) { - final PipeTsFileInsertionEvent tsFileInsertionEvent = - (PipeTsFileInsertionEvent) innerEvent; - tsFileInsertionEvent.disableMod4NonTransferPipes(extractor.isShouldTransferModFile()); - } - - if (!copiedEvent.increaseReferenceCount(PipeDataRegionAssigner.class.getName())) { - LOGGER.warn( - "The reference count of the event {} cannot be increased, skipping it.", - copiedEvent); - return; - } - extractor.extract(copiedEvent); - }); + matcher + .match(event) + .forEach( + extractor -> { + if (disruptor.isClosed()) { + return; + } + + if (event.getEvent().isGeneratedByPipe() && !extractor.isForwardingPipeRequests()) { + final ProgressReportEvent reportEvent = + new ProgressReportEvent( + extractor.getPipeName(), + extractor.getCreationTime(), + extractor.getPipeTaskMeta(), + extractor.getPipePattern(), + extractor.getRealtimeDataExtractionStartTime(), + extractor.getRealtimeDataExtractionEndTime()); + reportEvent.bindProgressIndex(event.getProgressIndex()); + if (!reportEvent.increaseReferenceCount(PipeDataRegionAssigner.class.getName())) { + LOGGER.warn( + "The reference count of the event {} cannot be increased, skipping it.", + reportEvent); + return; + } + extractor.extract(PipeRealtimeEventFactory.createRealtimeEvent(reportEvent)); + return; + } + + final PipeRealtimeEvent copiedEvent = + event.shallowCopySelfAndBindPipeTaskMetaForProgressReport( + extractor.getPipeName(), + extractor.getCreationTime(), + extractor.getPipeTaskMeta(), + extractor.getPipePattern(), + extractor.getRealtimeDataExtractionStartTime(), + extractor.getRealtimeDataExtractionEndTime()); + final EnrichedEvent innerEvent = copiedEvent.getEvent(); + if (innerEvent instanceof PipeTsFileInsertionEvent) { + final PipeTsFileInsertionEvent tsFileInsertionEvent = + (PipeTsFileInsertionEvent) innerEvent; + tsFileInsertionEvent.disableMod4NonTransferPipes( + extractor.isShouldTransferModFile()); + } + + if (!copiedEvent.increaseReferenceCount(PipeDataRegionAssigner.class.getName())) { + LOGGER.warn( + "The reference count of the event {} cannot be increased, skipping it.", + copiedEvent); + return; + } + extractor.extract(copiedEvent); + }); } public void startAssignTo(final PipeRealtimeDataRegionExtractor extractor) { - extractors.add(extractor); + matcher.register(extractor); } public void stopAssignTo(final PipeRealtimeDataRegionExtractor extractor) { - extractors.remove(extractor); + matcher.deregister(extractor); } public boolean notMoreExtractorNeededToBeAssigned() { - return extractors.isEmpty(); + return matcher.getRegisterCount() == 0; } + /** + * Clear the matcher and disruptor. The method {@link PipeDataRegionAssigner#publishToAssign} + * should not be used after calling this method. + */ @Override // use synchronized here for completely preventing reference count leaks under extreme thread // scheduling when closing public synchronized void close() { final long startTime = System.currentTimeMillis(); - extractors.clear(); + disruptor.shutdown(); + matcher.clear(); LOGGER.info( "Pipe: Assigner on data region {} shutdown internal disruptor within {} ms", dataRegionId, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java index 69a276b9d0a9..0769a54bbbab 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java @@ -110,7 +110,7 @@ public void listenToTsFile( return; } - assigner.assignToExtractor( + assigner.publishToAssign( PipeRealtimeEventFactory.createRealtimeEvent(tsFileResource, isLoaded)); } @@ -127,21 +127,20 @@ public void listenToInsertNode( return; } - assigner.assignToExtractor( + assigner.publishToAssign( PipeRealtimeEventFactory.createRealtimeEvent(insertNode, tsFileResource)); } public void listenToHeartbeat(boolean shouldPrintMessage) { dataRegionId2Assigner.forEach( (key, value) -> - value.assignToExtractor( + value.publishToAssign( PipeRealtimeEventFactory.createRealtimeEvent(key, shouldPrintMessage))); } public void listenToDeleteData(DeleteDataNode node) { dataRegionId2Assigner.forEach( - (key, value) -> - value.assignToExtractor(PipeRealtimeEventFactory.createRealtimeEvent(node))); + (key, value) -> value.publishToAssign(PipeRealtimeEventFactory.createRealtimeEvent(node))); } /////////////////////////////// singleton /////////////////////////////// diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/matcher/CachedSchemaPatternMatcher.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/matcher/CachedSchemaPatternMatcher.java index 30795c68dcff..015ece38f8bc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/matcher/CachedSchemaPatternMatcher.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/matcher/CachedSchemaPatternMatcher.java @@ -19,86 +19,190 @@ package org.apache.iotdb.db.pipe.extractor.dataregion.realtime.matcher; +import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.datastructure.pattern.PipePattern; -import org.apache.iotdb.commons.pipe.event.ProgressReportEvent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionExtractor; +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.HashSet; import java.util.Map; import java.util.Objects; +import java.util.Set; +import java.util.concurrent.CopyOnWriteArraySet; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.stream.Collectors; -public class CachedSchemaPatternMatcher { +public class CachedSchemaPatternMatcher implements PipeDataRegionMatcher { protected static final Logger LOGGER = LoggerFactory.getLogger(CachedSchemaPatternMatcher.class); - public static boolean match( - final PipeRealtimeEvent event, final PipeRealtimeDataRegionExtractor extractor) { - // HeartbeatEvent will be assigned to all extractors - if (event.getEvent() instanceof PipeHeartbeatEvent - || event.getEvent() instanceof ProgressReportEvent) { - return true; + protected final ReentrantReadWriteLock lock; + + protected final Set extractors; + protected final Cache> deviceToExtractorsCache; + + public CachedSchemaPatternMatcher() { + this.lock = new ReentrantReadWriteLock(); + // Should be thread-safe because the extractors will be returned by {@link #match} and + // iterated by {@link #assignToExtractor}, at the same time the extractors may be added or + // removed by {@link #register} and {@link #deregister}. + this.extractors = new CopyOnWriteArraySet<>(); + this.deviceToExtractorsCache = + Caffeine.newBuilder() + .maximumSize(PipeConfig.getInstance().getPipeExtractorMatcherCacheSize()) + .build(); + } + + @Override + public void register(final PipeRealtimeDataRegionExtractor extractor) { + lock.writeLock().lock(); + try { + extractors.add(extractor); + deviceToExtractorsCache.invalidateAll(); + } finally { + lock.writeLock().unlock(); } + } - // Deletion event will be assigned to extractors listened to it - if (event.getEvent() instanceof PipeSchemaRegionWritePlanEvent) { - return extractor.shouldExtractDeletion(); + @Override + public void deregister(final PipeRealtimeDataRegionExtractor extractor) { + lock.writeLock().lock(); + try { + extractors.remove(extractor); + deviceToExtractorsCache.invalidateAll(); + } finally { + lock.writeLock().unlock(); } + } - for (final Map.Entry entry : event.getSchemaInfo().entrySet()) { - final String device = entry.getKey(); - final String[] measurements = entry.getValue(); + @Override + public int getRegisterCount() { + lock.readLock().lock(); + try { + return extractors.size(); + } finally { + lock.readLock().unlock(); + } + } + + @Override + public Set match(final PipeRealtimeEvent event) { + final Set matchedExtractors = new HashSet<>(); + + lock.readLock().lock(); + try { + if (extractors.isEmpty()) { + return matchedExtractors; + } + + // HeartbeatEvent will be assigned to all extractors + if (event.getEvent() instanceof PipeHeartbeatEvent) { + return extractors; + } - if (!filterByDevice(device, extractor)) { - return false; + // Deletion event will be assigned to extractors listened to it + if (event.getEvent() instanceof PipeSchemaRegionWritePlanEvent) { + return extractors.stream() + .filter(PipeRealtimeDataRegionExtractor::shouldExtractDeletion) + .collect(Collectors.toSet()); } - // 2. filter matched candidate extractors by measurements - if (measurements.length == 0) { - // `measurements` is empty (only in case of tsfile event). match all extractors. - // - // case 1: the pattern can match all measurements of the device. - // in this case, the extractor can be matched without checking the measurements. - // - // case 2: the pattern may match some measurements of the device. - // in this case, we can't get all measurements efficiently here, - // so we just ASSUME the extractor matches and do more checks later. - return true; - } else { - final PipePattern pattern = extractor.getPipePattern(); - if (Objects.isNull(pattern) || pattern.isRoot() || pattern.coversDevice(device)) { - // The pattern can match all measurements of the device. - return true; + for (final Map.Entry entry : event.getSchemaInfo().entrySet()) { + final String device = entry.getKey(); + final String[] measurements = entry.getValue(); + + // 1. try to get matched extractors from cache, if not success, match them by device + final Set extractorsFilteredByDevice = + deviceToExtractorsCache.get(device, this::filterExtractorsByDevice); + // this would not happen + if (extractorsFilteredByDevice == null) { + LOGGER.warn("Match result NPE when handle device {}", device); + continue; + } + + // 2. filter matched candidate extractors by measurements + if (measurements.length == 0) { + // `measurements` is empty (only in case of tsfile event). match all extractors. + // + // case 1: the pattern can match all measurements of the device. + // in this case, the extractor can be matched without checking the measurements. + // + // case 2: the pattern may match some measurements of the device. + // in this case, we can't get all measurements efficiently here, + // so we just ASSUME the extractor matches and do more checks later. + matchedExtractors.addAll(extractorsFilteredByDevice); } else { - for (final String measurement : measurements) { - // Ignore null measurement for partial insert - if (measurement == null) { - continue; - } - - if (pattern.matchesMeasurement(device, measurement)) { - return true; - } - } + // `measurements` is not empty (only in case of tablet event). + // Match extractors by measurements. + extractorsFilteredByDevice.forEach( + extractor -> { + final PipePattern pattern = extractor.getPipePattern(); + if (Objects.isNull(pattern) || pattern.isRoot() || pattern.coversDevice(device)) { + // The pattern can match all measurements of the device. + matchedExtractors.add(extractor); + } else { + for (final String measurement : measurements) { + // Ignore null measurement for partial insert + if (measurement == null) { + continue; + } + + if (pattern.matchesMeasurement(device, measurement)) { + matchedExtractors.add(extractor); + // There would be no more matched extractors because the measurements are + // unique + break; + } + } + } + }); + } + + if (matchedExtractors.size() == extractors.size()) { + break; } } + } finally { + lock.readLock().unlock(); } - return false; + return matchedExtractors; } - private static boolean filterByDevice( - final String device, final PipeRealtimeDataRegionExtractor extractor) { - return extractor.shouldExtractInsertion() - && (Objects.isNull(extractor.getPipePattern()) - || extractor.getPipePattern().mayOverlapWithDevice(device)); + protected Set filterExtractorsByDevice(final String device) { + final Set filteredExtractors = new HashSet<>(); + + for (final PipeRealtimeDataRegionExtractor extractor : extractors) { + // Return if the extractor only extract deletion + if (!extractor.shouldExtractInsertion()) { + continue; + } + + final PipePattern pipePattern = extractor.getPipePattern(); + if (Objects.isNull(pipePattern) || pipePattern.mayOverlapWithDevice(device)) { + filteredExtractors.add(extractor); + } + } + + return filteredExtractors; } - private CachedSchemaPatternMatcher() { - // Utility class + @Override + public void clear() { + lock.writeLock().lock(); + try { + extractors.clear(); + deviceToExtractorsCache.invalidateAll(); + deviceToExtractorsCache.cleanUp(); + } finally { + lock.writeLock().unlock(); + } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/matcher/PipeDataRegionMatcher.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/matcher/PipeDataRegionMatcher.java new file mode 100644 index 000000000000..4e102a1f7cf6 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/matcher/PipeDataRegionMatcher.java @@ -0,0 +1,52 @@ +/* + * 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.pipe.extractor.dataregion.realtime.matcher; + +import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; +import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionExtractor; + +import java.util.Set; + +public interface PipeDataRegionMatcher { + + /** + * Register a extractor. If the extractor's pattern matches the event's schema info, the event + * will be assigned to the extractor. + */ + void register(PipeRealtimeDataRegionExtractor extractor); + + /** Deregister a extractor. */ + void deregister(PipeRealtimeDataRegionExtractor extractor); + + /** Get the number of registered extractors in this matcher. */ + int getRegisterCount(); + + /** + * Match the event's schema info with the registered extractors' patterns. If the event's schema + * info matches the pattern of a extractor, the extractor will be returned. + * + * @param event the event to be matched + * @return the matched extractors + */ + Set match(PipeRealtimeEvent event); + + /** Clear all the registered extractors and internal data structures. */ + void clear(); +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java index c21372158d21..5e0eb236ecb8 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java @@ -259,6 +259,7 @@ public class CommonConfig { private int pipeExtractorAssignerDisruptorRingBufferSize = 65536; private long pipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes = 50; // 50B + private long pipeExtractorMatcherCacheSize = 1024; private int pipeConnectorHandshakeTimeoutMs = 10 * 1000; // 10 seconds private int pipeConnectorTransferTimeoutMs = 15 * 60 * 1000; // 15 minutes @@ -1002,6 +1003,18 @@ public void setPipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes( pipeExtractorAssignerDisruptorRingBufferEntrySize); } + public long getPipeExtractorMatcherCacheSize() { + return pipeExtractorMatcherCacheSize; + } + + public void setPipeExtractorMatcherCacheSize(long pipeExtractorMatcherCacheSize) { + if (this.pipeExtractorMatcherCacheSize == pipeExtractorMatcherCacheSize) { + return; + } + this.pipeExtractorMatcherCacheSize = pipeExtractorMatcherCacheSize; + logger.info("pipeExtractorMatcherCacheSize is set to {}.", pipeExtractorMatcherCacheSize); + } + public int getPipeConnectorHandshakeTimeoutMs() { return pipeConnectorHandshakeTimeoutMs; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java index e768c67b371f..477056b0b0c2 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeConfig.java @@ -175,6 +175,10 @@ public long getPipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes() { return COMMON_CONFIG.getPipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes(); } + public long getPipeExtractorMatcherCacheSize() { + return COMMON_CONFIG.getPipeExtractorMatcherCacheSize(); + } + /////////////////////////////// Connector /////////////////////////////// public int getPipeConnectorHandshakeTimeoutMs() { @@ -496,6 +500,7 @@ public void printAllConfigs() { LOGGER.info( "PipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes: {}", getPipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes()); + LOGGER.info("PipeExtractorMatcherCacheSize: {}", getPipeExtractorMatcherCacheSize()); LOGGER.info("PipeConnectorHandshakeTimeoutMs: {}", getPipeConnectorHandshakeTimeoutMs()); LOGGER.info("PipeConnectorTransferTimeoutMs: {}", getPipeConnectorTransferTimeoutMs()); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java index fff467fa395f..0b53adba20f4 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java @@ -340,6 +340,14 @@ public static void loadPipeInternalConfig(CommonConfig config, TrimProperties pr config .getPipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes()))))); + config.setPipeExtractorMatcherCacheSize( + Integer.parseInt( + Optional.ofNullable(properties.getProperty("pipe_extractor_matcher_cache_size")) + .orElse( + properties.getProperty( + "pipe_extractor_matcher_cache_size", + String.valueOf(config.getPipeExtractorMatcherCacheSize()))))); + config.setPipeConnectorHandshakeTimeoutMs( Long.parseLong( Optional.ofNullable(properties.getProperty("pipe_sink_handshake_timeout_ms")) From 58bdce850171365d13c7ae8314bd6afdd533dcfb Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Sat, 5 Jul 2025 11:51:58 +0800 Subject: [PATCH 121/185] Fix null pointer and ClientPool exceptions --- .../protocol/thrift/async/IoTDBDataRegionAsyncConnector.java | 2 +- .../db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java index 62871f039fb9..502167f87385 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java @@ -408,7 +408,7 @@ private void transfer(final PipeTransferTsFileHandler pipeTransferTsFileHandler) AsyncPipeDataTransferServiceClient client = null; try { client = transferTsFileClientManager.borrowClient(); - pipeTransferTsFileHandler.transfer(clientManager, client); + pipeTransferTsFileHandler.transfer(transferTsFileClientManager, client); } catch (final Exception ex) { logOnClientException(client, ex); pipeTransferTsFileHandler.onError(ex); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java index a05a7be54359..5003dc95aa80 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java @@ -244,6 +244,10 @@ public long getTimePartitionId() { public boolean internallyIncreaseResourceReferenceCount(final String holderMessage) { extractTime = System.nanoTime(); try { + if (Objects.isNull(pipeName)) { + return true; + } + tsFile = PipeDataNodeResourceManager.tsfile().increaseFileReference(tsFile, true, pipeName); if (isWithMod) { modFile = From dbe3ed6fa7d698dc9cf0971ec626951943928ba5 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Sat, 5 Jul 2025 13:27:45 +0800 Subject: [PATCH 122/185] Fix null pointer when DecreaseResource --- .../db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java index 5003dc95aa80..89cc9cbffd9b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java @@ -272,6 +272,9 @@ public boolean internallyIncreaseResourceReferenceCount(final String holderMessa @Override public boolean internallyDecreaseResourceReferenceCount(final String holderMessage) { try { + if (pipeName == null) { + return true; + } PipeDataNodeResourceManager.tsfile().decreaseFileReference(tsFile, pipeName); if (isWithMod) { PipeDataNodeResourceManager.tsfile().decreaseFileReference(modFile, pipeName); From 2da92536c52621afdc8f6ba7c78c50c5f05f640f Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Sun, 6 Jul 2025 10:06:45 +0800 Subject: [PATCH 123/185] revert replace --- .../PipeRealtimePriorityBlockingQueue.java | 12 ++++++------ .../dataregion/tsfile/TsFileManager.java | 19 +++++++++++-------- 2 files changed, 17 insertions(+), 14 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java index 719f7ad4ea21..5373cc03694d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java @@ -72,7 +72,7 @@ public PipeRealtimePriorityBlockingQueue() { } @Override - public synchronized boolean directOffer(final Event event) { + public boolean directOffer(final Event event) { checkBeforeOffer(event); if (event instanceof TsFileInsertionEvent) { @@ -90,18 +90,18 @@ public synchronized boolean directOffer(final Event event) { } @Override - public synchronized boolean waitedOffer(final Event event) { + public boolean waitedOffer(final Event event) { return directOffer(event); } @Override - public synchronized boolean put(final Event event) { + public boolean put(final Event event) { directOffer(event); return true; } @Override - public synchronized Event directPoll() { + public Event directPoll() { Event event = null; final int pollHistoricalTsFileThreshold = PIPE_CONFIG.getPipeRealTimeQueuePollHistoricalTsFileThreshold(); @@ -146,7 +146,7 @@ public synchronized Event directPoll() { * available. */ @Override - public synchronized Event waitedPoll() { + public Event waitedPoll() { Event event = null; final int pollHistoricalTsFileThreshold = PIPE_CONFIG.getPipeRealTimeQueuePollHistoricalTsFileThreshold(); @@ -193,7 +193,7 @@ public synchronized Event waitedPoll() { } @Override - public synchronized Event peek() { + public Event peek() { final Event event = pendingQueue.peek(); if (Objects.nonNull(event)) { return event; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java index a9a9878627fb..fb7d33035768 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java @@ -278,14 +278,17 @@ public void replace( writeUnlock(); } - PipeDataNodeResourceManager.compaction() - .emitResult( - storageGroupName, - dataRegionId, - timePartition, - seqFileResources, - unseqFileResources, - targetFileResources); + // Currently disable + if (false) { + PipeDataNodeResourceManager.compaction() + .emitResult( + storageGroupName, + dataRegionId, + timePartition, + seqFileResources, + unseqFileResources, + targetFileResources); + } } public boolean contains(TsFileResource tsFileResource, boolean sequence) { From 64c07f86c6568912f6a190aa5bde59a43950b4d3 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Sun, 6 Jul 2025 10:15:52 +0800 Subject: [PATCH 124/185] Lock release --- .../db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java index 2d45a237e73c..9a1e082c92f2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java @@ -52,6 +52,8 @@ import org.apache.iotdb.db.queryengine.plan.Coordinator; import org.apache.iotdb.db.queryengine.plan.analyze.Analysis; import org.apache.iotdb.db.queryengine.plan.analyze.IPartitionFetcher; +import org.apache.iotdb.db.queryengine.plan.analyze.lock.DataNodeSchemaLockManager; +import org.apache.iotdb.db.queryengine.plan.analyze.lock.SchemaLockType; import org.apache.iotdb.db.queryengine.plan.analyze.schema.ISchemaFetcher; import org.apache.iotdb.db.queryengine.plan.analyze.schema.SchemaValidator; import org.apache.iotdb.db.queryengine.plan.execution.ExecutionResult; @@ -530,10 +532,13 @@ private Analysis executeTabletConversionOnException( final LoadTsFileDataTypeConverter loadTsFileDataTypeConverter = new LoadTsFileDataTypeConverter(isGeneratedByPipe); + DataNodeSchemaLockManager.getInstance().releaseReadLock(context); final TSStatus status = loadTsFileStatement.isConvertOnTypeMismatch() ? loadTsFileDataTypeConverter.convertForTreeModel(loadTsFileStatement).orElse(null) : null; + DataNodeSchemaLockManager.getInstance() + .takeReadLock(context, SchemaLockType.VALIDATE_VS_DELETION); if (status == null) { LOGGER.warn( From cabc774c822e314b695ed6c80b55b6b7831fdbe5 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Sun, 6 Jul 2025 10:45:33 +0800 Subject: [PATCH 125/185] fix-lock --- .../plan/analyze/load/LoadTsFileAnalyzer.java | 9 ++------- .../plan/scheduler/load/LoadTsFileScheduler.java | 2 +- .../load/converter/LoadTsFileDataTypeConverter.java | 12 +++++++++++- 3 files changed, 14 insertions(+), 9 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java index 9a1e082c92f2..df540b63371e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java @@ -52,8 +52,6 @@ import org.apache.iotdb.db.queryengine.plan.Coordinator; import org.apache.iotdb.db.queryengine.plan.analyze.Analysis; import org.apache.iotdb.db.queryengine.plan.analyze.IPartitionFetcher; -import org.apache.iotdb.db.queryengine.plan.analyze.lock.DataNodeSchemaLockManager; -import org.apache.iotdb.db.queryengine.plan.analyze.lock.SchemaLockType; import org.apache.iotdb.db.queryengine.plan.analyze.schema.ISchemaFetcher; import org.apache.iotdb.db.queryengine.plan.analyze.schema.SchemaValidator; import org.apache.iotdb.db.queryengine.plan.execution.ExecutionResult; @@ -406,7 +404,7 @@ private boolean handleSingleMiniFile(final int i) throws FileNotFoundException { final long startTime = System.nanoTime(); try { final LoadTsFileDataTypeConverter loadTsFileDataTypeConverter = - new LoadTsFileDataTypeConverter(isGeneratedByPipe); + new LoadTsFileDataTypeConverter(context, isGeneratedByPipe); final TSStatus status = loadTsFileDataTypeConverter @@ -531,14 +529,11 @@ private Analysis executeTabletConversionOnException( } final LoadTsFileDataTypeConverter loadTsFileDataTypeConverter = - new LoadTsFileDataTypeConverter(isGeneratedByPipe); - DataNodeSchemaLockManager.getInstance().releaseReadLock(context); + new LoadTsFileDataTypeConverter(context, isGeneratedByPipe); final TSStatus status = loadTsFileStatement.isConvertOnTypeMismatch() ? loadTsFileDataTypeConverter.convertForTreeModel(loadTsFileStatement).orElse(null) : null; - DataNodeSchemaLockManager.getInstance() - .takeReadLock(context, SchemaLockType.VALIDATE_VS_DELETION); if (status == null) { LOGGER.warn( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileScheduler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileScheduler.java index 59aa57ffd4e0..65eb21beee24 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileScheduler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileScheduler.java @@ -536,7 +536,7 @@ private boolean loadLocally(LoadSingleTsFileNode node) throws IoTDBException { private void convertFailedTsFilesToTabletsAndRetry() { final LoadTsFileDataTypeConverter loadTsFileDataTypeConverter = - new LoadTsFileDataTypeConverter(isGeneratedByPipe); + new LoadTsFileDataTypeConverter(queryContext, isGeneratedByPipe); final Iterator iterator = failedTsFileNodeIndexes.listIterator(); while (iterator.hasNext()) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/converter/LoadTsFileDataTypeConverter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/converter/LoadTsFileDataTypeConverter.java index ee2a8fe25475..81a1f0639beb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/converter/LoadTsFileDataTypeConverter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/converter/LoadTsFileDataTypeConverter.java @@ -22,8 +22,11 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.protocol.session.SessionManager; +import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.plan.Coordinator; import org.apache.iotdb.db.queryengine.plan.analyze.ClusterPartitionFetcher; +import org.apache.iotdb.db.queryengine.plan.analyze.lock.DataNodeSchemaLockManager; +import org.apache.iotdb.db.queryengine.plan.analyze.lock.SchemaLockType; import org.apache.iotdb.db.queryengine.plan.analyze.schema.ClusterSchemaFetcher; import org.apache.iotdb.db.queryengine.plan.statement.Statement; import org.apache.iotdb.db.queryengine.plan.statement.crud.LoadTsFileStatement; @@ -47,16 +50,20 @@ public class LoadTsFileDataTypeConverter { STATEMENT_EXCEPTION_VISITOR = new LoadConvertedInsertTabletStatementExceptionVisitor(); private final boolean isGeneratedByPipe; + private final MPPQueryContext context; private final LoadTreeStatementDataTypeConvertExecutionVisitor treeStatementDataTypeConvertExecutionVisitor = new LoadTreeStatementDataTypeConvertExecutionVisitor(this::executeForTreeModel); - public LoadTsFileDataTypeConverter(final boolean isGeneratedByPipe) { + public LoadTsFileDataTypeConverter( + final MPPQueryContext context, final boolean isGeneratedByPipe) { + this.context = context; this.isGeneratedByPipe = isGeneratedByPipe; } public Optional convertForTreeModel(final LoadTsFileStatement loadTsFileTreeStatement) { + DataNodeSchemaLockManager.getInstance().releaseReadLock(context); try { return loadTsFileTreeStatement.accept(treeStatementDataTypeConvertExecutionVisitor, null); } catch (Exception e) { @@ -64,6 +71,9 @@ public Optional convertForTreeModel(final LoadTsFileStatement loadTsFi "Failed to convert data types for tree model statement {}.", loadTsFileTreeStatement, e); return Optional.of( new TSStatus(TSStatusCode.LOAD_FILE_ERROR.getStatusCode()).setMessage(e.getMessage())); + } finally { + DataNodeSchemaLockManager.getInstance() + .takeReadLock(context, SchemaLockType.VALIDATE_VS_DELETION); } } From cbdcbc6cd5dc90f205ea1576986a20f66ab397fc Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Sun, 6 Jul 2025 14:12:49 +0800 Subject: [PATCH 126/185] Update PipeDataRegionAssigner.java continue fix partial Create PipeAssignerMetrics.java Update PipeDataRegionAssigner.java Update PipeHeartbeatEvent.java --- .../common/heartbeat/PipeHeartbeatEvent.java | 33 +++- .../assigner/PipeDataRegionAssigner.java | 20 +++ .../db/pipe/metric/PipeDataNodeMetrics.java | 3 + .../metric/source/PipeAssignerMetrics.java | 165 ++++++++++++++++++ 4 files changed, 213 insertions(+), 8 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/source/PipeAssignerMetrics.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/heartbeat/PipeHeartbeatEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/heartbeat/PipeHeartbeatEvent.java index 6860847baf1a..ad29b2854424 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/heartbeat/PipeHeartbeatEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/heartbeat/PipeHeartbeatEvent.java @@ -43,6 +43,7 @@ public class PipeHeartbeatEvent extends EnrichedEvent { private final String dataRegionId; private long timePublished; + private long timeAssigned; private long timeProcessed; private long timeTransferred; @@ -150,12 +151,22 @@ public void onPublished() { } } + public void onAssigned() { + if (shouldPrintMessage) { + timeAssigned = System.currentTimeMillis(); + if (timePublished != 0) { + PipeHeartbeatEventMetrics.getInstance() + .recordPublishedToAssignedTime(timeAssigned - timePublished); + } + } + } + public void onProcessed() { if (shouldPrintMessage) { timeProcessed = System.currentTimeMillis(); - if (timePublished != 0) { + if (timeAssigned != 0) { PipeHeartbeatEventMetrics.getInstance() - .recordAssignedToProcessedTime(timeProcessed - timePublished); + .recordAssignedToProcessedTime(timeProcessed - timeAssigned); } } } @@ -212,9 +223,13 @@ public String toString() { (timePublished != 0) ? DateTimeUtils.convertLongToDate(timePublished, "ms") : unknownMessage; + final String publishedToAssignedMessage = + (timeAssigned != 0 && timePublished != 0) + ? (timeAssigned - timePublished) + "ms" + : unknownMessage; final String assignedToProcessedMessage = - (timeProcessed != 0 && timePublished != 0) - ? (timeProcessed - timePublished) + "ms" + (timeProcessed != 0 && timeAssigned != 0) + ? (timeProcessed - timeAssigned) + "ms" : unknownMessage; final String processedToTransferredMessage = (timeTransferred != 0 && timeProcessed != 0) @@ -228,11 +243,11 @@ public String toString() { final String disruptorSizeMessage = Integer.toString(disruptorSize); final String extractorQueueTabletSizeMessage = - timePublished != 0 ? Integer.toString(extractorQueueTabletSize) : unknownMessage; + timeAssigned != 0 ? Integer.toString(extractorQueueTabletSize) : unknownMessage; final String extractorQueueTsFileSizeMessage = - timePublished != 0 ? Integer.toString(extractorQueueTsFileSize) : unknownMessage; + timeAssigned != 0 ? Integer.toString(extractorQueueTsFileSize) : unknownMessage; final String extractorQueueSizeMessage = - timePublished != 0 ? Integer.toString(extractorQueueSize) : unknownMessage; + timeAssigned != 0 ? Integer.toString(extractorQueueSize) : unknownMessage; final String connectorQueueTabletSizeMessage = timeProcessed != 0 ? Integer.toString(connectorQueueTabletSize) : unknownMessage; @@ -248,7 +263,9 @@ public String toString() { + dataRegionId + ", startTime=" + startTimeMessage - + ", publishedToProcessed=" + + ", publishedToAssigned=" + + publishedToAssignedMessage + + ", assignedToProcessed=" + assignedToProcessedMessage + ", processedToTransferred=" + processedToTransferredMessage diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java index e85e6b7fcfba..304e0833e6c3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java @@ -32,6 +32,7 @@ import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionExtractor; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.matcher.CachedSchemaPatternMatcher; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.matcher.PipeDataRegionMatcher; +import org.apache.iotdb.db.pipe.metric.source.PipeAssignerMetrics; import org.apache.iotdb.db.pipe.metric.source.PipeDataRegionEventCounter; import org.slf4j.Logger; @@ -70,6 +71,7 @@ public PipeDataRegionAssigner(final String dataRegionId) { this.matcher = new CachedSchemaPatternMatcher(); this.disruptor = new DisruptorQueue(this::assignToExtractor, this::onAssignedHook); this.dataRegionId = dataRegionId; + PipeAssignerMetrics.getInstance().register(this); } public void publishToAssign(final PipeRealtimeEvent event) { @@ -100,6 +102,10 @@ private void onAssignedHook(final PipeRealtimeEvent realtimeEvent) { realtimeEvent.decreaseReferenceCount(PipeDataRegionAssigner.class.getName(), false); final EnrichedEvent innerEvent = realtimeEvent.getEvent(); + if (innerEvent instanceof PipeHeartbeatEvent) { + ((PipeHeartbeatEvent) innerEvent).onAssigned(); + } + eventCounter.decreaseEventCount(innerEvent); } @@ -183,6 +189,8 @@ public boolean notMoreExtractorNeededToBeAssigned() { // use synchronized here for completely preventing reference count leaks under extreme thread // scheduling when closing public synchronized void close() { + PipeAssignerMetrics.getInstance().deregister(dataRegionId); + final long startTime = System.currentTimeMillis(); disruptor.shutdown(); matcher.clear(); @@ -191,4 +199,16 @@ public synchronized void close() { dataRegionId, System.currentTimeMillis() - startTime); } + + public int getTabletInsertionEventCount() { + return eventCounter.getTabletInsertionEventCount(); + } + + public int getTsFileInsertionEventCount() { + return eventCounter.getTsFileInsertionEventCount(); + } + + public int getPipeHeartbeatEventCount() { + return eventCounter.getPipeHeartbeatEventCount(); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeMetrics.java index f5a48b3f6b69..d7f1577a85c6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeMetrics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/PipeDataNodeMetrics.java @@ -30,6 +30,7 @@ import org.apache.iotdb.db.pipe.metric.schema.PipeSchemaRegionExtractorMetrics; import org.apache.iotdb.db.pipe.metric.schema.PipeSchemaRegionListenerMetrics; import org.apache.iotdb.db.pipe.metric.sink.PipeDataRegionConnectorMetrics; +import org.apache.iotdb.db.pipe.metric.source.PipeAssignerMetrics; import org.apache.iotdb.db.pipe.metric.source.PipeDataRegionExtractorMetrics; import org.apache.iotdb.metrics.AbstractMetricService; import org.apache.iotdb.metrics.metricsets.IMetricSet; @@ -40,6 +41,7 @@ public class PipeDataNodeMetrics implements IMetricSet { @Override public void bindTo(final AbstractMetricService metricService) { + PipeAssignerMetrics.getInstance().bindTo(metricService); PipeDataRegionExtractorMetrics.getInstance().bindTo(metricService); PipeProcessorMetrics.getInstance().bindTo(metricService); PipeDataRegionConnectorMetrics.getInstance().bindTo(metricService); @@ -56,6 +58,7 @@ public void bindTo(final AbstractMetricService metricService) { @Override public void unbindFrom(final AbstractMetricService metricService) { + PipeAssignerMetrics.getInstance().unbindFrom(metricService); PipeDataRegionExtractorMetrics.getInstance().unbindFrom(metricService); PipeProcessorMetrics.getInstance().unbindFrom(metricService); PipeDataRegionConnectorMetrics.getInstance().unbindFrom(metricService); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/source/PipeAssignerMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/source/PipeAssignerMetrics.java new file mode 100644 index 000000000000..3a35305584f6 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/source/PipeAssignerMetrics.java @@ -0,0 +1,165 @@ +/* + * 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.pipe.metric.source; + +import org.apache.iotdb.commons.service.metric.enums.Metric; +import org.apache.iotdb.commons.service.metric.enums.Tag; +import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.assigner.PipeDataRegionAssigner; +import org.apache.iotdb.metrics.AbstractMetricService; +import org.apache.iotdb.metrics.metricsets.IMetricSet; +import org.apache.iotdb.metrics.utils.MetricLevel; +import org.apache.iotdb.metrics.utils.MetricType; + +import com.google.common.collect.ImmutableSet; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +public class PipeAssignerMetrics implements IMetricSet { + + private static final Logger LOGGER = LoggerFactory.getLogger(PipeAssignerMetrics.class); + + private AbstractMetricService metricService; + + private final Map assignerMap = new HashMap<>(); + + //////////////////////////// bindTo & unbindFrom (metric framework) //////////////////////////// + + @Override + public void bindTo(AbstractMetricService metricService) { + this.metricService = metricService; + synchronized (this) { + for (String dataRegionId : assignerMap.keySet()) { + createMetrics(dataRegionId); + } + } + } + + private void createMetrics(String dataRegionId) { + createAutoGauge(dataRegionId); + } + + private void createAutoGauge(String dataRegionId) { + metricService.createAutoGauge( + Metric.UNASSIGNED_HEARTBEAT_COUNT.toString(), + MetricLevel.IMPORTANT, + assignerMap.get(dataRegionId), + PipeDataRegionAssigner::getPipeHeartbeatEventCount, + Tag.REGION.toString(), + dataRegionId); + metricService.createAutoGauge( + Metric.UNASSIGNED_TABLET_COUNT.toString(), + MetricLevel.IMPORTANT, + assignerMap.get(dataRegionId), + PipeDataRegionAssigner::getTabletInsertionEventCount, + Tag.REGION.toString(), + dataRegionId); + metricService.createAutoGauge( + Metric.UNASSIGNED_TSFILE_COUNT.toString(), + MetricLevel.IMPORTANT, + assignerMap.get(dataRegionId), + PipeDataRegionAssigner::getTsFileInsertionEventCount, + Tag.REGION.toString(), + dataRegionId); + } + + @Override + public void unbindFrom(AbstractMetricService metricService) { + ImmutableSet dataRegionIds = ImmutableSet.copyOf(assignerMap.keySet()); + for (String dataRegionId : dataRegionIds) { + deregister(dataRegionId); + } + if (!assignerMap.isEmpty()) { + LOGGER.warn("Failed to unbind from pipe assigner metrics, assigner map not empty"); + } + } + + private void removeMetrics(String dataRegionId) { + removeAutoGauge(dataRegionId); + } + + private void removeAutoGauge(String dataRegionId) { + metricService.remove( + MetricType.AUTO_GAUGE, + Metric.UNASSIGNED_HEARTBEAT_COUNT.toString(), + Tag.REGION.toString(), + dataRegionId); + metricService.remove( + MetricType.AUTO_GAUGE, + Metric.UNASSIGNED_TABLET_COUNT.toString(), + Tag.REGION.toString(), + dataRegionId); + metricService.remove( + MetricType.AUTO_GAUGE, + Metric.UNASSIGNED_TSFILE_COUNT.toString(), + Tag.REGION.toString(), + dataRegionId); + } + + //////////////////////////// register & deregister (pipe integration) //////////////////////////// + + public void register(@NonNull PipeDataRegionAssigner pipeDataRegionAssigner) { + String dataRegionId = pipeDataRegionAssigner.getDataRegionId(); + synchronized (this) { + assignerMap.putIfAbsent(dataRegionId, pipeDataRegionAssigner); + if (Objects.nonNull(metricService)) { + createMetrics(dataRegionId); + } + } + } + + public void deregister(String dataRegionId) { + synchronized (this) { + if (!assignerMap.containsKey(dataRegionId)) { + LOGGER.warn( + "Failed to deregister pipe assigner metrics, PipeDataRegionAssigner({}) does not exist", + dataRegionId); + return; + } + if (Objects.nonNull(metricService)) { + removeMetrics(dataRegionId); + } + assignerMap.remove(dataRegionId); + } + } + + //////////////////////////// singleton //////////////////////////// + + private static class PipeAssignerMetricsHolder { + + private static final PipeAssignerMetrics INSTANCE = new PipeAssignerMetrics(); + + private PipeAssignerMetricsHolder() { + // empty constructor + } + } + + public static PipeAssignerMetrics getInstance() { + return PipeAssignerMetrics.PipeAssignerMetricsHolder.INSTANCE; + } + + private PipeAssignerMetrics() { + // empty constructor + } +} From 79fca84a0d402c9ba7131c73dafd6a1744c6cddb Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Mon, 7 Jul 2025 18:13:37 +0800 Subject: [PATCH 127/185] Fix OOM caused by deleting gcSchemaInfo --- .../iotdb/db/pipe/event/realtime/PipeRealtimeEvent.java | 8 +++++++- .../realtime/assigner/PipeDataRegionAssigner.java | 1 + 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEvent.java index 65525ffe76ac..d4a39193467a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEvent.java @@ -84,6 +84,10 @@ public Map getSchemaInfo() { return device2Measurements; } + public void gcSchemaInfo() { + device2Measurements = null; + } + public boolean mayExtractorUseTablets(final PipeRealtimeDataRegionExtractor extractor) { final TsFileEpoch.State state = tsFileEpoch.getState(extractor); return state.equals(TsFileEpoch.State.EMPTY) || state.equals(TsFileEpoch.State.USING_TABLET); @@ -178,7 +182,9 @@ public PipeRealtimeEvent shallowCopySelfAndBindPipeTaskMetaForProgressReport( event.shallowCopySelfAndBindPipeTaskMetaForProgressReport( pipeName, creationTime, pipeTaskMeta, pattern, startTime, endTime), this.tsFileEpoch, - this.device2Measurements, + // device2Measurements is not used anymore, so it is not copied. + // If null is not passed, the field will not be GCed and may cause OOM. + null, pipeTaskMeta, pattern, startTime, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java index 304e0833e6c3..8e415772438c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java @@ -99,6 +99,7 @@ public void publishToAssign(final PipeRealtimeEvent event) { } private void onAssignedHook(final PipeRealtimeEvent realtimeEvent) { + realtimeEvent.gcSchemaInfo(); realtimeEvent.decreaseReferenceCount(PipeDataRegionAssigner.class.getName(), false); final EnrichedEvent innerEvent = realtimeEvent.getEvent(); From a4b33458db7061fd384d68916bd909f525abf7a3 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 8 Jul 2025 10:45:45 +0800 Subject: [PATCH 128/185] Remove some useless --- .../PipeRealtimeDataRegionLogExtractor.java | 66 +++--- .../consensus/index/ProgressIndexType.java | 8 +- .../index/impl/SegmentProgressIndex.java | 193 ------------------ 3 files changed, 44 insertions(+), 223 deletions(-) delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SegmentProgressIndex.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionLogExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionLogExtractor.java index 74d5c371dcee..4b300355c80e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionLogExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionLogExtractor.java @@ -82,7 +82,10 @@ private void extractTabletInsertion(PipeRealtimeEvent event) { private void extractTsFileInsertion(PipeRealtimeEvent event) { final PipeTsFileInsertionEvent tsFileInsertionEvent = (PipeTsFileInsertionEvent) event.getEvent(); - if (!tsFileInsertionEvent.isLoaded()) { + if (!(tsFileInsertionEvent.isLoaded() + // some insert nodes in the tsfile epoch are not captured by pipe + || tsFileInsertionEvent.getFileStartTime() + < event.getTsFileEpoch().getInsertNodeMinTime())) { // All data in the tsfile epoch has been extracted in tablet mode, so we should // simply ignore this event. event.decreaseReferenceCount(PipeRealtimeDataRegionLogExtractor.class.getName(), false); @@ -120,29 +123,46 @@ public boolean isNeedListenToInsertNode() { } @Override - protected Event doSupply(final PipeRealtimeEvent realtimeEvent) { - if (realtimeEvent.getEvent() instanceof PipeHeartbeatEvent) { - return supplyHeartbeat(realtimeEvent); - } else if (realtimeEvent.getEvent() instanceof PipeSchemaRegionWritePlanEvent - || realtimeEvent.getEvent() instanceof ProgressReportEvent) { - return supplyDirectly(realtimeEvent); - } else if (realtimeEvent.increaseReferenceCount( - PipeRealtimeDataRegionLogExtractor.class.getName())) { - return realtimeEvent.getEvent(); - } else { - // if the event's reference count can not be increased, it means the data represented by - // this event is not reliable anymore. the data has been lost. we simply discard this event - // and report the exception to PipeRuntimeAgent. - final String errorMessage = - String.format( - "Event %s can not be supplied because " - + "the reference count can not be increased, " - + "the data represented by this event is lost", - realtimeEvent.getEvent()); - LOGGER.error(errorMessage); - PipeDataNodeAgent.runtime() - .report(pipeTaskMeta, new PipeRuntimeNonCriticalException(errorMessage)); + public Event supply() { + PipeRealtimeEvent realtimeEvent = (PipeRealtimeEvent) pendingQueue.directPoll(); + + while (realtimeEvent != null) { + Event suppliedEvent = null; + + if (realtimeEvent.getEvent() instanceof PipeHeartbeatEvent) { + suppliedEvent = supplyHeartbeat(realtimeEvent); + } else if (realtimeEvent.getEvent() instanceof PipeSchemaRegionWritePlanEvent + || realtimeEvent.getEvent() instanceof ProgressReportEvent) { + suppliedEvent = supplyDirectly(realtimeEvent); + } else if (realtimeEvent.increaseReferenceCount( + PipeRealtimeDataRegionLogExtractor.class.getName())) { + suppliedEvent = realtimeEvent.getEvent(); + } else { + // if the event's reference count can not be increased, it means the data represented by + // this event is not reliable anymore. the data has been lost. we simply discard this event + // and report the exception to PipeRuntimeAgent. + final String errorMessage = + String.format( + "Event %s can not be supplied because " + + "the reference count can not be increased, " + + "the data represented by this event is lost", + realtimeEvent.getEvent()); + LOGGER.error(errorMessage); + PipeDataNodeAgent.runtime() + .report(pipeTaskMeta, new PipeRuntimeNonCriticalException(errorMessage)); + } + + realtimeEvent.decreaseReferenceCount( + PipeRealtimeDataRegionLogExtractor.class.getName(), false); + + if (suppliedEvent != null) { + return suppliedEvent; + } + + realtimeEvent = (PipeRealtimeEvent) pendingQueue.directPoll(); } + + // means the pending queue is empty. return null; } } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/ProgressIndexType.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/ProgressIndexType.java index d4b0f4ff18ba..58548e18c4b6 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/ProgressIndexType.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/ProgressIndexType.java @@ -24,7 +24,6 @@ import org.apache.iotdb.commons.consensus.index.impl.MetaProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.RecoverProgressIndex; -import org.apache.iotdb.commons.consensus.index.impl.SegmentProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.SimpleProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.StateProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.TimeWindowStateProgressIndex; @@ -44,8 +43,7 @@ public enum ProgressIndexType { HYBRID_PROGRESS_INDEX((short) 5), META_PROGRESS_INDEX((short) 6), TIME_WINDOW_STATE_PROGRESS_INDEX((short) 7), - STATE_PROGRESS_INDEX((short) 8), - SEGMENT_PROGRESS_INDEX((short) 9); + STATE_PROGRESS_INDEX((short) 8); private final short type; @@ -84,8 +82,6 @@ public static ProgressIndex deserializeFrom(ByteBuffer byteBuffer) { return TimeWindowStateProgressIndex.deserializeFrom(byteBuffer); case 8: return StateProgressIndex.deserializeFrom(byteBuffer); - case 9: - return SegmentProgressIndex.deserializeFrom(byteBuffer); default: throw new UnsupportedOperationException( String.format("Unsupported progress index type %s.", indexType)); @@ -111,8 +107,6 @@ public static ProgressIndex deserializeFrom(InputStream stream) throws IOExcepti return TimeWindowStateProgressIndex.deserializeFrom(stream); case 8: return StateProgressIndex.deserializeFrom(stream); - case 9: - return SegmentProgressIndex.deserializeFrom(stream); default: throw new UnsupportedOperationException( String.format("Unsupported progress index type %s.", indexType)); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SegmentProgressIndex.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SegmentProgressIndex.java deleted file mode 100644 index 5309e5439a85..000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SegmentProgressIndex.java +++ /dev/null @@ -1,193 +0,0 @@ -/* - * 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.consensus.index.impl; - -import org.apache.iotdb.commons.consensus.index.ProgressIndex; -import org.apache.iotdb.commons.consensus.index.ProgressIndexType; - -import org.apache.tsfile.utils.Pair; -import org.apache.tsfile.utils.RamUsageEstimator; -import org.apache.tsfile.utils.ReadWriteIOUtils; - -import javax.annotation.Nonnull; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.Objects; - -/** - * {@link SegmentProgressIndex} is a usual {@link ProgressIndex} with broken segments allowed. An - * {@link org.apache.iotdb.pipe.api.event.Event} is sent if its {@link ProgressIndex} <= the {@link - * #latestProgressIndex} and none of the {@link #brokenProgressIndexes}es has its {@link Pair#left} - * <= its {@link ProgressIndex} < {@link Pair#right}. If the {@link #brokenProgressIndexes} {@link - * List#isEmpty()}, the {@link ProgressIndex} behave just like the {@link #latestProgressIndex}. It - * is only used in the realtime data region extractor's {@link - * org.apache.iotdb.commons.pipe.agent.task.meta.PipeTaskMeta} to handle downgrading, and will never - * be in the insertNodes or tsFiles. - */ -public class SegmentProgressIndex extends ProgressIndex { - private static final long INSTANCE_SIZE = - RamUsageEstimator.shallowSizeOfInstance(SegmentProgressIndex.class); - public static final long LIST_SIZE = RamUsageEstimator.shallowSizeOfInstance(ArrayList.class); - public static final long PAIR_SIZE = RamUsageEstimator.shallowSizeOfInstance(Pair.class); - private ProgressIndex latestProgressIndex = MinimumProgressIndex.INSTANCE; - - // of the downgraded segments - private final LinkedList> brokenProgressIndexes = - new LinkedList<>(); - - public void recordStart(final ProgressIndex index) { - brokenProgressIndexes.add(new Pair<>(index, null)); - } - - public void recordEnd(final ProgressIndex index) { - brokenProgressIndexes.getLast().setRight(index); - } - - public void eliminate(final ProgressIndex index) { - final Iterator> iterator = brokenProgressIndexes.iterator(); - while (iterator.hasNext()) { - if (index.equals(iterator.next().getRight())) { - iterator.remove(); - return; - } - } - } - - @Override - public void serialize(final ByteBuffer byteBuffer) { - ProgressIndexType.SEGMENT_PROGRESS_INDEX.serialize(byteBuffer); - - latestProgressIndex.serialize(byteBuffer); - ReadWriteIOUtils.write(brokenProgressIndexes.size(), byteBuffer); - for (final Pair index : brokenProgressIndexes) { - index.getLeft().serialize(byteBuffer); - index.getRight().serialize(byteBuffer); - } - } - - @Override - public void serialize(final OutputStream stream) throws IOException { - ProgressIndexType.SEGMENT_PROGRESS_INDEX.serialize(stream); - - latestProgressIndex.serialize(stream); - ReadWriteIOUtils.write(brokenProgressIndexes.size(), stream); - for (final Pair index : brokenProgressIndexes) { - index.getLeft().serialize(stream); - index.getRight().serialize(stream); - } - } - - @Override - public boolean isAfter(final @Nonnull ProgressIndex progressIndex) { - return latestProgressIndex.isAfter(progressIndex) - && brokenProgressIndexes.stream() - .noneMatch( - pair -> - pair.getRight().isAfter(progressIndex) - && (progressIndex.isAfter(pair.getLeft()) - || progressIndex.equals(pair.getLeft()))); - } - - @Override - public boolean equals(final ProgressIndex progressIndex) { - if (progressIndex == null) { - return false; - } - if (this == progressIndex) { - return true; - } - if (progressIndex instanceof SegmentProgressIndex) { - final SegmentProgressIndex that = (SegmentProgressIndex) progressIndex; - return this.latestProgressIndex.equals(that.latestProgressIndex) - && this.brokenProgressIndexes.equals(that.brokenProgressIndexes); - } - return this.latestProgressIndex.equals(progressIndex); - } - - @Override - public ProgressIndex updateToMinimumEqualOrIsAfterProgressIndex( - final ProgressIndex progressIndex) { - return latestProgressIndex.updateToMinimumEqualOrIsAfterProgressIndex(progressIndex); - } - - @Override - public ProgressIndexType getType() { - return ProgressIndexType.SEGMENT_PROGRESS_INDEX; - } - - @Override - public TotalOrderSumTuple getTotalOrderSumTuple() { - throw new UnsupportedOperationException( - "This progressIndex is not for tsFile and shall never be used to sort resources"); - } - - @Override - public long ramBytesUsed() { - return INSTANCE_SIZE - + latestProgressIndex.ramBytesUsed() - + shallowSizeOfList(brokenProgressIndexes) - + PAIR_SIZE * brokenProgressIndexes.size() - + brokenProgressIndexes.stream() - .mapToLong(index -> index.getLeft().ramBytesUsed() + index.getRight().ramBytesUsed()) - .reduce(0L, Long::sum); - } - - public static SegmentProgressIndex deserializeFrom(final ByteBuffer byteBuffer) { - final SegmentProgressIndex segmentProgressIndex = new SegmentProgressIndex(); - segmentProgressIndex.latestProgressIndex = ProgressIndexType.deserializeFrom(byteBuffer); - final int size = ReadWriteIOUtils.readInt(byteBuffer); - for (int i = 0; i < size; i++) { - segmentProgressIndex.brokenProgressIndexes.add( - new Pair<>( - ProgressIndexType.deserializeFrom(byteBuffer), - ProgressIndexType.deserializeFrom(byteBuffer))); - } - return segmentProgressIndex; - } - - public static SegmentProgressIndex deserializeFrom(final InputStream stream) throws IOException { - final SegmentProgressIndex segmentProgressIndex = new SegmentProgressIndex(); - segmentProgressIndex.latestProgressIndex = ProgressIndexType.deserializeFrom(stream); - final int size = ReadWriteIOUtils.readInt(stream); - for (int i = 0; i < size; i++) { - segmentProgressIndex.brokenProgressIndexes.add( - new Pair<>( - ProgressIndexType.deserializeFrom(stream), - ProgressIndexType.deserializeFrom(stream))); - } - return segmentProgressIndex; - } - - private long shallowSizeOfList(final List list) { - return Objects.nonNull(list) - ? SegmentProgressIndex.LIST_SIZE - + RamUsageEstimator.alignObjectSize( - RamUsageEstimator.NUM_BYTES_ARRAY_HEADER - + (long) RamUsageEstimator.NUM_BYTES_OBJECT_REF * list.size()) - : 0L; - } -} From e60fc218291911bea39f5f41c47619a3e0d0bab7 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 8 Jul 2025 10:46:31 +0800 Subject: [PATCH 129/185] revert fileUtils --- .../apache/iotdb/commons/utils/FileUtils.java | 17 ----------------- 1 file changed, 17 deletions(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/FileUtils.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/FileUtils.java index 04577f423b28..7597b0826dfd 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/FileUtils.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/FileUtils.java @@ -114,23 +114,6 @@ public static void deleteFileOrDirectoryWithRetry(File file) { } } - public static boolean deleteFileChildrenQuietly(final File file) { - if (!file.exists() || !file.isDirectory()) { - return false; - } - - boolean result = true; - final File[] items = file.listFiles(); - if (items != null) { - for (final File item : items) { - if (item.isFile()) { - result &= org.apache.commons.io.FileUtils.deleteQuietly(item); - } - } - } - return result; - } - public static void deleteDirectoryAndEmptyParent(File folder) { deleteFileOrDirectory(folder); final File parentFolder = folder.getParentFile(); From a64ba166ae44693d2b1cb1ad8529d4e95c282751 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 8 Jul 2025 10:47:10 +0800 Subject: [PATCH 130/185] remove-useless --- .../PipeRealtimeDataRegionExtractor.java | 91 ------------------- ...PipeRealtimeDataRegionTsFileExtractor.java | 60 +++++++----- 2 files changed, 38 insertions(+), 113 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java index bfafc55763d2..c7a778357a44 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java @@ -20,9 +20,6 @@ package org.apache.iotdb.db.pipe.extractor.dataregion.realtime; import org.apache.iotdb.commons.consensus.DataRegionId; -import org.apache.iotdb.commons.consensus.index.ProgressIndex; -import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; -import org.apache.iotdb.commons.consensus.index.impl.SegmentProgressIndex; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeNonCriticalException; import org.apache.iotdb.commons.pipe.agent.task.connection.UnboundedBlockingPendingQueue; import org.apache.iotdb.commons.pipe.agent.task.meta.PipeTaskMeta; @@ -34,11 +31,8 @@ import org.apache.iotdb.commons.utils.TimePartitionUtils; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; -import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; -import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; import org.apache.iotdb.db.pipe.extractor.dataregion.DataRegionListeningFilter; -import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.assigner.PipeTsFileEpochProgressIndexKeeper; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.listener.PipeInsertionDataNodeListener; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.listener.PipeTimePartitionListener; import org.apache.iotdb.db.pipe.metric.source.PipeDataRegionEventCounter; @@ -206,10 +200,6 @@ public void customize( dataRegionId = String.valueOf(environment.getRegionId()); pipeTaskMeta = environment.getPipeTaskMeta(); - if (pipeTaskMeta.getProgressIndex() instanceof MinimumProgressIndex) { - pipeTaskMeta.updateProgressIndex(new SegmentProgressIndex()); - } - // Metrics related to TsFileEpoch are managed in PipeExtractorMetrics. These metrics are // indexed by the taskID of IoTDBDataRegionExtractor. To avoid PipeRealtimeDataRegionExtractor // holding a reference to IoTDBDataRegionExtractor, the taskID should be constructed to @@ -405,63 +395,6 @@ protected void extractDirectly(final PipeRealtimeEvent event) { } } - @Override - public Event supply() { - PipeRealtimeEvent realtimeEvent = getNextRealtimeEvent(); - - while (realtimeEvent != null) { - final EnrichedEvent innerEvent = realtimeEvent.getEvent(); - if (innerEvent instanceof PipeTsFileInsertionEvent - || innerEvent instanceof PipeInsertNodeTabletInsertionEvent) { - bindOrUpdateProgressIndexForRealtimeEvent(realtimeEvent); - } - - final Event suppliedEvent = doSupply(realtimeEvent); - - realtimeEvent.decreaseReferenceCount(PipeRealtimeDataRegionExtractor.class.getName(), false); - - if (suppliedEvent != null) { - return suppliedEvent; - } - - realtimeEvent = getNextRealtimeEvent(); - } - - // means the pending queue is empty. - return null; - } - - private PipeRealtimeEvent getNextRealtimeEvent() { - return (PipeRealtimeEvent) pendingQueue.directPoll(); - } - - // This may require some time thus we leave it for processor thread instead of writing thread - private boolean coarseFilterEvent(final PipeRealtimeEvent event) { - // 1. Check if time parsing is necessary. If not, it means that the timestamps of the data - // contained in this event are definitely within the time range [start time, end time]. - // 2. Check if the event's data timestamps may intersect with the time range. If not, it means - // that the data timestamps of this event are definitely not within the time range. - // 3. Check if pattern parsing is necessary. If not, it means that the paths of the data - // contained in this event are definitely covered by the pattern. - // 4. Check if the event's data paths may intersect with the pattern. If not, it means that the - // data of this event is definitely not overlapped with the pattern. - if ((!event.shouldParseTime() || event.getEvent().mayEventTimeOverlappedWithTimeRange()) - && (!event.shouldParsePattern() || event.getEvent().mayEventPathsOverlappedWithPattern())) { - if (sloppyTimeRange) { - // only skip parsing time for events whose data timestamps may intersect with the time range - event.skipParsingTime(); - } - if (sloppyPattern) { - // only skip parsing pattern for events whose data paths may intersect with the pattern - event.skipParsingPattern(); - } - return true; - } - return false; - } - - protected abstract Event doSupply(final PipeRealtimeEvent realtimeEvent); - protected Event supplyHeartbeat(final PipeRealtimeEvent event) { if (event.increaseReferenceCount(PipeRealtimeDataRegionExtractor.class.getName())) { return event.getEvent(); @@ -563,30 +496,6 @@ public final boolean isShouldTransferModFile() { return shouldTransferModFile; } - private void bindOrUpdateProgressIndexForRealtimeEvent(final PipeRealtimeEvent event) { - if (PipeTsFileEpochProgressIndexKeeper.getInstance() - .isProgressIndexAfterOrEquals( - dataRegionId, - pipeName, - event.getTsFileEpoch().getFilePath(), - getProgressIndex4RealtimeEvent(event))) { - event.skipReportOnCommit(); - if (LOGGER.isDebugEnabled()) { - LOGGER.debug( - "Pipe {} on data region {} skip commit of event {} because it was flushed prematurely.", - pipeName, - dataRegionId, - event.coreReportMessage()); - } - } - } - - private ProgressIndex getProgressIndex4RealtimeEvent(final PipeRealtimeEvent event) { - return event.getEvent() instanceof PipeTsFileInsertionEvent - ? ((PipeTsFileInsertionEvent) event.getEvent()).forceGetProgressIndex() - : event.getProgressIndex(); - } - @Override public String toString() { return "PipeRealtimeDataRegionExtractor{" diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionTsFileExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionTsFileExtractor.java index d881ec49943e..8072499b3daf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionTsFileExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionTsFileExtractor.java @@ -84,30 +84,46 @@ public boolean isNeedListenToInsertNode() { } @Override - protected Event doSupply(final PipeRealtimeEvent realtimeEvent) { - if (realtimeEvent.getEvent() instanceof PipeHeartbeatEvent) { - return supplyHeartbeat(realtimeEvent); - } else if (realtimeEvent.getEvent() instanceof PipeSchemaRegionWritePlanEvent - || realtimeEvent.getEvent() instanceof ProgressReportEvent) { - return supplyDirectly(realtimeEvent); - } else if (realtimeEvent.increaseReferenceCount( - PipeRealtimeDataRegionTsFileExtractor.class.getName())) { - return realtimeEvent.getEvent(); - } else { - // if the event's reference count can not be increased, it means the data represented by - // this event is not reliable anymore. the data has been lost. we simply discard this event - // and report the exception to PipeRuntimeAgent. - final String errorMessage = - String.format( - "Event %s can not be supplied because " - + "the reference count can not be increased, " - + "the data represented by this event is lost", - realtimeEvent.getEvent()); - LOGGER.error(errorMessage); - PipeDataNodeAgent.runtime() - .report(pipeTaskMeta, new PipeRuntimeNonCriticalException(errorMessage)); + public Event supply() { + PipeRealtimeEvent realtimeEvent = (PipeRealtimeEvent) pendingQueue.directPoll(); + + while (realtimeEvent != null) { + Event suppliedEvent = null; + + if (realtimeEvent.getEvent() instanceof PipeHeartbeatEvent) { + suppliedEvent = supplyHeartbeat(realtimeEvent); + } else if (realtimeEvent.getEvent() instanceof PipeSchemaRegionWritePlanEvent + || realtimeEvent.getEvent() instanceof ProgressReportEvent) { + suppliedEvent = supplyDirectly(realtimeEvent); + } else if (realtimeEvent.increaseReferenceCount( + PipeRealtimeDataRegionTsFileExtractor.class.getName())) { + suppliedEvent = realtimeEvent.getEvent(); + } else { + // if the event's reference count can not be increased, it means the data represented by + // this event is not reliable anymore. the data has been lost. we simply discard this event + // and report the exception to PipeRuntimeAgent. + final String errorMessage = + String.format( + "Event %s can not be supplied because " + + "the reference count can not be increased, " + + "the data represented by this event is lost", + realtimeEvent.getEvent()); + LOGGER.error(errorMessage); + PipeDataNodeAgent.runtime() + .report(pipeTaskMeta, new PipeRuntimeNonCriticalException(errorMessage)); + } + + realtimeEvent.decreaseReferenceCount( + PipeRealtimeDataRegionTsFileExtractor.class.getName(), false); + + if (suppliedEvent != null) { + return suppliedEvent; + } + + realtimeEvent = (PipeRealtimeEvent) pendingQueue.directPoll(); } + // means the pending queue is empty. return null; } } From 121b12a6303f4b9cd4c8133a4613e529c1de2ca4 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 8 Jul 2025 10:49:18 +0800 Subject: [PATCH 131/185] heartbeat-fix --- ...eRealtimeDataRegionHeartbeatExtractor.java | 28 +++++++++++++++---- 1 file changed, 22 insertions(+), 6 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHeartbeatExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHeartbeatExtractor.java index 7ebaa673d5d3..1df62eecc924 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHeartbeatExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHeartbeatExtractor.java @@ -27,13 +27,29 @@ public class PipeRealtimeDataRegionHeartbeatExtractor extends PipeRealtimeDataRegionExtractor { @Override - protected Event doSupply(PipeRealtimeEvent realtimeEvent) { - // only supply PipeHeartbeatEvent - if (realtimeEvent.getEvent() instanceof PipeHeartbeatEvent) { - return supplyHeartbeat(realtimeEvent); - } else if (realtimeEvent.getEvent() instanceof ProgressReportEvent) { - return supplyDirectly(realtimeEvent); + public Event supply() { + PipeRealtimeEvent realtimeEvent = (PipeRealtimeEvent) pendingQueue.directPoll(); + + while (realtimeEvent != null) { + Event suppliedEvent = null; + + // only supply PipeHeartbeatEvent + if (realtimeEvent.getEvent() instanceof PipeHeartbeatEvent) { + suppliedEvent = supplyHeartbeat(realtimeEvent); + } else if (realtimeEvent.getEvent() instanceof ProgressReportEvent) { + suppliedEvent = supplyDirectly(realtimeEvent); + } + + realtimeEvent.decreaseReferenceCount( + PipeRealtimeDataRegionHeartbeatExtractor.class.getName(), false); + + if (suppliedEvent != null) { + return suppliedEvent; + } + + realtimeEvent = (PipeRealtimeEvent) pendingQueue.directPoll(); } + return null; } From 6428005bffda4cbbbf0d2c4be118890c4704e72a Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 8 Jul 2025 11:22:13 +0800 Subject: [PATCH 132/185] revert realtime supply --- ...PipeRealtimeDataRegionHybridExtractor.java | 52 +++++++++++++------ 1 file changed, 35 insertions(+), 17 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java index bdb21cd08114..95fef2d7aba7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java @@ -314,24 +314,42 @@ private boolean mayTsFileLinkedCountReachDangerousThreshold(final PipeRealtimeEv } @Override - protected Event doSupply(final PipeRealtimeEvent realtimeEvent) { - // Used to judge the type of the event, not directly for supplying. - final Event eventToSupply = realtimeEvent.getEvent(); - if (eventToSupply instanceof TabletInsertionEvent) { - return supplyTabletInsertion(realtimeEvent); - } else if (eventToSupply instanceof TsFileInsertionEvent) { - return supplyTsFileInsertion(realtimeEvent); - } else if (eventToSupply instanceof PipeHeartbeatEvent) { - return supplyHeartbeat(realtimeEvent); - } else if (eventToSupply instanceof PipeSchemaRegionWritePlanEvent - || eventToSupply instanceof ProgressReportEvent) { - return supplyDirectly(realtimeEvent); - } else { - throw new UnsupportedOperationException( - String.format( - "Unsupported event type %s for hybrid realtime extractor %s to supply.", - eventToSupply.getClass(), this)); + public Event supply() { + PipeRealtimeEvent realtimeEvent = (PipeRealtimeEvent) pendingQueue.directPoll(); + + while (realtimeEvent != null) { + final Event suppliedEvent; + + // Used to judge the type of the event, not directly for supplying. + final Event eventToSupply = realtimeEvent.getEvent(); + if (eventToSupply instanceof TabletInsertionEvent) { + suppliedEvent = supplyTabletInsertion(realtimeEvent); + } else if (eventToSupply instanceof TsFileInsertionEvent) { + suppliedEvent = supplyTsFileInsertion(realtimeEvent); + } else if (eventToSupply instanceof PipeHeartbeatEvent) { + suppliedEvent = supplyHeartbeat(realtimeEvent); + } else if (eventToSupply instanceof PipeSchemaRegionWritePlanEvent + || eventToSupply instanceof ProgressReportEvent) { + suppliedEvent = supplyDirectly(realtimeEvent); + } else { + throw new UnsupportedOperationException( + String.format( + "Unsupported event type %s for hybrid realtime extractor %s to supply.", + eventToSupply.getClass(), this)); + } + + realtimeEvent.decreaseReferenceCount( + PipeRealtimeDataRegionHybridExtractor.class.getName(), false); + + if (suppliedEvent != null) { + return suppliedEvent; + } + + realtimeEvent = (PipeRealtimeEvent) pendingQueue.directPoll(); } + + // Means the pending queue is empty. + return null; } private Event supplyTabletInsertion(final PipeRealtimeEvent event) { From 1cea612f67a27e106d0f32a8b0f28a7132cafef4 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 8 Jul 2025 14:10:59 +0800 Subject: [PATCH 133/185] Update PipeRealtimeDataRegionLogExtractor.java --- .../realtime/PipeRealtimeDataRegionLogExtractor.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionLogExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionLogExtractor.java index 4b300355c80e..7549042e676c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionLogExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionLogExtractor.java @@ -82,10 +82,7 @@ private void extractTabletInsertion(PipeRealtimeEvent event) { private void extractTsFileInsertion(PipeRealtimeEvent event) { final PipeTsFileInsertionEvent tsFileInsertionEvent = (PipeTsFileInsertionEvent) event.getEvent(); - if (!(tsFileInsertionEvent.isLoaded() - // some insert nodes in the tsfile epoch are not captured by pipe - || tsFileInsertionEvent.getFileStartTime() - < event.getTsFileEpoch().getInsertNodeMinTime())) { + if (!(tsFileInsertionEvent.isLoaded())) { // All data in the tsfile epoch has been extracted in tablet mode, so we should // simply ignore this event. event.decreaseReferenceCount(PipeRealtimeDataRegionLogExtractor.class.getName(), false); From 7c217821898fc8a393909a977488059f7ec5acd5 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 8 Jul 2025 14:36:29 +0800 Subject: [PATCH 134/185] Remove synchornization --- .../task/connection/BlockingPendingQueue.java | 20 +++++++++---------- .../UnboundedBlockingPendingQueue.java | 2 +- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/connection/BlockingPendingQueue.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/connection/BlockingPendingQueue.java index 2fde7ffef1db..aa93b0932545 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/connection/BlockingPendingQueue.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/connection/BlockingPendingQueue.java @@ -50,7 +50,7 @@ protected BlockingPendingQueue( this.eventCounter = eventCounter; } - public synchronized boolean waitedOffer(final E event) { + public boolean waitedOffer(final E event) { checkBeforeOffer(event); try { final boolean offered = @@ -69,7 +69,7 @@ public synchronized boolean waitedOffer(final E event) { } } - public synchronized boolean directOffer(final E event) { + public boolean directOffer(final E event) { checkBeforeOffer(event); final boolean offered = pendingQueue.offer(event); if (offered) { @@ -78,7 +78,7 @@ public synchronized boolean directOffer(final E event) { return offered; } - public synchronized boolean put(final E event) { + public boolean put(final E event) { checkBeforeOffer(event); try { pendingQueue.put(event); @@ -91,13 +91,13 @@ public synchronized boolean put(final E event) { } } - public synchronized E directPoll() { + public E directPoll() { final E event = pendingQueue.poll(); eventCounter.decreaseEventCount(event); return event; } - public synchronized E waitedPoll() { + public E waitedPoll() { E event = null; try { event = @@ -112,22 +112,22 @@ public synchronized E waitedPoll() { return event; } - public synchronized E peek() { + public E peek() { return pendingQueue.peek(); } - public synchronized void clear() { + public void clear() { isClosed.set(true); pendingQueue.clear(); eventCounter.reset(); } /** DO NOT FORGET to set eventCounter to new value after invoking this method. */ - public synchronized void forEach(final Consumer action) { + public void forEach(final Consumer action) { pendingQueue.forEach(action); } - public synchronized void discardAllEvents() { + public void discardAllEvents() { isClosed.set(true); pendingQueue.removeIf( event -> { @@ -141,7 +141,7 @@ public synchronized void discardAllEvents() { eventCounter.reset(); } - public synchronized void discardEventsOfPipe(final String pipeNameToDrop, final int regionId) { + public void discardEventsOfPipe(final String pipeNameToDrop, final int regionId) { pendingQueue.removeIf( event -> { if (event instanceof EnrichedEvent diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/connection/UnboundedBlockingPendingQueue.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/connection/UnboundedBlockingPendingQueue.java index 1172ae19a8f2..785e89cfb9a8 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/connection/UnboundedBlockingPendingQueue.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/connection/UnboundedBlockingPendingQueue.java @@ -34,7 +34,7 @@ public UnboundedBlockingPendingQueue(final PipeEventCounter eventCounter) { pendingDeque = (BlockingDeque) pendingQueue; } - public synchronized E peekLast() { + public E peekLast() { return pendingDeque.peekLast(); } } From 6c9fe8643b58d6fa5c399c5cb176dbb6bc504a9f Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 8 Jul 2025 14:48:56 +0800 Subject: [PATCH 135/185] Create CachedSchemaPatternMatcherTest.java --- .../CachedSchemaPatternMatcherTest.java | 218 ++++++++++++++++++ 1 file changed, 218 insertions(+) create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/pattern/CachedSchemaPatternMatcherTest.java diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/pattern/CachedSchemaPatternMatcherTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/pattern/CachedSchemaPatternMatcherTest.java new file mode 100644 index 000000000000..2e3e57cd49e3 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/pattern/CachedSchemaPatternMatcherTest.java @@ -0,0 +1,218 @@ +/* + * 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.pipe.pattern; + +import org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant; +import org.apache.iotdb.commons.pipe.config.plugin.configuraion.PipeTaskRuntimeConfiguration; +import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskExtractorRuntimeEnvironment; +import org.apache.iotdb.commons.pipe.datastructure.pattern.PipePattern; +import org.apache.iotdb.commons.pipe.datastructure.pattern.PrefixPipePattern; +import org.apache.iotdb.commons.pipe.event.EnrichedEvent; +import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; +import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionExtractor; +import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.epoch.TsFileEpoch; +import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.matcher.CachedSchemaPatternMatcher; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; +import org.apache.iotdb.pipe.api.event.Event; + +import org.apache.tsfile.common.constant.TsFileConstant; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +public class CachedSchemaPatternMatcherTest { + + private static class MockedPipeRealtimeEvent extends PipeRealtimeEvent { + + public MockedPipeRealtimeEvent( + EnrichedEvent event, + TsFileEpoch tsFileEpoch, + Map device2Measurements, + PipePattern pattern) { + super(event, tsFileEpoch, device2Measurements, pattern); + } + + @Override + public boolean shouldParseTime() { + return false; + } + + @Override + public boolean shouldParsePattern() { + return false; + } + } + + private CachedSchemaPatternMatcher matcher; + private ExecutorService executorService; + private List extractors; + + @Before + public void setUp() { + matcher = new CachedSchemaPatternMatcher(); + executorService = Executors.newSingleThreadExecutor(); + extractors = new ArrayList<>(); + } + + @After + public void tearDown() { + executorService.shutdownNow(); + } + + @Test + public void testCachedMatcher() throws Exception { + PipeRealtimeDataRegionExtractor dataRegionExtractor = new PipeRealtimeDataRegionFakeExtractor(); + dataRegionExtractor.customize( + new PipeParameters( + new HashMap() { + { + put(PipeExtractorConstant.EXTRACTOR_PATTERN_KEY, "root"); + } + }), + new PipeTaskRuntimeConfiguration(new PipeTaskExtractorRuntimeEnvironment("1", 1, 1, null))); + extractors.add(dataRegionExtractor); + + int deviceExtractorNum = 10; + int seriesExtractorNum = 10; + for (int i = 0; i < deviceExtractorNum; i++) { + PipeRealtimeDataRegionExtractor deviceExtractor = new PipeRealtimeDataRegionFakeExtractor(); + int finalI1 = i; + deviceExtractor.customize( + new PipeParameters( + new HashMap() { + { + put(PipeExtractorConstant.EXTRACTOR_PATTERN_KEY, "root." + finalI1); + } + }), + new PipeTaskRuntimeConfiguration( + new PipeTaskExtractorRuntimeEnvironment("1", 1, 1, null))); + extractors.add(deviceExtractor); + for (int j = 0; j < seriesExtractorNum; j++) { + PipeRealtimeDataRegionExtractor seriesExtractor = new PipeRealtimeDataRegionFakeExtractor(); + int finalI = i; + int finalJ = j; + seriesExtractor.customize( + new PipeParameters( + new HashMap() { + { + put( + PipeExtractorConstant.EXTRACTOR_PATTERN_KEY, + "root." + finalI + "." + finalJ); + } + }), + new PipeTaskRuntimeConfiguration( + new PipeTaskExtractorRuntimeEnvironment("1", 1, 1, null))); + extractors.add(seriesExtractor); + } + } + + Future future = + executorService.submit(() -> extractors.forEach(extractor -> matcher.register(extractor))); + + int epochNum = 10000; + int deviceNum = 1000; + int seriesNum = 100; + Map deviceMap = + IntStream.range(0, deviceNum) + .mapToObj(String::valueOf) + .collect(Collectors.toMap(s -> "root." + s, s -> new String[0])); + String[] measurements = + IntStream.range(0, seriesNum).mapToObj(String::valueOf).toArray(String[]::new); + long totalTime = 0; + for (int i = 0; i < epochNum; i++) { + for (int j = 0; j < deviceNum; j++) { + MockedPipeRealtimeEvent event = + new MockedPipeRealtimeEvent( + null, null, Collections.singletonMap("root." + i, measurements), null); + long startTime = System.currentTimeMillis(); + matcher.match(event).forEach(extractor -> extractor.extract(event)); + totalTime += (System.currentTimeMillis() - startTime); + } + MockedPipeRealtimeEvent event = new MockedPipeRealtimeEvent(null, null, deviceMap, null); + long startTime = System.currentTimeMillis(); + matcher.match(event).forEach(extractor -> extractor.extract(event)); + totalTime += (System.currentTimeMillis() - startTime); + } + System.out.println("matcher.getRegisterCount() = " + matcher.getRegisterCount()); + System.out.println("totalTime = " + totalTime); + System.out.println( + "device match per second = " + + ((double) (epochNum * (deviceNum + 1)) / (double) (totalTime) * 1000.0)); + + future.get(); + } + + public static class PipeRealtimeDataRegionFakeExtractor extends PipeRealtimeDataRegionExtractor { + + public PipeRealtimeDataRegionFakeExtractor() { + pipePattern = new PrefixPipePattern(null); + } + + @Override + public Event supply() { + return null; + } + + @Override + protected void doExtract(PipeRealtimeEvent event) { + final boolean[] match = {false}; + event + .getSchemaInfo() + .forEach( + (k, v) -> { + if (v.length > 0) { + for (String s : v) { + match[0] = + match[0] + || (k + TsFileConstant.PATH_SEPARATOR + s) + .startsWith(getPatternString()); + } + } else { + match[0] = + match[0] + || (getPatternString().startsWith(k) || k.startsWith(getPatternString())); + } + }); + Assert.assertTrue(match[0]); + } + + @Override + public boolean isNeedListenToTsFile() { + return true; + } + + @Override + public boolean isNeedListenToInsertNode() { + return true; + } + } +} From b4c96bef2b577be7583718553371aac5568f1418 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 8 Jul 2025 17:09:37 +0800 Subject: [PATCH 136/185] continue-kill-sync --- .../connector/PipeRealtimePriorityBlockingQueue.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java index 5373cc03694d..272a0947593e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java @@ -335,19 +335,19 @@ public synchronized void replace( } @Override - public synchronized void clear() { + public void clear() { super.clear(); tsfileInsertEventDeque.clear(); } @Override - public synchronized void forEach(final Consumer action) { + public void forEach(final Consumer action) { super.forEach(action); tsfileInsertEventDeque.forEach(action); } @Override - public synchronized void discardAllEvents() { + public void discardAllEvents() { super.discardAllEvents(); tsfileInsertEventDeque.removeIf( event -> { @@ -362,7 +362,7 @@ public synchronized void discardAllEvents() { } @Override - public synchronized void discardEventsOfPipe(final String pipeNameToDrop, final int regionId) { + public void discardEventsOfPipe(final String pipeNameToDrop, final int regionId) { super.discardEventsOfPipe(pipeNameToDrop, regionId); tsfileInsertEventDeque.removeIf( event -> { From 31199d194d053130f3300ea73cdd78b3b7789f44 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 8 Jul 2025 18:52:19 +0800 Subject: [PATCH 137/185] change-default --- .../main/java/org/apache/iotdb/commons/conf/CommonConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java index 5e0eb236ecb8..d3f3822d9212 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java @@ -255,7 +255,7 @@ public class CommonConfig { private long pipeSubtaskExecutorCronHeartbeatEventIntervalSeconds = 20; private long pipeSubtaskExecutorForcedRestartIntervalMs = Long.MAX_VALUE; - private long pipeMaxWaitFinishTime = 2 * 60 * 1000; + private long pipeMaxWaitFinishTime = 10 * 1000; private int pipeExtractorAssignerDisruptorRingBufferSize = 65536; private long pipeExtractorAssignerDisruptorRingBufferEntrySizeInBytes = 50; // 50B From 2042779f286b7d02acdd4f05fee8ff547e31cd0b Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 8 Jul 2025 19:26:43 +0800 Subject: [PATCH 138/185] continue-progress-index-push-back-fix --- .../PipeRealtimeDataRegionExtractor.java | 35 +++++++++++++++++++ ...PipeRealtimeDataRegionHybridExtractor.java | 3 ++ ...PipeRealtimeDataRegionTsFileExtractor.java | 7 ++++ 3 files changed, 45 insertions(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java index c7a778357a44..0eaa94e0262c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.pipe.extractor.dataregion.realtime; import org.apache.iotdb.commons.consensus.DataRegionId; +import org.apache.iotdb.commons.consensus.index.ProgressIndex; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeNonCriticalException; import org.apache.iotdb.commons.pipe.agent.task.connection.UnboundedBlockingPendingQueue; import org.apache.iotdb.commons.pipe.agent.task.meta.PipeTaskMeta; @@ -31,8 +32,11 @@ import org.apache.iotdb.commons.utils.TimePartitionUtils; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; +import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; +import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; import org.apache.iotdb.db.pipe.extractor.dataregion.DataRegionListeningFilter; +import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.assigner.PipeTsFileEpochProgressIndexKeeper; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.listener.PipeInsertionDataNodeListener; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.listener.PipeTimePartitionListener; import org.apache.iotdb.db.pipe.metric.source.PipeDataRegionEventCounter; @@ -395,6 +399,13 @@ protected void extractDirectly(final PipeRealtimeEvent event) { } } + protected void maySkipIndex4Event(final PipeRealtimeEvent event) { + if (event.getEvent() instanceof PipeTsFileInsertionEvent + || event.getEvent() instanceof PipeInsertNodeTabletInsertionEvent) { + maySkipProgressIndexForRealtimeEvent(event); + } + } + protected Event supplyHeartbeat(final PipeRealtimeEvent event) { if (event.increaseReferenceCount(PipeRealtimeDataRegionExtractor.class.getName())) { return event.getEvent(); @@ -496,6 +507,30 @@ public final boolean isShouldTransferModFile() { return shouldTransferModFile; } + private void maySkipProgressIndexForRealtimeEvent(final PipeRealtimeEvent event) { + if (PipeTsFileEpochProgressIndexKeeper.getInstance() + .isProgressIndexAfterOrEquals( + dataRegionId, + pipeName, + event.getTsFileEpoch().getFilePath(), + getProgressIndex4RealtimeEvent(event))) { + event.skipReportOnCommit(); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug( + "Pipe {} on data region {} skip commit of event {} because it was flushed prematurely.", + pipeName, + dataRegionId, + event.coreReportMessage()); + } + } + } + + private ProgressIndex getProgressIndex4RealtimeEvent(final PipeRealtimeEvent event) { + return event.getEvent() instanceof PipeTsFileInsertionEvent + ? ((PipeTsFileInsertionEvent) event.getEvent()).forceGetProgressIndex() + : event.getProgressIndex(); + } + @Override public String toString() { return "PipeRealtimeDataRegionExtractor{" diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java index 95fef2d7aba7..1076a97b320d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java @@ -342,6 +342,7 @@ public Event supply() { PipeRealtimeDataRegionHybridExtractor.class.getName(), false); if (suppliedEvent != null) { + maySkipIndex4Event(realtimeEvent); return suppliedEvent; } @@ -446,6 +447,8 @@ private Event supplyTsFileInsertion(final PipeRealtimeEvent event) { LOGGER.error(errorMessage); PipeDataNodeAgent.runtime() .report(pipeTaskMeta, new PipeRuntimeNonCriticalException(errorMessage)); + PipeTsFileEpochProgressIndexKeeper.getInstance() + .eliminateProgressIndex(dataRegionId, pipeName, event.getTsFileEpoch().getFilePath()); return null; } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionTsFileExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionTsFileExtractor.java index 8072499b3daf..f189af5726de 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionTsFileExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionTsFileExtractor.java @@ -25,6 +25,7 @@ import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; +import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.assigner.PipeTsFileEpochProgressIndexKeeper; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.epoch.TsFileEpoch; import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.pipe.api.event.dml.insertion.TsFileInsertionEvent; @@ -50,6 +51,8 @@ protected void doExtract(PipeRealtimeEvent event) { } event.getTsFileEpoch().migrateState(this, state -> TsFileEpoch.State.USING_TSFILE); + PipeTsFileEpochProgressIndexKeeper.getInstance() + .registerProgressIndex(dataRegionId, pipeName, event.getTsFileEpoch().getResource()); if (!(event.getEvent() instanceof TsFileInsertionEvent)) { event.decreaseReferenceCount(PipeRealtimeDataRegionTsFileExtractor.class.getName(), false); @@ -111,12 +114,16 @@ public Event supply() { LOGGER.error(errorMessage); PipeDataNodeAgent.runtime() .report(pipeTaskMeta, new PipeRuntimeNonCriticalException(errorMessage)); + PipeTsFileEpochProgressIndexKeeper.getInstance() + .eliminateProgressIndex( + dataRegionId, pipeName, realtimeEvent.getTsFileEpoch().getFilePath()); } realtimeEvent.decreaseReferenceCount( PipeRealtimeDataRegionTsFileExtractor.class.getName(), false); if (suppliedEvent != null) { + maySkipIndex4Event(realtimeEvent); return suppliedEvent; } From 24fce7efb8b68c83d40ba72a64db734e6bef06e3 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 8 Jul 2025 19:55:12 +0800 Subject: [PATCH 139/185] Update PipeTsFileResourceManager.java --- .../db/pipe/resource/tsfile/PipeTsFileResourceManager.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java index 37b3014a499b..37d499ef4a23 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java @@ -76,7 +76,7 @@ public class PipeTsFileResourceManager { * @throws IOException when create hardlink or copy file failed */ public File increaseFileReference( - final File file, final boolean isTsFile, final @Nonnull String pipeName) throws IOException { + final File file, final boolean isTsFile, final @Nullable String pipeName) throws IOException { // If the file is already a hardlink or copied file, // just increase reference count and return it segmentLock.lock(file); @@ -123,7 +123,7 @@ public File increaseFileReference( } private boolean increaseReferenceIfExists( - final File file, final boolean isTsFile, final @Nonnull String pipeName) { + final File file, final boolean isTsFile, final @Nullable String pipeName) { final String path = file.getPath(); final PipeTsFileResource resource = hardlinkOrCopiedFileToPipeTsFileResourceMap From 53352971c9020ac19bf6e84b6b34f9fc0ca3fc0d Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 9 Jul 2025 09:52:31 +0800 Subject: [PATCH 140/185] Fixed msg NPE --- .../tablet/PipeInsertNodeTabletInsertionEvent.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java index 6951a02e6b08..ed7d59f0b35f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tablet/PipeInsertNodeTabletInsertionEvent.java @@ -388,7 +388,10 @@ public List toRawTabletInsertionEvents() { public String toString() { return String.format( "PipeInsertNodeTabletInsertionEvent{progressIndex=%s, isAligned=%s, isGeneratedByPipe=%s, dataContainers=%s}", - progressIndex, insertNode.isAligned(), insertNode.isGeneratedByPipe(), dataContainers) + progressIndex, + Objects.nonNull(insertNode) ? insertNode.isAligned() : null, + Objects.nonNull(insertNode) ? insertNode.isGeneratedByPipe() : null, + dataContainers) + " - " + super.toString(); } @@ -397,7 +400,9 @@ public String toString() { public String coreReportMessage() { return String.format( "PipeInsertNodeTabletInsertionEvent{progressIndex=%s, isAligned=%s, isGeneratedByPipe=%s}", - progressIndex, insertNode.isAligned(), insertNode.isGeneratedByPipe()) + progressIndex, + Objects.nonNull(insertNode) ? insertNode.isAligned() : null, + Objects.nonNull(insertNode) ? insertNode.isGeneratedByPipe() : null) + " - " + super.coreReportMessage(); } From c2ee6bfd5931e4cf1f1ccc0b2d1032b2bcc6b1b2 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 9 Jul 2025 10:02:36 +0800 Subject: [PATCH 141/185] Update DataNodeSchemaLockManager.java --- .../queryengine/plan/analyze/lock/DataNodeSchemaLockManager.java | 1 + 1 file changed, 1 insertion(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/lock/DataNodeSchemaLockManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/lock/DataNodeSchemaLockManager.java index 11478cf1f77f..c7bda59fd85f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/lock/DataNodeSchemaLockManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/lock/DataNodeSchemaLockManager.java @@ -54,6 +54,7 @@ public void releaseReadLock(final MPPQueryContext queryContext) { queryContext .getAcquiredLocks() .forEach(lockType -> locks[lockType.ordinal()].readLock().unlock()); + queryContext.getAcquiredLocks().clear(); } } From 5118fce5cace9bb53df17a98f38890d66ac53997 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 9 Jul 2025 12:26:45 +0800 Subject: [PATCH 142/185] Fix IT --- .../pipe/it/manual/IoTDBPipeTypeConversionISessionIT.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeTypeConversionISessionIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeTypeConversionISessionIT.java index 9af757acbdee..64d872001486 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeTypeConversionISessionIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeTypeConversionISessionIT.java @@ -381,13 +381,12 @@ private void createDataPipe(String diff, boolean isTSFile) { String sql = String.format( "create pipe test%s" - + " with source ('source'='iotdb-source','source.path'='root.test.**','realtime.mode'='%s','realtime.enable'='%s','history.enable'='%s')" + + " with source ('source'='iotdb-source','source.path'='root.test.**','realtime.mode'='%s','realtime.enable'='%s','history.enable'='true')" + " with processor ('processor'='do-nothing-processor')" + " with sink ('node-urls'='%s:%s','batch.enable'='false','sink.format'='%s')", diff, isTSFile ? "file" : "forced-log", !isTSFile, - isTSFile, receiverEnv.getIP(), receiverEnv.getPort(), isTSFile ? "tsfile" : "tablet"); From a09eb7c95fb113b5d4ae31e24199e6e29bd9015b Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 9 Jul 2025 14:37:44 +0800 Subject: [PATCH 143/185] Update IoTDBPipeTypeConversionIT.java --- .../apache/iotdb/pipe/it/manual/IoTDBPipeTypeConversionIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeTypeConversionIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeTypeConversionIT.java index 4ebce6b978c5..24b40d2e0c9c 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeTypeConversionIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeTypeConversionIT.java @@ -234,7 +234,7 @@ private void createDataPipe() { String sql = String.format( "create pipe test" - + " with source ('source'='iotdb-source','source.path'='root.test.**','realtime.mode'='forced-log','realtime.enable'='true','history.enable'='false')" + + " with source ('source'='iotdb-source','source.path'='root.test.**','realtime.mode'='forced-log')" + " with processor ('processor'='do-nothing-processor')" + " with sink ('node-urls'='%s:%s','batch.enable'='false','sink.format'='tablet')", receiverEnv.getIP(), receiverEnv.getPort()); From a27198321f36ac9be6cbc8da5f9fd65427fe0312 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 9 Jul 2025 15:28:06 +0800 Subject: [PATCH 144/185] Fixed tsfile concurrency problem --- .../PipeCompactedTsFileInsertionEvent.java | 1 + .../tsfile/PipeTsFileInsertionEvent.java | 14 +-- .../TsFileInsertionDataContainerProvider.java | 4 +- ...peHistoricalDataRegionTsFileExtractor.java | 1 + ...rce.java => PipeTsFilePublicResource.java} | 10 +- .../resource/tsfile/PipeTsFileResource.java | 7 +- .../tsfile/PipeTsFileResourceManager.java | 109 ++++++++++-------- .../PipeTsFileResourceManagerTest.java | 8 +- 8 files changed, 83 insertions(+), 71 deletions(-) rename iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/{PipeTsFileMemResource.java => PipeTsFilePublicResource.java} (97%) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java index fb0e808a0642..f7673c41662e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java @@ -49,6 +49,7 @@ public PipeCompactedTsFileInsertionEvent( final boolean shouldReportProgress) { super( tsFileResource, + null, bindIsWithMod(originalEvents), bindIsLoaded(originalEvents), bindIsGeneratedByHistoricalExtractor(originalEvents), diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java index 89cc9cbffd9b..de5281976d3d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java @@ -89,11 +89,13 @@ public class PipeTsFileInsertionEvent extends EnrichedEvent public PipeTsFileInsertionEvent(final TsFileResource resource, final boolean isLoaded) { // The modFile must be copied before the event is assigned to the listening pipes - this(resource, true, isLoaded, false, null, 0, null, null, Long.MIN_VALUE, Long.MAX_VALUE); + this( + resource, null, true, isLoaded, false, null, 0, null, null, Long.MIN_VALUE, Long.MAX_VALUE); } public PipeTsFileInsertionEvent( final TsFileResource resource, + final File tsFile, final boolean isWithMod, final boolean isLoaded, final boolean isGeneratedByHistoricalExtractor, @@ -110,7 +112,7 @@ public PipeTsFileInsertionEvent( // For events created for source, the tsFile is inherited from the assigner, because the // original tsFile may be gone, and we need to get the assigner's hard-linked tsFile to // hard-link it to each pipe dir - this.tsFile = resource.getTsFile(); + this.tsFile = Objects.isNull(tsFile) ? resource.getTsFile() : tsFile; final ModificationFile modFile = resource.getModFile(); this.isWithMod = isWithMod && modFile.exists(); @@ -244,10 +246,6 @@ public long getTimePartitionId() { public boolean internallyIncreaseResourceReferenceCount(final String holderMessage) { extractTime = System.nanoTime(); try { - if (Objects.isNull(pipeName)) { - return true; - } - tsFile = PipeDataNodeResourceManager.tsfile().increaseFileReference(tsFile, true, pipeName); if (isWithMod) { modFile = @@ -272,9 +270,6 @@ public boolean internallyIncreaseResourceReferenceCount(final String holderMessa @Override public boolean internallyDecreaseResourceReferenceCount(final String holderMessage) { try { - if (pipeName == null) { - return true; - } PipeDataNodeResourceManager.tsfile().decreaseFileReference(tsFile, pipeName); if (isWithMod) { PipeDataNodeResourceManager.tsfile().decreaseFileReference(modFile, pipeName); @@ -345,6 +340,7 @@ public PipeTsFileInsertionEvent shallowCopySelfAndBindPipeTaskMetaForProgressRep final long endTime) { return new PipeTsFileInsertionEvent( resource, + tsFile, isWithMod, isLoaded, isGeneratedByHistoricalExtractor, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/TsFileInsertionDataContainerProvider.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/TsFileInsertionDataContainerProvider.java index 18ef721b9248..1050950d1768 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/TsFileInsertionDataContainerProvider.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/container/TsFileInsertionDataContainerProvider.java @@ -29,7 +29,7 @@ import org.apache.iotdb.db.pipe.metric.overview.PipeTsFileToTabletsMetrics; import org.apache.iotdb.db.pipe.resource.PipeDataNodeResourceManager; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryManager; -import org.apache.iotdb.db.pipe.resource.tsfile.PipeTsFileMemResource; +import org.apache.iotdb.db.pipe.resource.tsfile.PipeTsFilePublicResource; import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.file.metadata.PlainDeviceID; @@ -81,7 +81,7 @@ public TsFileInsertionDataContainer provide() throws IOException { // Use scan container to save memory if ((double) PipeDataNodeResourceManager.memory().getUsedMemorySizeInBytes() / PipeMemoryManager.getTotalNonFloatingMemorySizeInBytes() - > PipeTsFileMemResource.MEMORY_SUFFICIENT_THRESHOLD) { + > PipeTsFilePublicResource.MEMORY_SUFFICIENT_THRESHOLD) { return new TsFileInsertionScanDataContainer( pipeName, creationTime, tsFile, pattern, startTime, endTime, pipeTaskMeta, sourceEvent); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java index 7909df1ccca6..41cc715eb523 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java @@ -562,6 +562,7 @@ public synchronized Event supply() { final PipeTsFileInsertionEvent event = new PipeTsFileInsertionEvent( resource, + null, shouldTransferModFile, false, true, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileMemResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFilePublicResource.java similarity index 97% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileMemResource.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFilePublicResource.java index 3673afb92e64..47134fe117c9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileMemResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFilePublicResource.java @@ -39,20 +39,22 @@ import java.util.Map; import java.util.Objects; -public class PipeTsFileMemResource extends PipeTsFileResource { - private static final Logger LOGGER = LoggerFactory.getLogger(PipeTsFileMemResource.class); +public class PipeTsFilePublicResource extends PipeTsFileResource { + private static final Logger LOGGER = LoggerFactory.getLogger(PipeTsFilePublicResource.class); public static final float MEMORY_SUFFICIENT_THRESHOLD = 0.7f; private PipeMemoryBlock allocatedMemoryBlock; private Map> deviceMeasurementsMap = null; private Map deviceIsAlignedMap = null; private Map measurementDataTypeMap = null; - public PipeTsFileMemResource() { - super(null); + public PipeTsFilePublicResource(File hardlinkOrCopiedFile) { + super(hardlinkOrCopiedFile); } @Override public void close() { + super.close(); + if (deviceMeasurementsMap != null) { deviceMeasurementsMap = null; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResource.java index 5065821ebe4f..8b37f8770944 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResource.java @@ -81,8 +81,9 @@ public boolean decreaseReferenceCount() { @Override public synchronized void close() { + boolean successful = false; try { - Files.deleteIfExists(hardlinkOrCopiedFile.toPath()); + successful = Files.deleteIfExists(hardlinkOrCopiedFile.toPath()); } catch (final Exception e) { LOGGER.error( "PipeTsFileResource: Failed to delete tsfile {} when closing, because {}. Please MANUALLY delete it.", @@ -91,6 +92,8 @@ public synchronized void close() { e); } - LOGGER.info("PipeTsFileResource: Closed tsfile {} and cleaned up.", hardlinkOrCopiedFile); + if (successful) { + LOGGER.info("PipeTsFileResource: Closed tsfile {} and cleaned up.", hardlinkOrCopiedFile); + } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java index 37d499ef4a23..bf8b6b86b439 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeTsFileResourceManager.java @@ -48,8 +48,8 @@ public class PipeTsFileResourceManager { // This is used to hold the assigner pinned tsFiles. // Also, it is used to provide metadata cache of the tsFile, and is shared by all the pipe's // tsFiles. - private final Map hardlinkOrCopiedFileToTsFileMemResourceMap = - new ConcurrentHashMap<>(); + private final Map + hardlinkOrCopiedFileToTsFilePublicResourceMap = new ConcurrentHashMap<>(); // PipeName -> TsFilePath -> PipeTsFileResource private final Map> @@ -81,7 +81,7 @@ public File increaseFileReference( // just increase reference count and return it segmentLock.lock(file); try { - if (increaseReferenceIfExists(file, isTsFile, pipeName)) { + if (increaseReferenceIfExists(file, pipeName)) { return file; } } finally { @@ -93,11 +93,8 @@ public File increaseFileReference( final File hardlinkOrCopiedFile = getHardlinkOrCopiedFileInPipeDir(file, pipeName); segmentLock.lock(hardlinkOrCopiedFile); try { - if (increaseReferenceIfExists(hardlinkOrCopiedFile, isTsFile, pipeName)) { - return hardlinkOrCopiedFileToPipeTsFileResourceMap - .computeIfAbsent(pipeName, pipe -> new ConcurrentHashMap<>()) - .get(hardlinkOrCopiedFile.getPath()) - .getFile(); + if (increaseReferenceIfExists(hardlinkOrCopiedFile, pipeName)) { + return getResourceMap(pipeName).get(hardlinkOrCopiedFile.getPath()).getFile(); } // If the file is a tsfile, create a hardlink in pipe dir and will return it. @@ -110,11 +107,16 @@ public File increaseFileReference( // If the file is not a hardlink or copied file, and there is no related hardlink or copied // file in pipe dir, create a hardlink or copy it to pipe dir, maintain a reference count for // the hardlink or copied file, and return the hardlink or copied file. - hardlinkOrCopiedFileToPipeTsFileResourceMap - .computeIfAbsent(pipeName, pipe -> new ConcurrentHashMap<>()) - .put(resultFile.getPath(), new PipeTsFileResource(resultFile)); + if (Objects.nonNull(pipeName)) { + hardlinkOrCopiedFileToPipeTsFileResourceMap + .computeIfAbsent(pipeName, k -> new ConcurrentHashMap<>()) + .put(resultFile.getPath(), new PipeTsFileResource(resultFile)); + } else { + hardlinkOrCopiedFileToTsFilePublicResourceMap.put( + resultFile.getPath(), new PipeTsFilePublicResource(resultFile)); + } - increaseMemReference(resultFile, isTsFile); + increasePublicReference(resultFile, pipeName); return resultFile; } finally { @@ -122,32 +124,29 @@ public File increaseFileReference( } } - private boolean increaseReferenceIfExists( - final File file, final boolean isTsFile, final @Nullable String pipeName) { + private boolean increaseReferenceIfExists(final File file, final @Nullable String pipeName) { final String path = file.getPath(); - final PipeTsFileResource resource = - hardlinkOrCopiedFileToPipeTsFileResourceMap - .computeIfAbsent(pipeName, pipe -> new ConcurrentHashMap<>()) - .get(path); + final PipeTsFileResource resource = getResourceMap(pipeName).get(path); if (resource != null) { resource.increaseReferenceCount(); - increaseMemReference(file, isTsFile); + increasePublicReference(file, pipeName); return true; } return false; } - private void increaseMemReference(final File file, final boolean isTsFile) { - if (!isTsFile) { + private void increasePublicReference(final File file, final String pipeName) { + if (Objects.isNull(pipeName)) { return; } // Increase the assigner's file to avoid hard-link or memory cache cleaning // Note that it does not exist for historical files - hardlinkOrCopiedFileToTsFileMemResourceMap.compute( - getCommonFilePath(file), + final String path = getCommonFilePath(file); + hardlinkOrCopiedFileToTsFilePublicResourceMap.compute( + path, (k, v) -> { if (Objects.isNull(v)) { - return new PipeTsFileMemResource(); + return new PipeTsFilePublicResource(new File(path)); } else { v.increaseReferenceCount(); return v; @@ -155,8 +154,8 @@ private void increaseMemReference(final File file, final boolean isTsFile) { }); } - public static File getHardlinkOrCopiedFileInPipeDir(final File file, final String pipeName) - throws IOException { + public static File getHardlinkOrCopiedFileInPipeDir( + final File file, final @Nullable String pipeName) throws IOException { try { return new File(getPipeTsFileDirPath(file, pipeName), getRelativeFilePath(file)); } catch (final Exception e) { @@ -169,7 +168,8 @@ public static File getHardlinkOrCopiedFileInPipeDir(final File file, final Strin } } - private static String getPipeTsFileDirPath(File file, final String pipeName) throws IOException { + private static String getPipeTsFileDirPath(File file, final @Nullable String pipeName) + throws IOException { while (!file.getName().equals(IoTDBConstant.SEQUENCE_FOLDER_NAME) && !file.getName().equals(IoTDBConstant.UNSEQUENCE_FOLDER_NAME) && !file.getName().equals(PipeConfig.getInstance().getPipeHardlinkBaseDirName())) { @@ -206,33 +206,31 @@ private static String getRelativeFilePath(File file) { * @param hardlinkOrCopiedFile the copied or hard-linked file */ public void decreaseFileReference( - final File hardlinkOrCopiedFile, final @Nonnull String pipeName) { + final File hardlinkOrCopiedFile, final @Nullable String pipeName) { segmentLock.lock(hardlinkOrCopiedFile); try { final String filePath = hardlinkOrCopiedFile.getPath(); - final PipeTsFileResource resource = - hardlinkOrCopiedFileToPipeTsFileResourceMap - .computeIfAbsent(pipeName, pipe -> new ConcurrentHashMap<>()) - .get(filePath); + final PipeTsFileResource resource = getResourceMap(pipeName).get(filePath); if (resource != null && resource.decreaseReferenceCount()) { - hardlinkOrCopiedFileToPipeTsFileResourceMap - .computeIfAbsent(pipeName, pipe -> new ConcurrentHashMap<>()) - .remove(filePath); + getResourceMap(pipeName).remove(filePath); } // Decrease the assigner's file to clear hard-link and memory cache // Note that it does not exist for historical files - decreaseMemReferenceIfExists(hardlinkOrCopiedFile); + decreasePublicReferenceIfExists(hardlinkOrCopiedFile, pipeName); } finally { segmentLock.unlock(hardlinkOrCopiedFile); } } - private void decreaseMemReferenceIfExists(final File file) { + private void decreasePublicReferenceIfExists(final File file, final @Nullable String pipeName) { + if (Objects.isNull(pipeName)) { + return; + } // Increase the assigner's file to avoid hard-link or memory cache cleaning // Note that it does not exist for historical files final String commonFilePath = getCommonFilePath(file); - if (hardlinkOrCopiedFileToTsFileMemResourceMap.containsKey(commonFilePath) - && hardlinkOrCopiedFileToTsFileMemResourceMap + if (hardlinkOrCopiedFileToTsFilePublicResourceMap.containsKey(commonFilePath) + && hardlinkOrCopiedFileToTsFilePublicResourceMap .get(commonFilePath) .decreaseReferenceCount()) { hardlinkOrCopiedFileToPipeTsFileResourceMap.remove(commonFilePath); @@ -265,7 +263,7 @@ public int getFileReferenceCount( ? hardlinkOrCopiedFileToPipeTsFileResourceMap .computeIfAbsent(pipeName, pipe -> new ConcurrentHashMap<>()) .get(hardlinkOrCopiedFile.getPath()) - : hardlinkOrCopiedFileToTsFileMemResourceMap.get( + : hardlinkOrCopiedFileToTsFilePublicResourceMap.get( getCommonFilePath(hardlinkOrCopiedFile)); return resource != null ? resource.getReferenceCount() : 0; } finally { @@ -286,8 +284,9 @@ public boolean cacheObjectsIfAbsent(final File hardlinkOrCopiedTsFile) throws IO || hardlinkOrCopiedTsFile.getParentFile().getParentFile() == null) { return false; } - final PipeTsFileMemResource resource = - hardlinkOrCopiedFileToTsFileMemResourceMap.get(getCommonFilePath(hardlinkOrCopiedTsFile)); + final PipeTsFilePublicResource resource = + hardlinkOrCopiedFileToTsFilePublicResourceMap.get( + getCommonFilePath(hardlinkOrCopiedTsFile)); return resource != null && resource.cacheObjectsIfAbsent(hardlinkOrCopiedTsFile); } finally { segmentLock.unlock(hardlinkOrCopiedTsFile); @@ -298,8 +297,9 @@ public Map> getDeviceMeasurementsMapFromCache( final File hardlinkOrCopiedTsFile) throws IOException { segmentLock.lock(hardlinkOrCopiedTsFile); try { - final PipeTsFileMemResource resource = - hardlinkOrCopiedFileToTsFileMemResourceMap.get(getCommonFilePath(hardlinkOrCopiedTsFile)); + final PipeTsFilePublicResource resource = + hardlinkOrCopiedFileToTsFilePublicResourceMap.get( + getCommonFilePath(hardlinkOrCopiedTsFile)); return resource == null ? null : resource.tryGetDeviceMeasurementsMap(hardlinkOrCopiedTsFile); } finally { segmentLock.unlock(hardlinkOrCopiedTsFile); @@ -310,8 +310,9 @@ public Map getDeviceIsAlignedMapFromCache( final File hardlinkOrCopiedTsFile, final boolean cacheOtherMetadata) throws IOException { segmentLock.lock(hardlinkOrCopiedTsFile); try { - final PipeTsFileMemResource resource = - hardlinkOrCopiedFileToTsFileMemResourceMap.get(getCommonFilePath(hardlinkOrCopiedTsFile)); + final PipeTsFilePublicResource resource = + hardlinkOrCopiedFileToTsFilePublicResourceMap.get( + getCommonFilePath(hardlinkOrCopiedTsFile)); return resource == null ? null : resource.tryGetDeviceIsAlignedMap(cacheOtherMetadata, hardlinkOrCopiedTsFile); @@ -324,8 +325,9 @@ public Map getMeasurementDataTypeMapFromCache( final File hardlinkOrCopiedTsFile) throws IOException { segmentLock.lock(hardlinkOrCopiedTsFile); try { - final PipeTsFileMemResource resource = - hardlinkOrCopiedFileToTsFileMemResourceMap.get(getCommonFilePath(hardlinkOrCopiedTsFile)); + final PipeTsFilePublicResource resource = + hardlinkOrCopiedFileToTsFilePublicResourceMap.get( + getCommonFilePath(hardlinkOrCopiedTsFile)); return resource == null ? null : resource.tryGetMeasurementDataTypeMap(hardlinkOrCopiedTsFile); @@ -334,8 +336,15 @@ public Map getMeasurementDataTypeMapFromCache( } } + public Map getResourceMap(final @Nullable String pipeName) { + return Objects.nonNull(pipeName) + ? hardlinkOrCopiedFileToPipeTsFileResourceMap.computeIfAbsent( + pipeName, k -> new ConcurrentHashMap<>()) + : hardlinkOrCopiedFileToTsFilePublicResourceMap; + } + public void pinTsFileResource( - final TsFileResource resource, final boolean withMods, final String pipeName) + final TsFileResource resource, final boolean withMods, final @Nullable String pipeName) throws IOException { increaseFileReference(resource.getTsFile(), true, pipeName); if (withMods && resource.getModFile().exists()) { @@ -343,7 +352,7 @@ public void pinTsFileResource( } } - public void unpinTsFileResource(final TsFileResource resource, final String pipeName) + public void unpinTsFileResource(final TsFileResource resource, final @Nullable String pipeName) throws IOException { final File pinnedFile = getHardlinkOrCopiedFileInPipeDir(resource.getTsFile(), pipeName); decreaseFileReference(pinnedFile, pipeName); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java index 5c5dc8402dfb..85b432a5114f 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/resource/PipeTsFileResourceManagerTest.java @@ -165,7 +165,7 @@ public void testIncreaseTsFile() throws IOException { final File pipeModFile = pipeTsFileResourceManager.increaseFileReference(originModFile, false, PIPE_NAME); Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, null)); - Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, null)); + Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, null)); Assert.assertTrue(Files.exists(originTsfile.toPath())); Assert.assertTrue(Files.exists(originModFile.toPath())); Assert.assertTrue(Files.exists(pipeTsfile.toPath())); @@ -181,7 +181,7 @@ public void testIncreaseTsFile() throws IOException { // test use copyFile to increase reference counts pipeTsFileResourceManager.increaseFileReference(pipeModFile, false, PIPE_NAME); - Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, null)); + Assert.assertEquals(2, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, null)); Assert.assertEquals(2, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, PIPE_NAME)); Assert.assertTrue(Files.exists(originModFile.toPath())); Assert.assertTrue(Files.exists(pipeModFile.toPath())); @@ -202,7 +202,7 @@ public void testDecreaseTsFile() throws IOException { final File pipeModFile = pipeTsFileResourceManager.increaseFileReference(originModFile, false, PIPE_NAME); Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, null)); - Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, null)); + Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, null)); Assert.assertTrue(Files.exists(pipeTsfile.toPath())); Assert.assertTrue(Files.exists(pipeModFile.toPath())); Assert.assertTrue(Files.exists(pipeTsfile.toPath())); @@ -216,7 +216,7 @@ public void testDecreaseTsFile() throws IOException { Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, PIPE_NAME)); Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeTsfile, null)); Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, PIPE_NAME)); - Assert.assertEquals(0, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, null)); + Assert.assertEquals(1, pipeTsFileResourceManager.getFileReferenceCount(pipeModFile, null)); Assert.assertFalse(Files.exists(originFile.toPath())); Assert.assertFalse(Files.exists(originModFile.toPath())); Assert.assertTrue(Files.exists(pipeTsfile.toPath())); From b482afa536d001f9bcad9207925d9a999f539fb5 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 9 Jul 2025 15:32:26 +0800 Subject: [PATCH 145/185] reduce heartbeat logger --- .../manager/pipe/agent/task/PipeConfigNodeTaskAgent.java | 2 +- .../apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/agent/task/PipeConfigNodeTaskAgent.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/agent/task/PipeConfigNodeTaskAgent.java index 911909d104ce..03641f411fa8 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/agent/task/PipeConfigNodeTaskAgent.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/agent/task/PipeConfigNodeTaskAgent.java @@ -208,7 +208,7 @@ protected void collectPipeMetaListInternal( return; } - LOGGER.info("Received pipe heartbeat request {} from config coordinator.", req.heartbeatId); + LOGGER.debug("Received pipe heartbeat request {} from config coordinator.", req.heartbeatId); final List pipeMetaBinaryList = new ArrayList<>(); final List pipeRemainingEventCountList = new ArrayList<>(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java index 4fc6bb7266c1..a26c930e518b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java @@ -471,7 +471,7 @@ protected void collectPipeMetaListInternal( PipeConfig.getInstance().getPipeMetaReportMaxLogNumPerRound(), PipeConfig.getInstance().getPipeMetaReportMaxLogIntervalRounds(), pipeMetaKeeper.getPipeMetaCount()); - LOGGER.info("Received pipe heartbeat request {} from config node.", req.heartbeatId); + LOGGER.debug("Received pipe heartbeat request {} from config node.", req.heartbeatId); final Set dataRegionIds = StorageEngine.getInstance().getAllDataRegionIds().stream() From e4c2c76696cc2ed2e8d596e4b462d2da72c8b154 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Wed, 9 Jul 2025 18:50:40 +0800 Subject: [PATCH 146/185] better logger --- .../db/pipe/agent/task/PipeDataNodeTaskAgent.java | 2 ++ .../iotdb/db/service/DataNodeShutdownHook.java | 13 ++++++++++++- 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java index a26c930e518b..d5ede5cad0a0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java @@ -619,6 +619,8 @@ public void persistAllProgressIndex2ConfigNode() { IoTDBDescriptor.getInstance().getConfig().getDataNodeId(), resp); if (TSStatusCode.SUCCESS_STATUS.getStatusCode() != result.getCode()) { LOGGER.warn("Failed to persist progress index to configNode, status: {}", result); + } else { + LOGGER.info("Successfully persisted all pipe's info to configNode."); } } catch (final Exception e) { LOGGER.warn(e.getMessage()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java index d795d8a5fbce..d09d5f1f87c2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java @@ -96,9 +96,16 @@ public void run() { if (PipeDataNodeAgent.task().getPipeCount() != 0) { for (Map.Entry entry : PipeDataNodeSinglePipeMetrics.getInstance().remainingEventAndTimeOperatorMap.entrySet()) { - while (entry.getValue().getRemainingNonHeartbeatEvents() > 0) { + boolean timeout = false; + while (true) { + if (entry.getValue().getRemainingNonHeartbeatEvents() > 0) { + logger.info( + "Successfully waited for pipe {} to finish.", entry.getValue().getPipeName()); + break; + } if (System.currentTimeMillis() - startTime > PipeConfig.getInstance().getPipeMaxWaitFinishTime()) { + timeout = true; break; } try { @@ -108,6 +115,10 @@ public void run() { logger.info("Interrupted when waiting for pipe to finish"); } } + if (timeout) { + logger.info("Timed out when waiting for pipes to finish, will break"); + break; + } } } // Persist progress index before shutdown to accurate recovery after restart From f2da2c9b21d35375e8baeec25c91c209cc74c2e0 Mon Sep 17 00:00:00 2001 From: Steve Yurong Su Date: Thu, 10 Jul 2025 10:31:12 +0800 Subject: [PATCH 147/185] Replace exceptions with debug logs in schema checks Changed device alignment and datatype mismatch checks to log debug messages instead of throwing exceptions. This allows the load process to continue while still providing diagnostic information when debug logging is enabled. --- .../plan/analyze/load/LoadTsFileAnalyzer.java | 30 +++++++++---------- 1 file changed, 14 insertions(+), 16 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java index df540b63371e..ccf5b2c215f7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java @@ -896,13 +896,12 @@ private void verifySchema(ISchemaTree schemaTree) // check device schema: is aligned or not final boolean isAlignedInTsFile = schemaCache.getDeviceIsAligned(device); final boolean isAlignedInIoTDB = iotdbDeviceSchemaInfo.isAligned(); - if (isAlignedInTsFile != isAlignedInIoTDB) { - throw new LoadAnalyzeException( - String.format( - "Device %s in TsFile is %s, but in IoTDB is %s.", - device, - isAlignedInTsFile ? "aligned" : "not aligned", - isAlignedInIoTDB ? "aligned" : "not aligned")); + if (LOGGER.isDebugEnabled() && isAlignedInTsFile != isAlignedInIoTDB) { + LOGGER.debug( + "Device {} in TsFile is {}, but in IoTDB is {}.", + device, + isAlignedInTsFile ? "aligned" : "not aligned", + isAlignedInIoTDB ? "aligned" : "not aligned"); } // check timeseries schema @@ -920,15 +919,14 @@ private void verifySchema(ISchemaTree schemaTree) } // check datatype - if (!tsFileSchema.getType().equals(iotdbSchema.getType())) { - throw new LoadAnalyzeTypeMismatchException( - String.format( - "Measurement %s%s%s datatype not match, TsFile: %s, IoTDB: %s", - device, - TsFileConstant.PATH_SEPARATOR, - iotdbSchema.getMeasurementId(), - tsFileSchema.getType(), - iotdbSchema.getType())); + if (LOGGER.isDebugEnabled() && !tsFileSchema.getType().equals(iotdbSchema.getType())) { + LOGGER.debug( + "Measurement {}{}{} datatype not match, TsFile: {}, IoTDB: {}", + device, + TsFileConstant.PATH_SEPARATOR, + iotdbSchema.getMeasurementId(), + tsFileSchema.getType(), + iotdbSchema.getType()); } // check encoding From f019de4b8b13e303f577f57fd53dc161ecd30f31 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 10 Jul 2025 11:49:32 +0800 Subject: [PATCH 148/185] Update PipeDataNodeSinglePipeMetrics.java --- .../pipe/metric/overview/PipeDataNodeSinglePipeMetrics.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeSinglePipeMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeSinglePipeMetrics.java index 9c19187932fc..1840093a3475 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeSinglePipeMetrics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/overview/PipeDataNodeSinglePipeMetrics.java @@ -55,7 +55,7 @@ public class PipeDataNodeSinglePipeMetrics implements IMetricSet { private static Histogram PIPE_DATANODE_INSERTNODE_TRANSFER_TIME_HISTOGRAM = DoNothingMetricManager.DO_NOTHING_HISTOGRAM; - private static Histogram PIPE_DATANODE_EVENT_TRANSFER_TIME_HISTOGRAM = + private static Histogram PIPE_DATANODE_TSFILE_TRANSFER_TIME_HISTOGRAM = DoNothingMetricManager.DO_NOTHING_HISTOGRAM; //////////////////////////// bindTo & unbindFrom (metric framework) //////////////////////////// @@ -69,7 +69,7 @@ public void bindTo(final AbstractMetricService metricService) { MetricLevel.IMPORTANT, Tag.NAME.toString(), "insert_node"); - PIPE_DATANODE_EVENT_TRANSFER_TIME_HISTOGRAM = + PIPE_DATANODE_TSFILE_TRANSFER_TIME_HISTOGRAM = metricService.getOrCreateHistogram( Metric.PIPE_DATANODE_EVENT_TRANSFER.toString(), MetricLevel.IMPORTANT, @@ -306,7 +306,7 @@ public void decreaseTsFileEventCount( operator.decreaseTsFileEventCount(); operator.getTsFileTransferTimer().update(transferTime, TimeUnit.NANOSECONDS); - PIPE_DATANODE_EVENT_TRANSFER_TIME_HISTOGRAM.update(transferTime); + PIPE_DATANODE_TSFILE_TRANSFER_TIME_HISTOGRAM.update(transferTime); } public void increaseHeartbeatEventCount(final String pipeName, final long creationTime) { From 27b76eec1791fce224b94904c01ee07eba4109e2 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 10 Jul 2025 12:32:32 +0800 Subject: [PATCH 149/185] Added timeout in shutdown hook --- .../iotdb/commons/pipe/agent/task/PipeTaskAgent.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java index 9d18744d0bdf..55aa6c5f17a6 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/PipeTaskAgent.java @@ -34,6 +34,7 @@ import org.apache.iotdb.commons.pipe.agent.task.meta.PipeTemporaryMetaInAgent; import org.apache.iotdb.commons.pipe.agent.task.progress.CommitterKey; import org.apache.iotdb.commons.pipe.agent.task.progress.PipeEventCommitManager; +import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.connector.limiter.PipeEndPointRateLimiter; import org.apache.iotdb.commons.subscription.config.SubscriptionConfig; import org.apache.iotdb.mpp.rpc.thrift.TPipeHeartbeatReq; @@ -55,6 +56,7 @@ import java.util.Objects; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; @@ -431,7 +433,11 @@ protected List handlePipeMetaChangesInternal( } public void dropAllPipeTasks() { - acquireWriteLock(); + if (!tryWriteLockWithTimeOut( + TimeUnit.MILLISECONDS.toSeconds(PipeConfig.getInstance().getPipeMaxWaitFinishTime()))) { + LOGGER.info("Failed to acquire lock when dropping all pipe tasks, will skip dropping"); + return; + } try { dropAllPipeTasksInternal(); } finally { From 9823ffea4834f8fe199ed3635551136c3a9ac8fe Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Thu, 10 Jul 2025 14:27:42 +0800 Subject: [PATCH 150/185] Fixed the issue that the isTotallyGeneratedByPipe flag is invalid when MemTable is null --- .../dataregion/flush/NotifyFlushMemTable.java | 7 ------- .../dataregion/memtable/AbstractMemTable.java | 13 ------------- .../dataregion/memtable/IMemTable.java | 4 ---- .../dataregion/memtable/TsFileProcessor.java | 19 +++++++++++++++---- 4 files changed, 15 insertions(+), 28 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/flush/NotifyFlushMemTable.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/flush/NotifyFlushMemTable.java index 9b730b7e3005..3eac19188e97 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/flush/NotifyFlushMemTable.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/flush/NotifyFlushMemTable.java @@ -36,11 +36,4 @@ public IMemTable copy() { public boolean isSignalMemTable() { return true; } - - @Override - public boolean isTotallyGeneratedByPipe() { - // Even though the `isTotallyGeneratedByPipe` for the corresponding memory table of this - // `NotifyFlushMemTable` might be true, we still return false to ensure data integrity. - return false; - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractMemTable.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractMemTable.java index 6df59ed12201..dcae00700ed8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractMemTable.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/AbstractMemTable.java @@ -67,7 +67,6 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Objects; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; @@ -122,8 +121,6 @@ public abstract class AbstractMemTable implements IMemTable { private static final String METRIC_POINT_IN = Metric.POINTS_IN.toString(); - private final AtomicBoolean isTotallyGeneratedByPipe = new AtomicBoolean(true); - protected AbstractMemTable() { this.database = null; this.dataRegionId = null; @@ -990,14 +987,4 @@ public void setDatabaseAndDataRegionId(String database, String dataRegionId) { this.database = database; this.dataRegionId = dataRegionId; } - - @Override - public void markAsNotGeneratedByPipe() { - this.isTotallyGeneratedByPipe.set(false); - } - - @Override - public boolean isTotallyGeneratedByPipe() { - return this.isTotallyGeneratedByPipe.get(); - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IMemTable.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IMemTable.java index 9de36b73eae8..3a7e9c55092f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IMemTable.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/IMemTable.java @@ -203,9 +203,5 @@ void queryForDeviceRegionScan( void setDatabaseAndDataRegionId(String database, String dataRegionId); - void markAsNotGeneratedByPipe(); - - boolean isTotallyGeneratedByPipe(); - void updateMemtablePointCountMetric(InsertNode insertNode, int pointsInserted); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java index e4f095b52c78..413b56af9ca3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java @@ -110,6 +110,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -187,6 +188,8 @@ public class TsFileProcessor { /** Total memtable size for mem control. */ private long totalMemTableSize; + private final AtomicBoolean isTotallyGeneratedByPipe = new AtomicBoolean(true); + private static final String FLUSH_QUERY_WRITE_LOCKED = "{}: {} get flushQueryLock write lock"; private static final String FLUSH_QUERY_WRITE_RELEASE = "{}: {} get flushQueryLock write lock released"; @@ -317,7 +320,7 @@ public void insert(InsertRowNode insertRowNode, long[] costsForMetrics) PipeDataNodeAgent.runtime().assignSimpleProgressIndexIfNeeded(insertRowNode); if (!insertRowNode.isGeneratedByPipe()) { - workMemTable.markAsNotGeneratedByPipe(); + markAsNotGeneratedByPipe(); } PipeInsertionDataNodeListener.getInstance() .listenToInsertNode( @@ -415,7 +418,7 @@ public void insert(InsertRowsNode insertRowsNode, long[] costsForMetrics) PipeDataNodeAgent.runtime().assignSimpleProgressIndexIfNeeded(insertRowsNode); if (!insertRowsNode.isGeneratedByPipe()) { - workMemTable.markAsNotGeneratedByPipe(); + markAsNotGeneratedByPipe(); } PipeInsertionDataNodeListener.getInstance() .listenToInsertNode( @@ -530,7 +533,7 @@ public void insertTablet( PipeDataNodeAgent.runtime().assignSimpleProgressIndexIfNeeded(insertTabletNode); if (!insertTabletNode.isGeneratedByPipe()) { - workMemTable.markAsNotGeneratedByPipe(); + markAsNotGeneratedByPipe(); } PipeInsertionDataNodeListener.getInstance() .listenToInsertNode( @@ -1578,7 +1581,7 @@ private void endFile() throws IOException, TsFileProcessorException { dataRegionInfo.getDataRegion().getDataRegionId(), tsFileResource, false, - workMemTable != null && workMemTable.isTotallyGeneratedByPipe()); + isTotallyGeneratedByPipe()); tsFileResource.serialize(); FileTimeIndexCacheRecorder.getInstance().logFileTimeIndex(tsFileResource); @@ -2170,4 +2173,12 @@ private void logFlushQueryReadUnlocked() { "{}: {} release flushQueryLock", storageGroupName, tsFileResource.getTsFile().getName()); } } + + private void markAsNotGeneratedByPipe() { + this.isTotallyGeneratedByPipe.set(false); + } + + private boolean isTotallyGeneratedByPipe() { + return this.isTotallyGeneratedByPipe.get(); + } } From a23f942013844aeaad98c8a33e7cb6d83c89b01e Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Thu, 10 Jul 2025 14:35:55 +0800 Subject: [PATCH 151/185] Simplify the function to set the isTotallyGeneratedByPipe flag --- .../dataregion/memtable/TsFileProcessor.java | 16 ++++------------ 1 file changed, 4 insertions(+), 12 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java index 413b56af9ca3..ad77ab22f748 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java @@ -320,7 +320,7 @@ public void insert(InsertRowNode insertRowNode, long[] costsForMetrics) PipeDataNodeAgent.runtime().assignSimpleProgressIndexIfNeeded(insertRowNode); if (!insertRowNode.isGeneratedByPipe()) { - markAsNotGeneratedByPipe(); + this.isTotallyGeneratedByPipe.set(false); } PipeInsertionDataNodeListener.getInstance() .listenToInsertNode( @@ -418,7 +418,7 @@ public void insert(InsertRowsNode insertRowsNode, long[] costsForMetrics) PipeDataNodeAgent.runtime().assignSimpleProgressIndexIfNeeded(insertRowsNode); if (!insertRowsNode.isGeneratedByPipe()) { - markAsNotGeneratedByPipe(); + this.isTotallyGeneratedByPipe.set(false); } PipeInsertionDataNodeListener.getInstance() .listenToInsertNode( @@ -533,7 +533,7 @@ public void insertTablet( PipeDataNodeAgent.runtime().assignSimpleProgressIndexIfNeeded(insertTabletNode); if (!insertTabletNode.isGeneratedByPipe()) { - markAsNotGeneratedByPipe(); + this.isTotallyGeneratedByPipe.set(false); } PipeInsertionDataNodeListener.getInstance() .listenToInsertNode( @@ -1581,7 +1581,7 @@ private void endFile() throws IOException, TsFileProcessorException { dataRegionInfo.getDataRegion().getDataRegionId(), tsFileResource, false, - isTotallyGeneratedByPipe()); + this.isTotallyGeneratedByPipe.get()); tsFileResource.serialize(); FileTimeIndexCacheRecorder.getInstance().logFileTimeIndex(tsFileResource); @@ -2173,12 +2173,4 @@ private void logFlushQueryReadUnlocked() { "{}: {} release flushQueryLock", storageGroupName, tsFileResource.getTsFile().getName()); } } - - private void markAsNotGeneratedByPipe() { - this.isTotallyGeneratedByPipe.set(false); - } - - private boolean isTotallyGeneratedByPipe() { - return this.isTotallyGeneratedByPipe.get(); - } } From 9937a915b869378d9eb6d41b78b709ab6b7b2704 Mon Sep 17 00:00:00 2001 From: Steve Yurong Su Date: Thu, 10 Jul 2025 14:44:26 +0800 Subject: [PATCH 152/185] Update LoadTsFileAnalyzer.java --- .../queryengine/plan/analyze/load/LoadTsFileAnalyzer.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java index ccf5b2c215f7..171f2b2f87f9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java @@ -896,8 +896,8 @@ private void verifySchema(ISchemaTree schemaTree) // check device schema: is aligned or not final boolean isAlignedInTsFile = schemaCache.getDeviceIsAligned(device); final boolean isAlignedInIoTDB = iotdbDeviceSchemaInfo.isAligned(); - if (LOGGER.isDebugEnabled() && isAlignedInTsFile != isAlignedInIoTDB) { - LOGGER.debug( + if (LOGGER.isInfoEnabled() && isAlignedInTsFile != isAlignedInIoTDB) { + LOGGER.info( "Device {} in TsFile is {}, but in IoTDB is {}.", device, isAlignedInTsFile ? "aligned" : "not aligned", @@ -919,8 +919,8 @@ private void verifySchema(ISchemaTree schemaTree) } // check datatype - if (LOGGER.isDebugEnabled() && !tsFileSchema.getType().equals(iotdbSchema.getType())) { - LOGGER.debug( + if (LOGGER.isInfoEnabled() && !tsFileSchema.getType().equals(iotdbSchema.getType())) { + LOGGER.info( "Measurement {}{}{} datatype not match, TsFile: {}, IoTDB: {}", device, TsFileConstant.PATH_SEPARATOR, From 94e9585d4640f45bcc18e4e3f5a281b9cdb27c3c Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 10 Jul 2025 17:30:17 +0800 Subject: [PATCH 153/185] Update PipeConfigNodeTaskAgent.java --- .../agent/task/PipeConfigNodeTaskAgent.java | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/agent/task/PipeConfigNodeTaskAgent.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/agent/task/PipeConfigNodeTaskAgent.java index 03641f411fa8..b79fd8da94f5 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/agent/task/PipeConfigNodeTaskAgent.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/agent/task/PipeConfigNodeTaskAgent.java @@ -207,21 +207,19 @@ protected void collectPipeMetaListInternal( if (isShutdown() || !PipeConfigNodeAgent.runtime().isLeaderReady()) { return; } - + final Optional logger = + PipeConfigNodeResourceManager.log() + .schedule( + PipeConfigNodeTaskAgent.class, + PipeConfig.getInstance().getPipeMetaReportMaxLogNumPerRound(), + PipeConfig.getInstance().getPipeMetaReportMaxLogIntervalRounds(), + pipeMetaKeeper.getPipeMetaCount()); LOGGER.debug("Received pipe heartbeat request {} from config coordinator.", req.heartbeatId); final List pipeMetaBinaryList = new ArrayList<>(); final List pipeRemainingEventCountList = new ArrayList<>(); final List pipeRemainingTimeList = new ArrayList<>(); try { - final Optional logger = - PipeConfigNodeResourceManager.log() - .schedule( - PipeConfigNodeTaskAgent.class, - PipeConfig.getInstance().getPipeMetaReportMaxLogNumPerRound(), - PipeConfig.getInstance().getPipeMetaReportMaxLogIntervalRounds(), - pipeMetaKeeper.getPipeMetaCount()); - for (final PipeMeta pipeMeta : pipeMetaKeeper.getPipeMetaList()) { pipeMetaBinaryList.add(pipeMeta.serialize()); @@ -244,7 +242,7 @@ protected void collectPipeMetaListInternal( remainingEventCount, estimatedRemainingTime)); } - LOGGER.info("Reported {} pipe metas.", pipeMetaBinaryList.size()); + logger.ifPresent(l -> l.info("Reported {} pipe metas.", pipeMetaBinaryList.size())); } catch (final IOException e) { throw new TException(e); } From 611d7aa01132ecb2c846975928ec2953290504a6 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Thu, 10 Jul 2025 17:46:01 +0800 Subject: [PATCH 154/185] Update PipeDataNodeTaskAgent.java --- .../pipe/agent/task/PipeDataNodeTaskAgent.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java index 52badab4da22..7f5d4e9cbf29 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeDataNodeTaskAgent.java @@ -381,6 +381,13 @@ private void collectPipeMetaListInternal(final TDataNodeHeartbeatResp resp) thro if (PipeDataNodeAgent.runtime().isShutdown()) { return; } + final Optional logger = + PipeDataNodeResourceManager.log() + .schedule( + PipeDataNodeTaskAgent.class, + PipeConfig.getInstance().getPipeMetaReportMaxLogNumPerRound(), + PipeConfig.getInstance().getPipeMetaReportMaxLogIntervalRounds(), + pipeMetaKeeper.getPipeMetaCount()); final Set dataRegionIds = StorageEngine.getInstance().getAllDataRegionIds().stream() @@ -392,13 +399,6 @@ private void collectPipeMetaListInternal(final TDataNodeHeartbeatResp resp) thro final List pipeRemainingEventCountList = new ArrayList<>(); final List pipeRemainingTimeList = new ArrayList<>(); try { - final Optional logger = - PipeDataNodeResourceManager.log() - .schedule( - PipeDataNodeTaskAgent.class, - PipeConfig.getInstance().getPipeMetaReportMaxLogNumPerRound(), - PipeConfig.getInstance().getPipeMetaReportMaxLogIntervalRounds(), - pipeMetaKeeper.getPipeMetaCount()); for (final PipeMeta pipeMeta : pipeMetaKeeper.getPipeMetaList()) { pipeMetaBinaryList.add(pipeMeta.serialize()); @@ -445,7 +445,7 @@ private void collectPipeMetaListInternal(final TDataNodeHeartbeatResp resp) thro remainingEventAndTime.getLeft(), remainingEventAndTime.getRight())); } - LOGGER.info("Reported {} pipe metas.", pipeMetaBinaryList.size()); + logger.ifPresent(l -> l.info("Reported {} pipe metas.", pipeMetaBinaryList.size())); } catch (final IOException | IllegalPathException e) { throw new TException(e); } From e2364c3e94a78f6e6bb2155ab8b5ce05a4d440ff Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 11 Jul 2025 10:59:22 +0800 Subject: [PATCH 155/185] Update IoTDBPipeTypeConversionISessionIT.java --- .../it/manual/IoTDBPipeTypeConversionISessionIT.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeTypeConversionISessionIT.java b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeTypeConversionISessionIT.java index 64d872001486..fc2b32e145bf 100644 --- a/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeTypeConversionISessionIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/pipe/it/manual/IoTDBPipeTypeConversionISessionIT.java @@ -44,6 +44,7 @@ import org.apache.tsfile.write.record.Tablet; import org.apache.tsfile.write.schema.IMeasurementSchema; import org.apache.tsfile.write.schema.MeasurementSchema; +import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -76,6 +77,7 @@ public void insertTablet() { } @Test + @Ignore("The receiver conversion is currently banned, will ignore conflict") public void insertTabletReceiveByTsFile() { prepareTypeConversionTest( (ISession senderSession, ISession receiverSession, Tablet tablet) -> { @@ -94,6 +96,7 @@ public void insertAlignedTablet() { } @Test + @Ignore("The receiver conversion is currently banned, will ignore conflict") public void insertAlignedTabletReceiveByTsFile() { prepareTypeConversionTest( (ISession senderSession, ISession receiverSession, Tablet tablet) -> { @@ -103,6 +106,7 @@ public void insertAlignedTabletReceiveByTsFile() { } @Test + @Ignore("The receiver conversion is currently banned, will ignore conflict") public void insertRecordsReceiveByTsFile() { prepareTypeConversionTest( (ISession senderSession, ISession receiverSession, Tablet tablet) -> { @@ -137,6 +141,7 @@ public void insertRecord() { } @Test + @Ignore("The receiver conversion is currently banned, will ignore conflict") public void insertRecordReceiveByTsFile() { prepareTypeConversionTest( (ISession senderSession, ISession receiverSession, Tablet tablet) -> { @@ -177,6 +182,7 @@ public void insertAlignedRecord() { } @Test + @Ignore("The receiver conversion is currently banned, will ignore conflict") public void insertAlignedRecordReceiveByTsFile() { prepareTypeConversionTest( (ISession senderSession, ISession receiverSession, Tablet tablet) -> { @@ -225,6 +231,7 @@ public void insertAlignedRecords() { } @Test + @Ignore("The receiver conversion is currently banned, will ignore conflict") public void insertAlignedRecordsReceiveByTsFile() { prepareTypeConversionTest( (ISession senderSession, ISession receiverSession, Tablet tablet) -> { @@ -253,6 +260,7 @@ public void insertStringRecordsOfOneDevice() { } @Test + @Ignore("The receiver conversion is currently banned, will ignore conflict") public void insertStringRecordsOfOneDeviceReceiveByTsFile() { prepareTypeConversionTest( (ISession senderSession, ISession receiverSession, Tablet tablet) -> { @@ -281,6 +289,7 @@ public void insertAlignedStringRecordsOfOneDevice() { } @Test + @Ignore("The receiver conversion is currently banned, will ignore conflict") public void insertAlignedStringRecordsOfOneDeviceReceiveByTsFile() { prepareTypeConversionTest( (ISession senderSession, ISession receiverSession, Tablet tablet) -> { From f7f25d7c1a019da7894020594e3cfa78adeeb1e4 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Fri, 11 Jul 2025 16:42:17 +0800 Subject: [PATCH 156/185] Add Tablet Batch and TSFile Batch related metrics --- .../evolvable/batch/PipeTabletEventBatch.java | 3 ++ .../batch/PipeTabletEventPlainBatch.java | 7 ++++ .../batch/PipeTabletEventTsFileBatch.java | 7 ++++ .../sink/PipeDataRegionConnectorMetrics.java | 34 +++++++++++++++++++ .../commons/service/metric/enums/Metric.java | 4 +++ 5 files changed, 55 insertions(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventBatch.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventBatch.java index 78a237a4102e..8d52055fcc97 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventBatch.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventBatch.java @@ -129,11 +129,14 @@ public boolean shouldEmit() { final long diff = System.currentTimeMillis() - firstEventProcessingTime; if (totalBufferSize >= getMaxBatchSizeInBytes() || diff >= maxDelayInMs) { allocatedMemoryBlock.updateCurrentMemoryEfficiencyAdjustMem((double) diff / maxDelayInMs); + recordMetric(diff, totalBufferSize); return true; } return false; } + protected abstract void recordMetric(final long timeInterval, final long bufferSize); + private long getMaxBatchSizeInBytes() { return allocatedMemoryBlock.getMemoryUsageInBytes(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventPlainBatch.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventPlainBatch.java index 33f228f42fb8..70ffaddb1800 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventPlainBatch.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventPlainBatch.java @@ -23,6 +23,7 @@ import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferTabletBatchReq; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; +import org.apache.iotdb.db.pipe.metric.sink.PipeDataRegionConnectorMetrics; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode; import org.apache.iotdb.db.storageengine.dataregion.wal.exception.WALPipeException; import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; @@ -70,6 +71,12 @@ protected boolean constructBatch(final TabletInsertionEvent event) return true; } + @Override + protected void recordMetric(long timeInterval, long bufferSize) { + PipeDataRegionConnectorMetrics.tabletBatchTimeIntervalHistogram.update(timeInterval); + PipeDataRegionConnectorMetrics.tabletBatchSizeHistogram.update(bufferSize); + } + @Override public synchronized void onSuccess() { super.onSuccess(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java index fd4ce71b6f92..98e27d4e7c60 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/payload/evolvable/batch/PipeTabletEventTsFileBatch.java @@ -25,6 +25,7 @@ import org.apache.iotdb.db.pipe.connector.util.PipeTabletEventSorter; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; +import org.apache.iotdb.db.pipe.metric.sink.PipeDataRegionConnectorMetrics; import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryWeightUtil; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertTabletNode; @@ -181,6 +182,12 @@ protected boolean constructBatch(final TabletInsertionEvent event) { return true; } + @Override + protected void recordMetric(long timeInterval, long bufferSize) { + PipeDataRegionConnectorMetrics.tsFileBatchTimeIntervalHistogram.update(timeInterval); + PipeDataRegionConnectorMetrics.tsFileBatchSizeHistogram.update(bufferSize); + } + private void bufferTablet( final String pipeName, final long creationTime, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/sink/PipeDataRegionConnectorMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/sink/PipeDataRegionConnectorMetrics.java index 80ebb272c96e..49ac24f3e4cd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/sink/PipeDataRegionConnectorMetrics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/metric/sink/PipeDataRegionConnectorMetrics.java @@ -23,7 +23,9 @@ import org.apache.iotdb.commons.service.metric.enums.Tag; import org.apache.iotdb.db.pipe.agent.task.subtask.connector.PipeConnectorSubtask; import org.apache.iotdb.metrics.AbstractMetricService; +import org.apache.iotdb.metrics.impl.DoNothingHistogram; import org.apache.iotdb.metrics.metricsets.IMetricSet; +import org.apache.iotdb.metrics.type.Histogram; import org.apache.iotdb.metrics.type.Rate; import org.apache.iotdb.metrics.type.Timer; import org.apache.iotdb.metrics.utils.MetricLevel; @@ -44,6 +46,14 @@ public class PipeDataRegionConnectorMetrics implements IMetricSet { private static final Logger LOGGER = LoggerFactory.getLogger(PipeDataRegionConnectorMetrics.class); + public static Histogram tabletBatchSizeHistogram = new DoNothingHistogram(); + + public static Histogram tsFileBatchSizeHistogram = new DoNothingHistogram(); + + public static Histogram tabletBatchTimeIntervalHistogram = new DoNothingHistogram(); + + public static Histogram tsFileBatchTimeIntervalHistogram = new DoNothingHistogram(); + @SuppressWarnings("java:S3077") private volatile AbstractMetricService metricService; @@ -66,6 +76,22 @@ public void bindTo(final AbstractMetricService metricService) { for (String taskID : taskIDs) { createMetrics(taskID); } + + tabletBatchSizeHistogram = + metricService.getOrCreateHistogram( + Metric.PIPE_INSERT_NODE_BATCH_SIZE.toString(), MetricLevel.IMPORTANT); + + tsFileBatchSizeHistogram = + metricService.getOrCreateHistogram( + Metric.PIPE_TSFILE_BATCH_SIZE.toString(), MetricLevel.IMPORTANT); + + tabletBatchTimeIntervalHistogram = + metricService.getOrCreateHistogram( + Metric.PIPE_INSERT_NODE_BATCH_TIME_COST.toString(), MetricLevel.IMPORTANT); + + tsFileBatchTimeIntervalHistogram = + metricService.getOrCreateHistogram( + Metric.PIPE_TSFILE_BATCH_TIME_COST.toString(), MetricLevel.IMPORTANT); } private void createMetrics(final String taskID) { @@ -230,6 +256,14 @@ public void unbindFrom(final AbstractMetricService metricService) { LOGGER.warn( "Failed to unbind from pipe data region connector metrics, connector map not empty"); } + + metricService.remove(MetricType.HISTOGRAM, Metric.PIPE_INSERT_NODE_BATCH_SIZE.toString()); + + metricService.remove(MetricType.HISTOGRAM, Metric.PIPE_TSFILE_BATCH_SIZE.toString()); + + metricService.remove(MetricType.HISTOGRAM, Metric.PIPE_INSERT_NODE_BATCH_TIME_COST.toString()); + + metricService.remove(MetricType.HISTOGRAM, Metric.PIPE_TSFILE_BATCH_TIME_COST.toString()); } private void removeMetrics(final String taskID) { diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/Metric.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/Metric.java index 1b177ee27e17..cd91c788884f 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/Metric.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/service/metric/enums/Metric.java @@ -139,6 +139,10 @@ public enum Metric { UNTRANSFERRED_TABLET_COUNT("untransferred_tablet_count"), UNTRANSFERRED_TSFILE_COUNT("untransferred_tsfile_count"), UNTRANSFERRED_HEARTBEAT_COUNT("untransferred_heartbeat_count"), + PIPE_INSERT_NODE_BATCH_SIZE("pipe_insert_node_batch_size"), + PIPE_TSFILE_BATCH_SIZE("pipe_tsfile_batch_size"), + PIPE_INSERT_NODE_BATCH_TIME_COST("pipe_insert_node_batch_time_cost"), + PIPE_TSFILE_BATCH_TIME_COST("pipe_tsfile_batch_time_cost"), PIPE_CONNECTOR_BATCH_SIZE("pipe_connector_batch_size"), PIPE_PENDING_HANDLERS_SIZE("pipe_pending_handlers_size"), PIPE_TOTAL_UNCOMPRESSED_SIZE("pipe_total_uncompressed_size"), From 3feb5f79c16d2a487696294656f14295b752c489 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 11 Jul 2025 18:11:13 +0800 Subject: [PATCH 157/185] Update CompactionUtils.java --- .../dataregion/compaction/execute/utils/CompactionUtils.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/CompactionUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/CompactionUtils.java index 3b251a0c909a..d0f79cb992b7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/CompactionUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/compaction/execute/utils/CompactionUtils.java @@ -287,6 +287,9 @@ public static void updateProgressIndexAndMark( List seqResources, List unseqResources) { for (TsFileResource targetResource : targetResources) { + // Initial value + targetResource.setGeneratedByPipe(true); + targetResource.setGeneratedByPipeConsensus(true); for (TsFileResource unseqResource : unseqResources) { targetResource.updateProgressIndex(unseqResource.getMaxProgressIndex()); targetResource.setGeneratedByPipe( From 9ced07f8d076071f54a5a4b4b69588216e17a98e Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Fri, 11 Jul 2025 20:21:14 +0800 Subject: [PATCH 158/185] Partial --- .../thrift/async/IoTDBDataRegionAsyncConnector.java | 2 +- .../pipe/connector/protocol/IoTDBConnector.java | 12 ++++++++++++ 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java index 502167f87385..07f803fe7413 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java @@ -419,7 +419,7 @@ private void transfer(final PipeTransferTsFileHandler pipeTransferTsFileHandler) }, executor); - if (PipeConfig.getInstance().isTransferTsFileSync()) { + if (PipeConfig.getInstance().isTransferTsFileSync() || !isRealtimeFirst) { try { completableFuture.get(); } catch (InterruptedException e) { diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBConnector.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBConnector.java index 561b61b2391c..a7dacb49553f 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBConnector.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/protocol/IoTDBConnector.java @@ -20,6 +20,7 @@ package org.apache.iotdb.commons.pipe.connector.protocol; import org.apache.iotdb.common.rpc.thrift.TEndPoint; +import org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant; import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskConnectorRuntimeEnvironment; import org.apache.iotdb.commons.pipe.connector.compressor.PipeCompressor; import org.apache.iotdb.commons.pipe.connector.compressor.PipeCompressorConfig; @@ -176,6 +177,7 @@ public abstract class IoTDBConnector implements PipeConnector { private final AtomicLong totalCompressedSize = new AtomicLong(0); protected String attributeSortedString; protected Timer compressionTimer; + protected boolean isRealtimeFirst; @Override public void validate(final PipeParameterValidator validator) throws Exception { @@ -465,6 +467,16 @@ public void customize( "IoTDBConnector {} = {}", CONNECTOR_EXCEPTION_DATA_CONVERT_ON_TYPE_MISMATCH_KEY, shouldReceiverConvertOnTypeMismatch); + isRealtimeFirst = + parameters.getBooleanOrDefault( + Arrays.asList( + PipeConnectorConstant.CONNECTOR_REALTIME_FIRST_KEY, + PipeConnectorConstant.SINK_REALTIME_FIRST_KEY), + PipeConnectorConstant.CONNECTOR_REALTIME_FIRST_DEFAULT_VALUE); + LOGGER.info( + "IoTDBConnector {} = {}", + PipeConnectorConstant.CONNECTOR_REALTIME_FIRST_KEY, + isRealtimeFirst); } protected LinkedHashSet parseNodeUrls(final PipeParameters parameters) From 5afc004fc128a9c22fc071cc30f70fc48f6b77ae Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Sat, 12 Jul 2025 11:53:50 +0800 Subject: [PATCH 159/185] Separated the pipe thread pool --- .../task/PipeConfigNodeSubtaskExecutor.java | 4 ++-- .../task/builder/PipeDataNodeTaskBuilder.java | 23 +++++-------------- .../PipeConnectorSubtaskExecutor.java | 7 ++++-- .../PipeProcessorSubtaskExecutor.java | 2 +- .../execution/PipeSubtaskExecutorManager.java | 18 ++++++++------- .../task/stage/PipeTaskConnectorStage.java | 6 +++-- .../PipeConnectorSubtaskManager.java | 14 ++++++++++- .../PipeConsensusSubtaskExecutor.java | 6 ++++- .../SubscriptionSubtaskExecutor.java | 9 +++----- .../stage/SubscriptionTaskConnectorStage.java | 4 ++-- .../iotdb/commons/conf/CommonConfig.java | 3 +-- .../task/execution/PipeSubtaskExecutor.java | 10 ++++---- 12 files changed, 58 insertions(+), 48 deletions(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/agent/task/PipeConfigNodeSubtaskExecutor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/agent/task/PipeConfigNodeSubtaskExecutor.java index 9aea3edfa38b..1ac64eb246f0 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/agent/task/PipeConfigNodeSubtaskExecutor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/agent/task/PipeConfigNodeSubtaskExecutor.java @@ -28,7 +28,7 @@ public class PipeConfigNodeSubtaskExecutor extends PipeSubtaskExecutor { private static final int THREAD_NUM = 1; private PipeConfigNodeSubtaskExecutor() { - super(THREAD_NUM, ThreadName.PIPE_CONFIGNODE_EXECUTOR_POOL, true); + super(THREAD_NUM, ThreadName.PIPE_CONFIGNODE_EXECUTOR_POOL.getName(), true); } /** @@ -36,7 +36,7 @@ private PipeConfigNodeSubtaskExecutor() { */ @TestOnly public PipeConfigNodeSubtaskExecutor(final Object ignored) { - super(THREAD_NUM, ThreadName.PIPE_CONFIGNODE_EXECUTOR_POOL, true); + super(THREAD_NUM, ThreadName.PIPE_CONFIGNODE_EXECUTOR_POOL.getName(), true); } private static class PipeSchemaSubtaskExecutorHolder { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/builder/PipeDataNodeTaskBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/builder/PipeDataNodeTaskBuilder.java index 5e7901cb7c77..f84cb73fd692 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/builder/PipeDataNodeTaskBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/builder/PipeDataNodeTaskBuilder.java @@ -27,7 +27,6 @@ import org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant; import org.apache.iotdb.commons.pipe.config.constant.SystemConstant; import org.apache.iotdb.db.pipe.agent.task.PipeDataNodeTask; -import org.apache.iotdb.db.pipe.agent.task.execution.PipeConnectorSubtaskExecutor; import org.apache.iotdb.db.pipe.agent.task.execution.PipeProcessorSubtaskExecutor; import org.apache.iotdb.db.pipe.agent.task.execution.PipeSubtaskExecutorManager; import org.apache.iotdb.db.pipe.agent.task.stage.PipeTaskConnectorStage; @@ -42,7 +41,6 @@ import org.slf4j.LoggerFactory; import java.util.Arrays; -import java.util.EnumMap; import java.util.HashMap; import java.util.Map; @@ -64,19 +62,8 @@ public class PipeDataNodeTaskBuilder { private final int regionId; private final PipeTaskMeta pipeTaskMeta; - private static final PipeProcessorSubtaskExecutor PROCESSOR_EXECUTOR; - private static final Map CONNECTOR_EXECUTOR_MAP; - - static { - PROCESSOR_EXECUTOR = PipeSubtaskExecutorManager.getInstance().getProcessorExecutor(); - CONNECTOR_EXECUTOR_MAP = new EnumMap<>(PipeType.class); - CONNECTOR_EXECUTOR_MAP.put( - PipeType.USER, PipeSubtaskExecutorManager.getInstance().getConnectorExecutor()); - CONNECTOR_EXECUTOR_MAP.put( - PipeType.SUBSCRIPTION, PipeSubtaskExecutorManager.getInstance().getSubscriptionExecutor()); - CONNECTOR_EXECUTOR_MAP.put( - PipeType.CONSENSUS, PipeSubtaskExecutorManager.getInstance().getConsensusExecutor()); - } + private static final PipeProcessorSubtaskExecutor PROCESSOR_EXECUTOR = + PipeSubtaskExecutorManager.getInstance().getProcessorExecutor(); protected final Map systemParameters = new HashMap<>(); @@ -117,7 +104,7 @@ public PipeDataNodeTask build() { pipeStaticMeta.getCreationTime(), connectorParameters, regionId, - CONNECTOR_EXECUTOR_MAP.get(pipeType)); + PipeSubtaskExecutorManager.getInstance().getSubscriptionExecutor()); } else { // user pipe or consensus pipe connectorStage = new PipeTaskConnectorStage( @@ -125,7 +112,9 @@ public PipeDataNodeTask build() { pipeStaticMeta.getCreationTime(), connectorParameters, regionId, - CONNECTOR_EXECUTOR_MAP.get(pipeType)); + pipeType.equals(PipeType.USER) + ? PipeSubtaskExecutorManager.getInstance().getConnectorExecutorSupplier() + : PipeSubtaskExecutorManager.getInstance().getConsensusExecutorSupplier()); } // The processor connects the extractor and connector. diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/execution/PipeConnectorSubtaskExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/execution/PipeConnectorSubtaskExecutor.java index fcb4888dbde5..96692c3cedd5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/execution/PipeConnectorSubtaskExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/execution/PipeConnectorSubtaskExecutor.java @@ -23,16 +23,19 @@ import org.apache.iotdb.commons.pipe.agent.task.execution.PipeSubtaskExecutor; import org.apache.iotdb.commons.pipe.config.PipeConfig; +import java.util.concurrent.atomic.AtomicInteger; + public class PipeConnectorSubtaskExecutor extends PipeSubtaskExecutor { + private static final AtomicInteger id = new AtomicInteger(0); public PipeConnectorSubtaskExecutor() { super( PipeConfig.getInstance().getPipeSubtaskExecutorMaxThreadNum(), - ThreadName.PIPE_CONNECTOR_EXECUTOR_POOL, + ThreadName.PIPE_CONNECTOR_EXECUTOR_POOL.getName() + "-" + id.getAndIncrement(), true); } - public PipeConnectorSubtaskExecutor(final int corePoolSize, final ThreadName threadName) { + public PipeConnectorSubtaskExecutor(final int corePoolSize, final String threadName) { super(corePoolSize, threadName, true); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/execution/PipeProcessorSubtaskExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/execution/PipeProcessorSubtaskExecutor.java index f25d28b0c2c4..9eddd27f22e6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/execution/PipeProcessorSubtaskExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/execution/PipeProcessorSubtaskExecutor.java @@ -28,7 +28,7 @@ public class PipeProcessorSubtaskExecutor extends PipeSubtaskExecutor { public PipeProcessorSubtaskExecutor() { super( PipeConfig.getInstance().getPipeSubtaskExecutorMaxThreadNum(), - ThreadName.PIPE_PROCESSOR_EXECUTOR_POOL, + ThreadName.PIPE_PROCESSOR_EXECUTOR_POOL.getName(), false); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/execution/PipeSubtaskExecutorManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/execution/PipeSubtaskExecutorManager.java index f0de4d8d58e5..e0a71454d235 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/execution/PipeSubtaskExecutorManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/execution/PipeSubtaskExecutorManager.java @@ -23,42 +23,44 @@ import org.apache.iotdb.db.pipe.consensus.PipeConsensusSubtaskExecutor; import org.apache.iotdb.db.subscription.task.execution.SubscriptionSubtaskExecutor; +import java.util.function.Supplier; + /** * PipeTaskExecutor is responsible for executing the pipe tasks, and it is scheduled by the * PipeTaskScheduler. It is a singleton class. */ public class PipeSubtaskExecutorManager { private final PipeProcessorSubtaskExecutor processorExecutor; - private final PipeConnectorSubtaskExecutor connectorExecutor; + private final Supplier connectorExecutorSupplier; private final SubscriptionSubtaskExecutor subscriptionExecutor; - private final PipeConsensusSubtaskExecutor consensusExecutor; + private final Supplier consensusExecutorSupplier; public PipeProcessorSubtaskExecutor getProcessorExecutor() { return processorExecutor; } - public PipeConnectorSubtaskExecutor getConnectorExecutor() { - return connectorExecutor; + public Supplier getConnectorExecutorSupplier() { + return connectorExecutorSupplier; } public SubscriptionSubtaskExecutor getSubscriptionExecutor() { return subscriptionExecutor; } - public PipeConsensusSubtaskExecutor getConsensusExecutor() { - return consensusExecutor; + public Supplier getConsensusExecutorSupplier() { + return consensusExecutorSupplier; } ///////////////////////// Singleton Instance Holder ///////////////////////// private PipeSubtaskExecutorManager() { processorExecutor = new PipeProcessorSubtaskExecutor(); - connectorExecutor = new PipeConnectorSubtaskExecutor(); + connectorExecutorSupplier = PipeConnectorSubtaskExecutor::new; subscriptionExecutor = SubscriptionConfig.getInstance().getSubscriptionEnabled() ? new SubscriptionSubtaskExecutor() : null; - consensusExecutor = new PipeConsensusSubtaskExecutor(); + consensusExecutorSupplier = PipeConsensusSubtaskExecutor::new; } private static class PipeTaskExecutorHolder { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/stage/PipeTaskConnectorStage.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/stage/PipeTaskConnectorStage.java index 345e6ab040c4..e5ae5dacd278 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/stage/PipeTaskConnectorStage.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/stage/PipeTaskConnectorStage.java @@ -28,13 +28,15 @@ import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.pipe.api.exception.PipeException; +import java.util.function.Supplier; + public class PipeTaskConnectorStage extends PipeTaskStage { protected final String pipeName; protected final long creationTime; protected final PipeParameters pipeConnectorParameters; protected final int regionId; - protected final PipeConnectorSubtaskExecutor executor; + protected final Supplier executor; protected String connectorSubtaskId; @@ -43,7 +45,7 @@ public PipeTaskConnectorStage( long creationTime, PipeParameters pipeConnectorParameters, int regionId, - PipeConnectorSubtaskExecutor executor) { + Supplier executor) { this.pipeName = pipeName; this.creationTime = creationTime; this.pipeConnectorParameters = pipeConnectorParameters; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeConnectorSubtaskManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeConnectorSubtaskManager.java index e556da428ef4..0f84e3e3dd73 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeConnectorSubtaskManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeConnectorSubtaskManager.java @@ -48,6 +48,7 @@ import java.util.Map; import java.util.TreeMap; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; public class PipeConnectorSubtaskManager { @@ -60,7 +61,7 @@ public class PipeConnectorSubtaskManager { attributeSortedString2SubtaskLifeCycleMap = new HashMap<>(); public synchronized String register( - final PipeConnectorSubtaskExecutor executor, + final Supplier executorSupplier, final PipeParameters pipeConnectorParameters, final PipeTaskConnectorRuntimeEnvironment environment) { final String connectorKey = @@ -109,6 +110,8 @@ public synchronized String register( environment.setAttributeSortedString(attributeSortedString); if (!attributeSortedString2SubtaskLifeCycleMap.containsKey(attributeSortedString)) { + final PipeConnectorSubtaskExecutor executor = executorSupplier.get(); + final List pipeConnectorSubtaskLifeCycleList = new ArrayList<>(connectorNum); @@ -169,6 +172,10 @@ public synchronized String register( pipeConnectorSubtaskLifeCycleList.add(pipeConnectorSubtaskLifeCycle); } + LOGGER.info( + "Pipe connector subtasks with attributes {} is bounded with connectorExecutor {}.", + attributeSortedString, + executor.getThreadName()); attributeSortedString2SubtaskLifeCycleMap.put( attributeSortedString, pipeConnectorSubtaskLifeCycleList); } @@ -192,10 +199,15 @@ public synchronized void deregister( final List lifeCycles = attributeSortedString2SubtaskLifeCycleMap.get(attributeSortedString); + + // Shall not be empty + final PipeConnectorSubtaskExecutor executor = lifeCycles.get(0).executor; + lifeCycles.removeIf(o -> o.deregister(pipeName, regionId)); if (lifeCycles.isEmpty()) { attributeSortedString2SubtaskLifeCycleMap.remove(attributeSortedString); + executor.shutdown(); } PipeEventCommitManager.getInstance().deregister(pipeName, creationTime, regionId); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/PipeConsensusSubtaskExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/PipeConsensusSubtaskExecutor.java index 72782ad228e7..d50e08463e2d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/PipeConsensusSubtaskExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/PipeConsensusSubtaskExecutor.java @@ -22,11 +22,15 @@ import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.db.pipe.agent.task.execution.PipeConnectorSubtaskExecutor; +import java.util.concurrent.atomic.AtomicInteger; + public class PipeConsensusSubtaskExecutor extends PipeConnectorSubtaskExecutor { + private static final AtomicInteger id = new AtomicInteger(0); + public PipeConsensusSubtaskExecutor() { super( PipeConfig.getInstance().getPipeSubtaskExecutorMaxThreadNum(), - ThreadName.PIPE_CONSENSUS_EXECUTOR_POOL); + ThreadName.PIPE_CONSENSUS_EXECUTOR_POOL + "-" + id.getAndIncrement()); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/execution/SubscriptionSubtaskExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/execution/SubscriptionSubtaskExecutor.java index f473a56a4478..5dcc5d8943cc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/execution/SubscriptionSubtaskExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/execution/SubscriptionSubtaskExecutor.java @@ -26,25 +26,22 @@ import org.apache.iotdb.db.pipe.agent.task.execution.PipeConnectorSubtaskExecutor; import org.apache.iotdb.db.subscription.task.subtask.SubscriptionReceiverSubtask; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; public class SubscriptionSubtaskExecutor extends PipeConnectorSubtaskExecutor { - - private static final Logger LOGGER = LoggerFactory.getLogger(SubscriptionSubtaskExecutor.class); + private static final AtomicInteger id = new AtomicInteger(0); private final AtomicLong submittedReceiverSubtasks = new AtomicLong(0); public SubscriptionSubtaskExecutor() { super( SubscriptionConfig.getInstance().getSubscriptionSubtaskExecutorMaxThreadNum(), - ThreadName.SUBSCRIPTION_EXECUTOR_POOL); + ThreadName.SUBSCRIPTION_EXECUTOR_POOL.getName() + "-" + id.getAndIncrement()); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/stage/SubscriptionTaskConnectorStage.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/stage/SubscriptionTaskConnectorStage.java index 003d712cc5b4..7d26701004ad 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/stage/SubscriptionTaskConnectorStage.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/subscription/task/stage/SubscriptionTaskConnectorStage.java @@ -36,7 +36,7 @@ public SubscriptionTaskConnectorStage( PipeParameters pipeConnectorParameters, int regionId, PipeConnectorSubtaskExecutor executor) { - super(pipeName, creationTime, pipeConnectorParameters, regionId, executor); + super(pipeName, creationTime, pipeConnectorParameters, regionId, () -> executor); } @Override @@ -44,7 +44,7 @@ protected void registerSubtask() { this.connectorSubtaskId = SubscriptionConnectorSubtaskManager.instance() .register( - executor, + executor.get(), pipeConnectorParameters, new PipeTaskConnectorRuntimeEnvironment(pipeName, creationTime, regionId)); } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java index d3f3822d9212..e0f982d7ddc9 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java @@ -210,8 +210,7 @@ public class CommonConfig { private int pipeRealTimeQueueMaxWaitingTsFileSize = 1; /** The maximum number of threads that can be used to execute subtasks in PipeSubtaskExecutor. */ - private int pipeSubtaskExecutorMaxThreadNum = - Math.max(5, Runtime.getRuntime().availableProcessors() / 2); + private int pipeSubtaskExecutorMaxThreadNum = 5; private int pipeDataStructureTabletRowSize = 2048; private int pipeDataStructureTabletSizeInBytes = 2097152; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/execution/PipeSubtaskExecutor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/execution/PipeSubtaskExecutor.java index 4ea7714962bf..533825e75e0a 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/execution/PipeSubtaskExecutor.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/execution/PipeSubtaskExecutor.java @@ -49,12 +49,14 @@ public abstract class PipeSubtaskExecutor { private final int corePoolSize; private int runningSubtaskNumber; + private final String threadName; protected PipeSubtaskExecutor( - final int corePoolSize, final ThreadName threadName, final boolean disableLogInThreadPool) { + final int corePoolSize, final String threadName, final boolean disableLogInThreadPool) { + this.threadName = threadName; underlyingThreadPool = (WrappedThreadPoolExecutor) - IoTDBThreadPoolFactory.newFixedThreadPool(corePoolSize, threadName.getName()); + IoTDBThreadPoolFactory.newFixedThreadPool(corePoolSize, threadName); if (disableLogInThreadPool) { underlyingThreadPool.disableErrorLog(); } @@ -171,7 +173,7 @@ protected final boolean hasAvailableThread() { // return getAvailableThreadCount() > 0; } - private int getAvailableThreadCount() { - return underlyingThreadPool.getCorePoolSize() - underlyingThreadPool.getActiveCount(); + public String getThreadName() { + return threadName; } } From 7f74a90d0ec4dd50b0ecfcfb0c1ca58568cce443 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Sat, 12 Jul 2025 12:27:54 +0800 Subject: [PATCH 160/185] Update PipeConnectorSubtaskManager.java --- .../task/subtask/connector/PipeConnectorSubtaskManager.java | 1 + 1 file changed, 1 insertion(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeConnectorSubtaskManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeConnectorSubtaskManager.java index 0f84e3e3dd73..300254f8b6d0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeConnectorSubtaskManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeConnectorSubtaskManager.java @@ -208,6 +208,7 @@ public synchronized void deregister( if (lifeCycles.isEmpty()) { attributeSortedString2SubtaskLifeCycleMap.remove(attributeSortedString); executor.shutdown(); + LOGGER.info("The executor {} has been successfully shutdown.", executor.getThreadName()); } PipeEventCommitManager.getInstance().deregister(pipeName, creationTime, regionId); From 0a0ee9c9a11146e0642031377936f876b7293645 Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Sat, 12 Jul 2025 12:44:48 +0800 Subject: [PATCH 161/185] Fix the problem of creating too many PipeTransferTsFileHandlers, causing OOM & fix the problem of RetryTSFile grabbing the lock, causing the failure to Drop Pipe --- .../async/IoTDBDataRegionAsyncConnector.java | 93 +++++++++++++------ .../handler/PipeTransferTsFileHandler.java | 40 ++++---- 2 files changed, 90 insertions(+), 43 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java index 07f803fe7413..fedd1f6a9e11 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java @@ -71,12 +71,14 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -98,10 +100,17 @@ public class IoTDBDataRegionAsyncConnector extends IoTDBConnector { private static final boolean isSplitTSFileBatchModeEnabled = true; private static final ExecutorService executor = - Executors.newFixedThreadPool(PipeConfig.getInstance().getPipeAsyncConnectorMaxClientNumber()); + new ThreadPoolExecutor( + PipeConfig.getInstance().getPipeAsyncConnectorMaxTsFileClientNumber(), + PipeConfig.getInstance().getPipeAsyncConnectorMaxTsFileClientNumber(), + 0L, + TimeUnit.MILLISECONDS, + new ArrayBlockingQueue( + 10 * PipeConfig.getInstance().getPipeSubtaskExecutorMaxThreadNum())); private final IoTDBDataRegionSyncConnector syncConnector = new IoTDBDataRegionSyncConnector(); private final BlockingQueue retryEventQueue = new LinkedBlockingQueue<>(); + private final BlockingQueue retryTSFileQueue = new LinkedBlockingQueue<>(); private final PipeDataRegionEventCounter retryEventQueueEventCounter = new PipeDataRegionEventCounter(); @@ -497,7 +506,7 @@ private void logOnClientException( * @see PipeConnector#transfer(TsFileInsertionEvent) for more details. */ private void transferQueuedEventsIfNecessary(final boolean forced) { - if (retryEventQueue.isEmpty() + if ((retryEventQueue.isEmpty() && retryTSFileQueue.isEmpty()) || (!forced && retryEventQueueEventCounter.getTabletInsertionEventCount() < PipeConfig.getInstance() @@ -505,38 +514,50 @@ private void transferQueuedEventsIfNecessary(final boolean forced) { && retryEventQueueEventCounter.getTsFileInsertionEventCount() < PipeConfig.getInstance() .getPipeAsyncConnectorForcedRetryTsFileEventQueueSizeThreshold() - && retryEventQueue.size() + && retryEventQueue.size() + retryTSFileQueue.size() < PipeConfig.getInstance() .getPipeAsyncConnectorForcedRetryTotalEventQueueSizeThreshold())) { return; } final long retryStartTime = System.currentTimeMillis(); - final int remainingEvents = retryEventQueue.size(); - while (!retryEventQueue.isEmpty()) { + final int remainingEvents = retryEventQueue.size() + retryTSFileQueue.size(); + while (!retryEventQueue.isEmpty() && !retryTSFileQueue.isEmpty()) { synchronized (this) { if (isClosed.get()) { return; } - if (retryEventQueue.isEmpty()) { + if (retryEventQueue.isEmpty() && retryTSFileQueue.isEmpty()) { break; } - final Event peekedEvent = retryEventQueue.peek(); + final Event peekedEvent; + final Event polledEvent; + if (!retryEventQueue.isEmpty()) { + peekedEvent = retryEventQueue.peek(); + + if (peekedEvent instanceof PipeInsertNodeTabletInsertionEvent) { + retryTransfer((PipeInsertNodeTabletInsertionEvent) peekedEvent); + } else if (peekedEvent instanceof PipeRawTabletInsertionEvent) { + retryTransfer((PipeRawTabletInsertionEvent) peekedEvent); + } else { + LOGGER.warn( + "IoTDBThriftAsyncConnector does not support transfer generic event: {}.", + peekedEvent); + } - if (peekedEvent instanceof PipeInsertNodeTabletInsertionEvent) { - retryTransfer((PipeInsertNodeTabletInsertionEvent) peekedEvent); - } else if (peekedEvent instanceof PipeRawTabletInsertionEvent) { - retryTransfer((PipeRawTabletInsertionEvent) peekedEvent); - } else if (peekedEvent instanceof PipeTsFileInsertionEvent) { - retryTransfer((PipeTsFileInsertionEvent) peekedEvent); + polledEvent = retryEventQueue.poll(); } else { - LOGGER.warn( - "IoTDBThriftAsyncConnector does not support transfer generic event: {}.", - peekedEvent); + if (transferTsFileCounter.get() != 0) { + return; + } + peekedEvent = retryTSFileQueue.peek(); + + retryTransfer((PipeTsFileInsertionEvent) peekedEvent); + + polledEvent = retryTSFileQueue.poll(); } - final Event polledEvent = retryEventQueue.poll(); retryEventQueueEventCounter.decreaseEventCount(polledEvent); if (polledEvent != peekedEvent) { LOGGER.error( @@ -559,16 +580,16 @@ private void transferQueuedEventsIfNecessary(final boolean forced) { && retryEventQueueEventCounter.getTsFileInsertionEventCount() < PipeConfig.getInstance() .getPipeAsyncConnectorForcedRetryTsFileEventQueueSizeThreshold() - && retryEventQueue.size() + && retryEventQueue.size() + retryTSFileQueue.size() < PipeConfig.getInstance() .getPipeAsyncConnectorForcedRetryTotalEventQueueSizeThreshold()) { return; } - if (remainingEvents <= retryEventQueue.size()) { + if (remainingEvents <= retryEventQueue.size() + retryTSFileQueue.size()) { throw new PipeException( "Failed to retry transferring events in the retry queue. Remaining events: " - + retryEventQueue.size() + + (retryEventQueue.size() + retryTSFileQueue.size()) + " (tablet events: " + retryEventQueueEventCounter.getTabletInsertionEventCount() + ", tsfile events: " @@ -646,8 +667,14 @@ public void addFailureEventToRetryQueue(final Event event) { return; } - retryEventQueue.offer(event); - retryEventQueueEventCounter.increaseEventCount(event); + if (event instanceof PipeTsFileInsertionEvent) { + retryTSFileQueue.offer((PipeTsFileInsertionEvent) event); + retryEventQueueEventCounter.increaseEventCount(event); + } else { + retryEventQueue.offer(event); + retryEventQueueEventCounter.increaseEventCount(event); + } + if (LOGGER.isDebugEnabled()) { LOGGER.debug("Added event {} to retry queue.", event); } @@ -687,6 +714,19 @@ public synchronized void discardEventsOfPipe(final String pipeNameToDrop, final } return false; }); + + retryTSFileQueue.removeIf( + event -> { + if (event instanceof EnrichedEvent + && pipeNameToDrop.equals(((EnrichedEvent) event).getPipeName()) + && regionId == ((EnrichedEvent) event).getRegionId()) { + ((EnrichedEvent) event) + .clearReferenceCount(IoTDBDataRegionAsyncConnector.class.getName()); + retryEventQueueEventCounter.decreaseEventCount(event); + return true; + } + return false; + }); } @Override @@ -729,8 +769,9 @@ public synchronized void close() { } public synchronized void clearRetryEventsReferenceCount() { - while (!retryEventQueue.isEmpty()) { - final Event event = retryEventQueue.poll(); + while (!retryEventQueue.isEmpty() || !retryTSFileQueue.isEmpty()) { + final Event event = + retryTSFileQueue.isEmpty() ? retryEventQueue.poll() : retryTSFileQueue.poll(); retryEventQueueEventCounter.decreaseEventCount(event); if (event instanceof EnrichedEvent) { ((EnrichedEvent) event).clearReferenceCount(IoTDBDataRegionAsyncConnector.class.getName()); @@ -741,7 +782,7 @@ public synchronized void clearRetryEventsReferenceCount() { //////////////////////// APIs provided for metric framework //////////////////////// public int getRetryEventQueueSize() { - return retryEventQueue.size(); + return retryEventQueue.size() + retryTSFileQueue.size(); } public int getBatchSize() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTsFileHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTsFileHandler.java index 1c586ce04ec1..da13e3724d13 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTsFileHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTsFileHandler.java @@ -47,7 +47,6 @@ import org.slf4j.LoggerFactory; import java.io.File; -import java.io.FileNotFoundException; import java.io.IOException; import java.io.RandomAccessFile; import java.util.Arrays; @@ -78,8 +77,8 @@ public class PipeTransferTsFileHandler extends PipeTransferTrackableHandler { private final boolean transferMod; private final int readFileBufferSize; - private final PipeTsFileMemoryBlock memoryBlock; - private final byte[] readBuffer; + private PipeTsFileMemoryBlock memoryBlock; + private byte[] readBuffer; private long position; private RandomAccessFile reader; @@ -98,7 +97,7 @@ public PipeTransferTsFileHandler( final File tsFile, final File modFile, final boolean transferMod) - throws FileNotFoundException, InterruptedException { + throws InterruptedException { super(connector); this.pipeName2WeightMap = pipeName2WeightMap; @@ -124,20 +123,8 @@ public PipeTransferTsFileHandler( Math.min( PipeConfig.getInstance().getPipeConnectorReadFileBufferSize(), transferMod ? Math.max(tsFile.length(), modFile.length()) : tsFile.length()); - memoryBlock = - PipeDataNodeResourceManager.memory() - .forceAllocateForTsFileWithRetry( - PipeConfig.getInstance().isPipeConnectorReadFileBufferMemoryControlEnabled() - ? readFileBufferSize - : 0); - readBuffer = new byte[readFileBufferSize]; position = 0; - reader = - Objects.nonNull(modFile) - ? new RandomAccessFile(modFile, "r") - : new RandomAccessFile(tsFile, "r"); - isSealSignalSent = new AtomicBoolean(false); } @@ -145,6 +132,23 @@ public void transfer( final IoTDBDataNodeAsyncClientManager clientManager, final AsyncPipeDataTransferServiceClient client) throws TException, IOException { + // Delay creation of resources to avoid OOM or too many open files + if (readBuffer == null) { + readBuffer = new byte[readFileBufferSize]; + PipeDataNodeResourceManager.memory() + .forceAllocateForTsFileWithRetry( + PipeConfig.getInstance().isPipeConnectorReadFileBufferMemoryControlEnabled() + ? readFileBufferSize + : 0); + } + + if (reader == null) { + reader = + Objects.nonNull(modFile) + ? new RandomAccessFile(modFile, "r") + : new RandomAccessFile(tsFile, "r"); + } + this.clientManager = clientManager; this.client = client; @@ -427,7 +431,9 @@ public void clearEventsReferenceCount() { @Override public void close() { super.close(); - memoryBlock.close(); + if (memoryBlock != null) { + memoryBlock.close(); + } } /** From 9db51d9e00bd6b4df8295ba53e55926032f7f634 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Sat, 12 Jul 2025 13:04:36 +0800 Subject: [PATCH 162/185] Optimized hybrid logger behaviour --- ...PipeRealtimeDataRegionHybridExtractor.java | 21 ++++++------------- 1 file changed, 6 insertions(+), 15 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java index 1076a97b320d..3298c6a707a2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java @@ -220,25 +220,16 @@ private boolean mayInsertNodeMemoryReachDangerousThreshold(final PipeRealtimeEve final boolean mayInsertNodeMemoryReachDangerousThreshold = floatingMemoryUsageInByte * pipeCount >= totalFloatingMemorySizeInBytes; if (mayInsertNodeMemoryReachDangerousThreshold && event.mayExtractorUseTablets(this)) { - logByLogManager( - l -> - l.info( - "Pipe task {}@{} canNotUseTabletAnyMore(1): The memory usage of the insert node {} has reached the dangerous threshold {}", - pipeName, - dataRegionId, - floatingMemoryUsageInByte * pipeCount, - totalFloatingMemorySizeInBytes)); + LOGGER.info( + "Pipe task {}@{} canNotUseTabletAnyMore(1): The memory usage of the insert node {} has reached the dangerous threshold {}", + pipeName, + dataRegionId, + floatingMemoryUsageInByte * pipeCount, + totalFloatingMemorySizeInBytes); } return mayInsertNodeMemoryReachDangerousThreshold; } - private void logByLogManager(final Consumer infoFunction) { - PipeDataNodeResourceManager.log() - .schedule( - PipeRealtimeDataRegionHybridExtractor.class, getTaskID(), Integer.MAX_VALUE, 100, 1) - .ifPresent(infoFunction); - } - /** * These judgements are deprecated, and are only reserved for manual operation and compatibility. */ From 52f28f00c22f0dcb3b32587f1129efa13b804e2d Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Sat, 12 Jul 2025 15:09:37 +0800 Subject: [PATCH 163/185] Optimized the downgrade logger to identify the tsFile --- .../PipeRealtimeDataRegionHybridExtractor.java | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java index 3298c6a707a2..51bd3819f546 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java @@ -41,7 +41,6 @@ import org.slf4j.LoggerFactory; import java.util.Objects; -import java.util.function.Consumer; public class PipeRealtimeDataRegionHybridExtractor extends PipeRealtimeDataRegionExtractor { @@ -221,9 +220,10 @@ private boolean mayInsertNodeMemoryReachDangerousThreshold(final PipeRealtimeEve floatingMemoryUsageInByte * pipeCount >= totalFloatingMemorySizeInBytes; if (mayInsertNodeMemoryReachDangerousThreshold && event.mayExtractorUseTablets(this)) { LOGGER.info( - "Pipe task {}@{} canNotUseTabletAnyMore(1): The memory usage of the insert node {} has reached the dangerous threshold {}", + "Pipe task {}@{} canNotUseTabletAnyMore(1) for tsFile {}: The memory usage of the insert node {} has reached the dangerous threshold {}", pipeName, dataRegionId, + event.getTsFileEpoch().getFilePath(), floatingMemoryUsageInByte * pipeCount, totalFloatingMemorySizeInBytes); } @@ -258,9 +258,10 @@ private boolean isHistoricalTsFileEventCountExceededLimit(final PipeRealtimeEven >= PipeConfig.getInstance().getPipeMaxAllowedHistoricalTsFilePerDataRegion(); if (isHistoricalTsFileEventCountExceededLimit && event.mayExtractorUseTablets(this)) { LOGGER.info( - "Pipe task {}@{} canNotUseTabletAnymoreDeprecated(1): The number of historical tsFile events {} has exceeded the limit {}", + "Pipe task {}@{} canNotUseTabletAnymoreDeprecated(1) for tsFile {}: The number of historical tsFile events {} has exceeded the limit {}", pipeName, dataRegionId, + event.getTsFileEpoch().getFilePath(), extractor.getHistoricalTsFileInsertionEventCount(), PipeConfig.getInstance().getPipeMaxAllowedHistoricalTsFilePerDataRegion()); } @@ -277,9 +278,10 @@ private boolean isRealtimeTsFileEventCountExceededLimit(final PipeRealtimeEvent >= PipeConfig.getInstance().getPipeMaxAllowedPendingTsFileEpochPerDataRegion(); if (isRealtimeTsFileEventCountExceededLimit && event.mayExtractorUseTablets(this)) { LOGGER.info( - "Pipe task {}@{} canNotUseTabletAnymoreDeprecated(2): The number of realtime tsFile events {} has exceeded the limit {}", + "Pipe task {}@{} canNotUseTabletAnymoreDeprecated(2) for tsFile {}: The number of realtime tsFile events {} has exceeded the limit {}", pipeName, dataRegionId, + event.getTsFileEpoch().getFilePath(), pendingQueue.getTsFileInsertionEventCount(), PipeConfig.getInstance().getPipeMaxAllowedPendingTsFileEpochPerDataRegion()); } @@ -295,9 +297,10 @@ private boolean mayTsFileLinkedCountReachDangerousThreshold(final PipeRealtimeEv >= PipeConfig.getInstance().getPipeMaxAllowedLinkedTsFileCount(); if (mayTsFileLinkedCountReachDangerousThreshold && event.mayExtractorUseTablets(this)) { LOGGER.info( - "Pipe task {}@{} canNotUseTabletAnymoreDeprecated(3): The number of linked tsFiles {} has reached the dangerous threshold {}", + "Pipe task {}@{} canNotUseTabletAnymoreDeprecated(3) for tsFile {}: The number of linked tsFiles {} has reached the dangerous threshold {}", pipeName, dataRegionId, + event.getTsFileEpoch().getFilePath(), PipeDataNodeResourceManager.tsfile().getLinkedTsFileCount(pipeName), PipeConfig.getInstance().getPipeMaxAllowedLinkedTsFileCount()); } From 659a894b3f31072b0b6a2f9aa26b8f778168befb Mon Sep 17 00:00:00 2001 From: luoluoyuyu Date: Sat, 12 Jul 2025 15:18:15 +0800 Subject: [PATCH 164/185] Modify PipeTransferTsFileHandler to apply for memory order --- .../thrift/async/handler/PipeTransferTsFileHandler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTsFileHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTsFileHandler.java index da13e3724d13..184237027cb9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTsFileHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTsFileHandler.java @@ -134,12 +134,12 @@ public void transfer( throws TException, IOException { // Delay creation of resources to avoid OOM or too many open files if (readBuffer == null) { - readBuffer = new byte[readFileBufferSize]; PipeDataNodeResourceManager.memory() .forceAllocateForTsFileWithRetry( PipeConfig.getInstance().isPipeConnectorReadFileBufferMemoryControlEnabled() ? readFileBufferSize : 0); + readBuffer = new byte[readFileBufferSize]; } if (reader == null) { From fac3bdbb94a5fd3a7ece7ccdbec73d229cea135a Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Sat, 12 Jul 2025 15:25:32 +0800 Subject: [PATCH 165/185] Fixed memoryBlock --- .../async/IoTDBDataRegionAsyncConnector.java | 34 +++++++++---------- .../handler/PipeTransferTsFileHandler.java | 2 +- 2 files changed, 17 insertions(+), 19 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java index fedd1f6a9e11..df2eb1fd0883 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java @@ -110,7 +110,7 @@ public class IoTDBDataRegionAsyncConnector extends IoTDBConnector { private final IoTDBDataRegionSyncConnector syncConnector = new IoTDBDataRegionSyncConnector(); private final BlockingQueue retryEventQueue = new LinkedBlockingQueue<>(); - private final BlockingQueue retryTSFileQueue = new LinkedBlockingQueue<>(); + private final BlockingQueue retryTsFileQueue = new LinkedBlockingQueue<>(); private final PipeDataRegionEventCounter retryEventQueueEventCounter = new PipeDataRegionEventCounter(); @@ -506,7 +506,7 @@ private void logOnClientException( * @see PipeConnector#transfer(TsFileInsertionEvent) for more details. */ private void transferQueuedEventsIfNecessary(final boolean forced) { - if ((retryEventQueue.isEmpty() && retryTSFileQueue.isEmpty()) + if ((retryEventQueue.isEmpty() && retryTsFileQueue.isEmpty()) || (!forced && retryEventQueueEventCounter.getTabletInsertionEventCount() < PipeConfig.getInstance() @@ -514,20 +514,20 @@ private void transferQueuedEventsIfNecessary(final boolean forced) { && retryEventQueueEventCounter.getTsFileInsertionEventCount() < PipeConfig.getInstance() .getPipeAsyncConnectorForcedRetryTsFileEventQueueSizeThreshold() - && retryEventQueue.size() + retryTSFileQueue.size() + && retryEventQueue.size() + retryTsFileQueue.size() < PipeConfig.getInstance() .getPipeAsyncConnectorForcedRetryTotalEventQueueSizeThreshold())) { return; } final long retryStartTime = System.currentTimeMillis(); - final int remainingEvents = retryEventQueue.size() + retryTSFileQueue.size(); - while (!retryEventQueue.isEmpty() && !retryTSFileQueue.isEmpty()) { + final int remainingEvents = retryEventQueue.size() + retryTsFileQueue.size(); + while (!retryEventQueue.isEmpty() && !retryTsFileQueue.isEmpty()) { synchronized (this) { if (isClosed.get()) { return; } - if (retryEventQueue.isEmpty() && retryTSFileQueue.isEmpty()) { + if (retryEventQueue.isEmpty() && retryTsFileQueue.isEmpty()) { break; } @@ -551,11 +551,9 @@ private void transferQueuedEventsIfNecessary(final boolean forced) { if (transferTsFileCounter.get() != 0) { return; } - peekedEvent = retryTSFileQueue.peek(); - + peekedEvent = retryTsFileQueue.peek(); retryTransfer((PipeTsFileInsertionEvent) peekedEvent); - - polledEvent = retryTSFileQueue.poll(); + polledEvent = retryTsFileQueue.poll(); } retryEventQueueEventCounter.decreaseEventCount(polledEvent); @@ -580,16 +578,16 @@ private void transferQueuedEventsIfNecessary(final boolean forced) { && retryEventQueueEventCounter.getTsFileInsertionEventCount() < PipeConfig.getInstance() .getPipeAsyncConnectorForcedRetryTsFileEventQueueSizeThreshold() - && retryEventQueue.size() + retryTSFileQueue.size() + && retryEventQueue.size() + retryTsFileQueue.size() < PipeConfig.getInstance() .getPipeAsyncConnectorForcedRetryTotalEventQueueSizeThreshold()) { return; } - if (remainingEvents <= retryEventQueue.size() + retryTSFileQueue.size()) { + if (remainingEvents <= retryEventQueue.size() + retryTsFileQueue.size()) { throw new PipeException( "Failed to retry transferring events in the retry queue. Remaining events: " - + (retryEventQueue.size() + retryTSFileQueue.size()) + + (retryEventQueue.size() + retryTsFileQueue.size()) + " (tablet events: " + retryEventQueueEventCounter.getTabletInsertionEventCount() + ", tsfile events: " @@ -668,7 +666,7 @@ public void addFailureEventToRetryQueue(final Event event) { } if (event instanceof PipeTsFileInsertionEvent) { - retryTSFileQueue.offer((PipeTsFileInsertionEvent) event); + retryTsFileQueue.offer((PipeTsFileInsertionEvent) event); retryEventQueueEventCounter.increaseEventCount(event); } else { retryEventQueue.offer(event); @@ -715,7 +713,7 @@ public synchronized void discardEventsOfPipe(final String pipeNameToDrop, final return false; }); - retryTSFileQueue.removeIf( + retryTsFileQueue.removeIf( event -> { if (event instanceof EnrichedEvent && pipeNameToDrop.equals(((EnrichedEvent) event).getPipeName()) @@ -769,9 +767,9 @@ public synchronized void close() { } public synchronized void clearRetryEventsReferenceCount() { - while (!retryEventQueue.isEmpty() || !retryTSFileQueue.isEmpty()) { + while (!retryEventQueue.isEmpty() || !retryTsFileQueue.isEmpty()) { final Event event = - retryTSFileQueue.isEmpty() ? retryEventQueue.poll() : retryTSFileQueue.poll(); + retryTsFileQueue.isEmpty() ? retryEventQueue.poll() : retryTsFileQueue.poll(); retryEventQueueEventCounter.decreaseEventCount(event); if (event instanceof EnrichedEvent) { ((EnrichedEvent) event).clearReferenceCount(IoTDBDataRegionAsyncConnector.class.getName()); @@ -782,7 +780,7 @@ public synchronized void clearRetryEventsReferenceCount() { //////////////////////// APIs provided for metric framework //////////////////////// public int getRetryEventQueueSize() { - return retryEventQueue.size() + retryTSFileQueue.size(); + return retryEventQueue.size() + retryTsFileQueue.size(); } public int getBatchSize() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTsFileHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTsFileHandler.java index 184237027cb9..f5581d2a85db 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTsFileHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTsFileHandler.java @@ -134,7 +134,7 @@ public void transfer( throws TException, IOException { // Delay creation of resources to avoid OOM or too many open files if (readBuffer == null) { - PipeDataNodeResourceManager.memory() + memoryBlock = PipeDataNodeResourceManager.memory() .forceAllocateForTsFileWithRetry( PipeConfig.getInstance().isPipeConnectorReadFileBufferMemoryControlEnabled() ? readFileBufferSize From 75cb7e77c9d2b15b3d6024bc7141d6265234cd76 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Sat, 12 Jul 2025 15:45:29 +0800 Subject: [PATCH 166/185] spotless --- .../async/handler/PipeTransferTsFileHandler.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTsFileHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTsFileHandler.java index f5581d2a85db..7353ea91e912 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTsFileHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/handler/PipeTransferTsFileHandler.java @@ -134,11 +134,12 @@ public void transfer( throws TException, IOException { // Delay creation of resources to avoid OOM or too many open files if (readBuffer == null) { - memoryBlock = PipeDataNodeResourceManager.memory() - .forceAllocateForTsFileWithRetry( - PipeConfig.getInstance().isPipeConnectorReadFileBufferMemoryControlEnabled() - ? readFileBufferSize - : 0); + memoryBlock = + PipeDataNodeResourceManager.memory() + .forceAllocateForTsFileWithRetry( + PipeConfig.getInstance().isPipeConnectorReadFileBufferMemoryControlEnabled() + ? readFileBufferSize + : 0); readBuffer = new byte[readFileBufferSize]; } From d12d80b1b58d93b9ff0ad4372fa7637f90f7f6aa Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Sat, 12 Jul 2025 17:05:53 +0800 Subject: [PATCH 167/185] Update IoTDBDataRegionAsyncConnector.java --- .../protocol/thrift/async/IoTDBDataRegionAsyncConnector.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java index df2eb1fd0883..2d9ffcf5051a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java @@ -522,7 +522,7 @@ private void transferQueuedEventsIfNecessary(final boolean forced) { final long retryStartTime = System.currentTimeMillis(); final int remainingEvents = retryEventQueue.size() + retryTsFileQueue.size(); - while (!retryEventQueue.isEmpty() && !retryTsFileQueue.isEmpty()) { + while (!retryEventQueue.isEmpty() || !retryTsFileQueue.isEmpty()) { synchronized (this) { if (isClosed.get()) { return; From 3913efa7be65168fb4756043553f40378c3e8ad8 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Sat, 12 Jul 2025 17:45:52 +0800 Subject: [PATCH 168/185] delete-uesless --- .../org/apache/iotdb/commons/pipe/config/PipeDescriptor.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java index ae9aa59101f5..15a1940dbbff 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/config/PipeDescriptor.java @@ -46,9 +46,6 @@ public static void loadPipeStaticConfig(CommonConfig config, TrimProperties prop config.setPipeHardlinkTsFileDirName( properties.getProperty( "pipe_hardlink_tsfile_dir_name", config.getPipeHardlinkTsFileDirName())); - config.setPipeProgressIndexPersistDirName( - properties.getProperty( - "pipe_progress_index_persist_dir_name", config.getPipeProgressIndexPersistDirName())); int pipeSubtaskExecutorMaxThreadNum = Integer.parseInt( properties.getProperty( From a3a9ee42a4eed835e7a00cba80359768699a6040 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Sat, 12 Jul 2025 19:01:07 +0800 Subject: [PATCH 169/185] Update PipeRealtimeExtractTest.java --- .../db/pipe/extractor/PipeRealtimeExtractTest.java | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipeRealtimeExtractTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipeRealtimeExtractTest.java index 8eb725f18aa5..c94baf458cb0 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipeRealtimeExtractTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipeRealtimeExtractTest.java @@ -150,32 +150,28 @@ public void testRealtimeExtractProcess() { "1", 1, Integer.parseInt(dataRegion1), - new PipeTaskMeta( - MinimumProgressIndex.INSTANCE, 1, Integer.parseInt(dataRegion1), false))); + new PipeTaskMeta(MinimumProgressIndex.INSTANCE, 1))); final PipeTaskRuntimeConfiguration configuration1 = new PipeTaskRuntimeConfiguration( new PipeTaskExtractorRuntimeEnvironment( "1", 1, Integer.parseInt(dataRegion1), - new PipeTaskMeta( - MinimumProgressIndex.INSTANCE, 1, Integer.parseInt(dataRegion1), false))); + new PipeTaskMeta(MinimumProgressIndex.INSTANCE, 1))); final PipeTaskRuntimeConfiguration configuration2 = new PipeTaskRuntimeConfiguration( new PipeTaskExtractorRuntimeEnvironment( "1", 1, Integer.parseInt(dataRegion2), - new PipeTaskMeta( - MinimumProgressIndex.INSTANCE, 1, Integer.parseInt(dataRegion2), false))); + new PipeTaskMeta(MinimumProgressIndex.INSTANCE, 1))); final PipeTaskRuntimeConfiguration configuration3 = new PipeTaskRuntimeConfiguration( new PipeTaskExtractorRuntimeEnvironment( "1", 1, Integer.parseInt(dataRegion2), - new PipeTaskMeta( - MinimumProgressIndex.INSTANCE, 1, Integer.parseInt(dataRegion2), false))); + new PipeTaskMeta(MinimumProgressIndex.INSTANCE, 1))); // Some parameters of extractor are validated and initialized during the validation process. extractor0.validate(new PipeParameterValidator(parameters0)); From 938b202994149ef2c8cc841ebb551dbc2f4731f8 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Sat, 12 Jul 2025 19:19:11 +0800 Subject: [PATCH 170/185] rename tsFile pending thread & unify the realtime parallel limit --- .../async/IoTDBDataRegionAsyncConnector.java | 18 +++++++----------- .../iotdb/commons/concurrent/ThreadName.java | 1 + 2 files changed, 8 insertions(+), 11 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java index 2d9ffcf5051a..ffeec7ae7544 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java @@ -21,6 +21,8 @@ import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.commons.client.async.AsyncPipeDataTransferServiceClient; +import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory; +import org.apache.iotdb.commons.concurrent.ThreadName; import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.connector.protocol.IoTDBConnector; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; @@ -71,14 +73,11 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -100,13 +99,9 @@ public class IoTDBDataRegionAsyncConnector extends IoTDBConnector { private static final boolean isSplitTSFileBatchModeEnabled = true; private static final ExecutorService executor = - new ThreadPoolExecutor( - PipeConfig.getInstance().getPipeAsyncConnectorMaxTsFileClientNumber(), - PipeConfig.getInstance().getPipeAsyncConnectorMaxTsFileClientNumber(), - 0L, - TimeUnit.MILLISECONDS, - new ArrayBlockingQueue( - 10 * PipeConfig.getInstance().getPipeSubtaskExecutorMaxThreadNum())); + IoTDBThreadPoolFactory.newFixedThreadPool( + PipeConfig.getInstance().getPipeRealTimeQueueMaxWaitingTsFileSize(), + ThreadName.PIPE_TSFILE_ASYNC_SEND_POOL.getName()); private final IoTDBDataRegionSyncConnector syncConnector = new IoTDBDataRegionSyncConnector(); private final BlockingQueue retryEventQueue = new LinkedBlockingQueue<>(); @@ -548,7 +543,8 @@ private void transferQueuedEventsIfNecessary(final boolean forced) { polledEvent = retryEventQueue.poll(); } else { - if (transferTsFileCounter.get() != 0) { + if (transferTsFileCounter.get() + >= PipeConfig.getInstance().getPipeRealTimeQueueMaxWaitingTsFileSize()) { return; } peekedEvent = retryTsFileQueue.peek(); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java index 3f1f312ae7cd..b8f21ef69aba 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java @@ -135,6 +135,7 @@ public enum ThreadName { PIPE_CONSENSUS_EXECUTOR_POOL("Pipe-Consensus-Executor-Pool"), PIPE_CONFIGNODE_EXECUTOR_POOL("Pipe-ConfigNode-Executor-Pool"), PIPE_SUBTASK_CALLBACK_EXECUTOR_POOL("Pipe-SubTask-Callback-Executor-Pool"), + PIPE_TSFILE_ASYNC_SEND_POOL("Pipe-TsFile-Async-Send-Pool"), PIPE_RUNTIME_META_SYNCER("Pipe-Runtime-Meta-Syncer"), PIPE_RUNTIME_HEARTBEAT("Pipe-Runtime-Heartbeat"), PIPE_RUNTIME_PROCEDURE_SUBMITTER("Pipe-Runtime-Procedure-Submitter"), From 4ede5b02ef794cc2ade4ab2da86f61a9af811393 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Sat, 12 Jul 2025 19:46:50 +0800 Subject: [PATCH 171/185] Separate executor --- .../IoTDBDataNodeAsyncClientManager.java | 36 +++++++++++++++++++ .../async/IoTDBDataRegionAsyncConnector.java | 9 +---- 2 files changed, 37 insertions(+), 8 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/client/IoTDBDataNodeAsyncClientManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/client/IoTDBDataNodeAsyncClientManager.java index 9cb1b0a5cd08..2fc41a618667 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/client/IoTDBDataNodeAsyncClientManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/client/IoTDBDataNodeAsyncClientManager.java @@ -23,6 +23,8 @@ import org.apache.iotdb.commons.client.ClientPoolFactory; import org.apache.iotdb.commons.client.IClientManager; import org.apache.iotdb.commons.client.async.AsyncPipeDataTransferServiceClient; +import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory; +import org.apache.iotdb.commons.concurrent.ThreadName; import org.apache.iotdb.commons.conf.CommonDescriptor; import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.connector.client.IoTDBClientManager; @@ -48,7 +50,9 @@ import java.util.Objects; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_LOAD_BALANCE_PRIORITY_STRATEGY; @@ -70,7 +74,12 @@ public class IoTDBDataNodeAsyncClientManager extends IoTDBClientManager // receiverAttributes -> IClientManager private static final Map> ASYNC_PIPE_DATA_TRANSFER_CLIENT_MANAGER_HOLDER = new ConcurrentHashMap<>(); + private static final Map TS_FILE_ASYNC_EXECUTOR_HOLDER = + new ConcurrentHashMap<>(); + private static final AtomicInteger id = new AtomicInteger(0); + private final IClientManager endPoint2Client; + private ExecutorService executor; private final LoadBalancer loadBalancer; @@ -123,6 +132,17 @@ public IoTDBDataNodeAsyncClientManager( } endPoint2Client = ASYNC_PIPE_DATA_TRANSFER_CLIENT_MANAGER_HOLDER.get(receiverAttributes); + if (isTSFileUsed) { + if (!TS_FILE_ASYNC_EXECUTOR_HOLDER.containsKey(receiverAttributes)) { + TS_FILE_ASYNC_EXECUTOR_HOLDER.putIfAbsent( + receiverAttributes, + IoTDBThreadPoolFactory.newFixedThreadPool( + PipeConfig.getInstance().getPipeRealTimeQueueMaxWaitingTsFileSize(), + ThreadName.PIPE_TSFILE_ASYNC_SEND_POOL.getName() + "-" + id.getAndIncrement())); + } + executor = TS_FILE_ASYNC_EXECUTOR_HOLDER.get(receiverAttributes); + } + RECEIVER_ATTRIBUTES_REF_COUNT.compute( receiverAttributes, (attributes, refCount) -> refCount == null ? 1 : refCount + 1); } @@ -336,6 +356,10 @@ public void updateLeaderCache(final String deviceId, final TEndPoint endPoint) { LEADER_CACHE_MANAGER.updateLeaderEndPoint(deviceId, endPoint); } + public ExecutorService getExecutor() { + return executor; + } + public void close() { isClosed = true; synchronized (IoTDBDataNodeAsyncClientManager.class) { @@ -352,6 +376,18 @@ public void close() { LOGGER.warn("Failed to close client manager.", e); } } + + final ExecutorService executor = + TS_FILE_ASYNC_EXECUTOR_HOLDER.remove(receiverAttributes); + if (executor != null) { + try { + executor.shutdown(); + LOGGER.info("Successfully shutdown executor {}.", executor); + } catch (final Exception e) { + LOGGER.warn("Failed to shutdown executor {}.", executor); + } + } + return null; } return refCount - 1; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java index ffeec7ae7544..0dacf0cb9122 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java @@ -21,8 +21,6 @@ import org.apache.iotdb.common.rpc.thrift.TEndPoint; import org.apache.iotdb.commons.client.async.AsyncPipeDataTransferServiceClient; -import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory; -import org.apache.iotdb.commons.concurrent.ThreadName; import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.connector.protocol.IoTDBConnector; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; @@ -76,7 +74,6 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -98,10 +95,6 @@ public class IoTDBDataRegionAsyncConnector extends IoTDBConnector { "Exception occurred while sending to receiver %s:%s."; private static final boolean isSplitTSFileBatchModeEnabled = true; - private static final ExecutorService executor = - IoTDBThreadPoolFactory.newFixedThreadPool( - PipeConfig.getInstance().getPipeRealTimeQueueMaxWaitingTsFileSize(), - ThreadName.PIPE_TSFILE_ASYNC_SEND_POOL.getName()); private final IoTDBDataRegionSyncConnector syncConnector = new IoTDBDataRegionSyncConnector(); private final BlockingQueue retryEventQueue = new LinkedBlockingQueue<>(); @@ -421,7 +414,7 @@ private void transfer(final PipeTransferTsFileHandler pipeTransferTsFileHandler) } return null; }, - executor); + transferTsFileClientManager.getExecutor()); if (PipeConfig.getInstance().isTransferTsFileSync() || !isRealtimeFirst) { try { From 78857eb8d684ed7a7c5553ea499ad870eebde9a7 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Sat, 12 Jul 2025 20:18:14 +0800 Subject: [PATCH 172/185] Fixed sleep logic --- .../subtask/PipeAbstractConnectorSubtask.java | 5 ---- .../task/subtask/PipeReportableSubtask.java | 24 +++++++++++++++++-- 2 files changed, 22 insertions(+), 7 deletions(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/subtask/PipeAbstractConnectorSubtask.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/subtask/PipeAbstractConnectorSubtask.java index 0ca1f1be8d95..427193c4f182 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/subtask/PipeAbstractConnectorSubtask.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/subtask/PipeAbstractConnectorSubtask.java @@ -34,16 +34,11 @@ import org.slf4j.LoggerFactory; import java.util.concurrent.ExecutorService; -import java.util.concurrent.atomic.AtomicLong; public abstract class PipeAbstractConnectorSubtask extends PipeReportableSubtask { private static final Logger LOGGER = LoggerFactory.getLogger(PipeAbstractConnectorSubtask.class); - // To ensure that high-priority tasks can obtain object locks first, a counter is now used to save - // the number of high-priority tasks. - protected final AtomicLong highPriorityLockTaskCount = new AtomicLong(0); - // For output (transfer events to the target system in connector) protected PipeConnector outputPipeConnector; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/subtask/PipeReportableSubtask.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/subtask/PipeReportableSubtask.java index aa50bdd75767..37916d8f25ec 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/subtask/PipeReportableSubtask.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/subtask/PipeReportableSubtask.java @@ -22,14 +22,20 @@ import org.apache.iotdb.commons.exception.pipe.PipeRuntimeConnectorRetryTimesConfigurableException; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeCriticalException; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeException; +import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.concurrent.atomic.AtomicLong; + public abstract class PipeReportableSubtask extends PipeSubtask { private static final Logger LOGGER = LoggerFactory.getLogger(PipeReportableSubtask.class); + // To ensure that high-priority tasks can obtain object locks first, a counter is now used to save + // the number of high-priority tasks. + protected final AtomicLong highPriorityLockTaskCount = new AtomicLong(0); protected PipeReportableSubtask(final String taskID, final long creationTime) { super(taskID, creationTime); @@ -84,7 +90,14 @@ private void onEnrichedEventFailure(final Throwable throwable) { throwable.getMessage(), throwable); try { - Thread.sleep(Math.min(1000L * retryCount.get(), 10000)); + synchronized (highPriorityLockTaskCount) { + // The wait operation will release the highPriorityLockTaskCount lock, so there will be + // no deadlock. + if (highPriorityLockTaskCount.get() == 0) { + highPriorityLockTaskCount.wait( + retryCount.get() * PipeConfig.getInstance().getPipeConnectorRetryIntervalMs()); + } + } } catch (final InterruptedException e) { LOGGER.warn( "Interrupted when retrying to execute subtask {} (creation time: {}, simple class: {})", @@ -151,7 +164,14 @@ private void onNonEnrichedEventFailure(final Throwable throwable) { throwable.getMessage(), throwable); try { - Thread.sleep(Math.min(1000L * retryCount.get(), 10000)); + synchronized (highPriorityLockTaskCount) { + // The wait operation will release the highPriorityLockTaskCount lock, so there will be + // no deadlock. + if (highPriorityLockTaskCount.get() == 0) { + highPriorityLockTaskCount.wait( + retryCount.get() * PipeConfig.getInstance().getPipeConnectorRetryIntervalMs()); + } + } } catch (final InterruptedException e) { LOGGER.warn( "Interrupted when retrying to execute subtask {} (creation time: {}, simple class: {})", From 2b0246caf222a742c0f896b4f2691f90a361eb72 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 14 Jul 2025 15:01:47 +0800 Subject: [PATCH 173/185] Reset default & Separate thread pool --- .../PipeConnectorSubtaskExecutor.java | 3 +- .../PipeConnectorSubtaskManager.java | 10 ++-- .../concurrent/IoTDBThreadPoolFactory.java | 15 ++++++ .../iotdb/commons/conf/CommonConfig.java | 3 +- .../task/execution/PipeSubtaskExecutor.java | 47 ++++++++++++++++--- 5 files changed, 66 insertions(+), 12 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/execution/PipeConnectorSubtaskExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/execution/PipeConnectorSubtaskExecutor.java index 96692c3cedd5..0850c81a59d6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/execution/PipeConnectorSubtaskExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/execution/PipeConnectorSubtaskExecutor.java @@ -31,7 +31,8 @@ public class PipeConnectorSubtaskExecutor extends PipeSubtaskExecutor { public PipeConnectorSubtaskExecutor() { super( PipeConfig.getInstance().getPipeSubtaskExecutorMaxThreadNum(), - ThreadName.PIPE_CONNECTOR_EXECUTOR_POOL.getName() + "-" + id.getAndIncrement(), + ThreadName.PIPE_CONNECTOR_EXECUTOR_POOL.getName() + "-" + id.get(), + ThreadName.PIPE_SUBTASK_CALLBACK_EXECUTOR_POOL.getName() + "-" + id.getAndIncrement(), true); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeConnectorSubtaskManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeConnectorSubtaskManager.java index 300254f8b6d0..e25542135c1b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeConnectorSubtaskManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeConnectorSubtaskManager.java @@ -173,9 +173,10 @@ public synchronized String register( } LOGGER.info( - "Pipe connector subtasks with attributes {} is bounded with connectorExecutor {}.", + "Pipe connector subtasks with attributes {} is bounded with connectorExecutor {} and callbackExecutor {}.", attributeSortedString, - executor.getThreadName()); + executor.getWorkingThreadName(), + executor.getCallbackThreadName()); attributeSortedString2SubtaskLifeCycleMap.put( attributeSortedString, pipeConnectorSubtaskLifeCycleList); } @@ -208,7 +209,10 @@ public synchronized void deregister( if (lifeCycles.isEmpty()) { attributeSortedString2SubtaskLifeCycleMap.remove(attributeSortedString); executor.shutdown(); - LOGGER.info("The executor {} has been successfully shutdown.", executor.getThreadName()); + LOGGER.info( + "The executor {} and {} has been successfully shutdown.", + executor.getWorkingThreadName(), + executor.getCallbackThreadName()); } PipeEventCommitManager.getInstance().deregister(pipeName, creationTime, regionId); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/IoTDBThreadPoolFactory.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/IoTDBThreadPoolFactory.java index 6234705ed2da..73d6ff22430d 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/IoTDBThreadPoolFactory.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/IoTDBThreadPoolFactory.java @@ -160,6 +160,21 @@ public static ExecutorService newSingleThreadExecutor( poolName); } + public static ExecutorService newSingleThreadExecutor( + String poolName, RejectedExecutionHandler handler) { + logger.info(NEW_SINGLE_THREAD_POOL_LOGGER_FORMAT, poolName); + return new WrappedSingleThreadExecutorService( + new ThreadPoolExecutor( + 1, + 1, + 0L, + TimeUnit.MILLISECONDS, + new LinkedBlockingQueue<>(), + new IoTThreadFactory(poolName), + handler), + poolName); + } + /** * see {@link Executors#newCachedThreadPool(java.util.concurrent.ThreadFactory)}. * diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java index cd7a700b2963..8ae979bd7349 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java @@ -208,7 +208,8 @@ public class CommonConfig { private int pipeRealTimeQueueMaxWaitingTsFileSize = 1; /** The maximum number of threads that can be used to execute subtasks in PipeSubtaskExecutor. */ - private int pipeSubtaskExecutorMaxThreadNum = 5; + private int pipeSubtaskExecutorMaxThreadNum = + Math.max(5, Runtime.getRuntime().availableProcessors() / 2); private int pipeDataStructureTabletRowSize = 2048; private int pipeDataStructureTabletSizeInBytes = 2097152; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/execution/PipeSubtaskExecutor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/execution/PipeSubtaskExecutor.java index 533825e75e0a..f16c6387cf57 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/execution/PipeSubtaskExecutor.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/agent/task/execution/PipeSubtaskExecutor.java @@ -30,18 +30,24 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.util.Map; +import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; +import java.util.concurrent.ThreadPoolExecutor; public abstract class PipeSubtaskExecutor { private static final Logger LOGGER = LoggerFactory.getLogger(PipeSubtaskExecutor.class); - private static final ExecutorService subtaskCallbackListeningExecutor = + private static final ExecutorService globalSubtaskCallbackListeningExecutor = IoTDBThreadPoolFactory.newSingleThreadExecutor( ThreadName.PIPE_SUBTASK_CALLBACK_EXECUTOR_POOL.getName()); + private final ExecutorService subtaskCallbackListeningExecutor; + protected final WrappedThreadPoolExecutor underlyingThreadPool; protected final ListeningExecutorService subtaskWorkerThreadPoolExecutor; @@ -49,18 +55,38 @@ public abstract class PipeSubtaskExecutor { private final int corePoolSize; private int runningSubtaskNumber; - private final String threadName; + private final String workingThreadName; + private final String callbackThreadName; + + protected PipeSubtaskExecutor( + final int corePoolSize, + final String workingThreadName, + final boolean disableLogInThreadPool) { + this(corePoolSize, workingThreadName, null, disableLogInThreadPool); + } protected PipeSubtaskExecutor( - final int corePoolSize, final String threadName, final boolean disableLogInThreadPool) { - this.threadName = threadName; + final int corePoolSize, + final String workingThreadName, + final @Nullable String callbackThreadName, + final boolean disableLogInThreadPool) { + this.workingThreadName = workingThreadName; + this.callbackThreadName = + Objects.nonNull(callbackThreadName) + ? callbackThreadName + : ThreadName.PIPE_SUBTASK_CALLBACK_EXECUTOR_POOL.getName(); underlyingThreadPool = (WrappedThreadPoolExecutor) - IoTDBThreadPoolFactory.newFixedThreadPool(corePoolSize, threadName); + IoTDBThreadPoolFactory.newFixedThreadPool(corePoolSize, workingThreadName); if (disableLogInThreadPool) { underlyingThreadPool.disableErrorLog(); } subtaskWorkerThreadPoolExecutor = MoreExecutors.listeningDecorator(underlyingThreadPool); + subtaskCallbackListeningExecutor = + Objects.nonNull(callbackThreadName) + ? IoTDBThreadPoolFactory.newSingleThreadExecutor( + callbackThreadName, new ThreadPoolExecutor.DiscardPolicy()) + : globalSubtaskCallbackListeningExecutor; registeredIdSubtaskMapper = new ConcurrentHashMap<>(); @@ -153,6 +179,9 @@ public final synchronized void shutdown() { } subtaskWorkerThreadPoolExecutor.shutdown(); + if (subtaskCallbackListeningExecutor != globalSubtaskCallbackListeningExecutor) { + subtaskCallbackListeningExecutor.shutdown(); + } } public final boolean isShutdown() { @@ -173,7 +202,11 @@ protected final boolean hasAvailableThread() { // return getAvailableThreadCount() > 0; } - public String getThreadName() { - return threadName; + public String getWorkingThreadName() { + return workingThreadName; + } + + public String getCallbackThreadName() { + return callbackThreadName; } } From df968f7f4080581993f7f56c80f0d46949f52692 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 14 Jul 2025 15:04:59 +0800 Subject: [PATCH 174/185] convert --- .../src/test/java/org/apache/iotdb/db/it/IoTDBLoadTsFileIT.java | 2 ++ pom.xml | 1 + 2 files changed, 3 insertions(+) diff --git a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBLoadTsFileIT.java b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBLoadTsFileIT.java index 5b4ae76332f5..13131dc617f1 100644 --- a/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBLoadTsFileIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/db/it/IoTDBLoadTsFileIT.java @@ -36,6 +36,7 @@ import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -901,6 +902,7 @@ public void testLoadLocally() throws Exception { } @Test + @Ignore("Load with conversion is currently banned") public void testLoadWithConvertOnTypeMismatch() throws Exception { List> measurementSchemas = diff --git a/pom.xml b/pom.xml index b25e37e92a5f..90b3fc63a212 100644 --- a/pom.xml +++ b/pom.xml @@ -47,6 +47,7 @@ distribution example library-udf + integration-test From db746af5a6bc56f0f76461704666b306bd417193 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 14 Jul 2025 15:05:44 +0800 Subject: [PATCH 175/185] revert-pom --- pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/pom.xml b/pom.xml index 90b3fc63a212..b25e37e92a5f 100644 --- a/pom.xml +++ b/pom.xml @@ -47,7 +47,6 @@ distribution example library-udf - integration-test From a836f48f099f2452ca982574f71b3f2ed2cde611 Mon Sep 17 00:00:00 2001 From: Steve Yurong Su Date: Mon, 14 Jul 2025 18:36:01 +0800 Subject: [PATCH 176/185] [To dev/1.3] Pipe: Add a tool for validating and repairing isGeneratedByPipe mark in tsfile resources (#15934) --- .../tsfile/mark-is-generated-by-pipe.bat | 59 ++++ .../tools/tsfile/mark-is-generated-by-pipe.sh | 51 ++++ ...atedByPipeMarkValidationAndRepairTool.java | 263 ++++++++++++++++++ 3 files changed, 373 insertions(+) create mode 100644 iotdb-core/datanode/src/assembly/resources/tools/tsfile/mark-is-generated-by-pipe.bat create mode 100644 iotdb-core/datanode/src/assembly/resources/tools/tsfile/mark-is-generated-by-pipe.sh create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/tools/validate/TsFileResourceIsGeneratedByPipeMarkValidationAndRepairTool.java diff --git a/iotdb-core/datanode/src/assembly/resources/tools/tsfile/mark-is-generated-by-pipe.bat b/iotdb-core/datanode/src/assembly/resources/tools/tsfile/mark-is-generated-by-pipe.bat new file mode 100644 index 000000000000..7d2b867bba9a --- /dev/null +++ b/iotdb-core/datanode/src/assembly/resources/tools/tsfile/mark-is-generated-by-pipe.bat @@ -0,0 +1,59 @@ +@REM +@REM Licensed to the Apache Software Foundation (ASF) under one +@REM or more contributor license agreements. See the NOTICE file +@REM distributed with this work for additional information +@REM regarding copyright ownership. The ASF licenses this file +@REM to you under the Apache License, Version 2.0 (the +@REM "License"); you may not use this file except in compliance +@REM with the License. You may obtain a copy of the License at +@REM +@REM http://www.apache.org/licenses/LICENSE-2.0 +@REM +@REM Unless required by applicable law or agreed to in writing, +@REM software distributed under the License is distributed on an +@REM "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +@REM KIND, either express or implied. See the License for the +@REM specific language governing permissions and limitations +@REM under the License. +@REM + +@echo off +echo ```````````````````````````````````````````````````````````````````````` +echo Starting Validating the isGeneratedByPipe Mark in TsFile Resources +echo ```````````````````````````````````````````````````````````````````````` + +if "%OS%" == "Windows_NT" setlocal + +pushd %~dp0..\.. +if NOT DEFINED IOTDB_HOME set IOTDB_HOME=%CD% +popd + +if NOT DEFINED MAIN_CLASS set MAIN_CLASS=org.apache.iotdb.db.tools.validate.TsFileResourceIsGeneratedByPipeMarkValidationAndRepairTool +if NOT DEFINED JAVA_HOME goto :err + +@REM ----------------------------------------------------------------------------- +@REM ***** CLASSPATH library setting ***** +@REM Ensure that any user defined CLASSPATH variables are not used on startup +set CLASSPATH="%IOTDB_HOME%\lib\*" + +goto okClasspath + +:append +set CLASSPATH=%CLASSPATH%;%1 +goto :eof + +@REM ----------------------------------------------------------------------------- +:okClasspath + +"%JAVA_HOME%\bin\java" -cp "%CLASSPATH%" %MAIN_CLASS% %* + +goto finally + +:err +echo JAVA_HOME environment variable must be set! +pause + +@REM ----------------------------------------------------------------------------- +:finally + +ENDLOCAL \ No newline at end of file diff --git a/iotdb-core/datanode/src/assembly/resources/tools/tsfile/mark-is-generated-by-pipe.sh b/iotdb-core/datanode/src/assembly/resources/tools/tsfile/mark-is-generated-by-pipe.sh new file mode 100644 index 000000000000..daabae671f3e --- /dev/null +++ b/iotdb-core/datanode/src/assembly/resources/tools/tsfile/mark-is-generated-by-pipe.sh @@ -0,0 +1,51 @@ +#!/bin/bash +# +# 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. +# + +echo ------------------------------------------------------------------------------------ +echo Starting Validating the isGeneratedByPipe Mark in TsFile Resources +echo ------------------------------------------------------------------------------------ + +source "$(dirname "$0")/../../sbin/iotdb-common.sh" +#get_iotdb_include and checkAllVariables is in iotdb-common.sh +VARS=$(get_iotdb_include "$*") +checkAllVariables +export IOTDB_HOME="${IOTDB_HOME}/.." +eval set -- "$VARS" + +if [ -n "$JAVA_HOME" ]; then + for java in "$JAVA_HOME"/bin/amd64/java "$JAVA_HOME"/bin/java; do + if [ -x "$java" ]; then + JAVA="$java" + break + fi + done +else + JAVA=java +fi + +CLASSPATH="" +for f in ${IOTDB_HOME}/lib/*.jar; do + CLASSPATH=${CLASSPATH}":"$f +done + +MAIN_CLASS=org.apache.iotdb.db.tools.validate.TsFileResourceIsGeneratedByPipeMarkValidationAndRepairTool + +"$JAVA" -cp "$CLASSPATH" "$MAIN_CLASS" "$@" +exit $? diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/tools/validate/TsFileResourceIsGeneratedByPipeMarkValidationAndRepairTool.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/tools/validate/TsFileResourceIsGeneratedByPipeMarkValidationAndRepairTool.java new file mode 100644 index 000000000000..e14f7e81d054 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/tools/validate/TsFileResourceIsGeneratedByPipeMarkValidationAndRepairTool.java @@ -0,0 +1,263 @@ +/* + * 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.tools.validate; + +import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; + +import org.apache.tsfile.common.constant.TsFileConstant; +import org.slf4j.Logger; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.ConcurrentSkipListSet; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; + +public class TsFileResourceIsGeneratedByPipeMarkValidationAndRepairTool { + + private static final Logger LOGGER = + org.slf4j.LoggerFactory.getLogger( + TsFileResourceIsGeneratedByPipeMarkValidationAndRepairTool.class); + + private static final String USAGE = + "Usage: --expected true|false --dirs ...\n" + + " --expected: whether the TsFileResource is expected to be generated by pipe\n" + + " --dirs: list of data directories to validate and repair"; + + private static final Set dataDirs = new ConcurrentSkipListSet<>(); + private static final AtomicBoolean expectedMark = new AtomicBoolean(true); + + private static final AtomicLong runtime = new AtomicLong(System.currentTimeMillis()); + + private static final AtomicInteger totalTsFileNum = new AtomicInteger(0); + private static final AtomicInteger toRepairTsFileNum = new AtomicInteger(0); + + // Usage: --expected true|false --dirs ... + public static void main(String[] args) throws IOException { + parseCommandLineArgs(args); + final List partitionDirs = findAllPartitionDirs(); + partitionDirs.parallelStream() + .forEach( + TsFileResourceIsGeneratedByPipeMarkValidationAndRepairTool + ::validateAndRepairTsFileResourcesInPartition); + printStatistics(); + } + + private static void parseCommandLineArgs(final String[] args) { + final Set argSet = + new ConcurrentSkipListSet<>( + args.length > 0 ? Arrays.asList(args) : Collections.emptyList()); + if (args.length == 0 + || argSet.contains("--help") + || argSet.contains("-h") + || !(argSet.contains("--expected") && argSet.contains("--dirs"))) { + LOGGER.info(USAGE); + System.exit(1); + } + + for (int i = 0; i < args.length; i++) { + if ("--expected".equals(args[i]) && i + 1 < args.length) { + expectedMark.set(Boolean.parseBoolean(args[++i])); + } else if ("--dirs".equals(args[i]) && i + 1 < args.length) { + i++; + while (i < args.length && !args[i].startsWith("--")) { + dataDirs.add(new File(args[i++])); + } + i--; + } else { + LOGGER.info("Unknown argument: {}", args[i]); + LOGGER.info(USAGE); + // Exit if an unknown argument is encountered + System.exit(1); + } + } + + if (dataDirs.isEmpty()) { + LOGGER.info("No data directories provided. Please specify with --dirs ..."); + System.exit(1); + } + + LOGGER.info("------------------------------------------------------"); + LOGGER.info("Expected mark: {}", expectedMark.get()); + LOGGER.info("Data directories: "); + for (File dir : dataDirs) { + LOGGER.info(" {}", dir.getAbsolutePath()); + } + LOGGER.info("------------------------------------------------------"); + } + + private static List findAllPartitionDirs() { + final List partitionDirs = new ArrayList<>(); + for (final File dataDir : dataDirs) { + if (dataDir.exists() && dataDir.isDirectory()) { + partitionDirs.addAll(findLeafDirectories(dataDir)); + } + } + return partitionDirs; + } + + public static List findLeafDirectories(File dir) { + List leafDirectories = new ArrayList<>(); + + File[] files = dir.listFiles(); + + if (files == null || files.length == 0) { + leafDirectories.add(dir); + return leafDirectories; + } + + for (File file : files) { + if (file.isDirectory()) { + leafDirectories.addAll(findLeafDirectories(file)); + } + } + + if (leafDirectories.isEmpty()) { + leafDirectories.add(dir); + } + + return leafDirectories; + } + + private static void validateAndRepairTsFileResourcesInPartition(final File partitionDir) { + final AtomicInteger totalResources = new AtomicInteger(); + final AtomicInteger toRepairResources = new AtomicInteger(); + + try { + final List resources = + loadAllTsFileResources(Collections.singletonList(partitionDir)); + totalResources.addAndGet(resources.size()); + + for (final TsFileResource resource : resources) { + try { + if (validateAndRepairSingleTsFileResource(resource)) { + toRepairResources.incrementAndGet(); + } + } catch (final Exception e) { + // Continue processing other resources even if one fails + LOGGER.warn( + "Error validating or repairing resource {}: {}", + resource.getTsFile().getAbsolutePath(), + e.getMessage(), + e); + } + } + } catch (final Exception e) { + LOGGER.warn( + "Error loading resources from partition {}: {}", + partitionDir.getAbsolutePath(), + e.getMessage(), + e); + } + + totalTsFileNum.addAndGet(totalResources.get()); + toRepairTsFileNum.addAndGet(toRepairResources.get()); + LOGGER.info( + "TimePartition {} has {} total resources, {} to repair resources. Process completed.", + partitionDir, + totalResources.get(), + toRepairResources.get()); + } + + private static List loadAllTsFileResources(List timePartitionDirs) + throws IOException { + final List resources = new ArrayList<>(); + + for (final File timePartitionDir : timePartitionDirs) { + for (final File tsfile : Objects.requireNonNull(timePartitionDir.listFiles())) { + final String filePath = tsfile.getAbsolutePath(); + if (!filePath.endsWith(TsFileConstant.TSFILE_SUFFIX) || !tsfile.isFile()) { + continue; + } + String resourcePath = tsfile.getAbsolutePath() + TsFileResource.RESOURCE_SUFFIX; + + if (!new File(resourcePath).exists()) { + LOGGER.info( + "{} is skipped because resource file is not exist.", tsfile.getAbsolutePath()); + continue; + } + + TsFileResource resource = new TsFileResource(tsfile); + resource.deserialize(); + resource.close(); + resources.add(resource); + } + } + + return resources; + } + + /** + * Validates and repairs a single TsFileResource. + * + * @param resource the TsFileResource to validate and repair + * @return true if the resource needs to be repaired and false if it is valid + */ + private static boolean validateAndRepairSingleTsFileResource(TsFileResource resource) { + if (resource.isGeneratedByPipe() == expectedMark.get()) { + // The resource is valid, no need to repair + return false; + } + + LOGGER.info( + "Repairing TsFileResource: {}, expected mark: {}, actual mark: {}", + resource.getTsFile().getAbsolutePath(), + expectedMark.get(), + resource.isGeneratedByPipe()); + + try { + repairSingleTsFileResource(resource); + + LOGGER.info( + "Marked TsFileResource as {} in resource: {}", + expectedMark.get(), + resource.getTsFile().getAbsolutePath()); + } catch (final Exception e) { + LOGGER.warn( + "ERROR: Failed to repair TsFileResource: {}, error: {}", + resource.getTsFile().getAbsolutePath(), + e.getMessage()); + } + + return true; + } + + private static void repairSingleTsFileResource(TsFileResource resource) throws IOException { + resource.setGeneratedByPipe(expectedMark.get()); + resource.serialize(); + } + + private static void printStatistics() { + LOGGER.info("------------------------------------------------------"); + LOGGER.info("Validation and repair completed. Statistics:"); + LOGGER.info( + "Total time taken: {} ms, total TsFile resources: {}, repaired TsFile resources: {}", + System.currentTimeMillis() - runtime.get(), + totalTsFileNum.get(), + toRepairTsFileNum.get()); + } +} From bec300fa31423778db32f45d2b05a5b16e743902 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 14 Jul 2025 19:10:58 +0800 Subject: [PATCH 177/185] Fixed flag setting --- .../realtime/listener/PipeInsertionDataNodeListener.java | 4 +--- .../apache/iotdb/db/storageengine/dataregion/DataRegion.java | 3 ++- .../db/storageengine/dataregion/memtable/TsFileProcessor.java | 4 ++-- .../iotdb/db/pipe/extractor/PipeRealtimeExtractTest.java | 2 +- 4 files changed, 6 insertions(+), 7 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java index 0769a54bbbab..9f00a6cc7769 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java @@ -96,9 +96,7 @@ public synchronized void stopListenAndAssign( public void listenToTsFile( final String dataRegionId, final TsFileResource tsFileResource, - final boolean isLoaded, - final boolean isGeneratedByPipe) { - tsFileResource.setGeneratedByPipe(isGeneratedByPipe); + final boolean isLoaded) { // We don't judge whether listenToTsFileExtractorCount.get() == 0 here on purpose // because extractors may use tsfile events when some exceptions occur in the // insert nodes listening process. diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java index 0d2737ab721b..3a416a0a19c7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java @@ -3296,8 +3296,9 @@ private boolean loadTsFileToUnSequence( } // Listen before the tsFile is added into tsFile manager to avoid it being compacted + tsFileResource.setGeneratedByPipe(isGeneratedByPipe); PipeInsertionDataNodeListener.getInstance() - .listenToTsFile(dataRegionId, tsFileResource, true, isGeneratedByPipe); + .listenToTsFile(dataRegionId, tsFileResource, true); tsFileManager.add(tsFileResource, false); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java index ad77ab22f748..bd02e5e646f0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java @@ -1140,6 +1140,7 @@ public Future asyncClose() { // we have to add the memtable into flushingList first and then set the shouldClose tag. // see https://issues.apache.org/jira/browse/IOTDB-510 IMemTable tmpMemTable = workMemTable == null ? new NotifyFlushMemTable() : workMemTable; + tsFileResource.setGeneratedByPipe(isTotallyGeneratedByPipe.get()); try { // When invoke closing TsFile after insert data to memTable, we shouldn't flush until invoke @@ -1580,8 +1581,7 @@ private void endFile() throws IOException, TsFileProcessorException { .listenToTsFile( dataRegionInfo.getDataRegion().getDataRegionId(), tsFileResource, - false, - this.isTotallyGeneratedByPipe.get()); + false); tsFileResource.serialize(); FileTimeIndexCacheRecorder.getInstance().logFileTimeIndex(tsFileResource); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipeRealtimeExtractTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipeRealtimeExtractTest.java index c94baf458cb0..afac224e8ee7 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipeRealtimeExtractTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/PipeRealtimeExtractTest.java @@ -323,7 +323,7 @@ private Future write2DataRegion( false), resource); PipeInsertionDataNodeListener.getInstance() - .listenToTsFile(dataRegionId, resource, false, false); + .listenToTsFile(dataRegionId, resource, false); } }); } From 43558c72fa2506984e9a7ca81906056db4e2f26c Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 14 Jul 2025 19:18:32 +0800 Subject: [PATCH 178/185] Update PipeInsertionDataNodeListener.java --- .../realtime/listener/PipeInsertionDataNodeListener.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java index 9f00a6cc7769..86ac909b0deb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java @@ -94,9 +94,7 @@ public synchronized void stopListenAndAssign( //////////////////////////// listen to events //////////////////////////// public void listenToTsFile( - final String dataRegionId, - final TsFileResource tsFileResource, - final boolean isLoaded) { + final String dataRegionId, final TsFileResource tsFileResource, final boolean isLoaded) { // We don't judge whether listenToTsFileExtractorCount.get() == 0 here on purpose // because extractors may use tsfile events when some exceptions occur in the // insert nodes listening process. From 636f822848450ecb7f0c4a16b639a03806c55eb4 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Mon, 14 Jul 2025 19:19:21 +0800 Subject: [PATCH 179/185] spotless --- .../apache/iotdb/db/storageengine/dataregion/DataRegion.java | 3 +-- .../storageengine/dataregion/memtable/TsFileProcessor.java | 5 +---- 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java index 3a416a0a19c7..17f430da427b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java @@ -3297,8 +3297,7 @@ private boolean loadTsFileToUnSequence( // Listen before the tsFile is added into tsFile manager to avoid it being compacted tsFileResource.setGeneratedByPipe(isGeneratedByPipe); - PipeInsertionDataNodeListener.getInstance() - .listenToTsFile(dataRegionId, tsFileResource, true); + PipeInsertionDataNodeListener.getInstance().listenToTsFile(dataRegionId, tsFileResource, true); tsFileManager.add(tsFileResource, false); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java index bd02e5e646f0..451986b2f52d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/memtable/TsFileProcessor.java @@ -1578,10 +1578,7 @@ private void endFile() throws IOException, TsFileProcessorException { // Listen after "endFile" to avoid unnecessary waiting for tsFile close // before resource serialization to avoid missing hardlink after restart PipeInsertionDataNodeListener.getInstance() - .listenToTsFile( - dataRegionInfo.getDataRegion().getDataRegionId(), - tsFileResource, - false); + .listenToTsFile(dataRegionInfo.getDataRegion().getDataRegionId(), tsFileResource, false); tsFileResource.serialize(); FileTimeIndexCacheRecorder.getInstance().logFileTimeIndex(tsFileResource); From f6fac844f3cfcacc7349bebd628486dd991ec0ed Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 15 Jul 2025 14:10:08 +0800 Subject: [PATCH 180/185] Fixed load tsFile --- .../org/apache/iotdb/db/storageengine/dataregion/DataRegion.java | 1 - .../apache/iotdb/db/storageengine/load/LoadTsFileManager.java | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java index 17f430da427b..555d90e6c7c7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java @@ -3296,7 +3296,6 @@ private boolean loadTsFileToUnSequence( } // Listen before the tsFile is added into tsFile manager to avoid it being compacted - tsFileResource.setGeneratedByPipe(isGeneratedByPipe); PipeInsertionDataNodeListener.getInstance().listenToTsFile(dataRegionId, tsFileResource, true); tsFileManager.add(tsFileResource, false); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/LoadTsFileManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/LoadTsFileManager.java index 25b594631aa2..28c952b3d726 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/LoadTsFileManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/LoadTsFileManager.java @@ -505,6 +505,7 @@ private void loadAll( final DataRegion dataRegion = entry.getKey().getDataRegion(); final TsFileResource tsFileResource = dataPartition2Resource.get(entry.getKey()); + tsFileResource.setGeneratedByPipe(isGeneratedByPipe); endTsFileResource( writer, tsFileResource, From b3dba84b3984a173337bd88058a16cecc34f8b05 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 15 Jul 2025 14:49:10 +0800 Subject: [PATCH 181/185] volatile --- .../db/storageengine/dataregion/tsfile/TsFileResource.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java index 2b8baeb5112b..323381537c4f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java @@ -168,10 +168,10 @@ public class TsFileResource { private ProgressIndex maxProgressIndex; /** used to prevent circular replication in PipeConsensus */ - private boolean isGeneratedByPipeConsensus = false; + private volatile boolean isGeneratedByPipeConsensus = false; /** used to prevent circular replication in Pipe */ - private boolean isGeneratedByPipe = false; + private volatile boolean isGeneratedByPipe = false; private InsertionCompactionCandidateStatus insertionCompactionCandidateStatus = InsertionCompactionCandidateStatus.NOT_CHECKED; From be62e310c6f7e16195bd646db14301f574c42190 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 15 Jul 2025 14:58:14 +0800 Subject: [PATCH 182/185] Atomic-reference --- .../dataregion/tsfile/TsFileResource.java | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java index 323381537c4f..337f0430a1d3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java @@ -165,7 +165,7 @@ public class TsFileResource { */ private TsFileResource originTsFileResource; - private ProgressIndex maxProgressIndex; + private AtomicReference maxProgressIndex = new AtomicReference<>(); /** used to prevent circular replication in PipeConsensus */ private volatile boolean isGeneratedByPipeConsensus = false; @@ -267,9 +267,9 @@ private void serializeTo(BufferedOutputStream outputStream) throws IOException { ReadWriteIOUtils.write((String) null, outputStream); } - if (maxProgressIndex != null) { + if (maxProgressIndex.get() != null) { TsFileResourceBlockType.PROGRESS_INDEX.serialize(outputStream); - maxProgressIndex.serialize(outputStream); + maxProgressIndex.get().serialize(outputStream); } else { TsFileResourceBlockType.EMPTY_BLOCK.serialize(outputStream); } @@ -301,7 +301,7 @@ public void deserialize() throws IOException { TsFileResourceBlockType.deserialize(ReadWriteIOUtils.readByte(inputStream)); switch (blockType) { case PROGRESS_INDEX: - maxProgressIndex = ProgressIndexType.deserializeFrom(inputStream); + maxProgressIndex.set(ProgressIndexType.deserializeFrom(inputStream)); break; case PIPE_MARK: isGeneratedByPipeConsensus = ReadWriteIOUtils.readBoolean(inputStream); @@ -1197,10 +1197,9 @@ public void updateProgressIndex(ProgressIndex progressIndex) { return; } - maxProgressIndex = - (maxProgressIndex == null - ? progressIndex - : maxProgressIndex.updateToMinimumEqualOrIsAfterProgressIndex(progressIndex)); + if (maxProgressIndex.compareAndSet(null, progressIndex)) { + maxProgressIndex.get().updateToMinimumEqualOrIsAfterProgressIndex(progressIndex); + } } public void setProgressIndex(ProgressIndex progressIndex) { @@ -1208,11 +1207,12 @@ public void setProgressIndex(ProgressIndex progressIndex) { return; } - maxProgressIndex = progressIndex; + maxProgressIndex.set(progressIndex); } public ProgressIndex getMaxProgressIndex() { - return maxProgressIndex == null ? MinimumProgressIndex.INSTANCE : maxProgressIndex; + final ProgressIndex index = maxProgressIndex.get(); + return index == null ? MinimumProgressIndex.INSTANCE : index; } public boolean isEmpty() { From aa1ffc842e9c789d8dfc4674aef921ffcdfd69de Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 15 Jul 2025 16:32:31 +0800 Subject: [PATCH 183/185] fix --- .../db/storageengine/dataregion/tsfile/TsFileResource.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java index 337f0430a1d3..c1a67400f4de 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java @@ -165,7 +165,7 @@ public class TsFileResource { */ private TsFileResource originTsFileResource; - private AtomicReference maxProgressIndex = new AtomicReference<>(); + private final AtomicReference maxProgressIndex = new AtomicReference<>(); /** used to prevent circular replication in PipeConsensus */ private volatile boolean isGeneratedByPipeConsensus = false; @@ -1197,7 +1197,7 @@ public void updateProgressIndex(ProgressIndex progressIndex) { return; } - if (maxProgressIndex.compareAndSet(null, progressIndex)) { + if (!maxProgressIndex.compareAndSet(null, progressIndex)) { maxProgressIndex.get().updateToMinimumEqualOrIsAfterProgressIndex(progressIndex); } } From 0217c0f776d6bd580702822345121d215c00adef Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 15 Jul 2025 16:38:37 +0800 Subject: [PATCH 184/185] for-ci --- .../subtask/connector/PipeRealtimePriorityBlockingQueue.java | 1 - 1 file changed, 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java index 272a0947593e..409447b5b2fb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/subtask/connector/PipeRealtimePriorityBlockingQueue.java @@ -203,7 +203,6 @@ public Event peek() { public synchronized void replace( String dataRegionId, Set sourceFiles, List targetFiles) { - // TODO: return if pipe sink is for pipe consensus final int regionId = Integer.parseInt(dataRegionId); final Map> eventsToBeRemovedGroupByCommitterKey = From 71b7a4cf9e27043b14b3e68288ccbeb99fa76c52 Mon Sep 17 00:00:00 2001 From: Caideyipi <87789683+Caideyipi@users.noreply.github.com> Date: Tue, 15 Jul 2025 16:44:20 +0800 Subject: [PATCH 185/185] for-ci-2 --- .../event/common/tsfile/PipeCompactedTsFileInsertionEvent.java | 1 - .../iotdb/db/pipe/resource/tsfile/PipeCompactionManager.java | 1 - 2 files changed, 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java index f7673c41662e..9160ddf55af4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java @@ -74,7 +74,6 @@ public PipeCompactedTsFileInsertionEvent( // init fields of EnrichedEvent this.committerKey = committerKey; - // TODO pipe consensus: handle rebootTimes isPatternParsed = bindIsPatternParsed(originalEvents); isTimeParsed = bindIsTimeParsed(originalEvents); this.shouldReportOnCommit = shouldReportProgress; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeCompactionManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeCompactionManager.java index 29d880c720d7..03860fda8f85 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeCompactionManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/resource/tsfile/PipeCompactionManager.java @@ -57,7 +57,6 @@ public void emitResult( for (final PipeConnectorSubtaskLifeCycle lifeCycle : pipeConnectorSubtaskLifeCycles) { final UnboundedBlockingPendingQueue pendingQueue = lifeCycle.getPendingQueue(); - // TODO: support non realtime priority blocking queue if (pendingQueue instanceof PipeRealtimePriorityBlockingQueue) { final PipeRealtimePriorityBlockingQueue realtimePriorityBlockingQueue = (PipeRealtimePriorityBlockingQueue) pendingQueue;