From cf53a0f81ca2e92ec07f0314e38a93541277e550 Mon Sep 17 00:00:00 2001 From: XNX02 <202100130044@mail.sdu.edu.cn> Date: Mon, 10 Mar 2025 01:59:00 +0800 Subject: [PATCH 01/27] clusterId --- .../event/realtime/PipeRealtimeEvent.java | 35 +++++++++++++++++-- .../realtime/PipeRealtimeEventFactory.java | 6 ++-- .../realtime/epoch/TsFileEpochManager.java | 6 ++-- .../PipeInsertionDataNodeListener.java | 6 ++++ .../pipeconsensus/PipeConsensusReceiver.java | 2 +- .../thrift/IoTDBDataNodeReceiver.java | 5 +++ .../plan/planner/LogicalPlanVisitor.java | 1 + .../plan/planner/TreeModelPlanner.java | 25 +++++++++---- .../plan/planner/plan/node/PlanNode.java | 9 +++++ .../load/LoadTsFileDispatcherImpl.java | 8 +++-- .../scheduler/load/LoadTsFileScheduler.java | 32 ++++++++++++++++- .../statement/crud/InsertBaseStatement.java | 10 ++++++ .../statement/crud/LoadTsFileStatement.java | 10 ++++++ .../storageengine/dataregion/DataRegion.java | 9 +++-- .../dataregion/tsfile/TsFileResource.java | 16 +++++++++ .../storageengine/load/LoadTsFileManager.java | 2 +- .../CachedSchemaPatternMatcherTest.java | 2 +- .../connector/client/IoTDBSyncClient.java | 10 ++++++ .../client/IoTDBSyncClientManager.java | 8 +++-- .../commons/pipe/event/EnrichedEvent.java | 3 ++ .../pipe/receiver/IoTDBFileReceiver.java | 6 ++-- .../src/main/thrift/client.thrift | 1 + 22 files changed, 184 insertions(+), 28 deletions(-) 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 e68311da074b9..327d24a1aea53 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 @@ -44,6 +44,12 @@ public class PipeRealtimeEvent extends EnrichedEvent { private Map device2Measurements; + private final String originClusterId; + + public String getOriginClusterId() { + return originClusterId; + } + public PipeRealtimeEvent( final EnrichedEvent event, final TsFileEpoch tsFileEpoch, @@ -58,7 +64,27 @@ public PipeRealtimeEvent( null, true, Long.MIN_VALUE, - Long.MAX_VALUE); + Long.MAX_VALUE, + null); + } + + public PipeRealtimeEvent( + final EnrichedEvent event, + final TsFileEpoch tsFileEpoch, + final Map device2Measurements, + final String originClusterId) { + this( + event, + tsFileEpoch, + device2Measurements, + null, + null, + null, + null, + true, + Long.MIN_VALUE, + Long.MAX_VALUE, + originClusterId); } public PipeRealtimeEvent( @@ -71,7 +97,8 @@ public PipeRealtimeEvent( final String userName, final boolean skipIfNoPrivileges, final long startTime, - final long endTime) { + final long endTime, + final String originClusterId) { // PipeTaskMeta is used to report the progress of the event, the PipeRealtimeEvent // is only used in the realtime event extractor, which does not need to report the progress // of the event, so the pipeTaskMeta is always null. @@ -89,6 +116,7 @@ public PipeRealtimeEvent( this.event = event; this.tsFileEpoch = tsFileEpoch; this.device2Measurements = device2Measurements; + this.originClusterId = originClusterId; } public EnrichedEvent getEvent() { @@ -228,7 +256,8 @@ public PipeRealtimeEvent shallowCopySelfAndBindPipeTaskMetaForProgressReport( userName, skipIfNoPrivileges, startTime, - endTime); + endTime, + null); } @Override 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 92f4aaa7960c5..b2ec6356f4a2a 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 @@ -117,7 +117,7 @@ public static PipeRealtimeEvent createRealtimeEvent( public static PipeRealtimeEvent createRealtimeEvent( final String dataRegionId, final boolean shouldPrintMessage) { return new PipeRealtimeEvent( - new PipeHeartbeatEvent(dataRegionId, shouldPrintMessage), null, null); + new PipeHeartbeatEvent(dataRegionId, shouldPrintMessage), null, null, null); } public static PipeRealtimeEvent createRealtimeEvent( @@ -137,11 +137,11 @@ public static PipeRealtimeEvent createRealtimeEvent( deleteDataNodeEvent); } - return new PipeRealtimeEvent(deleteDataNodeEvent, null, null); + return new PipeRealtimeEvent(deleteDataNodeEvent, null, null, null); } public static PipeRealtimeEvent createRealtimeEvent(final ProgressReportEvent event) { - return new PipeRealtimeEvent(event, null, null); + return new PipeRealtimeEvent(event, null, null, null); } private PipeRealtimeEventFactory() { 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 869f9244e87cf..4f95758c6c380 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 @@ -70,7 +70,8 @@ public PipeRealtimeEvent bindPipeTsFileInsertionEvent( event, epoch, resource.getDevices().stream() - .collect(Collectors.toMap(Functions.identity(), device -> EMPTY_MEASUREMENT_ARRAY))); + .collect(Collectors.toMap(Functions.identity(), device -> EMPTY_MEASUREMENT_ARRAY)), + resource.getOriginClusterId()); } public PipeRealtimeEvent bindPipeInsertNodeTabletInsertionEvent( @@ -83,7 +84,8 @@ public PipeRealtimeEvent bindPipeInsertNodeTabletInsertionEvent( epoch, node instanceof InsertRowsNode ? getDevice2MeasurementsMapFromInsertRowsNode((InsertRowsNode) node) - : Collections.singletonMap(node.getDeviceID(), node.getMeasurements())); + : Collections.singletonMap(node.getDeviceID(), node.getMeasurements()), + node.getOriginClusterId()); } private Map getDevice2MeasurementsMapFromInsertRowsNode( 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 f1f8c22ac5168..5a63494d8cb3f 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 @@ -26,12 +26,15 @@ 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.assigner.PipeDataRegionAssigner; +import org.apache.iotdb.db.pipe.extractor.schemaregion.IoTDBSchemaRegionExtractor; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.AbstractDeleteDataNode; 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.Collections; import java.util.Objects; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicInteger; @@ -50,6 +53,8 @@ public class PipeInsertionDataNodeListener { private final ConcurrentMap dataRegionId2Assigner = new ConcurrentHashMap<>(); + private final Set ingoreCulsterId = + Collections.newSetFromMap(new ConcurrentHashMap<>()); private final AtomicInteger listenToTsFileExtractorCount = new AtomicInteger(0); private final AtomicInteger listenToInsertNodeExtractorCount = new AtomicInteger(0); @@ -103,6 +108,7 @@ public void listenToTsFile( final boolean isLoaded, final boolean isGeneratedByPipe) { tsFileResource.setGeneratedByPipe(isGeneratedByPipe); + tsFileResource.setOriginClusterId(tsFileResource.getOriginClusterId()); // 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/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java index eb3284d1233f1..366f92418e467 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java @@ -656,7 +656,7 @@ private TSStatus loadFileToDataRegion(String filePath, ProgressIndex progressInd StorageEngine.getInstance().getDataRegion(((DataRegionId) consensusGroupId)); if (region != null) { TsFileResource resource = generateTsFileResource(filePath, progressIndex); - region.loadNewTsFile(resource, true, false); + region.loadNewTsFile(resource, true, false, null); } else { // Data region is null indicates that dr has been removed or migrated. In those cases, there // is no need to replicate data. we just return success to avoid leader keeping retry 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 0420f6cd278d0..d3f8ac984f1ee 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 @@ -440,6 +440,7 @@ public synchronized TPipeTransferResp receive(final TPipeTransferReq req) { private TPipeTransferResp handleTransferTabletInsertNode( final PipeTransferTabletInsertNodeReq req) { final InsertBaseStatement statement = req.constructStatement(); + statement.setOriginClusterId(clusterIdFromHandshakeRequest); return new TPipeTransferResp( statement.isEmpty() ? RpcUtils.SUCCESS_STATUS @@ -448,6 +449,7 @@ private TPipeTransferResp handleTransferTabletInsertNode( private TPipeTransferResp handleTransferTabletBinary(final PipeTransferTabletBinaryReq req) { final InsertBaseStatement statement = req.constructStatement(); + statement.setOriginClusterId(clusterIdFromHandshakeRequest); return new TPipeTransferResp( statement.isEmpty() ? RpcUtils.SUCCESS_STATUS @@ -456,6 +458,7 @@ private TPipeTransferResp handleTransferTabletBinary(final PipeTransferTabletBin private TPipeTransferResp handleTransferTabletRaw(final PipeTransferTabletRawReq req) { final InsertTabletStatement statement = req.constructStatement(); + statement.setOriginClusterId(clusterIdFromHandshakeRequest); return new TPipeTransferResp( statement.isEmpty() ? RpcUtils.SUCCESS_STATUS @@ -596,6 +599,7 @@ private TSStatus loadTsFileSync(final String dataBaseName, final String fileAbso statement.setVerifySchema(validateTsFile.get()); statement.setAutoCreateDatabase(false); statement.setDatabase(dataBaseName); + statement.setOriginClusterId(clusterIdFromHandshakeRequest); return executeStatementAndClassifyExceptions(statement); } @@ -745,6 +749,7 @@ private TPipeTransferResp handleTransferSlice(final PipeTransferSliceReq pipeTra * message field. */ private TSStatus executeStatementAndAddRedirectInfo(final InsertBaseStatement statement) { + statement.setOriginClusterId(clusterIdFromHandshakeRequest); final TSStatus result = executeStatementAndClassifyExceptions(statement); if (result.getCode() == TSStatusCode.REDIRECTION_RECOMMEND.getStatusCode() diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanVisitor.java index 376f582eeddf8..36c07e7f2f893 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanVisitor.java @@ -491,6 +491,7 @@ public PlanNode visitInsertRow(InsertRowStatement insertRowStatement, MPPQueryCo insertRowStatement.getValues(), insertRowStatement.isNeedInferType()); insertNode.setFailedMeasurementNumber(insertRowStatement.getFailedMeasurementNumber()); + insertNode.setOriginClusterId(insertRowStatement.getOriginClusterId()); return insertNode; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TreeModelPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TreeModelPlanner.java index 88bdeb87a384b..d5ae132621a7b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TreeModelPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TreeModelPlanner.java @@ -134,13 +134,24 @@ public IScheduler doSchedule( instanceof LoadTsFileStatement; if (statement instanceof LoadTsFileStatement || isPipeEnrichedTsFileLoad) { scheduler = - new LoadTsFileScheduler( - distributedPlan, - context, - stateMachine, - syncInternalServiceClientManager, - partitionFetcher, - isPipeEnrichedTsFileLoad); + statement instanceof PipeEnrichedStatement + ? new LoadTsFileScheduler( + distributedPlan, + context, + stateMachine, + syncInternalServiceClientManager, + partitionFetcher, + true, + ((LoadTsFileStatement) ((PipeEnrichedStatement) statement).getInnerStatement()) + .getOriginClusterId()) + : new LoadTsFileScheduler( + distributedPlan, + context, + stateMachine, + syncInternalServiceClientManager, + partitionFetcher, + false); + } else { scheduler = new ClusterScheduler( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNode.java index 4e3196c8d42c4..0adf39f84564c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNode.java @@ -50,6 +50,7 @@ public abstract class PlanNode implements IConsensusRequest { protected PlanNodeId id; protected boolean isGeneratedByPipe = false; + protected String originClusterId = null; protected PlanNode() {} @@ -74,6 +75,14 @@ public void markAsGeneratedByPipe() { isGeneratedByPipe = true; } + public String getOriginClusterId() { + return originClusterId; + } + + public void setOriginClusterId(String originClusterId) { + this.originClusterId = originClusterId; + } + public abstract List getChildren(); public abstract void addChild(PlanNode child); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileDispatcherImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileDispatcherImpl.java index bab730c8af2c3..e761713ab7289 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileDispatcherImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileDispatcherImpl.java @@ -84,16 +84,19 @@ public class LoadTsFileDispatcherImpl implements IFragInstanceDispatcher { internalServiceClientManager; private final ExecutorService executor; private final boolean isGeneratedByPipe; + private final String originClusterId; public LoadTsFileDispatcherImpl( IClientManager internalServiceClientManager, - boolean isGeneratedByPipe) { + boolean isGeneratedByPipe, + String originClusterId) { this.internalServiceClientManager = internalServiceClientManager; this.localhostIpAddr = IoTDBDescriptor.getInstance().getConfig().getInternalAddress(); this.localhostInternalPort = IoTDBDescriptor.getInstance().getConfig().getInternalPort(); this.executor = IoTDBThreadPoolFactory.newCachedThreadPool(LoadTsFileDispatcherImpl.class.getName()); this.isGeneratedByPipe = isGeneratedByPipe; + this.originClusterId = originClusterId; } public void setUuid(String uuid) { @@ -176,7 +179,8 @@ public void dispatchLocally(FragmentInstance instance) throws FragmentInstanceDi .loadNewTsFile( tsFileResource, ((LoadSingleTsFileNode) planNode).isDeleteAfterLoad(), - isGeneratedByPipe); + isGeneratedByPipe, + originClusterId); } catch (LoadFileException e) { LOGGER.warn("Load TsFile Node {} error.", planNode, e); TSStatus resultStatus = new TSStatus(); 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 d8f90fbef0925..55fe7d402e277 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 @@ -140,6 +140,7 @@ public class LoadTsFileScheduler implements IScheduler { private final Set allReplicaSets; private final boolean isGeneratedByPipe; private final LoadTsFileDataCacheMemoryBlock block; + private final String originClusterId; public LoadTsFileScheduler( DistributedQueryPlan distributedQueryPlan, @@ -153,11 +154,40 @@ public LoadTsFileScheduler( this.tsFileNodeList = new ArrayList<>(); this.failedTsFileNodeIndexes = new ArrayList<>(); this.fragmentId = distributedQueryPlan.getRootSubPlan().getPlanFragment().getId(); - this.dispatcher = new LoadTsFileDispatcherImpl(internalServiceClientManager, isGeneratedByPipe); + this.dispatcher = + new LoadTsFileDispatcherImpl(internalServiceClientManager, isGeneratedByPipe, null); this.partitionFetcher = new DataPartitionBatchFetcher(partitionFetcher); this.allReplicaSets = new HashSet<>(); this.isGeneratedByPipe = isGeneratedByPipe; this.block = LoadTsFileMemoryManager.getInstance().allocateDataCacheMemoryBlock(); + this.originClusterId = null; + + for (FragmentInstance fragmentInstance : distributedQueryPlan.getInstances()) { + tsFileNodeList.add((LoadSingleTsFileNode) fragmentInstance.getFragment().getPlanNodeTree()); + } + } + + public LoadTsFileScheduler( + DistributedQueryPlan distributedQueryPlan, + MPPQueryContext queryContext, + QueryStateMachine stateMachine, + IClientManager internalServiceClientManager, + IPartitionFetcher partitionFetcher, + boolean isGeneratedByPipe, + String originClusterId) { + this.queryContext = queryContext; + this.stateMachine = stateMachine; + this.tsFileNodeList = new ArrayList<>(); + this.failedTsFileNodeIndexes = new ArrayList<>(); + this.fragmentId = distributedQueryPlan.getRootSubPlan().getPlanFragment().getId(); + this.dispatcher = + new LoadTsFileDispatcherImpl( + internalServiceClientManager, isGeneratedByPipe, originClusterId); + this.partitionFetcher = new DataPartitionBatchFetcher(partitionFetcher); + this.allReplicaSets = new HashSet<>(); + this.isGeneratedByPipe = isGeneratedByPipe; + this.block = LoadTsFileMemoryManager.getInstance().allocateDataCacheMemoryBlock(); + this.originClusterId = originClusterId; for (FragmentInstance fragmentInstance : distributedQueryPlan.getInstances()) { tsFileNodeList.add((LoadSingleTsFileNode) fragmentInstance.getFragment().getPlanNodeTree()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertBaseStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertBaseStatement.java index af95bc4edb900..2770080fb4797 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertBaseStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertBaseStatement.java @@ -102,6 +102,16 @@ public abstract class InsertBaseStatement extends Statement { @TableModel protected String databaseName; + public String getOriginClusterId() { + return originClusterId; + } + + public void setOriginClusterId(String originClusterId) { + this.originClusterId = originClusterId; + } + + protected String originClusterId; + // endregion public PartialPath getDevicePath() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatement.java index 0c9c0eaaf48b4..16f6da5939fc8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatement.java @@ -68,6 +68,16 @@ public class LoadTsFileStatement extends Statement { private final List resources; private final List writePointCountList; + public String getOriginClusterId() { + return originClusterId; + } + + public void setOriginClusterId(String originClusterId) { + this.originClusterId = originClusterId; + } + + private String originClusterId; + public LoadTsFileStatement(String filePath) throws FileNotFoundException { this.file = new File(filePath); this.databaseLevel = IoTDBDescriptor.getInstance().getConfig().getDefaultStorageGroupLevel(); 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 263de360061d8..479457c2c7bd9 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 @@ -2947,7 +2947,8 @@ public int compact() { public void loadNewTsFile( final TsFileResource newTsFileResource, final boolean deleteOriginFile, - final boolean isGeneratedByPipe) + final boolean isGeneratedByPipe, + final String originClusterId) throws LoadFileException { final File tsfileToBeInserted = newTsFileResource.getTsFile(); final long newFilePartitionId = newTsFileResource.getTimePartitionWithCheck(); @@ -2980,7 +2981,8 @@ public void loadNewTsFile( newTsFileResource, newFilePartitionId, deleteOriginFile, - isGeneratedByPipe); + isGeneratedByPipe, + originClusterId); FileMetrics.getInstance() .addTsFile( @@ -3048,7 +3050,8 @@ private boolean loadTsFileToUnSequence( final TsFileResource tsFileResource, final long filePartitionId, final boolean deleteOriginFile, - boolean isGeneratedByPipe) + boolean isGeneratedByPipe, + final String originClusterId) throws LoadFileException, DiskSpaceInsufficientException { final File targetFile = fsFactory.getFile( 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 204fe1f32d438..a572d638feab8 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 @@ -202,6 +202,16 @@ public class TsFileResource implements PersistentResource { /** used to prevent circular replication in Pipe */ private boolean isGeneratedByPipe = false; + public String getOriginClusterId() { + return originClusterId; + } + + public void setOriginClusterId(String originClusterId) { + this.originClusterId = originClusterId; + } + + private String originClusterId; + private InsertionCompactionCandidateStatus insertionCompactionCandidateStatus = InsertionCompactionCandidateStatus.NOT_CHECKED; @@ -308,6 +318,9 @@ private void serializeTo(BufferedOutputStream outputStream) throws IOException { ReadWriteIOUtils.write(isGeneratedByPipeConsensus, outputStream); ReadWriteIOUtils.write(isGeneratedByPipe, outputStream); + if (originClusterId != null) { + ReadWriteIOUtils.write(originClusterId, outputStream); + } } /** deserialize from disk */ @@ -344,6 +357,9 @@ public void deserialize() throws IOException { isGeneratedByPipeConsensus = ReadWriteIOUtils.readBoolean(inputStream); isGeneratedByPipe = ReadWriteIOUtils.readBoolean(inputStream); } + if (inputStream.available() > 0) { + originClusterId = ReadWriteIOUtils.readString(inputStream); + } } } 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 74bf838a8d3f1..a56a02fbe193b 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 @@ -517,7 +517,7 @@ private void loadAll(boolean isGeneratedByPipe, ProgressIndex progressIndex) final DataRegion dataRegion = entry.getKey().getDataRegion(); final TsFileResource tsFileResource = dataPartition2Resource.get(entry.getKey()); endTsFileResource(writer, tsFileResource, progressIndex); - dataRegion.loadNewTsFile(tsFileResource, true, isGeneratedByPipe); + dataRegion.loadNewTsFile(tsFileResource, true, isGeneratedByPipe, null); // Metrics dataRegion 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 index 61f91fea8a3a4..2dd637a35af01 100644 --- 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 @@ -59,7 +59,7 @@ public MockedPipeRealtimeEvent( final EnrichedEvent event, final TsFileEpoch tsFileEpoch, final Map device2Measurements) { - super(event, tsFileEpoch, device2Measurements); + super(event, tsFileEpoch, device2Measurements, null); } @Override diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClient.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClient.java index d79c430474c32..9915402136f18 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClient.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClient.java @@ -52,6 +52,16 @@ public class IoTDBSyncClient extends IClientRPCService.Client private final int port; private final TEndPoint endPoint; + public String getClusterId() { + return clusterId; + } + + public void setClusterId(String clusterId) { + this.clusterId = clusterId; + } + + private String clusterId; + public IoTDBSyncClient( ThriftClientProperty property, String ipAddress, diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClientManager.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClientManager.java index 17a48e1dd9d8f..9b39267842eab 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClientManager.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClientManager.java @@ -249,10 +249,14 @@ public void sendHandshakeReq(final Pair clientAndStatu } else { clientAndStatus.setRight(true); client.setTimeout(CONNECTION_TIMEOUT_MS.get()); + if (resp.isSetClusterId()) { + client.setClusterId(resp.getClusterId()); + } LOGGER.info( - "Handshake success. Target server ip: {}, port: {}", + "Handshake success. Target server ip: {}, port: {}, clusterId:{}", client.getIpAddress(), - client.getPort()); + client.getPort(), + client.getClusterId()); } } catch (Exception e) { LOGGER.warn( 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 6b4ae19464aec..563364e084fac 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 @@ -79,6 +79,8 @@ public abstract class EnrichedEvent implements Event { protected String userName; protected boolean skipIfNoPrivileges; + // protected final String originClusterId; + protected EnrichedEvent( final String pipeName, final long creationTime, @@ -101,6 +103,7 @@ protected EnrichedEvent( this.skipIfNoPrivileges = skipIfNoPrivileges; this.startTime = startTime; this.endTime = endTime; + // this.originClusterId = originClusterId; isPatternParsed = (treePattern == null || treePattern.isRoot()) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiver.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiver.java index 1e0f6305f2d8f..53c907eb90b65 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiver.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiver.java @@ -74,6 +74,8 @@ public abstract class IoTDBFileReceiver implements IoTDBReceiver { protected String username = CONNECTOR_IOTDB_USER_DEFAULT_VALUE; protected String password = CONNECTOR_IOTDB_PASSWORD_DEFAULT_VALUE; + protected String clusterIdFromHandshakeRequest; + private static final boolean IS_FSYNC_ENABLED = PipeConfig.getInstance().getPipeFileReceiverFsyncEnabled(); private File writingFile; @@ -187,7 +189,7 @@ protected TPipeTransferResp handleTransferHandshakeV1(final PipeTransferHandshak getSenderHost(), getSenderPort(), newReceiverDir.getPath()); - return new TPipeTransferResp(RpcUtils.SUCCESS_STATUS); + return new TPipeTransferResp(RpcUtils.SUCCESS_STATUS).setClusterId(getClusterId()); } protected abstract String getReceiverFileBaseDir() throws Exception; @@ -211,7 +213,7 @@ protected TPipeTransferResp handleTransferHandshakeV2(final PipeTransferHandshak } // Reject to handshake if the request does not contain sender's clusterId. - final String clusterIdFromHandshakeRequest = + clusterIdFromHandshakeRequest = req.getParams().get(PipeTransferHandshakeConstant.HANDSHAKE_KEY_CLUSTER_ID); if (clusterIdFromHandshakeRequest == null) { final TSStatus status = diff --git a/iotdb-protocol/thrift-datanode/src/main/thrift/client.thrift b/iotdb-protocol/thrift-datanode/src/main/thrift/client.thrift index f658017b60057..ac2a3a6af68f1 100644 --- a/iotdb-protocol/thrift-datanode/src/main/thrift/client.thrift +++ b/iotdb-protocol/thrift-datanode/src/main/thrift/client.thrift @@ -497,6 +497,7 @@ struct TPipeTransferReq { struct TPipeTransferResp { 1:required common.TSStatus status 2:optional binary body + 3:optional string clusterId } struct TPipeSubscribeReq { From 1b0ad1e9b6ff58febd6b45270c4b210d776b9423 Mon Sep 17 00:00:00 2001 From: XNX02 <202100130044@mail.sdu.edu.cn> Date: Mon, 10 Mar 2025 15:31:52 +0800 Subject: [PATCH 02/27] update --- .../PipeRealtimeDataRegionExtractor.java | 16 ++++++++++------ .../assigner/PipeDataRegionAssigner.java | 2 +- .../listener/PipeInsertionDataNodeListener.java | 5 +++-- .../db/storageengine/dataregion/DataRegion.java | 2 +- .../dataregion/memtable/TsFileProcessor.java | 3 ++- .../pipe/extractor/PipeRealtimeExtractTest.java | 2 +- .../commons/pipe/extractor/IoTDBExtractor.java | 10 ++++------ 7 files changed, 22 insertions(+), 18 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 e07385fd8138a..a39ef37b9a36c 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 @@ -113,6 +113,8 @@ public abstract class PipeRealtimeDataRegionExtractor implements PipeExtractor { protected boolean isForwardingPipeRequests; + protected boolean isDoubleLiving; + private boolean shouldTransferModFile; // Whether to transfer mods private boolean sloppyTimeRange; // true to disable time range filter after extraction @@ -241,22 +243,19 @@ public void customize( ? TimePartitionUtils.getTimePartitionId(realtimeDataExtractionEndTime) : TimePartitionUtils.getTimePartitionId(realtimeDataExtractionEndTime) - 1; - final boolean isDoubleLiving = + isDoubleLiving = parameters.getBooleanOrDefault( Arrays.asList( PipeExtractorConstant.EXTRACTOR_MODE_DOUBLE_LIVING_KEY, PipeExtractorConstant.SOURCE_MODE_DOUBLE_LIVING_KEY), PipeExtractorConstant.EXTRACTOR_MODE_DOUBLE_LIVING_DEFAULT_VALUE); - if (isDoubleLiving) { - isForwardingPipeRequests = false; - } else { + isForwardingPipeRequests = parameters.getBooleanOrDefault( Arrays.asList( PipeExtractorConstant.EXTRACTOR_FORWARDING_PIPE_REQUESTS_KEY, PipeExtractorConstant.SOURCE_FORWARDING_PIPE_REQUESTS_KEY), PipeExtractorConstant.EXTRACTOR_FORWARDING_PIPE_REQUESTS_DEFAULT_VALUE); - } if (parameters.hasAnyAttributes(EXTRACTOR_MODS_KEY, SOURCE_MODS_KEY)) { shouldTransferModFile = @@ -539,6 +538,11 @@ public final boolean isForwardingPipeRequests() { return isForwardingPipeRequests; } + public final boolean isDoubleLiving(){ + return isDoubleLiving; + } + + public abstract boolean isNeedListenToTsFile(); public abstract boolean isNeedListenToInsertNode(); @@ -567,7 +571,7 @@ public String toString() { .add("startTimePartitionIdLowerBound", startTimePartitionIdLowerBound) .add("endTimePartitionIdUpperBound", endTimePartitionIdUpperBound) .add("dataRegionTimePartitionIdBound", dataRegionTimePartitionIdBound) - .add("isForwardingPipeRequests", isForwardingPipeRequests) + .add("isForwardingPipeRequests", isForwardingPipeRequests).add("isDoubleLiving", isDoubleLiving) .add("shouldTransferModFile", shouldTransferModFile) .add("sloppyTimeRange", sloppyTimeRange) .add("sloppyPattern", sloppyPattern) 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 3184d6a3524ae..e08456c718241 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 @@ -146,7 +146,7 @@ private void assignToExtractor( return; } - if (event.getEvent().isGeneratedByPipe() && !extractor.isForwardingPipeRequests()) { + 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)) { 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 5a63494d8cb3f..752d88635e26b 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 @@ -106,9 +106,10 @@ public void listenToTsFile( final String databaseName, final TsFileResource tsFileResource, final boolean isLoaded, - final boolean isGeneratedByPipe) { + final boolean isGeneratedByPipe, + final String originClusterId) { tsFileResource.setGeneratedByPipe(isGeneratedByPipe); - tsFileResource.setOriginClusterId(tsFileResource.getOriginClusterId()); + tsFileResource.setOriginClusterId(originClusterId); // 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 479457c2c7bd9..bf61712b71894 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 @@ -3142,7 +3142,7 @@ private boolean loadTsFileToUnSequence( // Listen before the tsFile is added into tsFile manager to avoid it being compacted PipeInsertionDataNodeListener.getInstance() - .listenToTsFile(dataRegionId, databaseName, tsFileResource, true, isGeneratedByPipe); + .listenToTsFile(dataRegionId, databaseName, tsFileResource, true, isGeneratedByPipe,originClusterId); 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 ecd6dc2be3971..f24981d7ea73e 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 @@ -1291,7 +1291,8 @@ public Future asyncClose() { dataRegionInfo.getDataRegion().getDatabaseName(), tsFileResource, false, - tmpMemTable.isTotallyGeneratedByPipe()); + tmpMemTable.isTotallyGeneratedByPipe(), + null); // When invoke closing TsFile after insert data to memTable, we shouldn't flush until invoke // flushing memTable in System module. 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 63d5661c5c71a..f4ea7dc42c80e 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 @@ -317,7 +317,7 @@ private Future write2DataRegion(int writeNum, String dataRegionId, int startN false), resource); PipeInsertionDataNodeListener.getInstance() - .listenToTsFile(dataRegionId, dataRegionId, resource, false, false); + .listenToTsFile(dataRegionId, dataRegionId, resource, false, false,null); } }); } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/extractor/IoTDBExtractor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/extractor/IoTDBExtractor.java index 04bf535d78f88..d348d16448878 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/extractor/IoTDBExtractor.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/extractor/IoTDBExtractor.java @@ -60,7 +60,7 @@ public abstract class IoTDBExtractor implements PipeExtractor { protected PipeTaskMeta pipeTaskMeta; protected boolean isForwardingPipeRequests; - + protected boolean isDoubleLiving; // The value is always true after the first start even the extractor is closed protected final AtomicBoolean hasBeenStarted = new AtomicBoolean(false); protected String userName; @@ -148,22 +148,20 @@ public void customize( taskID = pipeName + "_" + regionId + "_" + creationTime; pipeTaskMeta = environment.getPipeTaskMeta(); - final boolean isDoubleLiving = + isDoubleLiving = parameters.getBooleanOrDefault( Arrays.asList( PipeExtractorConstant.EXTRACTOR_MODE_DOUBLE_LIVING_KEY, PipeExtractorConstant.SOURCE_MODE_DOUBLE_LIVING_KEY), PipeExtractorConstant.EXTRACTOR_MODE_DOUBLE_LIVING_DEFAULT_VALUE); - if (isDoubleLiving) { - isForwardingPipeRequests = false; - } else { + isForwardingPipeRequests = parameters.getBooleanOrDefault( Arrays.asList( PipeExtractorConstant.EXTRACTOR_FORWARDING_PIPE_REQUESTS_KEY, PipeExtractorConstant.SOURCE_FORWARDING_PIPE_REQUESTS_KEY), PipeExtractorConstant.EXTRACTOR_FORWARDING_PIPE_REQUESTS_DEFAULT_VALUE); - } + userName = parameters.getStringByKeys( From 9af04457418c30695972b8b84dafcf5fb08ecd08 Mon Sep 17 00:00:00 2001 From: XNX02 <202100130044@mail.sdu.edu.cn> Date: Tue, 11 Mar 2025 00:34:53 +0800 Subject: [PATCH 03/27] update --- .../task/builder/PipeDataNodeTaskBuilder.java | 15 +++---- .../task/stage/PipeTaskConnectorStage.java | 6 +++ .../task/stage/PipeTaskExtractorStage.java | 16 +++++++- .../PipeConnectorSubtaskManager.java | 21 ++++++++++ .../async/IoTDBDataRegionAsyncConnector.java | 5 +++ .../sync/IoTDBDataNodeSyncConnector.java | 4 ++ .../sync/IoTDBDataRegionSyncConnector.java | 1 - .../pipe/event/UserDefinedEnrichedEvent.java | 5 +++ .../deletion/PipeDeleteDataNodeEvent.java | 6 +++ .../common/heartbeat/PipeHeartbeatEvent.java | 5 +++ .../PipeSchemaRegionWritePlanEvent.java | 16 ++++++-- .../PipeInsertNodeTabletInsertionEvent.java | 13 +++++- .../tablet/PipeRawTabletInsertionEvent.java | 5 +++ .../common/terminate/PipeTerminateEvent.java | 5 +++ .../tsfile/PipeTsFileInsertionEvent.java | 9 +++++ .../event/realtime/PipeRealtimeEvent.java | 40 ++++--------------- .../realtime/PipeRealtimeEventFactory.java | 9 +++-- .../dataregion/IoTDBDataRegionExtractor.java | 9 +++++ .../PipeRealtimeDataRegionExtractor.java | 26 ++++++++---- .../assigner/PipeDataRegionAssigner.java | 5 ++- .../realtime/epoch/TsFileEpochManager.java | 6 +-- .../PipeInsertionDataNodeListener.java | 5 --- .../IoTDBSchemaRegionExtractor.java | 5 ++- .../SchemaRegionListeningQueue.java | 8 +++- .../storageengine/dataregion/DataRegion.java | 3 +- .../dataregion/memtable/TsFileProcessor.java | 2 +- .../extractor/PipeRealtimeExtractTest.java | 2 +- .../CachedSchemaPatternMatcherTest.java | 2 +- .../connector/client/IoTDBSyncClient.java | 10 ----- .../client/IoTDBSyncClientManager.java | 16 ++++++-- .../commons/pipe/event/EnrichedEvent.java | 2 + .../commons/pipe/event/PipeSnapshotEvent.java | 5 +++ .../pipe/event/PipeWritePlanEvent.java | 31 ++++++++++++++ .../pipe/event/ProgressReportEvent.java | 5 +++ .../pipe/extractor/IoTDBExtractor.java | 13 +++--- .../IoTDBNonDataRegionExtractor.java | 17 +++++++- 36 files changed, 255 insertions(+), 98 deletions(-) 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 1b517419c9852..f420ba58d9d0a 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 @@ -95,13 +95,6 @@ public PipeDataNodeTask build() { checkConflict(extractorParameters, connectorParameters); // We first build the extractor and connector, then build the processor. - final PipeTaskExtractorStage extractorStage = - new PipeTaskExtractorStage( - pipeStaticMeta.getPipeName(), - pipeStaticMeta.getCreationTime(), - extractorParameters, - regionId, - pipeTaskMeta); final PipeTaskConnectorStage connectorStage; final PipeType pipeType = pipeStaticMeta.getPipeType(); @@ -124,6 +117,14 @@ public PipeDataNodeTask build() { CONNECTOR_EXECUTOR_MAP.get(pipeType)); } + final PipeTaskExtractorStage extractorStage = + new PipeTaskExtractorStage( + pipeStaticMeta.getPipeName(), + pipeStaticMeta.getCreationTime(), + extractorParameters, + regionId, + pipeTaskMeta, + connectorStage.getPipeConnectorClusterIds()); // The processor connects the extractor and connector. final PipeTaskProcessorStage processorStage = new PipeTaskProcessorStage( 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 345e6ab040c4a..da5d883a6cc24 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,6 +28,8 @@ import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.pipe.api.exception.PipeException; +import java.util.Set; + public class PipeTaskConnectorStage extends PipeTaskStage { protected final String pipeName; @@ -86,4 +88,8 @@ public void dropSubtask() throws PipeException { public UnboundedBlockingPendingQueue getPipeConnectorPendingQueue() { return PipeConnectorSubtaskManager.instance().getPipeConnectorPendingQueue(connectorSubtaskId); } + + public Set getPipeConnectorClusterIds() { + return PipeConnectorSubtaskManager.instance().getSinkClusterIds(connectorSubtaskId); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/stage/PipeTaskExtractorStage.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/stage/PipeTaskExtractorStage.java index f6a2030e5fa76..ff0d96cc625fd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/stage/PipeTaskExtractorStage.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/stage/PipeTaskExtractorStage.java @@ -26,6 +26,8 @@ import org.apache.iotdb.commons.pipe.config.plugin.configuraion.PipeTaskRuntimeConfiguration; import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskExtractorRuntimeEnvironment; 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.storageengine.StorageEngine; import org.apache.iotdb.pipe.api.PipeExtractor; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; @@ -35,6 +37,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Set; + public class PipeTaskExtractorStage extends PipeTaskStage { private static final Logger LOGGER = LoggerFactory.getLogger(PipeTaskExtractorStage.class); @@ -46,7 +50,8 @@ public PipeTaskExtractorStage( long creationTime, PipeParameters extractorParameters, int regionId, - PipeTaskMeta pipeTaskMeta) { + PipeTaskMeta pipeTaskMeta, + Set originClusterIds) { pipeExtractor = StorageEngine.getInstance().getAllDataRegionIds().contains(new DataRegionId(regionId)) ? PipeDataNodeAgent.plugin().dataRegion().reflectExtractor(extractorParameters) @@ -64,6 +69,15 @@ public PipeTaskExtractorStage( new PipeTaskExtractorRuntimeEnvironment( pipeName, creationTime, regionId, pipeTaskMeta)); pipeExtractor.customize(extractorParameters, runtimeConfiguration); + + if (pipeExtractor instanceof IoTDBDataRegionExtractor) { + ((IoTDBDataRegionExtractor) pipeExtractor).setSinkClusterIds(originClusterIds); + } + + if (pipeExtractor instanceof IoTDBSchemaRegionExtractor) { + ((IoTDBSchemaRegionExtractor) pipeExtractor).setSinkClusterIds(originClusterIds); + } + } catch (Exception e) { try { pipeExtractor.close(); 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 86ba75c37fc0d..bdf34a7809be9 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 @@ -29,6 +29,8 @@ import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskConnectorRuntimeEnvironment; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; import org.apache.iotdb.db.pipe.agent.task.execution.PipeConnectorSubtaskExecutor; +import org.apache.iotdb.db.pipe.connector.protocol.thrift.async.IoTDBDataRegionAsyncConnector; +import org.apache.iotdb.db.pipe.connector.protocol.thrift.sync.IoTDBDataNodeSyncConnector; import org.apache.iotdb.db.pipe.metric.source.PipeDataRegionEventCounter; import org.apache.iotdb.db.storageengine.StorageEngine; import org.apache.iotdb.pipe.api.PipeConnector; @@ -45,6 +47,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.TreeMap; public class PipeConnectorSubtaskManager { @@ -56,6 +59,7 @@ public class PipeConnectorSubtaskManager { private final Map> attributeSortedString2SubtaskLifeCycleMap = new HashMap<>(); + private final Map> attributeSortedStringToSinkClusterIdsMap = new HashMap<>(); public synchronized String register( final PipeConnectorSubtaskExecutor executor, @@ -116,6 +120,7 @@ public synchronized String register( : new UnboundedBlockingPendingQueue<>(new PipeDataRegionEventCounter()); for (int connectorIndex = 0; connectorIndex < connectorNum; connectorIndex++) { + final PipeConnector pipeConnector = isDataRegionConnector ? PipeDataNodeAgent.plugin().dataRegion().reflectConnector(pipeConnectorParameters) @@ -156,6 +161,17 @@ public synchronized String register( final PipeConnectorSubtaskLifeCycle pipeConnectorSubtaskLifeCycle = new PipeConnectorSubtaskLifeCycle(executor, pipeConnectorSubtask, pendingQueue); pipeConnectorSubtaskLifeCycleList.add(pipeConnectorSubtaskLifeCycle); + + if (pipeConnector instanceof IoTDBDataNodeSyncConnector) { + attributeSortedStringToSinkClusterIdsMap.put( + attributeSortedString, + ((IoTDBDataNodeSyncConnector) pipeConnector).getEndPointsClusterIds()); + } + if (pipeConnector instanceof IoTDBDataRegionAsyncConnector) { + attributeSortedStringToSinkClusterIdsMap.put( + attributeSortedString, + ((IoTDBDataRegionAsyncConnector) pipeConnector).getEndPointsClusterIds()); + } } attributeSortedString2SubtaskLifeCycleMap.put( @@ -185,6 +201,7 @@ public synchronized void deregister( if (lifeCycles.isEmpty()) { attributeSortedString2SubtaskLifeCycleMap.remove(attributeSortedString); + attributeSortedStringToSinkClusterIdsMap.remove(attributeSortedString); } PipeEventCommitManager.getInstance().deregister(pipeName, creationTime, regionId); @@ -233,6 +250,10 @@ private String generateAttributeSortedString(final PipeParameters pipeConnectorP return sortedStringSourceMap.toString(); } + public Set getSinkClusterIds(String attributeSortedString) { + return attributeSortedStringToSinkClusterIdsMap.get(attributeSortedString); + } + ///////////////////////// Singleton Instance Holder ///////////////////////// private PipeConnectorSubtaskManager() { 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 8ef6d3d9d6e7e..3492e63c0f3fa 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 @@ -70,6 +70,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Set; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.LinkedBlockingQueue; @@ -590,6 +591,10 @@ public synchronized void close() { super.close(); } + public Set getEndPointsClusterIds() { + return retryConnector.getEndPointsClusterIds(); + } + //////////////////////// APIs provided for metric framework //////////////////////// public int getRetryEventQueueSize() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataNodeSyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataNodeSyncConnector.java index dbd96fd7d1b2c..c2324a4b60895 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataNodeSyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataNodeSyncConnector.java @@ -106,4 +106,8 @@ protected IoTDBSyncClientManager constructClient( shouldMarkAsPipeRequest); return clientManager; } + + public Set getEndPointsClusterIds() { + return clientManager.getEndPointsClusterIds(); + } } 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 56816220a813b..2c1f313e55993 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 @@ -265,7 +265,6 @@ private void doTransferWrapper(final Pair endPo private void doTransfer( final TEndPoint endPoint, final PipeTabletEventPlainBatch batchToTransfer) { final Pair clientAndStatus = clientManager.getClient(endPoint); - final TPipeTransferResp resp; try { final TPipeTransferReq uncompressedReq = batchToTransfer.toTPipeTransferReq(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/UserDefinedEnrichedEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/UserDefinedEnrichedEvent.java index cdfbc273593f9..fdcddf7907812 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/UserDefinedEnrichedEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/UserDefinedEnrichedEvent.java @@ -103,6 +103,11 @@ public boolean isGeneratedByPipe() { return enrichedEvent.isGeneratedByPipe(); } + @Override + public String getOriginClusterId() { + return enrichedEvent.getOriginClusterId(); + } + @Override public boolean mayEventTimeOverlappedWithTimeRange() { return enrichedEvent.mayEventTimeOverlappedWithTimeRange(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java index 692144edfb062..f60fafea5d23d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java @@ -44,6 +44,7 @@ public class PipeDeleteDataNodeEvent extends EnrichedEvent implements Serializab private AbstractDeleteDataNode deleteDataNode; private DeletionResource deletionResource; private boolean isGeneratedByPipe; + private String originClusterId; private ProgressIndex progressIndex; public PipeDeleteDataNodeEvent() { @@ -145,6 +146,11 @@ public boolean isGeneratedByPipe() { return isGeneratedByPipe; } + @Override + public String getOriginClusterId() { + return originClusterId; + } + @Override public void throwIfNoPrivilege() { if (skipIfNoPrivileges || !(deleteDataNode instanceof RelationalDeleteDataNode)) { 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 1339611e7bfe8..6f454aa237627 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 @@ -140,6 +140,11 @@ public boolean isGeneratedByPipe() { return false; } + @Override + public String getOriginClusterId() { + return null; + } + @Override public boolean mayEventTimeOverlappedWithTimeRange() { return true; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java index 85517fc23a7d1..b12435a91162c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java @@ -41,7 +41,12 @@ public PipeSchemaRegionWritePlanEvent() { } public PipeSchemaRegionWritePlanEvent(final PlanNode planNode, final boolean isGeneratedByPipe) { - this(planNode, null, 0, null, null, null, null, true, isGeneratedByPipe); + this(planNode, null, 0, null, null, null, null, true, isGeneratedByPipe, null); + } + + public PipeSchemaRegionWritePlanEvent( + final PlanNode planNode, final boolean isGeneratedByPipe, final String originClusterId) { + this(planNode, null, 0, null, null, null, null, true, isGeneratedByPipe, originClusterId); } public PipeSchemaRegionWritePlanEvent( @@ -53,7 +58,8 @@ public PipeSchemaRegionWritePlanEvent( final TablePattern tablePattern, final String userName, final boolean skipIfNoPrivileges, - final boolean isGeneratedByPipe) { + final boolean isGeneratedByPipe, + final String originClusterId) { super( pipeName, creationTime, @@ -62,7 +68,8 @@ public PipeSchemaRegionWritePlanEvent( tablePattern, userName, skipIfNoPrivileges, - isGeneratedByPipe); + isGeneratedByPipe, + originClusterId); this.planNode = planNode; } @@ -90,7 +97,8 @@ public EnrichedEvent shallowCopySelfAndBindPipeTaskMetaForProgressReport( tablePattern, userName, skipIfNoPrivileges, - isGeneratedByPipe); + isGeneratedByPipe, + null); } @Override 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 84976cdab5712..ffc40111a5a08 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 @@ -91,6 +91,7 @@ public class PipeInsertNodeTabletInsertionEvent extends PipeInsertionEvent private final WALEntryHandler walEntryHandler; private final boolean isAligned; private final boolean isGeneratedByPipe; + private final String originClusterId; private final AtomicReference allocatedMemoryBlock; private volatile List tablets; @@ -111,7 +112,8 @@ public PipeInsertNodeTabletInsertionEvent( final Set tableNames, final ProgressIndex progressIndex, final boolean isAligned, - final boolean isGeneratedByPipe) { + final boolean isGeneratedByPipe, + final String originClusterId) { this( isTableModel, databaseNameFromDataRegion, @@ -121,6 +123,7 @@ public PipeInsertNodeTabletInsertionEvent( progressIndex, isAligned, isGeneratedByPipe, + originClusterId, null, 0, null, @@ -141,6 +144,7 @@ private PipeInsertNodeTabletInsertionEvent( final ProgressIndex progressIndex, final boolean isAligned, final boolean isGeneratedByPipe, + final String originClusterId, final String pipeName, final long creationTime, final PipeTaskMeta pipeTaskMeta, @@ -169,6 +173,7 @@ private PipeInsertNodeTabletInsertionEvent( this.progressIndex = progressIndex; this.isAligned = isAligned; this.isGeneratedByPipe = isGeneratedByPipe; + this.originClusterId = originClusterId; this.allocatedMemoryBlock = new AtomicReference<>(); } @@ -272,6 +277,7 @@ public PipeInsertNodeTabletInsertionEvent shallowCopySelfAndBindPipeTaskMetaForP progressIndex, isAligned, isGeneratedByPipe, + originClusterId, pipeName, creationTime, pipeTaskMeta, @@ -288,6 +294,11 @@ public boolean isGeneratedByPipe() { return isGeneratedByPipe; } + @Override + public String getOriginClusterId() { + return originClusterId; + } + @Override public void throwIfNoPrivilege() { if (skipIfNoPrivileges || !isTableModelEvent()) { 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 69076810175e6..cc1f2f514ec85 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 @@ -293,6 +293,11 @@ public boolean isGeneratedByPipe() { throw new UnsupportedOperationException("isGeneratedByPipe() is not supported!"); } + @Override + public String getOriginClusterId() { + throw new UnsupportedOperationException("getOriginClusterId() is not supported!"); + } + @Override public boolean mayEventTimeOverlappedWithTimeRange() { final long[] timestamps = tablet.getTimestamps(); 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 7cfd9f139a2b7..c557f2ab7a967 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 @@ -92,6 +92,11 @@ public boolean isGeneratedByPipe() { return false; } + @Override + public String getOriginClusterId() { + return null; + } + @Override public boolean mayEventTimeOverlappedWithTimeRange() { return true; 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 fb2395ecd44e0..53115977ab1b4 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 @@ -85,6 +85,8 @@ public class PipeTsFileInsertionEvent extends PipeInsertionEvent private final boolean isGeneratedByPipeConsensus; private final boolean isGeneratedByHistoricalExtractor; + private final String originClusterId; + private final AtomicBoolean isClosed; private final AtomicReference eventParser; @@ -162,6 +164,8 @@ public PipeTsFileInsertionEvent( this.isGeneratedByPipeConsensus = resource.isGeneratedByPipeConsensus(); this.isGeneratedByHistoricalExtractor = isGeneratedByHistoricalExtractor; + this.originClusterId = resource.getOriginClusterId(); + isClosed = new AtomicBoolean(resource.isClosed()); // Register close listener if TsFile is not closed if (!isClosed.get()) { @@ -428,6 +432,11 @@ public boolean isGeneratedByPipe() { return isGeneratedByPipe; } + @Override + public String getOriginClusterId() { + return originClusterId; + } + @Override public void throwIfNoPrivilege() throws IOException { if (!isTableModelEvent()) { 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 327d24a1aea53..247956c610cd6 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 @@ -44,12 +44,6 @@ public class PipeRealtimeEvent extends EnrichedEvent { private Map device2Measurements; - private final String originClusterId; - - public String getOriginClusterId() { - return originClusterId; - } - public PipeRealtimeEvent( final EnrichedEvent event, final TsFileEpoch tsFileEpoch, @@ -64,27 +58,7 @@ public PipeRealtimeEvent( null, true, Long.MIN_VALUE, - Long.MAX_VALUE, - null); - } - - public PipeRealtimeEvent( - final EnrichedEvent event, - final TsFileEpoch tsFileEpoch, - final Map device2Measurements, - final String originClusterId) { - this( - event, - tsFileEpoch, - device2Measurements, - null, - null, - null, - null, - true, - Long.MIN_VALUE, - Long.MAX_VALUE, - originClusterId); + Long.MAX_VALUE); } public PipeRealtimeEvent( @@ -97,8 +71,7 @@ public PipeRealtimeEvent( final String userName, final boolean skipIfNoPrivileges, final long startTime, - final long endTime, - final String originClusterId) { + final long endTime) { // PipeTaskMeta is used to report the progress of the event, the PipeRealtimeEvent // is only used in the realtime event extractor, which does not need to report the progress // of the event, so the pipeTaskMeta is always null. @@ -116,7 +89,6 @@ public PipeRealtimeEvent( this.event = event; this.tsFileEpoch = tsFileEpoch; this.device2Measurements = device2Measurements; - this.originClusterId = originClusterId; } public EnrichedEvent getEvent() { @@ -256,8 +228,7 @@ public PipeRealtimeEvent shallowCopySelfAndBindPipeTaskMetaForProgressReport( userName, skipIfNoPrivileges, startTime, - endTime, - null); + endTime); } @Override @@ -265,6 +236,11 @@ public boolean isGeneratedByPipe() { return event.isGeneratedByPipe(); } + @Override + public String getOriginClusterId() { + return event.getOriginClusterId(); + } + @Override public String toString() { return "PipeRealtimeEvent{" 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 b2ec6356f4a2a..d860e62442701 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 @@ -96,7 +96,8 @@ public static PipeRealtimeEvent createRealtimeEvent( : null, insertNode.getProgressIndex(), insertNode.isAligned(), - insertNode.isGeneratedByPipe()); + insertNode.isGeneratedByPipe(), + insertNode.getOriginClusterId()); // if using IoTV2, assign a replicateIndex for this event if (DataRegionConsensusImpl.getInstance() instanceof PipeConsensus @@ -117,7 +118,7 @@ public static PipeRealtimeEvent createRealtimeEvent( public static PipeRealtimeEvent createRealtimeEvent( final String dataRegionId, final boolean shouldPrintMessage) { return new PipeRealtimeEvent( - new PipeHeartbeatEvent(dataRegionId, shouldPrintMessage), null, null, null); + new PipeHeartbeatEvent(dataRegionId, shouldPrintMessage), null, null); } public static PipeRealtimeEvent createRealtimeEvent( @@ -137,11 +138,11 @@ public static PipeRealtimeEvent createRealtimeEvent( deleteDataNodeEvent); } - return new PipeRealtimeEvent(deleteDataNodeEvent, null, null, null); + return new PipeRealtimeEvent(deleteDataNodeEvent, null, null); } public static PipeRealtimeEvent createRealtimeEvent(final ProgressReportEvent event) { - return new PipeRealtimeEvent(event, null, null, null); + return new PipeRealtimeEvent(event, null, null); } private PipeRealtimeEventFactory() { 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 b8e5bec509a48..f3ea94341f6b7 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 @@ -58,6 +58,7 @@ import java.util.Arrays; import java.util.Objects; +import java.util.Set; import java.util.concurrent.atomic.AtomicReference; import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.EXTRACTOR_DATABASE_KEY; @@ -697,6 +698,14 @@ public void close() throws Exception { } } + public Set getSinkClusterIds() { + return realtimeExtractor.getSinkClusterIds(); + } + + public void setSinkClusterIds(Set sinkClusterIds) { + realtimeExtractor.setSinkClusterIds(sinkClusterIds); + } + //////////////////////////// APIs provided for detecting stuck //////////////////////////// public boolean shouldExtractInsertion() { 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 a39ef37b9a36c..db4f48618909a 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 @@ -113,6 +113,8 @@ public abstract class PipeRealtimeDataRegionExtractor implements PipeExtractor { protected boolean isForwardingPipeRequests; + protected Set sinkClusterIds; + protected boolean isDoubleLiving; private boolean shouldTransferModFile; // Whether to transfer mods @@ -250,12 +252,12 @@ public void customize( PipeExtractorConstant.SOURCE_MODE_DOUBLE_LIVING_KEY), PipeExtractorConstant.EXTRACTOR_MODE_DOUBLE_LIVING_DEFAULT_VALUE); - isForwardingPipeRequests = - parameters.getBooleanOrDefault( - Arrays.asList( - PipeExtractorConstant.EXTRACTOR_FORWARDING_PIPE_REQUESTS_KEY, - PipeExtractorConstant.SOURCE_FORWARDING_PIPE_REQUESTS_KEY), - PipeExtractorConstant.EXTRACTOR_FORWARDING_PIPE_REQUESTS_DEFAULT_VALUE); + isForwardingPipeRequests = + parameters.getBooleanOrDefault( + Arrays.asList( + PipeExtractorConstant.EXTRACTOR_FORWARDING_PIPE_REQUESTS_KEY, + PipeExtractorConstant.SOURCE_FORWARDING_PIPE_REQUESTS_KEY), + PipeExtractorConstant.EXTRACTOR_FORWARDING_PIPE_REQUESTS_DEFAULT_VALUE); if (parameters.hasAnyAttributes(EXTRACTOR_MODS_KEY, SOURCE_MODS_KEY)) { shouldTransferModFile = @@ -538,10 +540,17 @@ public final boolean isForwardingPipeRequests() { return isForwardingPipeRequests; } - public final boolean isDoubleLiving(){ + public final boolean isDoubleLiving() { return isDoubleLiving; } + public Set getSinkClusterIds() { + return sinkClusterIds; + } + + public void setSinkClusterIds(Set sinkClusterIds) { + this.sinkClusterIds = sinkClusterIds; + } public abstract boolean isNeedListenToTsFile(); @@ -571,7 +580,8 @@ public String toString() { .add("startTimePartitionIdLowerBound", startTimePartitionIdLowerBound) .add("endTimePartitionIdUpperBound", endTimePartitionIdUpperBound) .add("dataRegionTimePartitionIdBound", dataRegionTimePartitionIdBound) - .add("isForwardingPipeRequests", isForwardingPipeRequests).add("isDoubleLiving", isDoubleLiving) + .add("isForwardingPipeRequests", isForwardingPipeRequests) + .add("isDoubleLiving", isDoubleLiving) .add("shouldTransferModFile", shouldTransferModFile) .add("sloppyTimeRange", sloppyTimeRange) .add("sloppyPattern", sloppyPattern) 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 e08456c718241..1521ffb2bf954 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 @@ -146,7 +146,10 @@ private void assignToExtractor( return; } - if ((event.getEvent().isGeneratedByPipe() && !extractor.isForwardingPipeRequests())) { + if ((event.getEvent().isGeneratedByPipe() && !extractor.isForwardingPipeRequests()) + || (extractor.isDoubleLiving() + && Objects.nonNull(event.getOriginClusterId()) + && extractor.getSinkClusterIds().contains(event.getOriginClusterId()))) { // The frequency of progress reports is limited by the counter, while progress // reports to TsFileInsertionEvent are not limited. if (!(event.getEvent() instanceof TsFileInsertionEvent)) { 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 4f95758c6c380..869f9244e87cf 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 @@ -70,8 +70,7 @@ public PipeRealtimeEvent bindPipeTsFileInsertionEvent( event, epoch, resource.getDevices().stream() - .collect(Collectors.toMap(Functions.identity(), device -> EMPTY_MEASUREMENT_ARRAY)), - resource.getOriginClusterId()); + .collect(Collectors.toMap(Functions.identity(), device -> EMPTY_MEASUREMENT_ARRAY))); } public PipeRealtimeEvent bindPipeInsertNodeTabletInsertionEvent( @@ -84,8 +83,7 @@ public PipeRealtimeEvent bindPipeInsertNodeTabletInsertionEvent( epoch, node instanceof InsertRowsNode ? getDevice2MeasurementsMapFromInsertRowsNode((InsertRowsNode) node) - : Collections.singletonMap(node.getDeviceID(), node.getMeasurements()), - node.getOriginClusterId()); + : Collections.singletonMap(node.getDeviceID(), node.getMeasurements())); } private Map getDevice2MeasurementsMapFromInsertRowsNode( 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 752d88635e26b..ec33d92595b88 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 @@ -26,15 +26,12 @@ 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.assigner.PipeDataRegionAssigner; -import org.apache.iotdb.db.pipe.extractor.schemaregion.IoTDBSchemaRegionExtractor; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.AbstractDeleteDataNode; 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.Collections; import java.util.Objects; -import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicInteger; @@ -53,8 +50,6 @@ public class PipeInsertionDataNodeListener { private final ConcurrentMap dataRegionId2Assigner = new ConcurrentHashMap<>(); - private final Set ingoreCulsterId = - Collections.newSetFromMap(new ConcurrentHashMap<>()); private final AtomicInteger listenToTsFileExtractorCount = new AtomicInteger(0); private final AtomicInteger listenToInsertNodeExtractorCount = new AtomicInteger(0); 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 7f5b8df681117..8770e992f4205 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 @@ -203,7 +203,8 @@ protected Optional trimRealtimeEventByPrivilege( ((PipeSchemaRegionWritePlanEvent) event).getPlanNode(), userName); if (result.isPresent()) { return Optional.of( - new PipeSchemaRegionWritePlanEvent(result.get(), event.isGeneratedByPipe())); + new PipeSchemaRegionWritePlanEvent( + result.get(), event.isGeneratedByPipe(), event.getOriginClusterId())); } if (skipIfNoPrivileges) { return Optional.empty(); @@ -225,7 +226,7 @@ protected Optional trimRealtimeEventByPipePattern( .map( planNode1 -> new PipeSchemaRegionWritePlanEvent( - planNode1, event.isGeneratedByPipe()))); + planNode1, event.isGeneratedByPipe(), event.getOriginClusterId()))); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/schemaregion/SchemaRegionListeningQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/schemaregion/SchemaRegionListeningQueue.java index 9cc752dcecfd4..130ab3af8cc04 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/schemaregion/SchemaRegionListeningQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/schemaregion/SchemaRegionListeningQueue.java @@ -54,12 +54,16 @@ public synchronized void tryListenToNode(final PlanNode node) { case PIPE_ENRICHED_WRITE: event = new PipeSchemaRegionWritePlanEvent( - ((PipeEnrichedWritePlanNode) node).getWritePlanNode(), true); + ((PipeEnrichedWritePlanNode) node).getWritePlanNode(), + true, + node.getOriginClusterId()); break; case PIPE_ENRICHED_NON_WRITE: event = new PipeSchemaRegionWritePlanEvent( - ((PipeEnrichedNonWritePlanNode) node).getNonWritePlanNode(), true); + ((PipeEnrichedNonWritePlanNode) node).getNonWritePlanNode(), + true, + node.getOriginClusterId()); break; default: event = new PipeSchemaRegionWritePlanEvent(node, false); 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 bf61712b71894..86df070fad298 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 @@ -3142,7 +3142,8 @@ private boolean loadTsFileToUnSequence( // Listen before the tsFile is added into tsFile manager to avoid it being compacted PipeInsertionDataNodeListener.getInstance() - .listenToTsFile(dataRegionId, databaseName, tsFileResource, true, isGeneratedByPipe,originClusterId); + .listenToTsFile( + dataRegionId, databaseName, tsFileResource, true, isGeneratedByPipe, originClusterId); 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 f24981d7ea73e..d430e52466348 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 @@ -1292,7 +1292,7 @@ public Future asyncClose() { tsFileResource, false, tmpMemTable.isTotallyGeneratedByPipe(), - null); + null); // When invoke closing TsFile after insert data to memTable, we shouldn't flush until invoke // flushing memTable in System module. 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 f4ea7dc42c80e..0e7dec3494f0d 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 @@ -317,7 +317,7 @@ private Future write2DataRegion(int writeNum, String dataRegionId, int startN false), resource); PipeInsertionDataNodeListener.getInstance() - .listenToTsFile(dataRegionId, dataRegionId, resource, false, false,null); + .listenToTsFile(dataRegionId, dataRegionId, resource, false, false, null); } }); } 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 index 2dd637a35af01..61f91fea8a3a4 100644 --- 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 @@ -59,7 +59,7 @@ public MockedPipeRealtimeEvent( final EnrichedEvent event, final TsFileEpoch tsFileEpoch, final Map device2Measurements) { - super(event, tsFileEpoch, device2Measurements, null); + super(event, tsFileEpoch, device2Measurements); } @Override diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClient.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClient.java index 9915402136f18..d79c430474c32 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClient.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClient.java @@ -52,16 +52,6 @@ public class IoTDBSyncClient extends IClientRPCService.Client private final int port; private final TEndPoint endPoint; - public String getClusterId() { - return clusterId; - } - - public void setClusterId(String clusterId) { - this.clusterId = clusterId; - } - - private String clusterId; - public IoTDBSyncClient( ThriftClientProperty property, String ipAddress, diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClientManager.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClientManager.java index 9b39267842eab..680a04e231706 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClientManager.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/client/IoTDBSyncClientManager.java @@ -36,9 +36,12 @@ import java.io.Closeable; import java.io.IOException; +import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import static org.apache.iotdb.commons.pipe.config.constant.PipeConnectorConstant.CONNECTOR_LOAD_BALANCE_PRIORITY_STRATEGY; @@ -61,6 +64,8 @@ public abstract class IoTDBSyncClientManager extends IoTDBClientManager implemen private final LoadBalancer loadBalancer; + private final Set endPointsClusterIds = Collections.synchronizedSet(new HashSet<>()); + protected IoTDBSyncClientManager( List endPoints, boolean useSSL, @@ -250,13 +255,12 @@ public void sendHandshakeReq(final Pair clientAndStatu clientAndStatus.setRight(true); client.setTimeout(CONNECTION_TIMEOUT_MS.get()); if (resp.isSetClusterId()) { - client.setClusterId(resp.getClusterId()); + endPointsClusterIds.add(resp.getClusterId()); } LOGGER.info( - "Handshake success. Target server ip: {}, port: {}, clusterId:{}", + "Handshake success. Target server ip: {}, port: {}", client.getIpAddress(), - client.getPort(), - client.getClusterId()); + client.getPort()); } } catch (Exception e) { LOGGER.warn( @@ -276,6 +280,10 @@ protected abstract PipeTransferHandshakeV2Req buildHandshakeV2Req(Map getEndPointsClusterIds() { + return endPointsClusterIds; + } + public Pair getClient() { return loadBalancer.getClient(); } 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 563364e084fac..f79a0e6298a76 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 @@ -406,6 +406,8 @@ public PipeTaskMeta getPipeTaskMeta() { public abstract boolean isGeneratedByPipe(); + public abstract String getOriginClusterId(); + /** Whether the {@link EnrichedEvent} need to be committed in order. */ public boolean needToCommit() { return true; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/PipeSnapshotEvent.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/PipeSnapshotEvent.java index ae6aefb3c103e..085f2395acbfe 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/PipeSnapshotEvent.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/PipeSnapshotEvent.java @@ -71,6 +71,11 @@ public boolean isGeneratedByPipe() { return false; } + @Override + public String getOriginClusterId() { + return null; + } + @Override public boolean mayEventTimeOverlappedWithTimeRange() { return true; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/PipeWritePlanEvent.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/PipeWritePlanEvent.java index f2000836c751e..e85492305403d 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/PipeWritePlanEvent.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/PipeWritePlanEvent.java @@ -28,6 +28,8 @@ public abstract class PipeWritePlanEvent extends EnrichedEvent implements Serial protected boolean isGeneratedByPipe; + protected String originClusterId; + protected ProgressIndex progressIndex; protected PipeWritePlanEvent( @@ -52,6 +54,30 @@ protected PipeWritePlanEvent( this.isGeneratedByPipe = isGeneratedByPipe; } + protected PipeWritePlanEvent( + final String pipeName, + final long creationTime, + final PipeTaskMeta pipeTaskMeta, + final TreePattern treePattern, + final TablePattern tablePattern, + final String userName, + final boolean skipIfNoPrivileges, + final boolean isGeneratedByPipe, + final String originClusterId) { + super( + pipeName, + creationTime, + pipeTaskMeta, + treePattern, + tablePattern, + userName, + skipIfNoPrivileges, + Long.MIN_VALUE, + Long.MAX_VALUE); + this.isGeneratedByPipe = isGeneratedByPipe; + this.originClusterId = originClusterId; + } + /** {@link PipeWritePlanEvent} does not share resources with other events. */ @Override public boolean internallyIncreaseResourceReferenceCount(final String holderMessage) { @@ -79,6 +105,11 @@ public boolean isGeneratedByPipe() { return isGeneratedByPipe; } + @Override + public String getOriginClusterId() { + return originClusterId; + } + @Override public boolean mayEventTimeOverlappedWithTimeRange() { return true; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/ProgressReportEvent.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/ProgressReportEvent.java index 536c22ae98917..980110a309b65 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/ProgressReportEvent.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/ProgressReportEvent.java @@ -103,6 +103,11 @@ public boolean isGeneratedByPipe() { return false; } + @Override + public String getOriginClusterId() { + return null; + } + @Override public boolean mayEventTimeOverlappedWithTimeRange() { return true; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/extractor/IoTDBExtractor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/extractor/IoTDBExtractor.java index d348d16448878..30cd5305874e9 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/extractor/IoTDBExtractor.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/extractor/IoTDBExtractor.java @@ -155,13 +155,12 @@ public void customize( PipeExtractorConstant.SOURCE_MODE_DOUBLE_LIVING_KEY), PipeExtractorConstant.EXTRACTOR_MODE_DOUBLE_LIVING_DEFAULT_VALUE); - isForwardingPipeRequests = - parameters.getBooleanOrDefault( - Arrays.asList( - PipeExtractorConstant.EXTRACTOR_FORWARDING_PIPE_REQUESTS_KEY, - PipeExtractorConstant.SOURCE_FORWARDING_PIPE_REQUESTS_KEY), - PipeExtractorConstant.EXTRACTOR_FORWARDING_PIPE_REQUESTS_DEFAULT_VALUE); - + isForwardingPipeRequests = + parameters.getBooleanOrDefault( + Arrays.asList( + PipeExtractorConstant.EXTRACTOR_FORWARDING_PIPE_REQUESTS_KEY, + PipeExtractorConstant.SOURCE_FORWARDING_PIPE_REQUESTS_KEY), + PipeExtractorConstant.EXTRACTOR_FORWARDING_PIPE_REQUESTS_DEFAULT_VALUE); userName = parameters.getStringByKeys( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/extractor/IoTDBNonDataRegionExtractor.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/extractor/IoTDBNonDataRegionExtractor.java index 9ff381efb8221..22801607f79bb 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/extractor/IoTDBNonDataRegionExtractor.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/extractor/IoTDBNonDataRegionExtractor.java @@ -47,6 +47,7 @@ import java.util.List; import java.util.Objects; import java.util.Optional; +import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; @TreeModel @@ -70,6 +71,8 @@ public abstract class IoTDBNonDataRegionExtractor extends IoTDBExtractor { protected abstract AbstractPipeListeningQueue getListeningQueue(); + protected Set sinkClusterIds; + @Override public void customize( final PipeParameters parameters, final PipeExtractorRuntimeConfiguration configuration) @@ -224,7 +227,11 @@ public EnrichedEvent supply() throws Exception { if (Objects.isNull(realtimeEvent) || !isTypeListened(realtimeEvent) - || (!isForwardingPipeRequests && realtimeEvent.isGeneratedByPipe())) { + || (!isForwardingPipeRequests && realtimeEvent.isGeneratedByPipe()) + || (isDoubleLiving + && Objects.nonNull(realtimeEvent.getOriginClusterId()) + && Objects.nonNull(sinkClusterIds) + && sinkClusterIds.contains(realtimeEvent.getOriginClusterId()))) { final ProgressReportEvent event = new ProgressReportEvent( pipeName, @@ -284,6 +291,14 @@ protected abstract Optional trimRealtimeEventByPipePattern( protected abstract void confineHistoricalEventTransferTypes(final PipeSnapshotEvent event); + public Set getSinkClusterIds() { + return sinkClusterIds; + } + + public void setSinkClusterIds(Set sinkClusterIds) { + this.sinkClusterIds = sinkClusterIds; + } + @Override public void close() throws Exception { getListeningQueue().returnIterator(iterator); From 690caa0b2f9ea96c3d1101ad5034331152bd810b Mon Sep 17 00:00:00 2001 From: XNX02 <202100130044@mail.sdu.edu.cn> Date: Tue, 11 Mar 2025 01:09:12 +0800 Subject: [PATCH 04/27] schema --- .../tsfile/PipeTsFileInsertionEvent.java | 43 ++++++++++++++++++- .../realtime/PipeRealtimeEventFactory.java | 5 ++- ...lDataRegionTsFileAndDeletionExtractor.java | 1 + .../PipeInsertionDataNodeListener.java | 8 +++- .../dataregion/tsfile/TsFileResource.java | 16 ------- .../event/TsFileInsertionEventParserTest.java | 1 + 6 files changed, 52 insertions(+), 22 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 53115977ab1b4..f0acfd70d2be9 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 @@ -101,7 +101,8 @@ public PipeTsFileInsertionEvent( final String databaseNameFromDataRegion, final TsFileResource resource, final boolean isLoaded, - final boolean isGeneratedByHistoricalExtractor) { + final boolean isGeneratedByHistoricalExtractor, + final String originClusterId) { // The modFile must be copied before the event is assigned to the listening pipes this( isTableModelEvent, @@ -110,6 +111,7 @@ public PipeTsFileInsertionEvent( true, isLoaded, isGeneratedByHistoricalExtractor, + originClusterId, null, 0, null, @@ -137,6 +139,42 @@ public PipeTsFileInsertionEvent( final boolean skipIfNoPrivileges, final long startTime, final long endTime) { + this( + isTableModelEvent, + databaseNameFromDataRegion, + resource, + true, + isLoaded, + isGeneratedByHistoricalExtractor, + null, + pipeName, + creationTime, + pipeTaskMeta, + treePattern, + tablePattern, + userName, + skipIfNoPrivileges, + startTime, + endTime); + } + + public PipeTsFileInsertionEvent( + final Boolean isTableModelEvent, + final String databaseNameFromDataRegion, + final TsFileResource resource, + final boolean isWithMod, + final boolean isLoaded, + final boolean isGeneratedByHistoricalExtractor, + final String originClusterId, + final String pipeName, + final long creationTime, + final PipeTaskMeta pipeTaskMeta, + final TreePattern treePattern, + final TablePattern tablePattern, + final String userName, + final boolean skipIfNoPrivileges, + final long startTime, + final long endTime) { super( pipeName, creationTime, @@ -164,7 +202,7 @@ public PipeTsFileInsertionEvent( this.isGeneratedByPipeConsensus = resource.isGeneratedByPipeConsensus(); this.isGeneratedByHistoricalExtractor = isGeneratedByHistoricalExtractor; - this.originClusterId = resource.getOriginClusterId(); + this.originClusterId = originClusterId; isClosed = new AtomicBoolean(resource.isClosed()); // Register close listener if TsFile is not closed @@ -416,6 +454,7 @@ public PipeTsFileInsertionEvent shallowCopySelfAndBindPipeTaskMetaForProgressRep isWithMod, isLoaded, isGeneratedByHistoricalExtractor, + null, pipeName, creationTime, pipeTaskMeta, 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 d860e62442701..37361991a8b5b 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 @@ -51,10 +51,11 @@ public static PipeRealtimeEvent createRealtimeEvent( final Boolean isTableModel, final String databaseNameFromDataRegion, final TsFileResource resource, - final boolean isLoaded) { + final boolean isLoaded, + final String originClusterId) { PipeTsFileInsertionEvent tsFileInsertionEvent = new PipeTsFileInsertionEvent( - isTableModel, databaseNameFromDataRegion, resource, isLoaded, false); + isTableModel, databaseNameFromDataRegion, resource, isLoaded, false, originClusterId); // if using IoTV2, assign a replicateIndex for this event if (DataRegionConsensusImpl.getInstance() instanceof PipeConsensus diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java index 381db6a25cfe2..9236807cc8919 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java @@ -804,6 +804,7 @@ private Event supplyTsFileEvent(final TsFileResource resource) { shouldTransferModFile, false, true, + null, pipeName, creationTime, pipeTaskMeta, 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 ec33d92595b88..4aa55c027b3df 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 @@ -104,7 +104,6 @@ public void listenToTsFile( final boolean isGeneratedByPipe, final String originClusterId) { tsFileResource.setGeneratedByPipe(isGeneratedByPipe); - tsFileResource.setOriginClusterId(originClusterId); // 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. @@ -118,7 +117,12 @@ public void listenToTsFile( assigner.publishToAssign( PipeRealtimeEventFactory.createRealtimeEvent( - dataRegionId, assigner.isTableModel(), databaseName, tsFileResource, isLoaded)); + dataRegionId, + assigner.isTableModel(), + databaseName, + tsFileResource, + isLoaded, + originClusterId)); } public void listenToInsertNode( 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 a572d638feab8..204fe1f32d438 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 @@ -202,16 +202,6 @@ public class TsFileResource implements PersistentResource { /** used to prevent circular replication in Pipe */ private boolean isGeneratedByPipe = false; - public String getOriginClusterId() { - return originClusterId; - } - - public void setOriginClusterId(String originClusterId) { - this.originClusterId = originClusterId; - } - - private String originClusterId; - private InsertionCompactionCandidateStatus insertionCompactionCandidateStatus = InsertionCompactionCandidateStatus.NOT_CHECKED; @@ -318,9 +308,6 @@ private void serializeTo(BufferedOutputStream outputStream) throws IOException { ReadWriteIOUtils.write(isGeneratedByPipeConsensus, outputStream); ReadWriteIOUtils.write(isGeneratedByPipe, outputStream); - if (originClusterId != null) { - ReadWriteIOUtils.write(originClusterId, outputStream); - } } /** deserialize from disk */ @@ -357,9 +344,6 @@ public void deserialize() throws IOException { isGeneratedByPipeConsensus = ReadWriteIOUtils.readBoolean(inputStream); isGeneratedByPipe = ReadWriteIOUtils.readBoolean(inputStream); } - if (inputStream.available() > 0) { - originClusterId = ReadWriteIOUtils.readString(inputStream); - } } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/TsFileInsertionEventParserTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/TsFileInsertionEventParserTest.java index 1318b466b79db..b6da5dd841543 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/TsFileInsertionEventParserTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/TsFileInsertionEventParserTest.java @@ -567,6 +567,7 @@ private void testTsFilePointNum( false, false, null, + null, 0, null, null, From 23029c9118fce0b2779977f4d09df3b206a8ca8d Mon Sep 17 00:00:00 2001 From: XNX02 <202100130044@mail.sdu.edu.cn> Date: Tue, 11 Mar 2025 15:03:12 +0800 Subject: [PATCH 05/27] config --- .../write/pipe/payload/PipeEnrichedPlan.java | 12 ++++++++++++ .../iotdb/confignode/manager/ConfigManager.java | 9 +++++---- .../confignode/manager/PermissionManager.java | 8 ++++++-- .../pipe/agent/task/PipeConfigNodeSubtask.java | 7 +++++++ .../protocol/IoTDBConfigRegionConnector.java | 5 +++++ .../receiver/protocol/IoTDBConfigNodeReceiver.java | 14 ++++++++++---- .../manager/schema/ClusterSchemaManager.java | 10 +++++++--- .../src/main/thrift/confignode.thrift | 6 ++++++ 8 files changed, 58 insertions(+), 13 deletions(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlan.java index e35c16ab7b1c8..185f07286effb 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlan.java @@ -31,6 +31,8 @@ public class PipeEnrichedPlan extends ConfigPhysicalPlan { private ConfigPhysicalPlan innerPlan; + private String originClusterIds; + public PipeEnrichedPlan() { super(ConfigPhysicalPlanType.PipeEnriched); } @@ -40,10 +42,20 @@ public PipeEnrichedPlan(ConfigPhysicalPlan innerPlan) { this.innerPlan = innerPlan; } + public PipeEnrichedPlan(ConfigPhysicalPlan innerPlan, String originClusterIds) { + super(ConfigPhysicalPlanType.PipeEnriched); + this.innerPlan = innerPlan; + this.originClusterIds = originClusterIds; + } + public ConfigPhysicalPlan getInnerPlan() { return innerPlan; } + public String getOriginClusterIds() { + return originClusterIds; + } + @Override protected void serializeImpl(DataOutputStream stream) throws IOException { stream.writeShort(getType().getPlanType()); 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 614bdafaefe8d..2fcc789e8f947 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 @@ -733,7 +733,7 @@ public DataSet showTTL(ShowTTLPlan showTTLPlan) { public TSStatus setDatabase(final DatabaseSchemaPlan databaseSchemaPlan) { final TSStatus status = confirmLeader(); if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - return clusterSchemaManager.setDatabase(databaseSchemaPlan, false); + return clusterSchemaManager.setDatabase(databaseSchemaPlan, false, null); } else { return status; } @@ -743,7 +743,7 @@ public TSStatus setDatabase(final DatabaseSchemaPlan databaseSchemaPlan) { public TSStatus alterDatabase(final DatabaseSchemaPlan databaseSchemaPlan) { final TSStatus status = confirmLeader(); if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - return clusterSchemaManager.alterDatabase(databaseSchemaPlan, false); + return clusterSchemaManager.alterDatabase(databaseSchemaPlan, false, null); } else { return status; } @@ -1255,7 +1255,7 @@ public SubscriptionManager getSubscriptionManager() { public TSStatus operatePermission(final AuthorPlan authorPlan) { final TSStatus status = confirmLeader(); if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - return permissionManager.operatePermission(authorPlan, false); + return permissionManager.operatePermission(authorPlan, false, null); } else { return status; } @@ -2159,7 +2159,8 @@ public TSStatus deleteTimeSeries(TDeleteTimeSeriesReq req) { status = clusterSchemaManager.setDatabase( new DatabaseSchemaPlan(ConfigPhysicalPlanType.CreateDatabase, databaseSchema), - isGeneratedByPipe); + isGeneratedByPipe, + null); if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { failedStatus.add(status); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/PermissionManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/PermissionManager.java index f17a63cfa8221..e2ec75745b02f 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/PermissionManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/PermissionManager.java @@ -59,7 +59,8 @@ public PermissionManager(ConfigManager configManager, AuthorInfo authorInfo) { * @param isGeneratedByPipe whether the plan is operated by pipe receiver * @return TSStatus */ - public TSStatus operatePermission(AuthorPlan authorPlan, boolean isGeneratedByPipe) { + public TSStatus operatePermission( + AuthorPlan authorPlan, boolean isGeneratedByPipe, String originClusterIds) { TSStatus tsStatus; // If the permissions change, clear the cache content affected by the operation LOGGER.info("Auth: run auth plan: {}", authorPlan.toString()); @@ -69,7 +70,10 @@ public TSStatus operatePermission(AuthorPlan authorPlan, boolean isGeneratedByPi || authorPlan.getAuthorType() == ConfigPhysicalPlanType.CreateUserWithRawPassword) { tsStatus = getConsensusManager() - .write(isGeneratedByPipe ? new PipeEnrichedPlan(authorPlan) : authorPlan); + .write( + isGeneratedByPipe + ? new PipeEnrichedPlan(authorPlan, originClusterIds) + : authorPlan); } else { List allDataNodes = configManager.getNodeManager().getRegisteredDataNodes(); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/agent/task/PipeConfigNodeSubtask.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/agent/task/PipeConfigNodeSubtask.java index 62fc0b6e560d8..e79a1642e0514 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/agent/task/PipeConfigNodeSubtask.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/agent/task/PipeConfigNodeSubtask.java @@ -31,6 +31,7 @@ import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.event.ProgressReportEvent; import org.apache.iotdb.confignode.manager.pipe.agent.PipeConfigNodeAgent; +import org.apache.iotdb.confignode.manager.pipe.connector.protocol.IoTDBConfigRegionConnector; import org.apache.iotdb.confignode.manager.pipe.extractor.IoTDBConfigRegionExtractor; import org.apache.iotdb.confignode.manager.pipe.metric.sink.PipeConfigRegionConnectorMetrics; import org.apache.iotdb.pipe.api.PipeExtractor; @@ -147,6 +148,12 @@ private void initConnector(final Map connectorAttributes) throws // 4. Handshake outputPipeConnector.handshake(); + if (outputPipeConnector instanceof IoTDBConfigRegionConnector + && extractor instanceof IoTDBConfigRegionExtractor) { + ((IoTDBConfigRegionExtractor) extractor) + .setSinkClusterIds( + ((IoTDBConfigRegionConnector) outputPipeConnector).getSinkClusterIds()); + } } catch (final Exception e) { try { outputPipeConnector.close(); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/protocol/IoTDBConfigRegionConnector.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/protocol/IoTDBConfigRegionConnector.java index 4453cf4e06354..2ae4d46297025 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/protocol/IoTDBConfigRegionConnector.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/connector/protocol/IoTDBConfigRegionConnector.java @@ -53,6 +53,7 @@ import java.util.Collections; import java.util.List; import java.util.Objects; +import java.util.Set; @TreeModel @TableModel @@ -275,4 +276,8 @@ private void doTransfer(final PipeConfigRegionSnapshotEvent snapshotEvent) LOGGER.info("Successfully transferred config region snapshot {}.", snapshotFile); } + + public Set getSinkClusterIds() { + return clientManager.getEndPointsClusterIds(); + } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java index 4cbb001fb0940..266ea5daf91f4 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java @@ -553,11 +553,13 @@ private TSStatus executePlan(final ConfigPhysicalPlan plan) throws ConsensusExce ConfigNodeDescriptor.getInstance().getConf().getDefaultDataRegionGroupNumPerDatabase()); schema.setMaxSchemaRegionGroupNum(schema.getMinSchemaRegionGroupNum()); schema.setMaxDataRegionGroupNum(schema.getMinDataRegionGroupNum()); - return configManager.getClusterSchemaManager().setDatabase((DatabaseSchemaPlan) plan, true); + return configManager + .getClusterSchemaManager() + .setDatabase((DatabaseSchemaPlan) plan, true, clusterIdFromHandshakeRequest); case AlterDatabase: return configManager .getClusterSchemaManager() - .alterDatabase((DatabaseSchemaPlan) plan, true); + .alterDatabase((DatabaseSchemaPlan) plan, true, clusterIdFromHandshakeRequest); case DeleteDatabase: return configManager.deleteDatabases( new TDeleteDatabasesReq( @@ -754,10 +756,14 @@ private TSStatus executePlan(final ConfigPhysicalPlan plan) throws ConsensusExce case RRevokeUserSysPri: case RGrantUserRole: case RRevokeUserRole: - return configManager.getPermissionManager().operatePermission((AuthorPlan) plan, true); + return configManager + .getPermissionManager() + .operatePermission((AuthorPlan) plan, true, clusterIdFromHandshakeRequest); case CreateSchemaTemplate: default: - return configManager.getConsensusManager().write(new PipeEnrichedPlan(plan)); + return configManager + .getConsensusManager() + .write(new PipeEnrichedPlan(plan, clusterIdFromHandshakeRequest)); } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java index 65e9f2d82a479..1952b603fb38f 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java @@ -159,7 +159,9 @@ public ClusterSchemaManager( /** Set Database */ public TSStatus setDatabase( - final DatabaseSchemaPlan databaseSchemaPlan, final boolean isGeneratedByPipe) { + final DatabaseSchemaPlan databaseSchemaPlan, + final boolean isGeneratedByPipe, + final String originClusterIds) { TSStatus result; final TDatabaseSchema schema = databaseSchemaPlan.getSchema(); @@ -213,7 +215,9 @@ public TSStatus setDatabase( /** Alter Database */ public TSStatus alterDatabase( - final DatabaseSchemaPlan databaseSchemaPlan, final boolean isGeneratedByPipe) { + final DatabaseSchemaPlan databaseSchemaPlan, + final boolean isGeneratedByPipe, + final String originClusterId) { TSStatus result; final TDatabaseSchema databaseSchema = databaseSchemaPlan.getSchema(); @@ -260,7 +264,7 @@ public TSStatus alterDatabase( getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan(databaseSchemaPlan) + ? new PipeEnrichedPlan(databaseSchemaPlan, originClusterId) : databaseSchemaPlan); PartitionMetrics.bindDatabaseReplicationFactorMetricsWhenUpdate( MetricService.getInstance(), diff --git a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift index 1d03a9c9453c3..7d3fca8b83aac 100644 --- a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift +++ b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift @@ -167,6 +167,7 @@ struct TDeleteDatabasesReq { 1: required list prefixPathList 2: optional bool isGeneratedByPipe 3: optional bool isTableModel + 4: optional string originClusterId } struct TSetSchemaReplicationFactorReq { @@ -545,6 +546,7 @@ struct TCreateTriggerReq { struct TDropTriggerReq { 1: required string triggerName 2: optional bool isGeneratedByPipe + 3: optional string originClusterId } struct TGetLocationForTriggerResp { @@ -735,6 +737,7 @@ struct TSetSchemaTemplateReq { 2: required string name 3: required string path 4: optional bool isGeneratedByPipe + 5: optional string originClusterId } struct TGetPathsSetTemplatesReq { @@ -864,12 +867,14 @@ struct TDeleteTimeSeriesReq { 1: required string queryId 2: required binary pathPatternTree 3: optional bool isGeneratedByPipe + 4: optional string originClusterId } struct TDeleteLogicalViewReq { 1: required string queryId 2: required binary pathPatternTree 3: optional bool isGeneratedByPipe + 4: optional string originClusterId } struct TAlterLogicalViewReq { @@ -1005,6 +1010,7 @@ struct TUnsetSchemaTemplateReq { 2: required string templateName 3: required string path 4: optional bool isGeneratedByPipe + 5: optional String originClusterId } struct TCreateModelReq { From 8fdca088f7553953051278d842d951c04ef5e026 Mon Sep 17 00:00:00 2001 From: XNX02 <202100130044@mail.sdu.edu.cn> Date: Tue, 11 Mar 2025 21:36:09 +0800 Subject: [PATCH 06/27] config procudure --- .../write/pipe/payload/PipeEnrichedPlan.java | 26 ++++++-- .../confignode/manager/ConfigManager.java | 28 +++++---- .../confignode/manager/ProcedureManager.java | 60 +++++++++++++++---- .../iotdb/confignode/manager/TTLManager.java | 23 +++++-- .../confignode/manager/TriggerManager.java | 4 +- .../protocol/IoTDBConfigNodeReceiver.java | 41 +++++++++---- .../manager/schema/ClusterSchemaManager.java | 30 ++++++---- .../procedure/env/ConfigNodeProcedureEnv.java | 5 +- .../procedure/impl/StateMachineProcedure.java | 25 ++++++++ .../impl/node/AbstractNodeProcedure.java | 4 ++ .../schema/DeactivateTemplateProcedure.java | 10 ++++ .../impl/schema/DeleteDatabaseProcedure.java | 22 ++++++- .../schema/DeleteLogicalViewProcedure.java | 10 ++++ .../schema/DeleteTimeSeriesProcedure.java | 23 ++++++- .../impl/schema/SetTTLProcedure.java | 30 +++++++++- .../impl/schema/SetTemplateProcedure.java | 21 ++++++- .../impl/schema/UnsetTemplateProcedure.java | 27 ++++++++- .../AbstractAlterOrDropTableProcedure.java | 12 ++++ .../schema/table/AddTableColumnProcedure.java | 14 ++++- .../schema/table/CreateTableProcedure.java | 10 ++++ .../impl/trigger/DropTriggerProcedure.java | 6 ++ .../src/main/thrift/confignode.thrift | 3 +- .../src/main/thrift/datanode.thrift | 2 + 23 files changed, 359 insertions(+), 77 deletions(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlan.java index 185f07286effb..7d7c90e9eb8c7 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlan.java @@ -31,7 +31,7 @@ public class PipeEnrichedPlan extends ConfigPhysicalPlan { private ConfigPhysicalPlan innerPlan; - private String originClusterIds; + private String originClusterId; public PipeEnrichedPlan() { super(ConfigPhysicalPlanType.PipeEnriched); @@ -42,18 +42,18 @@ public PipeEnrichedPlan(ConfigPhysicalPlan innerPlan) { this.innerPlan = innerPlan; } - public PipeEnrichedPlan(ConfigPhysicalPlan innerPlan, String originClusterIds) { + public PipeEnrichedPlan(ConfigPhysicalPlan innerPlan, String originClusterId) { super(ConfigPhysicalPlanType.PipeEnriched); this.innerPlan = innerPlan; - this.originClusterIds = originClusterIds; + this.originClusterId = originClusterId; } public ConfigPhysicalPlan getInnerPlan() { return innerPlan; } - public String getOriginClusterIds() { - return originClusterIds; + public String getOriginClusterId() { + return originClusterId; } @Override @@ -61,11 +61,27 @@ protected void serializeImpl(DataOutputStream stream) throws IOException { stream.writeShort(getType().getPlanType()); ByteBuffer buffer = innerPlan.serializeToByteBuffer(); stream.write(buffer.array(), 0, buffer.limit()); + + if (originClusterId == null) { + stream.writeBoolean(false); + } else { + stream.writeBoolean(true); + stream.writeUTF(originClusterId); + } } @Override protected void deserializeImpl(ByteBuffer buffer) throws IOException { innerPlan = ConfigPhysicalPlan.Factory.create(buffer); + + if (buffer.hasRemaining() && buffer.get() == 1) { // Read boolean + int strLength = buffer.getShort(); + byte[] bytes = new byte[strLength]; + buffer.get(bytes); + originClusterId = new String(bytes); + } else { + originClusterId = null; + } } @Override 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 2fcc789e8f947..104bcb95bdadd 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 @@ -651,9 +651,9 @@ public TSStatus setTTL(SetTTLPlan setTTLPlan) { TSStatus status = confirmLeader(); if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { if (setTTLPlan.getTTL() == TTLCache.NULL_TTL) { - return ttlManager.unsetTTL(setTTLPlan, false); + return ttlManager.unsetTTL(setTTLPlan, false, null); } else { - return ttlManager.setTTL(setTTLPlan, false); + return ttlManager.setTTL(setTTLPlan, false, null); } } else { return status; @@ -768,7 +768,8 @@ public synchronized TSStatus deleteDatabases(final TDeleteDatabasesReq tDeleteRe return procedureManager.deleteDatabases( new ArrayList<>(deleteDatabaseSchemaMap.values()), - tDeleteReq.isSetIsGeneratedByPipe() && tDeleteReq.isIsGeneratedByPipe()); + tDeleteReq.isSetIsGeneratedByPipe() && tDeleteReq.isIsGeneratedByPipe(), + tDeleteReq.isSetOriginClusterId() ? tDeleteReq.getOriginClusterId() : null); } else { return status; } @@ -1970,7 +1971,8 @@ public synchronized TSStatus setSchemaTemplate(TSetSchemaTemplateReq req) { req.getQueryId(), req.getName(), req.getPath(), - req.isSetIsGeneratedByPipe() && req.isIsGeneratedByPipe()); + req.isSetIsGeneratedByPipe() && req.isIsGeneratedByPipe(), + req.getOriginClusterId()); } else { return status; } @@ -2041,7 +2043,8 @@ public TSStatus deactivateSchemaTemplate(final TDeactivateSchemaTemplateReq req) return procedureManager.deactivateTemplate( req.getQueryId(), templateSetInfo, - req.isSetIsGeneratedByPipe() && req.isIsGeneratedByPipe()); + req.isSetIsGeneratedByPipe() && req.isIsGeneratedByPipe(), + req.getOriginClusterId()); } @Override @@ -2058,7 +2061,8 @@ public synchronized TSStatus unsetSchemaTemplate(TUnsetSchemaTemplateReq req) { req.getQueryId(), checkResult.right, new PartialPath(req.getPath()), - req.isSetIsGeneratedByPipe() && req.isIsGeneratedByPipe()); + req.isSetIsGeneratedByPipe() && req.isIsGeneratedByPipe(), + req.getOriginClusterId()); } catch (IllegalPathException e) { return RpcUtils.getStatus(e.getErrorCode(), e.getMessage()); } @@ -2086,7 +2090,7 @@ public TSStatus alterSchemaTemplate(TAlterSchemaTemplateReq req) { TemplateAlterOperationUtil.parseOperationType(buffer); if (operationType.equals(TemplateAlterOperationType.EXTEND_TEMPLATE)) { return clusterSchemaManager.extendSchemaTemplate( - TemplateAlterOperationUtil.parseTemplateExtendInfo(buffer), false); + TemplateAlterOperationUtil.parseTemplateExtendInfo(buffer), false, null); } return RpcUtils.getStatus(TSStatusCode.UNSUPPORTED_OPERATION); } else { @@ -2102,6 +2106,7 @@ public TSStatus deleteTimeSeries(TDeleteTimeSeriesReq req) { PathPatternTree rawPatternTree = PathPatternTree.deserialize(ByteBuffer.wrap(req.getPathPatternTree())); boolean isGeneratedByPipe = req.isSetIsGeneratedByPipe() && req.isIsGeneratedByPipe(); + String originClusterId = req.getOriginClusterId(); /** * If delete pattern is prefix path (such as root.db.**), it may be optimized to delete * database plus create database. We need to determine two conditions: whether the pattern @@ -2126,7 +2131,8 @@ public TSStatus deleteTimeSeries(TDeleteTimeSeriesReq req) { deleteTimeSeriesPatternPaths.add(path); } if (!canOptimize) { - return procedureManager.deleteTimeSeries(queryId, rawPatternTree, isGeneratedByPipe); + return procedureManager.deleteTimeSeries( + queryId, rawPatternTree, isGeneratedByPipe, originClusterId); } // check if the database is using template try { @@ -2143,14 +2149,14 @@ public TSStatus deleteTimeSeries(TDeleteTimeSeriesReq req) { deleteTimeSeriesPatternTree.constructTree(); status = procedureManager.deleteTimeSeries( - queryId, deleteTimeSeriesPatternTree, isGeneratedByPipe); + queryId, deleteTimeSeriesPatternTree, isGeneratedByPipe, originClusterId); } if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { // 2. delete database List failedStatus = new ArrayList<>(); status = procedureManager.deleteDatabases( - new ArrayList<>(deleteDatabaseSchemas), isGeneratedByPipe); + new ArrayList<>(deleteDatabaseSchemas), isGeneratedByPipe, originClusterId); if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { failedStatus.add(status); } @@ -2160,7 +2166,7 @@ public TSStatus deleteTimeSeries(TDeleteTimeSeriesReq req) { clusterSchemaManager.setDatabase( new DatabaseSchemaPlan(ConfigPhysicalPlanType.CreateDatabase, databaseSchema), isGeneratedByPipe, - null); + originClusterId); if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { failedStatus.add(status); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java index 002fcf859ffcf..470ce7795a0f9 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java @@ -246,7 +246,9 @@ public TSStatus testSubProcedure() { } public TSStatus deleteDatabases( - final List deleteSgSchemaList, final boolean isGeneratedByPipe) { + final List deleteSgSchemaList, + final boolean isGeneratedByPipe, + final String originClusterId) { final List procedures = new ArrayList<>(); final long startCheckTimeForProcedures = System.currentTimeMillis(); for (final TDatabaseSchema databaseSchema : deleteSgSchemaList) { @@ -262,7 +264,7 @@ public TSStatus deleteDatabases( if (Boolean.FALSE.equals(procedureIdDuplicatePair.getRight())) { DeleteDatabaseProcedure procedure = - new DeleteDatabaseProcedure(databaseSchema, isGeneratedByPipe); + new DeleteDatabaseProcedure(databaseSchema, isGeneratedByPipe, originClusterId); this.executor.submitProcedure(procedure); procedures.add(procedure); break; @@ -296,7 +298,10 @@ public TSStatus deleteDatabases( } public TSStatus deleteTimeSeries( - String queryId, PathPatternTree patternTree, boolean isGeneratedByPipe) { + String queryId, + PathPatternTree patternTree, + boolean isGeneratedByPipe, + String originClusterId) { DeleteTimeSeriesProcedure procedure = null; synchronized (this) { boolean hasOverlappedTask = false; @@ -324,7 +329,8 @@ public TSStatus deleteTimeSeries( TSStatusCode.OVERLAP_WITH_EXISTING_TASK, "Some other task is deleting some target timeseries."); } - procedure = new DeleteTimeSeriesProcedure(queryId, patternTree, isGeneratedByPipe); + procedure = + new DeleteTimeSeriesProcedure(queryId, patternTree, isGeneratedByPipe, originClusterId); this.executor.submitProcedure(procedure); } } @@ -364,7 +370,10 @@ public TSStatus deleteLogicalView(TDeleteLogicalViewReq req) { } procedure = new DeleteLogicalViewProcedure( - queryId, patternTree, req.isSetIsGeneratedByPipe() && req.isIsGeneratedByPipe()); + queryId, + patternTree, + req.isSetIsGeneratedByPipe() && req.isIsGeneratedByPipe(), + req.getOriginClusterId()); this.executor.submitProcedure(procedure); } } @@ -413,7 +422,11 @@ public TSStatus alterLogicalView(final TAlterLogicalViewReq req) { } public TSStatus setSchemaTemplate( - String queryId, String templateName, String templateSetPath, boolean isGeneratedByPipe) { + String queryId, + String templateName, + String templateSetPath, + boolean isGeneratedByPipe, + String originClusterId) { SetTemplateProcedure procedure = null; synchronized (this) { boolean hasOverlappedTask = false; @@ -442,7 +455,8 @@ public TSStatus setSchemaTemplate( "Some other task is setting template on target path."); } procedure = - new SetTemplateProcedure(queryId, templateName, templateSetPath, isGeneratedByPipe); + new SetTemplateProcedure( + queryId, templateName, templateSetPath, isGeneratedByPipe, originClusterId); this.executor.submitProcedure(procedure); } } @@ -450,7 +464,10 @@ public TSStatus setSchemaTemplate( } public TSStatus deactivateTemplate( - String queryId, Map> templateSetInfo, boolean isGeneratedByPipe) { + String queryId, + Map> templateSetInfo, + boolean isGeneratedByPipe, + String originClusterId) { DeactivateTemplateProcedure procedure = null; synchronized (this) { boolean hasOverlappedTask = false; @@ -489,7 +506,9 @@ public TSStatus deactivateTemplate( TSStatusCode.OVERLAP_WITH_EXISTING_TASK, "Some other task is deactivating some target template from target path."); } - procedure = new DeactivateTemplateProcedure(queryId, templateSetInfo, isGeneratedByPipe); + procedure = + new DeactivateTemplateProcedure( + queryId, templateSetInfo, isGeneratedByPipe, originClusterId); this.executor.submitProcedure(procedure); } } @@ -497,7 +516,11 @@ public TSStatus deactivateTemplate( } public TSStatus unsetSchemaTemplate( - String queryId, Template template, PartialPath path, boolean isGeneratedByPipe) { + String queryId, + Template template, + PartialPath path, + boolean isGeneratedByPipe, + String originClusterId) { UnsetTemplateProcedure procedure = null; synchronized (this) { boolean hasOverlappedTask = false; @@ -527,7 +550,8 @@ public TSStatus unsetSchemaTemplate( "Some other task is unsetting target template from target path " + path.getFullPath()); } - procedure = new UnsetTemplateProcedure(queryId, template, path, isGeneratedByPipe); + procedure = + new UnsetTemplateProcedure(queryId, template, path, isGeneratedByPipe, originClusterId); this.executor.submitProcedure(procedure); } } @@ -1259,8 +1283,10 @@ && new UpdateProcedurePlan(createTriggerProcedure).getSerializedSize() > planSiz * @return {@link TSStatusCode#SUCCESS_STATUS} if the trigger has been dropped successfully, * {@link TSStatusCode#DROP_TRIGGER_ERROR} otherwise */ - public TSStatus dropTrigger(String triggerName, boolean isGeneratedByPipe) { - DropTriggerProcedure procedure = new DropTriggerProcedure(triggerName, isGeneratedByPipe); + public TSStatus dropTrigger( + String triggerName, boolean isGeneratedByPipe, String originClusterId) { + DropTriggerProcedure procedure = + new DropTriggerProcedure(triggerName, isGeneratedByPipe, originClusterId); executor.submitProcedure(procedure); TSStatus status = waitingProcedureFinished(procedure); if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { @@ -1695,6 +1721,14 @@ public TSStatus setTTL(SetTTLPlan setTTLPlan, final boolean isGeneratedByPipe) { return waitingProcedureFinished(procedure); } + public TSStatus setTTL( + SetTTLPlan setTTLPlan, final boolean isGeneratedByPipe, final String originalClusterId) { + SetTTLProcedure procedure = + new SetTTLProcedure(setTTLPlan, isGeneratedByPipe, originalClusterId); + executor.submitProcedure(procedure); + return waitingProcedureFinished(procedure); + } + /** * Waiting until the specific procedure finished. * diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/TTLManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/TTLManager.java index 110d68a02c6c9..35797f73367cf 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/TTLManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/TTLManager.java @@ -55,7 +55,10 @@ public TTLManager(IManager configManager, TTLInfo ttlInfo) { } /** Set ttl when creating database. */ - public TSStatus setTTL(DatabaseSchemaPlan databaseSchemaPlan, final boolean isGeneratedByPipe) + public TSStatus setTTL( + DatabaseSchemaPlan databaseSchemaPlan, + final boolean isGeneratedByPipe, + final String originClusterId) throws IllegalPathException { long ttl = databaseSchemaPlan.getSchema().getTTL(); if (ttl < 0) { @@ -67,10 +70,13 @@ public TSStatus setTTL(DatabaseSchemaPlan databaseSchemaPlan, final boolean isGe new SetTTLPlan( PathUtils.splitPathToDetachedNodes(databaseSchemaPlan.getSchema().getName()), ttl); setTTLPlan.setDataBase(true); - return configManager.getProcedureManager().setTTL(setTTLPlan, isGeneratedByPipe); + return configManager + .getProcedureManager() + .setTTL(setTTLPlan, isGeneratedByPipe, originClusterId); } - public TSStatus setTTL(SetTTLPlan setTTLPlan, final boolean isGeneratedByPipe) { + public TSStatus setTTL( + SetTTLPlan setTTLPlan, final boolean isGeneratedByPipe, final String originClusterId) { PartialPath path = new PartialPath(setTTLPlan.getPathPattern()); if (!checkIsPathValidated(path)) { TSStatus errorStatus = new TSStatus(TSStatusCode.ILLEGAL_PARAMETER.getStatusCode()); @@ -89,10 +95,13 @@ public TSStatus setTTL(SetTTLPlan setTTLPlan, final boolean isGeneratedByPipe) { // if path matches database, then set both path and path.** setTTLPlan.setDataBase(configManager.getPartitionManager().isDatabaseExist(path.getFullPath())); - return configManager.getProcedureManager().setTTL(setTTLPlan, isGeneratedByPipe); + return configManager + .getProcedureManager() + .setTTL(setTTLPlan, isGeneratedByPipe, originClusterId); } - public TSStatus unsetTTL(SetTTLPlan setTTLPlan, final boolean isGeneratedByPipe) { + public TSStatus unsetTTL( + SetTTLPlan setTTLPlan, final boolean isGeneratedByPipe, final String originClusterId) { PartialPath path = new PartialPath(setTTLPlan.getPathPattern()); if (!checkIsPathValidated(path)) { TSStatus errorStatus = new TSStatus(TSStatusCode.ILLEGAL_PARAMETER.getStatusCode()); @@ -105,7 +114,9 @@ public TSStatus unsetTTL(SetTTLPlan setTTLPlan, final boolean isGeneratedByPipe) // if path matches database, then unset both path and path.** setTTLPlan.setDataBase(configManager.getPartitionManager().isDatabaseExist(path.getFullPath())); - return configManager.getProcedureManager().setTTL(setTTLPlan, isGeneratedByPipe); + return configManager + .getProcedureManager() + .setTTL(setTTLPlan, isGeneratedByPipe, originClusterId); } public DataSet showTTL(ShowTTLPlan showTTLPlan) { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/TriggerManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/TriggerManager.java index 5f64c4963125f..eff18c7f2b258 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/TriggerManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/TriggerManager.java @@ -136,7 +136,9 @@ public TSStatus dropTrigger(TDropTriggerReq req) { return configManager .getProcedureManager() .dropTrigger( - req.getTriggerName(), req.isSetIsGeneratedByPipe() && req.isIsGeneratedByPipe()); + req.getTriggerName(), + req.isSetIsGeneratedByPipe() && req.isIsGeneratedByPipe(), + req.getOriginClusterId()); } public TGetTriggerTableResp getTriggerTable(boolean onlyStateful) { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java index 266ea5daf91f4..ebfbe70cd5e7f 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java @@ -566,51 +566,67 @@ private TSStatus executePlan(final ConfigPhysicalPlan plan) throws ConsensusExce Collections.singletonList(((DeleteDatabasePlan) plan).getName())) .setIsGeneratedByPipe(true) .setIsTableModel( - PathUtils.isTableModelDatabase(((DeleteDatabasePlan) plan).getName()))); + PathUtils.isTableModelDatabase(((DeleteDatabasePlan) plan).getName())) + .setOriginClusterId(clusterIdFromHandshakeRequest)); case ExtendSchemaTemplate: return configManager .getClusterSchemaManager() - .extendSchemaTemplate(((ExtendSchemaTemplatePlan) plan).getTemplateExtendInfo(), true); + .extendSchemaTemplate( + ((ExtendSchemaTemplatePlan) plan).getTemplateExtendInfo(), + true, + clusterIdFromHandshakeRequest); case CommitSetSchemaTemplate: return configManager.setSchemaTemplate( new TSetSchemaTemplateReq( queryId, ((CommitSetSchemaTemplatePlan) plan).getName(), ((CommitSetSchemaTemplatePlan) plan).getPath()) - .setIsGeneratedByPipe(true)); + .setIsGeneratedByPipe(true) + .setOriginClusterId(clusterIdFromHandshakeRequest)); case PipeUnsetTemplate: return configManager.unsetSchemaTemplate( new TUnsetSchemaTemplateReq( queryId, ((PipeUnsetSchemaTemplatePlan) plan).getName(), ((PipeUnsetSchemaTemplatePlan) plan).getPath()) - .setIsGeneratedByPipe(true)); + .setIsGeneratedByPipe(true) + .setOriginClusterId(clusterIdFromHandshakeRequest)); case PipeDeleteTimeSeries: return configManager.deleteTimeSeries( new TDeleteTimeSeriesReq( queryId, ((PipeDeleteTimeSeriesPlan) plan).getPatternTreeBytes()) - .setIsGeneratedByPipe(true)); + .setIsGeneratedByPipe(true) + .setOriginClusterId(clusterIdFromHandshakeRequest)); case PipeDeleteLogicalView: return configManager.deleteLogicalView( new TDeleteLogicalViewReq( queryId, ((PipeDeleteLogicalViewPlan) plan).getPatternTreeBytes()) - .setIsGeneratedByPipe(true)); + .setIsGeneratedByPipe(true) + .setOriginClusterId(clusterIdFromHandshakeRequest)); case PipeDeactivateTemplate: return configManager .getProcedureManager() .deactivateTemplate( - queryId, ((PipeDeactivateTemplatePlan) plan).getTemplateSetInfo(), true); + queryId, + ((PipeDeactivateTemplatePlan) plan).getTemplateSetInfo(), + true, + clusterIdFromHandshakeRequest); case UpdateTriggerStateInTable: // TODO: Record complete message in trigger return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()); case DeleteTriggerInTable: return configManager.dropTrigger( new TDropTriggerReq(((DeleteTriggerInTablePlan) plan).getTriggerName()) - .setIsGeneratedByPipe(true)); + .setIsGeneratedByPipe(true) + .setOriginClusterId(clusterIdFromHandshakeRequest)); case SetTTL: return ((SetTTLPlan) plan).getTTL() == TTLCache.NULL_TTL - ? configManager.getTTLManager().unsetTTL((SetTTLPlan) plan, true) - : configManager.getTTLManager().setTTL((SetTTLPlan) plan, true); + ? configManager + .getTTLManager() + .unsetTTL((SetTTLPlan) plan, true, clusterIdFromHandshakeRequest) + : configManager + .getTTLManager() + .setTTL((SetTTLPlan) plan, true, clusterIdFromHandshakeRequest); case PipeCreateTable: return executeIdempotentCreateTable((PipeCreateTablePlan) plan, queryId); case AddTableColumn: @@ -627,7 +643,8 @@ private TSStatus executePlan(final ConfigPhysicalPlan plan) throws ConsensusExce ((AddTableColumnPlan) plan).getTableName(), queryId, ((AddTableColumnPlan) plan).getColumnSchemaList(), - true)); + true, + clusterIdFromHandshakeRequest)); case SetTableProperties: return configManager .getProcedureManager() @@ -780,7 +797,7 @@ private TSStatus executeIdempotentCreateTable( table.getTableName(), queryId, ProcedureType.CREATE_TABLE_PROCEDURE, - new CreateTableProcedure(database, table, true)); + new CreateTableProcedure(database, table, true, clusterIdFromHandshakeRequest)); if (result.getCode() == TSStatusCode.TABLE_ALREADY_EXISTS.getStatusCode()) { // If the table already exists, we shall add the sender table's columns to the // receiver's table, inner procedure guaranteeing that the columns existing at the diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java index 1952b603fb38f..0ecd2073e9ff5 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java @@ -183,11 +183,14 @@ public TSStatus setDatabase( getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan(databaseSchemaPlan) + ? new PipeEnrichedPlan(databaseSchemaPlan, originClusterIds) : databaseSchemaPlan); // set ttl if (schema.isSetTTL()) { - result = configManager.getTTLManager().setTTL(databaseSchemaPlan, isGeneratedByPipe); + result = + configManager + .getTTLManager() + .setTTL(databaseSchemaPlan, isGeneratedByPipe, originClusterIds); } // Bind Database metrics PartitionMetrics.bindDatabaseRelatedMetricsWhenUpdate( @@ -282,14 +285,16 @@ public TSStatus alterDatabase( /** Delete DatabaseSchema. */ public TSStatus deleteDatabase( - final DeleteDatabasePlan deleteDatabasePlan, final boolean isGeneratedByPipe) { + final DeleteDatabasePlan deleteDatabasePlan, + final boolean isGeneratedByPipe, + final String originClusterId) { TSStatus result; try { result = getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan(deleteDatabasePlan) + ? new PipeEnrichedPlan(deleteDatabasePlan, originClusterId) : deleteDatabasePlan); } catch (final ConsensusException e) { LOGGER.warn(CONSENSUS_WRITE_ERROR, e); @@ -923,12 +928,13 @@ public TSStatus rollbackPreUnsetSchemaTemplate(int templateId, PartialPath path) } public TSStatus unsetSchemaTemplateInBlackList( - int templateId, PartialPath path, boolean isGeneratedByPipe) { + int templateId, PartialPath path, boolean isGeneratedByPipe, String originClusterId) { try { return getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan(new UnsetSchemaTemplatePlan(templateId, path)) + ? new PipeEnrichedPlan( + new UnsetSchemaTemplatePlan(templateId, path), originClusterId) : new UnsetSchemaTemplatePlan(templateId, path)); } catch (ConsensusException e) { LOGGER.warn(CONSENSUS_WRITE_ERROR, e); @@ -994,7 +1000,7 @@ public synchronized TSStatus dropSchemaTemplate(String templateName) { } public synchronized TSStatus extendSchemaTemplate( - TemplateExtendInfo templateExtendInfo, boolean isGeneratedByPipe) { + TemplateExtendInfo templateExtendInfo, boolean isGeneratedByPipe, String originClusterId) { if (templateExtendInfo.getEncodings() != null) { for (int i = 0; i < templateExtendInfo.getDataTypes().size(); i++) { try { @@ -1037,7 +1043,7 @@ public synchronized TSStatus extendSchemaTemplate( getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan(extendSchemaTemplatePlan) + ? new PipeEnrichedPlan(extendSchemaTemplatePlan, originClusterId) : extendSchemaTemplatePlan); } catch (ConsensusException e) { LOGGER.warn(CONSENSUS_WRITE_ERROR, e); @@ -1285,12 +1291,16 @@ public synchronized TSStatus addTableColumn( final String database, final String tableName, final List columnSchemaList, - final boolean isGeneratedByPipe) { + final boolean isGeneratedByPipe, + final String originClusterId) { final AddTableColumnPlan addTableColumnPlan = new AddTableColumnPlan(database, tableName, columnSchemaList, false); try { return getConsensusManager() - .write(isGeneratedByPipe ? new PipeEnrichedPlan(addTableColumnPlan) : addTableColumnPlan); + .write( + isGeneratedByPipe + ? new PipeEnrichedPlan(addTableColumnPlan, originClusterId) + : addTableColumnPlan); } catch (final ConsensusException e) { LOGGER.warn(e.getMessage(), e); return RpcUtils.getStatus(TSStatusCode.INTERNAL_SERVER_ERROR, e.getMessage()); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/env/ConfigNodeProcedureEnv.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/env/ConfigNodeProcedureEnv.java index e721e3dd0140e..788c9044b4008 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/env/ConfigNodeProcedureEnv.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/env/ConfigNodeProcedureEnv.java @@ -137,9 +137,10 @@ public ConfigManager getConfigManager() { * @param isGeneratedByPipe whether the deletion is triggered by pipe request * @return tsStatus */ - public TSStatus deleteDatabaseConfig(final String name, final boolean isGeneratedByPipe) { + public TSStatus deleteDatabaseConfig( + final String name, final boolean isGeneratedByPipe, final String originClusterId) { return getClusterSchemaManager() - .deleteDatabase(new DeleteDatabasePlan(name), isGeneratedByPipe); + .deleteDatabase(new DeleteDatabasePlan(name), isGeneratedByPipe, originClusterId); } /** diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/StateMachineProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/StateMachineProcedure.java index edd8d7f69e5ee..215e1b5e7fe01 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/StateMachineProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/StateMachineProcedure.java @@ -64,6 +64,8 @@ public abstract class StateMachineProcedure extends Procedure /** Mark whether this procedure is called by a pipe forwarded request. */ protected boolean isGeneratedByPipe; + protected String originClusterId; + private boolean isStateDeserialized = false; protected StateMachineProcedure() { @@ -74,6 +76,11 @@ protected StateMachineProcedure(final boolean isGeneratedByPipe) { this.isGeneratedByPipe = isGeneratedByPipe; } + protected StateMachineProcedure(final boolean isGeneratedByPipe, final String originClusterId) { + this.isGeneratedByPipe = isGeneratedByPipe; + this.originClusterId = originClusterId; + } + public enum Flow { HAS_MORE_STATE, NO_MORE_STATE, @@ -275,6 +282,13 @@ public void serialize(DataOutputStream stream) throws IOException { for (int state : states) { stream.writeInt(state); } + + if (originClusterId == null) { + stream.writeBoolean(false); + } else { + stream.writeBoolean(true); + stream.writeUTF(originClusterId); + } } @Override @@ -291,6 +305,17 @@ public void deserialize(ByteBuffer byteBuffer) { } } this.setStateDeserialized(true); + if (byteBuffer.hasRemaining()) { + boolean hasClusterId = byteBuffer.get() != 0; + if (hasClusterId) { + int strLength = byteBuffer.getShort(); + byte[] bytes = new byte[strLength]; + byteBuffer.get(bytes); + originClusterId = new String(bytes); + } else { + originClusterId = null; + } + } } /** diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/node/AbstractNodeProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/node/AbstractNodeProcedure.java index b141027917366..fd115eabc7170 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/node/AbstractNodeProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/node/AbstractNodeProcedure.java @@ -39,6 +39,10 @@ protected AbstractNodeProcedure(boolean isGeneratedByPipe) { super(isGeneratedByPipe); } + protected AbstractNodeProcedure(final boolean isGeneratedByPipe, final String originClusterId) { + super(isGeneratedByPipe, originClusterId); + } + @Override protected ProcedureLockState acquireLock(ConfigNodeProcedureEnv configNodeProcedureEnv) { configNodeProcedureEnv.getSchedulerLock().lock(); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeactivateTemplateProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeactivateTemplateProcedure.java index cffa43f3f73c3..650b1c4146f2d 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeactivateTemplateProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeactivateTemplateProcedure.java @@ -95,6 +95,16 @@ public DeactivateTemplateProcedure( setTemplateSetInfo(templateSetInfo); } + public DeactivateTemplateProcedure( + String queryId, + Map> templateSetInfo, + boolean isGeneratedByPipe, + String originClusterId) { + super(isGeneratedByPipe, originClusterId); + this.queryId = queryId; + setTemplateSetInfo(templateSetInfo); + } + @Override protected Flow executeFromState(ConfigNodeProcedureEnv env, DeactivateTemplateState state) throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java index 5fc8158716360..8efadf7346865 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java @@ -75,6 +75,15 @@ public DeleteDatabaseProcedure( this.deleteDatabaseSchema = deleteDatabaseSchema; } + public DeleteDatabaseProcedure( + final TDatabaseSchema deleteDatabaseSchema, + final boolean isGeneratedByPipe, + final String originClusterId) { + super(isGeneratedByPipe); + this.deleteDatabaseSchema = deleteDatabaseSchema; + this.originClusterId = originClusterId; + } + public TDatabaseSchema getDeleteDatabaseSchema() { return deleteDatabaseSchema; } @@ -205,7 +214,8 @@ protected Flow executeFromState( // Delete DatabasePartitionTable final TSStatus deleteConfigResult = - env.deleteDatabaseConfig(deleteDatabaseSchema.getName(), isGeneratedByPipe); + env.deleteDatabaseConfig( + deleteDatabaseSchema.getName(), isGeneratedByPipe, originClusterId); if (deleteConfigResult.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { LOG.info( @@ -314,7 +324,8 @@ public boolean equals(final Object that) { && thatProc.getCurrentState().equals(this.getCurrentState()) && thatProc.getCycles() == this.getCycles() && thatProc.isGeneratedByPipe == this.isGeneratedByPipe - && thatProc.deleteDatabaseSchema.equals(this.getDeleteDatabaseSchema()); + && thatProc.deleteDatabaseSchema.equals(this.getDeleteDatabaseSchema()) + && Objects.equals(thatProc.originClusterId, this.originClusterId); } return false; } @@ -322,6 +333,11 @@ public boolean equals(final Object that) { @Override public int hashCode() { return Objects.hash( - getProcId(), getCurrentState(), getCycles(), isGeneratedByPipe, deleteDatabaseSchema); + getProcId(), + getCurrentState(), + getCycles(), + isGeneratedByPipe, + deleteDatabaseSchema, + originClusterId); } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteLogicalViewProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteLogicalViewProcedure.java index 6b67da9aab9c0..cbc95e5b394a2 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteLogicalViewProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteLogicalViewProcedure.java @@ -89,6 +89,16 @@ public DeleteLogicalViewProcedure( setPatternTree(patternTree); } + public DeleteLogicalViewProcedure( + final String queryId, + final PathPatternTree patternTree, + final boolean isGeneratedByPipe, + final String originClusterId) { + super(isGeneratedByPipe, originClusterId); + this.queryId = queryId; + setPatternTree(patternTree); + } + @Override protected Flow executeFromState( final ConfigNodeProcedureEnv env, final DeleteLogicalViewState state) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java index b92a675c60e1c..237fb1758ca01 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java @@ -94,6 +94,16 @@ public DeleteTimeSeriesProcedure( setPatternTree(patternTree); } + public DeleteTimeSeriesProcedure( + final String queryId, + final PathPatternTree patternTree, + final boolean isGeneratedByPipe, + final String originClusterId) { + super(isGeneratedByPipe, originClusterId); + this.queryId = queryId; + setPatternTree(patternTree); + } + @Override protected Flow executeFromState( final ConfigNodeProcedureEnv env, final DeleteTimeSeriesState state) @@ -278,7 +288,8 @@ private void collectPayload4Pipe(final ConfigNodeProcedureEnv env) { .getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan(new PipeDeleteTimeSeriesPlan(patternTreeBytes)) + ? new PipeEnrichedPlan( + new PipeDeleteTimeSeriesPlan(patternTreeBytes), originClusterId) : new PipeDeleteTimeSeriesPlan(patternTreeBytes)); } catch (final ConsensusException e) { LOGGER.warn(CONSENSUS_WRITE_ERROR, e); @@ -385,13 +396,19 @@ public boolean equals(final Object o) { && this.getCurrentState().equals(that.getCurrentState()) && this.getCycles() == getCycles() && this.isGeneratedByPipe == that.isGeneratedByPipe - && this.patternTree.equals(that.patternTree); + && this.patternTree.equals(that.patternTree) + && Objects.equals(this.originClusterId, that.originClusterId); } @Override public int hashCode() { return Objects.hash( - getProcId(), getCurrentState(), getCycles(), isGeneratedByPipe, patternTree); + getProcId(), + getCurrentState(), + getCycles(), + isGeneratedByPipe, + patternTree, + originClusterId); } private class DeleteTimeSeriesRegionTaskExecutor diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTTLProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTTLProcedure.java index de81a0681a198..cc64436f393bb 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTTLProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTTLProcedure.java @@ -65,6 +65,12 @@ public SetTTLProcedure(SetTTLPlan plan, final boolean isGeneratedByPipe) { this.plan = plan; } + public SetTTLProcedure( + SetTTLPlan plan, final boolean isGeneratedByPipe, final String originClusterId) { + super(isGeneratedByPipe, originClusterId); + this.plan = plan; + } + @Override protected Flow executeFromState(ConfigNodeProcedureEnv env, SetTTLState state) throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException { @@ -91,7 +97,8 @@ private void setConfigNodeTTL(ConfigNodeProcedureEnv env) { res = env.getConfigManager() .getConsensusManager() - .write(isGeneratedByPipe ? new PipeEnrichedPlan(this.plan) : this.plan); + .write( + isGeneratedByPipe ? new PipeEnrichedPlan(this.plan, originClusterId) : this.plan); } catch (ConsensusException e) { LOGGER.warn("Failed in the write API executing the consensus layer due to: ", e); res = new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()); @@ -157,6 +164,13 @@ public void serialize(DataOutputStream stream) throws IOException { : ProcedureType.SET_TTL_PROCEDURE.getTypeCode()); super.serialize(stream); ReadWriteIOUtils.write(plan.serializeToByteBuffer(), stream); + + if (originClusterId == null) { + stream.writeBoolean(false); + } else { + stream.writeBoolean(true); + ReadWriteIOUtils.write(originClusterId, stream); + } } @Override @@ -165,6 +179,15 @@ public void deserialize(ByteBuffer byteBuffer) { try { ReadWriteIOUtils.readInt(byteBuffer); this.plan = (SetTTLPlan) ConfigPhysicalPlan.Factory.create(byteBuffer); + + if (byteBuffer.hasRemaining()) { + boolean hasClusterId = byteBuffer.get() != 0; + if (hasClusterId) { + this.originClusterId = ReadWriteIOUtils.readString(byteBuffer); + } else { + this.originClusterId = null; + } + } } catch (IOException e) { LOGGER.error("IO error when deserialize setTTL plan.", e); } @@ -179,11 +202,12 @@ public boolean equals(Object o) { return false; } return this.plan.equals(((SetTTLProcedure) o).plan) - && this.isGeneratedByPipe == (((SetTTLProcedure) o).isGeneratedByPipe); + && this.isGeneratedByPipe == (((SetTTLProcedure) o).isGeneratedByPipe) + && Objects.equals(this.originClusterId, ((SetTTLProcedure) o).originClusterId); } @Override public int hashCode() { - return Objects.hash(plan, isGeneratedByPipe); + return Objects.hash(plan, isGeneratedByPipe, originClusterId); } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTemplateProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTemplateProcedure.java index 9180904cade14..fa1ca49ca7af5 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTemplateProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTemplateProcedure.java @@ -98,6 +98,19 @@ public SetTemplateProcedure( this.templateSetPath = templateSetPath; } + public SetTemplateProcedure( + final String queryId, + final String templateName, + final String templateSetPath, + final boolean isGeneratedByPipe, + final String originClusterId) { + super(isGeneratedByPipe); + this.queryId = queryId; + this.templateName = templateName; + this.templateSetPath = templateSetPath; + this.originClusterId = originClusterId; + } + @Override protected Flow executeFromState(final ConfigNodeProcedureEnv env, final SetTemplateState state) throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException { @@ -355,7 +368,7 @@ private void commitSetTemplate(final ConfigNodeProcedureEnv env) { .getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan(commitSetSchemaTemplatePlan) + ? new PipeEnrichedPlan(commitSetSchemaTemplatePlan, originClusterId) : commitSetSchemaTemplatePlan); } catch (final ConsensusException e) { LOGGER.warn(CONSENSUS_WRITE_ERROR, e); @@ -596,7 +609,8 @@ public boolean equals(final Object o) { && Objects.equals(getCycles(), that.getCycles()) && Objects.equals(isGeneratedByPipe, that.isGeneratedByPipe) && Objects.equals(templateName, that.templateName) - && Objects.equals(templateSetPath, that.templateSetPath); + && Objects.equals(templateSetPath, that.templateSetPath) + && Objects.equals(originClusterId, that.originClusterId); } @Override @@ -607,6 +621,7 @@ public int hashCode() { getCycles(), isGeneratedByPipe, templateName, - templateSetPath); + templateSetPath, + originClusterId); } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/UnsetTemplateProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/UnsetTemplateProcedure.java index 049c6a46eacfc..8c0773b1ea5c2 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/UnsetTemplateProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/UnsetTemplateProcedure.java @@ -81,6 +81,18 @@ public UnsetTemplateProcedure( this.path = path; } + public UnsetTemplateProcedure( + String queryId, + Template template, + PartialPath path, + boolean isGeneratedByPipe, + String originClusterId) { + super(isGeneratedByPipe, originClusterId); + this.queryId = queryId; + this.template = template; + this.path = path; + } + @Override protected Flow executeFromState(ConfigNodeProcedureEnv env, UnsetTemplateState state) throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException { @@ -196,7 +208,8 @@ private void unsetTemplate(ConfigNodeProcedureEnv env) { TSStatus status = env.getConfigManager() .getClusterSchemaManager() - .unsetSchemaTemplateInBlackList(template.getId(), path, isGeneratedByPipe); + .unsetSchemaTemplateInBlackList( + template.getId(), path, isGeneratedByPipe, originClusterId); if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { setNextState(UnsetTemplateState.CLEAN_DATANODE_TEMPLATE_CACHE); } else { @@ -364,12 +377,20 @@ public boolean equals(Object o) { && Objects.equals(isGeneratedByPipe, that.isGeneratedByPipe) && Objects.equals(queryId, that.queryId) && Objects.equals(template, that.template) - && Objects.equals(path, that.path); + && Objects.equals(path, that.path) + && Objects.equals(originClusterId, that.originClusterId); } @Override public int hashCode() { return Objects.hash( - getProcId(), getCurrentState(), getCycles(), isGeneratedByPipe, queryId, template, path); + getProcId(), + getCurrentState(), + getCycles(), + isGeneratedByPipe, + queryId, + template, + path, + originClusterId); } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/AbstractAlterOrDropTableProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/AbstractAlterOrDropTableProcedure.java index ea99a07c41157..4fde4030cb21d 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/AbstractAlterOrDropTableProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/AbstractAlterOrDropTableProcedure.java @@ -73,6 +73,18 @@ protected AbstractAlterOrDropTableProcedure( this.queryId = queryId; } + protected AbstractAlterOrDropTableProcedure( + final String database, + final String tableName, + final String queryId, + final boolean isGeneratedByPipe, + final String originClusterId) { + super(isGeneratedByPipe, originClusterId); + this.database = database; + this.tableName = tableName; + this.queryId = queryId; + } + public String getDatabase() { return database; } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/AddTableColumnProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/AddTableColumnProcedure.java index 37c3b83eaf189..2a9592f04e298 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/AddTableColumnProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/AddTableColumnProcedure.java @@ -63,6 +63,17 @@ public AddTableColumnProcedure( this.addedColumnList = addedColumnList; } + public AddTableColumnProcedure( + final String database, + final String tableName, + final String queryId, + final List addedColumnList, + final boolean isGeneratedByPipe, + final String originClusterId) { + super(database, tableName, queryId, isGeneratedByPipe, originClusterId); + this.addedColumnList = addedColumnList; + } + @Override protected Flow executeFromState(final ConfigNodeProcedureEnv env, final AddTableColumnState state) throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException { @@ -129,7 +140,8 @@ private void addColumn(final ConfigNodeProcedureEnv env) { final TSStatus status = env.getConfigManager() .getClusterSchemaManager() - .addTableColumn(database, tableName, addedColumnList, isGeneratedByPipe); + .addTableColumn( + database, tableName, addedColumnList, isGeneratedByPipe, originClusterId); if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { setFailure(new ProcedureException(new IoTDBException(status.getMessage(), status.getCode()))); } else { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/CreateTableProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/CreateTableProcedure.java index a018412a89a55..0855825244f90 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/CreateTableProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/CreateTableProcedure.java @@ -71,6 +71,16 @@ public CreateTableProcedure( this.table = table; } + public CreateTableProcedure( + final String database, + final TsTable table, + final boolean isGeneratedByPipe, + final String originClusterId) { + super(isGeneratedByPipe, originClusterId); + this.database = database; + this.table = table; + } + @Override protected Flow executeFromState(final ConfigNodeProcedureEnv env, final CreateTableState state) throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/trigger/DropTriggerProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/trigger/DropTriggerProcedure.java index 19bfcdc30d2ba..42a9104e0df37 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/trigger/DropTriggerProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/trigger/DropTriggerProcedure.java @@ -57,6 +57,12 @@ public DropTriggerProcedure(String triggerName, boolean isGeneratedByPipe) { this.triggerName = triggerName; } + public DropTriggerProcedure( + String triggerName, boolean isGeneratedByPipe, String originClusterId) { + super(isGeneratedByPipe, originClusterId); + this.triggerName = triggerName; + } + @Override protected Flow executeFromState(ConfigNodeProcedureEnv env, DropTriggerState state) { if (triggerName == null) { diff --git a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift index 7d3fca8b83aac..148908a46700c 100644 --- a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift +++ b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift @@ -1003,6 +1003,7 @@ struct TDeactivateSchemaTemplateReq { 2: required binary pathPatternTree 3: optional string templateName 4: optional bool isGeneratedByPipe + 5: optional string originClusterId } struct TUnsetSchemaTemplateReq { @@ -1010,7 +1011,7 @@ struct TUnsetSchemaTemplateReq { 2: required string templateName 3: required string path 4: optional bool isGeneratedByPipe - 5: optional String originClusterId + 5: optional string originClusterId } struct TCreateModelReq { diff --git a/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift b/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift index 5d69c528478e6..661e12526d367 100644 --- a/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift +++ b/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift @@ -432,12 +432,14 @@ struct TDeleteDataForDeleteSchemaReq { 1: required list dataRegionIdList 2: required binary pathPatternTree 3: optional bool isGeneratedByPipe + 4: optional string originClusterId } struct TDeleteTimeSeriesReq { 1: required list schemaRegionIdList 2: required binary pathPatternTree 3: optional bool isGeneratedByPipe + 4: optional string originClusterId } struct TConstructSchemaBlackListWithTemplateReq { From 897d41a3656387f802adfb2b9edac15ffe1dd271 Mon Sep 17 00:00:00 2001 From: XNX02 <202100130044@mail.sdu.edu.cn> Date: Tue, 11 Mar 2025 23:29:17 +0800 Subject: [PATCH 07/27] update originclusteid --- .../ConfigRegionStateMachine.java | 4 +-- .../confignode/manager/ConfigManager.java | 8 ++++-- .../confignode/manager/PermissionManager.java | 2 +- .../confignode/manager/ProcedureManager.java | 14 +++++++--- .../event/PipeConfigRegionWritePlanEvent.java | 28 ++++++++++++++++--- .../extractor/ConfigRegionListeningQueue.java | 17 +++++++---- .../protocol/IoTDBConfigNodeReceiver.java | 21 +++++++++----- .../manager/schema/ClusterSchemaManager.java | 22 +++++++++------ .../schema/DeactivateTemplateProcedure.java | 17 ++++++++--- .../impl/schema/DeleteDatabaseProcedure.java | 3 +- .../schema/DeleteLogicalViewProcedure.java | 16 ++++++++--- .../schema/DeleteTimeSeriesProcedure.java | 6 ++-- .../schema/table/CreateTableProcedure.java | 3 +- .../schema/table/DeleteDevicesProcedure.java | 18 +++++++++++- .../table/DropTableColumnProcedure.java | 14 +++++++++- .../impl/schema/table/DropTableProcedure.java | 12 +++++++- .../table/RenameTableColumnProcedure.java | 19 +++++++++++-- .../table/SetTablePropertiesProcedure.java | 17 +++++++++-- .../impl/sync/AuthOperationProcedure.java | 21 ++++++++++++-- .../impl/trigger/DropTriggerProcedure.java | 13 +++++++-- .../ConfigRegionListeningQueueTest.java | 4 +-- .../src/main/thrift/confignode.thrift | 3 ++ .../src/main/thrift/datanode.thrift | 6 +++- 23 files changed, 225 insertions(+), 63 deletions(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/statemachine/ConfigRegionStateMachine.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/statemachine/ConfigRegionStateMachine.java index d4e61c22f75e1..db7bb560056e6 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/statemachine/ConfigRegionStateMachine.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/statemachine/ConfigRegionStateMachine.java @@ -134,7 +134,7 @@ protected TSStatus write(ConfigPhysicalPlan plan) { } if (result.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - PipeConfigNodeAgent.runtime().listener().tryListenToPlan(plan, false); + PipeConfigNodeAgent.runtime().listener().tryListenToPlan(plan, false, null); } return result; @@ -428,7 +428,7 @@ private void initStandAloneConfigNode() { // Recover the linked queue. // Note that the "nextPlan"s may contain create and drop pipe operations // and will affect whether the queue listen to the plans. - PipeConfigNodeAgent.runtime().listener().tryListenToPlan(nextPlan, false); + PipeConfigNodeAgent.runtime().listener().tryListenToPlan(nextPlan, false, null); } } catch (UnknownPhysicalPlanTypeException e) { LOGGER.error("Try listen to plan failed", e); 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 104bcb95bdadd..4628746ddde36 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 @@ -2683,14 +2683,16 @@ public TSStatus alterOrDropTable(final TAlterOrDropTableReq req) { req.getDatabase(), req.getTableName(), ReadWriteIOUtils.readString(req.updateInfo), - false); + false, + null); case COMMENT_COLUMN: return clusterSchemaManager.setTableColumnComment( req.getDatabase(), req.getTableName(), ReadWriteIOUtils.readString(req.updateInfo), ReadWriteIOUtils.readString(req.updateInfo), - false); + false, + null); default: throw new IllegalArgumentException(); } @@ -2703,7 +2705,7 @@ public TSStatus alterOrDropTable(final TAlterOrDropTableReq req) { public TDeleteTableDeviceResp deleteDevice(final TDeleteTableDeviceReq req) { final TSStatus status = confirmLeader(); return status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode() - ? procedureManager.deleteDevices(req, false) + ? procedureManager.deleteDevices(req, false, null) : new TDeleteTableDeviceResp(status); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/PermissionManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/PermissionManager.java index e2ec75745b02f..8e268dcaaedc0 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/PermissionManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/PermissionManager.java @@ -80,7 +80,7 @@ public TSStatus operatePermission( tsStatus = configManager .getProcedureManager() - .operateAuthPlan(authorPlan, allDataNodes, isGeneratedByPipe); + .operateAuthPlan(authorPlan, allDataNodes, isGeneratedByPipe, originClusterIds); } return tsStatus; } catch (final ConsensusException e) { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java index 470ce7795a0f9..9b38d8d900e25 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java @@ -1703,10 +1703,13 @@ public TSStatus dropSubscription(TUnsubscribeReq req) { } public TSStatus operateAuthPlan( - AuthorPlan authorPlan, List dns, boolean isGeneratedByPipe) { + AuthorPlan authorPlan, + List dns, + boolean isGeneratedByPipe, + String originClusterId) { try { AuthOperationProcedure procedure = - new AuthOperationProcedure(authorPlan, dns, isGeneratedByPipe); + new AuthOperationProcedure(authorPlan, dns, isGeneratedByPipe, originClusterId); executor.submitProcedure(procedure); return waitingProcedureFinished(procedure); } catch (Exception e) { @@ -1887,7 +1890,9 @@ public TSStatus dropTable(final TAlterOrDropTableReq req) { } public TDeleteTableDeviceResp deleteDevices( - final TDeleteTableDeviceReq req, final boolean isGeneratedByPipe) { + final TDeleteTableDeviceReq req, + final boolean isGeneratedByPipe, + final String originClusterId) { long procedureId; DeleteDevicesProcedure procedure = null; synchronized (this) { @@ -1915,7 +1920,8 @@ public TDeleteTableDeviceResp deleteDevices( req.getPatternInfo(), req.getFilterInfo(), req.getModInfo(), - isGeneratedByPipe); + isGeneratedByPipe, + originClusterId); this.executor.submitProcedure(procedure); } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/event/PipeConfigRegionWritePlanEvent.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/event/PipeConfigRegionWritePlanEvent.java index ec3824de284af..351b1bb42767d 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/event/PipeConfigRegionWritePlanEvent.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/event/PipeConfigRegionWritePlanEvent.java @@ -42,7 +42,24 @@ public PipeConfigRegionWritePlanEvent() { public PipeConfigRegionWritePlanEvent( final ConfigPhysicalPlan configPhysicalPlan, final boolean isGeneratedByPipe) { - this(configPhysicalPlan, null, 0, null, null, null, null, true, isGeneratedByPipe); + this(configPhysicalPlan, null, 0, null, null, null, null, true, isGeneratedByPipe, null); + } + + public PipeConfigRegionWritePlanEvent( + final ConfigPhysicalPlan configPhysicalPlan, + final boolean isGeneratedByPipe, + final String originClusterId) { + this( + configPhysicalPlan, + null, + 0, + null, + null, + null, + null, + true, + isGeneratedByPipe, + originClusterId); } public PipeConfigRegionWritePlanEvent( @@ -54,7 +71,8 @@ public PipeConfigRegionWritePlanEvent( final TablePattern tablePattern, final String userName, final boolean skipIfNoPrivileges, - final boolean isGeneratedByPipe) { + final boolean isGeneratedByPipe, + final String originClusterId) { super( pipeName, creationTime, @@ -63,7 +81,8 @@ public PipeConfigRegionWritePlanEvent( tablePattern, userName, skipIfNoPrivileges, - isGeneratedByPipe); + isGeneratedByPipe, + originClusterId); this.configPhysicalPlan = configPhysicalPlan; } @@ -91,7 +110,8 @@ public EnrichedEvent shallowCopySelfAndBindPipeTaskMetaForProgressReport( tablePattern, userName, skipIfNoPrivileges, - false); + false, + null); } @Override diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/ConfigRegionListeningQueue.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/ConfigRegionListeningQueue.java index 9192b7feeef8c..0570ab66538c7 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/ConfigRegionListeningQueue.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/ConfigRegionListeningQueue.java @@ -64,12 +64,17 @@ public class ConfigRegionListeningQueue extends AbstractPipeListeningQueue /////////////////////////////// Function /////////////////////////////// public synchronized void tryListenToPlan( - final ConfigPhysicalPlan plan, final boolean isGeneratedByPipe) { + final ConfigPhysicalPlan plan, + final boolean isGeneratedByPipe, + final String originClusterId) { if (ConfigRegionListeningFilter.shouldPlanBeListened(plan)) { final PipeConfigRegionWritePlanEvent event; switch (plan.getType()) { case PipeEnriched: - tryListenToPlan(((PipeEnrichedPlan) plan).getInnerPlan(), true); + tryListenToPlan( + ((PipeEnrichedPlan) plan).getInnerPlan(), + true, + (((PipeEnrichedPlan) plan).getOriginClusterId())); return; case UnsetTemplate: // Different clusters have different template ids, so we need to @@ -85,7 +90,8 @@ public synchronized void tryListenToPlan( .getTemplate(((UnsetSchemaTemplatePlan) plan).getTemplateId()) .getName(), ((UnsetSchemaTemplatePlan) plan).getPath().getFullPath()), - isGeneratedByPipe); + isGeneratedByPipe, + originClusterId); } catch (final MetadataException e) { LOGGER.warn("Failed to collect UnsetTemplatePlan", e); return; @@ -104,14 +110,15 @@ public synchronized void tryListenToPlan( ((CommitCreateTablePlan) plan).getDatabase(), ((CommitCreateTablePlan) plan).getTableName()) .orElse(null)), - isGeneratedByPipe); + isGeneratedByPipe, + originClusterId); } catch (final MetadataException e) { LOGGER.warn("Failed to collect CommitCreateTablePlan", e); return; } break; default: - event = new PipeConfigRegionWritePlanEvent(plan, isGeneratedByPipe); + event = new PipeConfigRegionWritePlanEvent(plan, isGeneratedByPipe, originClusterId); } tryListen(event); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java index ebfbe70cd5e7f..9de7bcdb03203 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java @@ -659,7 +659,8 @@ private TSStatus executePlan(final ConfigPhysicalPlan plan) throws ConsensusExce ((SetTablePropertiesPlan) plan).getTableName(), queryId, ((SetTablePropertiesPlan) plan).getProperties(), - true)); + true, + clusterIdFromHandshakeRequest)); case CommitDeleteColumn: return configManager .getProcedureManager() @@ -674,7 +675,8 @@ private TSStatus executePlan(final ConfigPhysicalPlan plan) throws ConsensusExce ((CommitDeleteColumnPlan) plan).getTableName(), queryId, ((CommitDeleteColumnPlan) plan).getColumnName(), - true)); + true, + clusterIdFromHandshakeRequest)); case RenameTableColumn: return configManager .getProcedureManager() @@ -690,7 +692,8 @@ private TSStatus executePlan(final ConfigPhysicalPlan plan) throws ConsensusExce queryId, ((RenameTableColumnPlan) plan).getOldName(), ((RenameTableColumnPlan) plan).getNewName(), - true)); + true, + clusterIdFromHandshakeRequest)); case CommitDeleteTable: return configManager .getProcedureManager() @@ -704,7 +707,8 @@ private TSStatus executePlan(final ConfigPhysicalPlan plan) throws ConsensusExce ((CommitDeleteTablePlan) plan).getDatabase(), ((CommitDeleteTablePlan) plan).getTableName(), queryId, - true)); + true, + clusterIdFromHandshakeRequest)); case SetTableComment: return configManager .getClusterSchemaManager() @@ -712,7 +716,8 @@ private TSStatus executePlan(final ConfigPhysicalPlan plan) throws ConsensusExce ((SetTableCommentPlan) plan).getDatabase(), ((SetTableCommentPlan) plan).getTableName(), ((SetTableCommentPlan) plan).getComment(), - true); + true, + clusterIdFromHandshakeRequest); case SetTableColumnComment: return configManager .getClusterSchemaManager() @@ -721,7 +726,8 @@ private TSStatus executePlan(final ConfigPhysicalPlan plan) throws ConsensusExce ((SetTableColumnCommentPlan) plan).getTableName(), ((SetTableColumnCommentPlan) plan).getColumnName(), ((SetTableColumnCommentPlan) plan).getComment(), - true); + true, + clusterIdFromHandshakeRequest); case PipeDeleteDevices: return configManager .getProcedureManager() @@ -733,7 +739,8 @@ private TSStatus executePlan(final ConfigPhysicalPlan plan) throws ConsensusExce ByteBuffer.wrap(((PipeDeleteDevicesPlan) plan).getPatternBytes()), ByteBuffer.wrap(((PipeDeleteDevicesPlan) plan).getFilterBytes()), ByteBuffer.wrap(((PipeDeleteDevicesPlan) plan).getModBytes())), - true) + true, + clusterIdFromHandshakeRequest) .getStatus(); case CreateUser: case CreateUserWithRawPassword: diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java index 0ecd2073e9ff5..343ce6c13a9b4 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java @@ -1326,14 +1326,15 @@ public synchronized TSStatus renameTableColumn( final String tableName, final String oldName, final String newName, - final boolean isGeneratedByPipe) { + final boolean isGeneratedByPipe, + final String originClusterId) { final RenameTableColumnPlan renameTableColumnPlan = new RenameTableColumnPlan(database, tableName, oldName, newName); try { return getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan(renameTableColumnPlan) + ? new PipeEnrichedPlan(renameTableColumnPlan, originClusterId) : renameTableColumnPlan); } catch (final ConsensusException e) { LOGGER.warn(e.getMessage(), e); @@ -1345,14 +1346,15 @@ public synchronized TSStatus setTableProperties( final String database, final String tableName, final Map properties, - final boolean isGeneratedByPipe) { + final boolean isGeneratedByPipe, + final String originClusterId) { final SetTablePropertiesPlan setTablePropertiesPlan = new SetTablePropertiesPlan(database, tableName, properties); try { return getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan(setTablePropertiesPlan) + ? new PipeEnrichedPlan(setTablePropertiesPlan, originClusterId) : setTablePropertiesPlan); } catch (final ConsensusException e) { LOGGER.warn(e.getMessage(), e); @@ -1364,13 +1366,16 @@ public synchronized TSStatus setTableComment( final String database, final String tableName, final String comment, - final boolean isGeneratedByPipe) { + final boolean isGeneratedByPipe, + final String originClusterId) { final SetTableCommentPlan setTableCommentPlan = new SetTableCommentPlan(database, tableName, comment); try { return getConsensusManager() .write( - isGeneratedByPipe ? new PipeEnrichedPlan(setTableCommentPlan) : setTableCommentPlan); + isGeneratedByPipe + ? new PipeEnrichedPlan(setTableCommentPlan, originClusterId) + : setTableCommentPlan); } catch (final ConsensusException e) { LOGGER.warn(e.getMessage(), e); return RpcUtils.getStatus(TSStatusCode.INTERNAL_SERVER_ERROR, e.getMessage()); @@ -1382,14 +1387,15 @@ public synchronized TSStatus setTableColumnComment( final String tableName, final String columnName, final String comment, - final boolean isGeneratedByPipe) { + final boolean isGeneratedByPipe, + final String originClusterId) { final SetTableColumnCommentPlan setTableColumnCommentPlan = new SetTableColumnCommentPlan(database, tableName, columnName, comment); try { return getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan(setTableColumnCommentPlan) + ? new PipeEnrichedPlan(setTableColumnCommentPlan, originClusterId) : setTableColumnCommentPlan); } catch (final ConsensusException e) { LOGGER.warn(e.getMessage(), e); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeactivateTemplateProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeactivateTemplateProcedure.java index 650b1c4146f2d..74d8ae3327e18 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeactivateTemplateProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeactivateTemplateProcedure.java @@ -261,7 +261,8 @@ private void deactivateTemplate(ConfigNodeProcedureEnv env) { CnToDnAsyncRequestType.DEACTIVATE_TEMPLATE, ((dataNodeLocation, consensusGroupIdList) -> new TDeactivateTemplateReq(consensusGroupIdList, dataNodeRequest) - .setIsGeneratedByPipe(isGeneratedByPipe))); + .setIsGeneratedByPipe(isGeneratedByPipe) + .setOriginClusterId(originClusterId))); deleteTimeSeriesTask.execute(); } @@ -273,7 +274,8 @@ private void collectPayload4Pipe(ConfigNodeProcedureEnv env) { .getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan(new PipeDeactivateTemplatePlan(templateSetInfo)) + ? new PipeEnrichedPlan( + new PipeDeactivateTemplatePlan(templateSetInfo), originClusterId) : new PipeDeactivateTemplatePlan(templateSetInfo)); } catch (ConsensusException e) { LOGGER.warn(CONSENSUS_WRITE_ERROR, e); @@ -431,13 +433,20 @@ public boolean equals(Object o) { && Objects.equals(getCycles(), that.getCycles()) && Objects.equals(isGeneratedByPipe, that.isGeneratedByPipe) && Objects.equals(queryId, that.queryId) - && Objects.equals(templateSetInfo, that.templateSetInfo); + && Objects.equals(templateSetInfo, that.templateSetInfo) + && Objects.equals(originClusterId, that.originClusterId); } @Override public int hashCode() { return Objects.hash( - getProcId(), getCurrentState(), getCycles(), isGeneratedByPipe, queryId, templateSetInfo); + getProcId(), + getCurrentState(), + getCycles(), + isGeneratedByPipe, + queryId, + templateSetInfo, + originClusterId); } private class DeactivateTemplateRegionTaskExecutor diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java index 8efadf7346865..2c6f628b5574d 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteDatabaseProcedure.java @@ -79,9 +79,8 @@ public DeleteDatabaseProcedure( final TDatabaseSchema deleteDatabaseSchema, final boolean isGeneratedByPipe, final String originClusterId) { - super(isGeneratedByPipe); + super(isGeneratedByPipe, originClusterId); this.deleteDatabaseSchema = deleteDatabaseSchema; - this.originClusterId = originClusterId; } public TDatabaseSchema getDeleteDatabaseSchema() { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteLogicalViewProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteLogicalViewProcedure.java index cbc95e5b394a2..d1d116e5ebbd1 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteLogicalViewProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteLogicalViewProcedure.java @@ -223,7 +223,8 @@ private void deleteViewSchema(final ConfigNodeProcedureEnv env) { CnToDnAsyncRequestType.DELETE_VIEW, ((dataNodeLocation, consensusGroupIdList) -> new TDeleteViewSchemaReq(consensusGroupIdList, patternTreeBytes) - .setIsGeneratedByPipe(isGeneratedByPipe))); + .setIsGeneratedByPipe(isGeneratedByPipe) + .setOriginClusterId(originClusterId))); deleteTimeSeriesTask.execute(); } @@ -235,7 +236,8 @@ private void collectPayload4Pipe(final ConfigNodeProcedureEnv env) { .getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan(new PipeDeleteLogicalViewPlan(patternTreeBytes)) + ? new PipeEnrichedPlan( + new PipeDeleteLogicalViewPlan(patternTreeBytes), originClusterId) : new PipeDeleteLogicalViewPlan(patternTreeBytes)); } catch (final ConsensusException e) { LOGGER.warn(CONSENSUS_WRITE_ERROR, e); @@ -338,13 +340,19 @@ public boolean equals(final Object o) { && this.getCurrentState().equals(that.getCurrentState()) && this.getCycles() == that.getCycles() && isGeneratedByPipe == that.isGeneratedByPipe - && patternTree.equals(that.patternTree); + && patternTree.equals(that.patternTree) + && Objects.equals(originClusterId, that.originClusterId); } @Override public int hashCode() { return Objects.hash( - getProcId(), getCurrentState(), getCycles(), isGeneratedByPipe, patternTree); + getProcId(), + getCurrentState(), + getCycles(), + isGeneratedByPipe, + patternTree, + originClusterId); } private class DeleteLogicalViewRegionTaskExecutor diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java index 237fb1758ca01..2309ab4e71fd5 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java @@ -263,7 +263,8 @@ private void executeDeleteData( new TDeleteDataForDeleteSchemaReq( new ArrayList<>(consensusGroupIdList), preparePatternTreeBytesData(patternTree)) - .setIsGeneratedByPipe(isGeneratedByPipe))); + .setIsGeneratedByPipe(isGeneratedByPipe) + .setOriginClusterId(originClusterId))); deleteDataTask.execute(); } @@ -276,7 +277,8 @@ private void deleteTimeSeriesSchema(final ConfigNodeProcedureEnv env) { CnToDnAsyncRequestType.DELETE_TIMESERIES, ((dataNodeLocation, consensusGroupIdList) -> new TDeleteTimeSeriesReq(consensusGroupIdList, patternTreeBytes) - .setIsGeneratedByPipe(isGeneratedByPipe))); + .setIsGeneratedByPipe(isGeneratedByPipe) + .setOriginCluster(originClusterId))); deleteTimeSeriesTask.execute(); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/CreateTableProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/CreateTableProcedure.java index 0855825244f90..75fdc50028ef6 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/CreateTableProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/CreateTableProcedure.java @@ -180,7 +180,8 @@ private void commitCreateTable(final ConfigNodeProcedureEnv env) { final TSStatus status = SchemaUtils.executeInConsensusLayer( isGeneratedByPipe - ? new PipeEnrichedPlan(new CommitCreateTablePlan(database, table.getTableName())) + ? new PipeEnrichedPlan( + new CommitCreateTablePlan(database, table.getTableName()), originClusterId) : new CommitCreateTablePlan(database, table.getTableName()), env, LOGGER); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/DeleteDevicesProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/DeleteDevicesProcedure.java index 131a633b268d2..39934f9a1674c 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/DeleteDevicesProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/DeleteDevicesProcedure.java @@ -95,6 +95,21 @@ public DeleteDevicesProcedure( this.modBytes = modBytes; } + public DeleteDevicesProcedure( + final String database, + final String tableName, + final String queryId, + final @Nonnull byte[] patternBytes, + final @Nonnull byte[] filterBytes, + final @Nonnull byte[] modBytes, + final boolean isGeneratedByPipe, + final String originClusterId) { + super(database, tableName, queryId, isGeneratedByPipe, originClusterId); + this.patternBytes = patternBytes; + this.filterBytes = filterBytes; + this.modBytes = modBytes; + } + @Override protected Flow executeFromState(final ConfigNodeProcedureEnv env, final DeleteDevicesState state) throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException { @@ -289,7 +304,8 @@ private void collectPayload4Pipe(final ConfigNodeProcedureEnv env) { isGeneratedByPipe ? new PipeEnrichedPlan( new PipeDeleteDevicesPlan( - database, tableName, patternBytes, filterBytes, modBytes)) + database, tableName, patternBytes, filterBytes, modBytes), + originClusterId) : new PipeDeleteDevicesPlan( database, tableName, patternBytes, filterBytes, modBytes)); } catch (final ConsensusException e) { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/DropTableColumnProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/DropTableColumnProcedure.java index daf63032fef4e..d50998db38e20 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/DropTableColumnProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/DropTableColumnProcedure.java @@ -75,6 +75,17 @@ public DropTableColumnProcedure( this.columnName = columnName; } + public DropTableColumnProcedure( + final String database, + final String tableName, + final String queryId, + final String columnName, + final boolean isGeneratedByPipe, + final String originClusterId) { + super(database, tableName, queryId, isGeneratedByPipe, originClusterId); + this.columnName = columnName; + } + @Override protected String getActionMessage() { return "drop table column"; @@ -202,7 +213,8 @@ private void dropColumn(final ConfigNodeProcedureEnv env) { final TSStatus status = SchemaUtils.executeInConsensusLayer( isGeneratedByPipe - ? new PipeEnrichedPlan(new CommitDeleteColumnPlan(database, tableName, columnName)) + ? new PipeEnrichedPlan( + new CommitDeleteColumnPlan(database, tableName, columnName), originClusterId) : new CommitDeleteColumnPlan(database, tableName, columnName), env, LOGGER); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/DropTableProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/DropTableProcedure.java index eb4f94fda9295..e50c41fc235da 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/DropTableProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/DropTableProcedure.java @@ -66,6 +66,15 @@ public DropTableProcedure( super(database, tableName, queryId, isGeneratedByPipe); } + public DropTableProcedure( + final String database, + final String tableName, + final String queryId, + final boolean isGeneratedByPipe, + final String originClusterId) { + super(database, tableName, queryId, isGeneratedByPipe, originClusterId); + } + // Not used @Override protected String getActionMessage() { @@ -190,7 +199,8 @@ private void dropTable(final ConfigNodeProcedureEnv env) { final TSStatus status = SchemaUtils.executeInConsensusLayer( isGeneratedByPipe - ? new PipeEnrichedPlan(new CommitDeleteTablePlan(database, tableName)) + ? new PipeEnrichedPlan( + new CommitDeleteTablePlan(database, tableName), originClusterId) : new CommitDeleteTablePlan(database, tableName), env, LOGGER); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/RenameTableColumnProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/RenameTableColumnProcedure.java index 565370064fb29..3613c5ea139c5 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/RenameTableColumnProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/RenameTableColumnProcedure.java @@ -64,6 +64,19 @@ public RenameTableColumnProcedure( this.newName = newName; } + public RenameTableColumnProcedure( + final String database, + final String tableName, + final String queryId, + final String oldName, + final String newName, + final boolean isGeneratedByPipe, + final String originClusterId) { + super(database, tableName, queryId, isGeneratedByPipe, originClusterId); + this.oldName = oldName; + this.newName = newName; + } + @Override protected Flow executeFromState( final ConfigNodeProcedureEnv env, final RenameTableColumnState state) @@ -129,7 +142,8 @@ private void renameColumn(final ConfigNodeProcedureEnv env) { final TSStatus status = env.getConfigManager() .getClusterSchemaManager() - .renameTableColumn(database, tableName, oldName, newName, isGeneratedByPipe); + .renameTableColumn( + database, tableName, oldName, newName, isGeneratedByPipe, originClusterId); if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { setFailure(new ProcedureException(new IoTDBException(status.getMessage(), status.getCode()))); } else { @@ -169,7 +183,8 @@ private void rollbackRenameColumn(final ConfigNodeProcedureEnv env) { final TSStatus status = env.getConfigManager() .getClusterSchemaManager() - .renameTableColumn(database, tableName, newName, oldName, isGeneratedByPipe); + .renameTableColumn( + database, tableName, newName, oldName, isGeneratedByPipe, originClusterId); if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { setFailure(new ProcedureException(new IoTDBException(status.getMessage(), status.getCode()))); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/SetTablePropertiesProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/SetTablePropertiesProcedure.java index 19c6020ac55ff..10f3b0e4b6444 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/SetTablePropertiesProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/SetTablePropertiesProcedure.java @@ -70,6 +70,17 @@ public SetTablePropertiesProcedure( this.updatedProperties = properties; } + public SetTablePropertiesProcedure( + final String database, + final String tableName, + final String queryId, + final Map properties, + final boolean isGeneratedByPipe, + final String originClusterId) { + super(database, tableName, queryId, isGeneratedByPipe, originClusterId); + this.updatedProperties = properties; + } + @Override protected Flow executeFromState( final ConfigNodeProcedureEnv env, final SetTablePropertiesState state) @@ -145,7 +156,8 @@ private void setProperties(final ConfigNodeProcedureEnv env) { final TSStatus status = env.getConfigManager() .getClusterSchemaManager() - .setTableProperties(database, tableName, updatedProperties, isGeneratedByPipe); + .setTableProperties( + database, tableName, updatedProperties, isGeneratedByPipe, originClusterId); if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { setFailure(new ProcedureException(new IoTDBException(status.getMessage(), status.getCode()))); } else { @@ -193,7 +205,8 @@ private void rollbackSetProperties(final ConfigNodeProcedureEnv env) { final TSStatus status = env.getConfigManager() .getClusterSchemaManager() - .setTableProperties(database, tableName, originalProperties, isGeneratedByPipe); + .setTableProperties( + database, tableName, originalProperties, isGeneratedByPipe, originClusterId); if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { setFailure(new ProcedureException(new IoTDBException(status.getMessage(), status.getCode()))); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/sync/AuthOperationProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/sync/AuthOperationProcedure.java index 9d4fb61f85de1..8fe60b2250e40 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/sync/AuthOperationProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/sync/AuthOperationProcedure.java @@ -87,6 +87,19 @@ public AuthOperationProcedure( this.timeoutMS = commonConfig.getDatanodeTokenTimeoutMS(); } + public AuthOperationProcedure( + AuthorPlan plan, + List alldns, + boolean isGeneratedByPipe, + String originClusterId) { + super(isGeneratedByPipe, originClusterId); + this.user = plan.getUserName(); + this.role = plan.getRoleName(); + this.plan = plan; + this.datanodes = alldns; + this.timeoutMS = commonConfig.getDatanodeTokenTimeoutMS(); + } + @Override protected Flow executeFromState(ConfigNodeProcedureEnv env, AuthOperationProcedureState state) { try { @@ -147,7 +160,7 @@ private void writePlan(ConfigNodeProcedureEnv env) { res = env.getConfigManager() .getConsensusManager() - .write(isGeneratedByPipe ? new PipeEnrichedPlan(plan) : plan); + .write(isGeneratedByPipe ? new PipeEnrichedPlan(plan, originClusterId) : plan); } catch (ConsensusException e) { LOGGER.warn(CONSENSUS_WRITE_ERROR, e); res = new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()); @@ -254,11 +267,13 @@ public boolean equals(Object o) { && Objects.equals(plan, that.plan) && Objects.equals(dataNodesToInvalid, that.dataNodesToInvalid) && Objects.equals(datanodes, that.datanodes) - && Objects.equals(isGeneratedByPipe, that.isGeneratedByPipe); + && Objects.equals(isGeneratedByPipe, that.isGeneratedByPipe) + && Objects.equals(originClusterId, that.originClusterId); } @Override public int hashCode() { - return Objects.hash(plan, timeoutMS, dataNodesToInvalid, datanodes, isGeneratedByPipe); + return Objects.hash( + plan, timeoutMS, dataNodesToInvalid, datanodes, isGeneratedByPipe, originClusterId); } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/trigger/DropTriggerProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/trigger/DropTriggerProcedure.java index 42a9104e0df37..a32259d57b470 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/trigger/DropTriggerProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/trigger/DropTriggerProcedure.java @@ -104,7 +104,8 @@ protected Flow executeFromState(ConfigNodeProcedureEnv env, DropTriggerState sta .getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan(new DeleteTriggerInTablePlan(triggerName)) + ? new PipeEnrichedPlan( + new DeleteTriggerInTablePlan(triggerName), originClusterId) : new DeleteTriggerInTablePlan(triggerName)); setNextState(DropTriggerState.CONFIG_NODE_DROPPED); break; @@ -187,7 +188,8 @@ public boolean equals(Object that) { && thatProc.getCurrentState().equals(this.getCurrentState()) && thatProc.getCycles() == this.getCycles() && thatProc.isGeneratedByPipe == this.isGeneratedByPipe - && (thatProc.triggerName).equals(this.triggerName); + && (thatProc.triggerName).equals(this.triggerName) + && Objects.equals(thatProc.originClusterId, this.originClusterId); } return false; } @@ -195,6 +197,11 @@ public boolean equals(Object that) { @Override public int hashCode() { return Objects.hash( - getProcId(), getCurrentState(), getCycles(), isGeneratedByPipe, triggerName); + getProcId(), + getCurrentState(), + getCycles(), + isGeneratedByPipe, + triggerName, + originClusterId); } } diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/ConfigRegionListeningQueueTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/ConfigRegionListeningQueueTest.java index f25dbe0bc9878..03038a7a8c2fb 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/ConfigRegionListeningQueueTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/ConfigRegionListeningQueueTest.java @@ -81,8 +81,8 @@ public void testSnapshot() throws TException, IOException, AuthException { false, new ArrayList<>())); - PipeConfigNodeAgent.runtime().listener().tryListenToPlan(plan1, false); - PipeConfigNodeAgent.runtime().listener().tryListenToPlan(plan2, false); + PipeConfigNodeAgent.runtime().listener().tryListenToPlan(plan1, false, null); + PipeConfigNodeAgent.runtime().listener().tryListenToPlan(plan2, false, null); // tryListenToSnapshots() cannot be tested here since we cannot operate the reference count of // the original or deserialized events diff --git a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift index 148908a46700c..e10e220ed61e7 100644 --- a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift +++ b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift @@ -161,6 +161,7 @@ struct TSetDataNodeStatusReq { struct TDeleteDatabaseReq { 1: required string prefixPath 2: optional bool isGeneratedByPipe + 3: optional string originClusterId } struct TDeleteDatabasesReq { @@ -541,6 +542,7 @@ struct TCreateTriggerReq { 10: optional binary jarFile 11: optional string jarMD5 12: optional bool isGeneratedByPipe + 13: optional string originClusterId } struct TDropTriggerReq { @@ -881,6 +883,7 @@ struct TAlterLogicalViewReq { 1: required string queryId 2: required binary viewBinary 3: optional bool isGeneratedByPipe + 4: optional string originClusterId } // Subscription topic diff --git a/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift b/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift index 661e12526d367..22d987ee68d27 100644 --- a/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift +++ b/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift @@ -386,6 +386,7 @@ struct TLoadCommandReq { 3: optional bool isGeneratedByPipe 4: optional binary progressIndex 5: optional list regionIds + 6: optional string originClusterId } struct TAttributeUpdateReq { @@ -439,7 +440,7 @@ struct TDeleteTimeSeriesReq { 1: required list schemaRegionIdList 2: required binary pathPatternTree 3: optional bool isGeneratedByPipe - 4: optional string originClusterId + 4: optional string originCluster } struct TConstructSchemaBlackListWithTemplateReq { @@ -456,6 +457,7 @@ struct TDeactivateTemplateReq { 1: required list schemaRegionIdList 2: required map> templateSetInfo 3: optional bool isGeneratedByPipe + 4: optional string originClusterId } struct TCountPathsUsingTemplateReq { @@ -572,12 +574,14 @@ struct TDeleteViewSchemaReq { 1: required list schemaRegionIdList 2: required binary pathPatternTree 3: optional bool isGeneratedByPipe + 4: optional string originClusterId } struct TAlterViewReq { 1: required list schemaRegionIdList 2: required list viewBinaryList 3: optional bool isGeneratedByPipe + 4: optional string originClusterId } // ==================================================== From 1a1cb5ec88fe8f2cef1ff5130ab05469b3d443b6 Mon Sep 17 00:00:00 2001 From: XNX02 <202100130044@mail.sdu.edu.cn> Date: Tue, 11 Mar 2025 23:50:46 +0800 Subject: [PATCH 08/27] update configregionxtratctor --- .../manager/pipe/extractor/IoTDBConfigRegionExtractor.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/IoTDBConfigRegionExtractor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/IoTDBConfigRegionExtractor.java index ace61b9531476..a870e6317f141 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/IoTDBConfigRegionExtractor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/IoTDBConfigRegionExtractor.java @@ -229,7 +229,8 @@ protected Optional trimRealtimeEventByPrivilege( TABLE_PRIVILEGE_PARSE_VISITOR.process(plan, userName); if (result.isPresent()) { return Optional.of( - new PipeConfigRegionWritePlanEvent(result.get(), event.isGeneratedByPipe())); + new PipeConfigRegionWritePlanEvent( + result.get(), event.isGeneratedByPipe(), event.getOriginClusterId())); } if (skipIfNoPrivileges) { return Optional.empty(); @@ -246,7 +247,8 @@ protected Optional trimRealtimeEventByPipePattern( tablePattern) .map( configPhysicalPlan -> - new PipeConfigRegionWritePlanEvent(configPhysicalPlan, event.isGeneratedByPipe())); + new PipeConfigRegionWritePlanEvent( + configPhysicalPlan, event.isGeneratedByPipe(), event.getOriginClusterId())); } public static Optional parseConfigPlan( From 998f4919b4785bd02241fdbe92d3ca9851cd6059 Mon Sep 17 00:00:00 2001 From: XNX02 <202100130044@mail.sdu.edu.cn> Date: Wed, 12 Mar 2025 04:01:07 +0800 Subject: [PATCH 09/27] fix clusterId --- .../dataregion/DataExecutionVisitor.java | 1 + .../protocol/thrift/IoTDBDataNodeReceiver.java | 18 ++++++++++-------- .../plan/planner/LogicalPlanVisitor.java | 8 +++++--- .../plan/planner/TreeModelPlanner.java | 3 +-- .../node/pipe/PipeEnrichedDeleteDataNode.java | 12 ++++++++++++ .../plan/node/pipe/PipeEnrichedInsertNode.java | 12 ++++++++++++ .../pipe/PipeEnrichedNonWritePlanNode.java | 5 +++++ .../node/pipe/PipeEnrichedWritePlanNode.java | 7 +++++++ .../node/write/InsertMultiTabletsNode.java | 17 ++++++++++++++++- .../plan/node/write/InsertRowsNode.java | 15 ++++++++++++++- .../node/write/InsertRowsOfOneDeviceNode.java | 6 ++++++ .../statement/pipe/PipeEnrichedStatement.java | 12 ++++++++++++ .../storageengine/dataregion/DataRegion.java | 1 + 13 files changed, 102 insertions(+), 15 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/statemachine/dataregion/DataExecutionVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/statemachine/dataregion/DataExecutionVisitor.java index 43d58119b0029..772af236e4886 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/statemachine/dataregion/DataExecutionVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/statemachine/dataregion/DataExecutionVisitor.java @@ -285,6 +285,7 @@ public TSStatus visitDeleteData( public TSStatus visitPipeEnrichedDeleteDataNode( final PipeEnrichedDeleteDataNode node, final DataRegion context) { node.getDeleteDataNode().markAsGeneratedByPipe(); + node.getDeleteDataNode().setOriginClusterId(node.getOriginClusterId()); return node.getDeleteDataNode().accept(this, context); } } 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 d3f8ac984f1ee..0cc116f0e0ab2 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 @@ -440,7 +440,6 @@ public synchronized TPipeTransferResp receive(final TPipeTransferReq req) { private TPipeTransferResp handleTransferTabletInsertNode( final PipeTransferTabletInsertNodeReq req) { final InsertBaseStatement statement = req.constructStatement(); - statement.setOriginClusterId(clusterIdFromHandshakeRequest); return new TPipeTransferResp( statement.isEmpty() ? RpcUtils.SUCCESS_STATUS @@ -449,7 +448,6 @@ private TPipeTransferResp handleTransferTabletInsertNode( private TPipeTransferResp handleTransferTabletBinary(final PipeTransferTabletBinaryReq req) { final InsertBaseStatement statement = req.constructStatement(); - statement.setOriginClusterId(clusterIdFromHandshakeRequest); return new TPipeTransferResp( statement.isEmpty() ? RpcUtils.SUCCESS_STATUS @@ -458,7 +456,6 @@ private TPipeTransferResp handleTransferTabletBinary(final PipeTransferTabletBin private TPipeTransferResp handleTransferTabletRaw(final PipeTransferTabletRawReq req) { final InsertTabletStatement statement = req.constructStatement(); - statement.setOriginClusterId(clusterIdFromHandshakeRequest); return new TPipeTransferResp( statement.isEmpty() ? RpcUtils.SUCCESS_STATUS @@ -599,7 +596,6 @@ private TSStatus loadTsFileSync(final String dataBaseName, final String fileAbso statement.setVerifySchema(validateTsFile.get()); statement.setAutoCreateDatabase(false); statement.setDatabase(dataBaseName); - statement.setOriginClusterId(clusterIdFromHandshakeRequest); return executeStatementAndClassifyExceptions(statement); } @@ -695,6 +691,7 @@ private TPipeTransferResp handleTransferSchemaPlan(final PipeTransferPlanNodeReq .alterLogicalViewByPipe((AlterLogicalViewNode) req.getPlanNode())); } final Object statement = PLAN_TO_STATEMENT_VISITOR.process(req.getPlanNode(), null); + return statement instanceof Statement ? new TPipeTransferResp(executeStatementAndClassifyExceptions((Statement) statement)) : new TPipeTransferResp( @@ -749,7 +746,6 @@ private TPipeTransferResp handleTransferSlice(final PipeTransferSliceReq pipeTra * message field. */ private TSStatus executeStatementAndAddRedirectInfo(final InsertBaseStatement statement) { - statement.setOriginClusterId(clusterIdFromHandshakeRequest); final TSStatus result = executeStatementAndClassifyExceptions(statement); if (result.getCode() == TSStatusCode.REDIRECTION_RECOMMEND.getStatusCode() @@ -912,7 +908,9 @@ private TSStatus executeStatementForTableModel( return Coordinator.getInstance() .executeForTableModel( - shouldMarkAsPipeRequest.get() ? new PipeEnrichedStatement(statement) : statement, + shouldMarkAsPipeRequest.get() + ? new PipeEnrichedStatement(statement, clusterIdFromHandshakeRequest) + : statement, relationalSqlParser, SESSION_MANAGER.getCurrSession(), SESSION_MANAGER.requestQueryId(), @@ -936,7 +934,9 @@ private TSStatus executeStatementForTableModel( // Retry after creating the database return Coordinator.getInstance() .executeForTableModel( - shouldMarkAsPipeRequest.get() ? new PipeEnrichedStatement(statement) : statement, + shouldMarkAsPipeRequest.get() + ? new PipeEnrichedStatement(statement, clusterIdFromHandshakeRequest) + : statement, relationalSqlParser, SESSION_MANAGER.getCurrSession(), SESSION_MANAGER.requestQueryId(), @@ -987,7 +987,9 @@ private void autoCreateDatabaseIfNecessary(final String database) { private TSStatus executeStatementForTreeModel(final Statement statement) { return Coordinator.getInstance() .executeForTreeModel( - shouldMarkAsPipeRequest.get() ? new PipeEnrichedStatement(statement) : statement, + shouldMarkAsPipeRequest.get() + ? new PipeEnrichedStatement(statement, clusterIdFromHandshakeRequest) + : statement, SESSION_MANAGER.requestQueryId(), SESSION_MANAGER.getSessionInfo(SESSION_MANAGER.getCurrSession()), "", diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanVisitor.java index 36c07e7f2f893..82ffa7698fa34 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/LogicalPlanVisitor.java @@ -504,12 +504,14 @@ public PlanNode visitPipeEnrichedStatement( if (node instanceof LoadTsFileNode) { return node; } else if (node instanceof InsertNode) { - return new PipeEnrichedInsertNode((InsertNode) node); + return new PipeEnrichedInsertNode( + (InsertNode) node, pipeEnrichedStatement.getOriginClusterId()); } else if (node instanceof DeleteDataNode) { - return new PipeEnrichedDeleteDataNode((DeleteDataNode) node); + return new PipeEnrichedDeleteDataNode( + (DeleteDataNode) node, pipeEnrichedStatement.getOriginClusterId()); } - return new PipeEnrichedWritePlanNode(node); + return new PipeEnrichedWritePlanNode(node, pipeEnrichedStatement.getOriginClusterId()); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TreeModelPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TreeModelPlanner.java index d5ae132621a7b..c3d75a7328101 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TreeModelPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TreeModelPlanner.java @@ -142,8 +142,7 @@ public IScheduler doSchedule( syncInternalServiceClientManager, partitionFetcher, true, - ((LoadTsFileStatement) ((PipeEnrichedStatement) statement).getInnerStatement()) - .getOriginClusterId()) + ((PipeEnrichedStatement) statement).getOriginClusterId()) : new LoadTsFileScheduler( distributedPlan, context, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedDeleteDataNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedDeleteDataNode.java index 973e71e97b5ac..285ce19ad3970 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedDeleteDataNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedDeleteDataNode.java @@ -58,6 +58,13 @@ public PipeEnrichedDeleteDataNode(final AbstractDeleteDataNode deleteDataNode) { this.deleteDataNode = deleteDataNode; } + public PipeEnrichedDeleteDataNode( + final AbstractDeleteDataNode deleteDataNode, final String originClusterId) { + super(deleteDataNode.getPlanNodeId()); + this.deleteDataNode = deleteDataNode; + this.originClusterId = originClusterId; + } + public PlanNode getDeleteDataNode() { return deleteDataNode; } @@ -72,6 +79,11 @@ public void markAsGeneratedByPipe() { deleteDataNode.markAsGeneratedByPipe(); } + @Override + public void setOriginClusterId(final String originClusterId) { + deleteDataNode.setOriginClusterId(originClusterId); + } + @Override public PlanNodeId getPlanNodeId() { return deleteDataNode.getPlanNodeId(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedInsertNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedInsertNode.java index 102222c77eab3..62eeb6edd27c8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedInsertNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedInsertNode.java @@ -63,6 +63,13 @@ public PipeEnrichedInsertNode(final InsertNode insertNode) { this.insertNode = insertNode; } + public PipeEnrichedInsertNode(final InsertNode insertNode, final String originClusterId) { + super(insertNode.getPlanNodeId()); + this.insertNode = insertNode; + this.originClusterId = originClusterId; + insertNode.setOriginClusterId(originClusterId); + } + public InsertNode getInsertNode() { return insertNode; } @@ -77,6 +84,11 @@ public void markAsGeneratedByPipe() { insertNode.markAsGeneratedByPipe(); } + @Override + public void setOriginClusterId(final String originClusterId) { + insertNode.setOriginClusterId(originClusterId); + } + @Override public PlanNodeId getPlanNodeId() { return insertNode.getPlanNodeId(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedNonWritePlanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedNonWritePlanNode.java index 5d4c5eef5cd7e..c81565fb7b04f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedNonWritePlanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedNonWritePlanNode.java @@ -81,6 +81,11 @@ public void markAsGeneratedByPipe() { nonWritePlanNode.markAsGeneratedByPipe(); } + @Override + public void setOriginClusterId(final String originClusterId) { + nonWritePlanNode.setOriginClusterId(originClusterId); + } + @Override public PlanNodeId getPlanNodeId() { return nonWritePlanNode.getPlanNodeId(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedWritePlanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedWritePlanNode.java index e8ba967b26ea4..c01fa0a4bd663 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedWritePlanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedWritePlanNode.java @@ -81,6 +81,13 @@ public PipeEnrichedWritePlanNode(final WritePlanNode schemaWriteNode) { this.writePlanNode = schemaWriteNode; } + public PipeEnrichedWritePlanNode( + final WritePlanNode schemaWriteNode, final String originClusterId) { + super(schemaWriteNode.getPlanNodeId()); + this.writePlanNode = schemaWriteNode; + this.originClusterId = originClusterId; + } + public WritePlanNode getWritePlanNode() { return writePlanNode; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertMultiTabletsNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertMultiTabletsNode.java index eaf0a0ef2c112..35cc148327864 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertMultiTabletsNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertMultiTabletsNode.java @@ -112,6 +112,13 @@ public InsertMultiTabletsNode( this.insertTabletNodeList = insertTabletNodeList; } + public InsertMultiTabletsNode(PlanNodeId id, String originClusterId) { + super(id); + parentInsertTabletNodeIndexList = new ArrayList<>(); + insertTabletNodeList = new ArrayList<>(); + this.originClusterId = originClusterId; + } + public List getParentInsertTabletNodeIndexList() { return parentInsertTabletNodeIndexList; } @@ -151,7 +158,9 @@ public List splitByPartition(IAnalysis analysis) { if (tmpNode != null) { tmpNode.addInsertTabletNode((InsertTabletNode) subNode, i); } else { - tmpNode = new InsertMultiTabletsNode(this.getPlanNodeId()); + tmpNode = + new InsertMultiTabletsNode( + this.getPlanNodeId(), insertTabletNode.getOriginClusterId()); tmpNode.setDataRegionReplicaSet(dataRegionReplicaSet); tmpNode.addInsertTabletNode((InsertTabletNode) subNode, i); splitMap.put(dataRegionReplicaSet, tmpNode); @@ -262,6 +271,12 @@ public void markAsGeneratedByRemoteConsensusLeader() { insertTabletNodeList.forEach(InsertTabletNode::markAsGeneratedByRemoteConsensusLeader); } + @Override + public void setOriginClusterId(final String originClusterId) { + this.originClusterId = originClusterId; + insertTabletNodeList.forEach(node -> node.setOriginClusterId(originClusterId)); + } + @Override public boolean equals(Object o) { if (this == o) return true; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowsNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowsNode.java index d34be33dbaf43..ebca30e5bb037 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowsNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowsNode.java @@ -73,6 +73,13 @@ public InsertRowsNode(PlanNodeId id) { insertRowNodeIndexList = new ArrayList<>(); } + public InsertRowsNode(PlanNodeId id, String originClusterId) { + super(id); + insertRowNodeList = new ArrayList<>(); + insertRowNodeIndexList = new ArrayList<>(); + this.originClusterId = originClusterId; + } + @Override public InsertNode mergeInsertNode(List insertNodes) { List list = new ArrayList<>(); @@ -261,6 +268,12 @@ public void markAsGeneratedByRemoteConsensusLeader() { insertRowNodeList.forEach(InsertRowNode::markAsGeneratedByRemoteConsensusLeader); } + @Override + public void setOriginClusterId(final String originClusterId) { + this.originClusterId = originClusterId; + insertRowNodeList.forEach(insertRowNode -> insertRowNode.setOriginClusterId(originClusterId)); + } + @Override public List splitByPartition(IAnalysis analysis) { Map splitMap = new HashMap<>(); @@ -282,7 +295,7 @@ public List splitByPartition(IAnalysis analysis) { if (tmpNode != null) { tmpNode.addOneInsertRowNode(insertRowNode, i); } else { - tmpNode = new InsertRowsNode(this.getPlanNodeId()); + tmpNode = new InsertRowsNode(this.getPlanNodeId(), insertRowNode.getOriginClusterId()); tmpNode.setDataRegionReplicaSet(dataRegionReplicaSet); tmpNode.addOneInsertRowNode(insertRowNode, i); splitMap.put(dataRegionReplicaSet, tmpNode); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowsOfOneDeviceNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowsOfOneDeviceNode.java index 14c28b4de5efb..aabeb681858b0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowsOfOneDeviceNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowsOfOneDeviceNode.java @@ -307,6 +307,12 @@ public void markAsGeneratedByRemoteConsensusLeader() { insertRowNodeList.forEach(InsertRowNode::markAsGeneratedByRemoteConsensusLeader); } + @Override + public void setOriginClusterId(final String originClusterId) { + this.originClusterId = originClusterId; + insertRowNodeList.forEach(insertRowNode -> insertRowNode.setOriginClusterId(originClusterId)); + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/pipe/PipeEnrichedStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/pipe/PipeEnrichedStatement.java index c9c125a981efa..77f80a847464c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/pipe/PipeEnrichedStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/pipe/PipeEnrichedStatement.java @@ -37,11 +37,19 @@ public class PipeEnrichedStatement extends Statement { private Statement innerStatement; + private String originClusterId; + public PipeEnrichedStatement(final Statement innerStatement) { statementType = StatementType.PIPE_ENRICHED; this.innerStatement = innerStatement; } + public PipeEnrichedStatement(final Statement innerStatement, final String originClusterId) { + statementType = StatementType.PIPE_ENRICHED; + this.innerStatement = innerStatement; + this.originClusterId = originClusterId; + } + public Statement getInnerStatement() { return innerStatement; } @@ -50,6 +58,10 @@ public void setInnerStatement(final Statement innerStatement) { this.innerStatement = innerStatement; } + public String getOriginClusterId() { + return originClusterId; + } + @Override public R accept(final StatementVisitor visitor, final C context) { return visitor.visitPipeEnrichedStatement(this, context); 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 86df070fad298..e23179efc1050 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 @@ -1419,6 +1419,7 @@ private List insertToTsFileProcessors( if (insertRowNode.isGeneratedByRemoteConsensusLeader()) { v.markAsGeneratedByRemoteConsensusLeader(); } + v.setOriginClusterId(insertRowsNode.getOriginClusterId()); } if (v.isAligned() != insertRowNode.isAligned()) { v.setMixingAlignment(true); From 3d828a3950690acf2ed6c6021ae48c90707b9c21 Mon Sep 17 00:00:00 2001 From: XNX02 <202100130044@mail.sdu.edu.cn> Date: Fri, 14 Mar 2025 14:28:09 +0800 Subject: [PATCH 10/27] update --- .../iotdb/confignode/manager/ConfigManager.java | 2 +- .../statemachine/dataregion/DataExecutionVisitor.java | 1 + .../schemaregion/SchemaRegionListeningQueue.java | 2 +- .../execution/executor/RegionWriteExecutor.java | 1 + .../plan/node/pipe/PipeEnrichedDeleteDataNode.java | 2 +- .../plan/node/pipe/PipeEnrichedInsertNode.java | 3 +-- .../plan/node/pipe/PipeEnrichedWritePlanNode.java | 7 ++++++- .../plan/node/write/InsertMultiTabletsNode.java | 11 +---------- .../plan/planner/plan/node/write/InsertRowsNode.java | 9 +-------- .../iotdb/db/storageengine/dataregion/DataRegion.java | 3 ++- .../src/main/thrift/confignode.thrift | 1 + 11 files changed, 17 insertions(+), 25 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 4628746ddde36..253a5a3a43879 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 @@ -2354,7 +2354,7 @@ public TPipeConfigTransferResp handleTransferConfigPlan(TPipeConfigTransferReq r .setType(req.type) .setBody(req.body) : new TPipeTransferReq(req.version, req.type, req.body)); - return new TPipeConfigTransferResp(result.status).setBody(result.body); + return new TPipeConfigTransferResp(result.status).setBody(result.body).setClusterId(result.clusterId); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/statemachine/dataregion/DataExecutionVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/statemachine/dataregion/DataExecutionVisitor.java index 772af236e4886..2dd565b7066d9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/statemachine/dataregion/DataExecutionVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/statemachine/dataregion/DataExecutionVisitor.java @@ -237,6 +237,7 @@ public TSStatus visitInsertRowsOfOneDevice( @Override public TSStatus visitPipeEnrichedInsertNode(PipeEnrichedInsertNode node, DataRegion context) { node.getInsertNode().markAsGeneratedByPipe(); + node.getInsertNode().setOriginClusterId(node.getOriginClusterId()); return node.getInsertNode().accept(this, context); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/schemaregion/SchemaRegionListeningQueue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/schemaregion/SchemaRegionListeningQueue.java index 130ab3af8cc04..677eccc29cd26 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/schemaregion/SchemaRegionListeningQueue.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/schemaregion/SchemaRegionListeningQueue.java @@ -56,7 +56,7 @@ public synchronized void tryListenToNode(final PlanNode node) { new PipeSchemaRegionWritePlanEvent( ((PipeEnrichedWritePlanNode) node).getWritePlanNode(), true, - node.getOriginClusterId()); + ((PipeEnrichedWritePlanNode) node).getWritePlanNode().getOriginClusterId()); break; case PIPE_ENRICHED_NON_WRITE: event = diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/executor/RegionWriteExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/executor/RegionWriteExecutor.java index e9fac5c1d32a2..6bb47fade6476 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/executor/RegionWriteExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/executor/RegionWriteExecutor.java @@ -1033,6 +1033,7 @@ private RegionExecutionResult executeCreateOrUpdateTableDevice( @Override public RegionExecutionResult visitPipeEnrichedWritePlanNode( final PipeEnrichedWritePlanNode node, final WritePlanNodeExecutionContext context) { + node.setOriginClusterId(node.getOriginClusterId()); return node.getWritePlanNode().accept(pipeExecutionVisitor, context); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedDeleteDataNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedDeleteDataNode.java index 285ce19ad3970..fc5a4db83bca9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedDeleteDataNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedDeleteDataNode.java @@ -195,7 +195,7 @@ public List splitByPartition(final IAnalysis analysis) { plan -> plan instanceof PipeEnrichedDeleteDataNode ? plan - : new PipeEnrichedDeleteDataNode((DeleteDataNode) plan)) + : new PipeEnrichedDeleteDataNode((DeleteDataNode) plan, originClusterId)) .collect(Collectors.toList()); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedInsertNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedInsertNode.java index 62eeb6edd27c8..eb7f7cf490099 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedInsertNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedInsertNode.java @@ -67,7 +67,6 @@ public PipeEnrichedInsertNode(final InsertNode insertNode, final String originCl super(insertNode.getPlanNodeId()); this.insertNode = insertNode; this.originClusterId = originClusterId; - insertNode.setOriginClusterId(originClusterId); } public InsertNode getInsertNode() { @@ -152,7 +151,7 @@ public List splitByPartition(final IAnalysis analysis) { plan -> plan instanceof PipeEnrichedInsertNode ? plan - : new PipeEnrichedInsertNode((InsertNode) plan)) + : new PipeEnrichedInsertNode((InsertNode) plan, originClusterId)) .collect(Collectors.toList()); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedWritePlanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedWritePlanNode.java index c01fa0a4bd663..4961948481c81 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedWritePlanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedWritePlanNode.java @@ -102,6 +102,11 @@ public void markAsGeneratedByPipe() { writePlanNode.markAsGeneratedByPipe(); } + @Override + public void setOriginClusterId(final String originClusterId) { + writePlanNode.setOriginClusterId(originClusterId); + } + @Override public PlanNodeId getPlanNodeId() { return writePlanNode.getPlanNodeId(); @@ -198,7 +203,7 @@ public List splitByPartition(final IAnalysis analysis) { plan -> plan instanceof PipeEnrichedWritePlanNode ? plan - : new PipeEnrichedWritePlanNode(plan)) + : new PipeEnrichedWritePlanNode(plan, originClusterId)) .collect(Collectors.toList()); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertMultiTabletsNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertMultiTabletsNode.java index 35cc148327864..1d1d157d5dd16 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertMultiTabletsNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertMultiTabletsNode.java @@ -112,13 +112,6 @@ public InsertMultiTabletsNode( this.insertTabletNodeList = insertTabletNodeList; } - public InsertMultiTabletsNode(PlanNodeId id, String originClusterId) { - super(id); - parentInsertTabletNodeIndexList = new ArrayList<>(); - insertTabletNodeList = new ArrayList<>(); - this.originClusterId = originClusterId; - } - public List getParentInsertTabletNodeIndexList() { return parentInsertTabletNodeIndexList; } @@ -158,9 +151,7 @@ public List splitByPartition(IAnalysis analysis) { if (tmpNode != null) { tmpNode.addInsertTabletNode((InsertTabletNode) subNode, i); } else { - tmpNode = - new InsertMultiTabletsNode( - this.getPlanNodeId(), insertTabletNode.getOriginClusterId()); + tmpNode = new InsertMultiTabletsNode(this.getPlanNodeId()); tmpNode.setDataRegionReplicaSet(dataRegionReplicaSet); tmpNode.addInsertTabletNode((InsertTabletNode) subNode, i); splitMap.put(dataRegionReplicaSet, tmpNode); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowsNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowsNode.java index ebca30e5bb037..482c1e1546aa0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowsNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowsNode.java @@ -73,13 +73,6 @@ public InsertRowsNode(PlanNodeId id) { insertRowNodeIndexList = new ArrayList<>(); } - public InsertRowsNode(PlanNodeId id, String originClusterId) { - super(id); - insertRowNodeList = new ArrayList<>(); - insertRowNodeIndexList = new ArrayList<>(); - this.originClusterId = originClusterId; - } - @Override public InsertNode mergeInsertNode(List insertNodes) { List list = new ArrayList<>(); @@ -295,7 +288,7 @@ public List splitByPartition(IAnalysis analysis) { if (tmpNode != null) { tmpNode.addOneInsertRowNode(insertRowNode, i); } else { - tmpNode = new InsertRowsNode(this.getPlanNodeId(), insertRowNode.getOriginClusterId()); + tmpNode = new InsertRowsNode(this.getPlanNodeId()); tmpNode.setDataRegionReplicaSet(dataRegionReplicaSet); tmpNode.addOneInsertRowNode(insertRowNode, i); splitMap.put(dataRegionReplicaSet, tmpNode); 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 e23179efc1050..f0b295231f254 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 @@ -1419,7 +1419,7 @@ private List insertToTsFileProcessors( if (insertRowNode.isGeneratedByRemoteConsensusLeader()) { v.markAsGeneratedByRemoteConsensusLeader(); } - v.setOriginClusterId(insertRowsNode.getOriginClusterId()); + v.setOriginClusterId(insertRowNode.getOriginClusterId()); } if (v.isAligned() != insertRowNode.isAligned()) { v.setMixingAlignment(true); @@ -3428,6 +3428,7 @@ public void insert(InsertRowsOfOneDeviceNode insertRowsOfOneDeviceNode) if (insertRowNode.isGeneratedByRemoteConsensusLeader()) { v.markAsGeneratedByRemoteConsensusLeader(); } + v.setOriginClusterId(insertRowNode.getOriginClusterId()); } v.addOneInsertRowNode(insertRowNode, finalI); v.updateProgressIndex(insertRowNode.getProgressIndex()); diff --git a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift index e10e220ed61e7..57bcb0bf1c3bc 100644 --- a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift +++ b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift @@ -863,6 +863,7 @@ struct TPipeConfigTransferReq { struct TPipeConfigTransferResp { 1: required common.TSStatus status 2: optional binary body + 3: optional string clusterId } struct TDeleteTimeSeriesReq { From 1efd28e2a7cea3899f1acb528a6a3fe867ffaedb Mon Sep 17 00:00:00 2001 From: XNX02 <202100130044@mail.sdu.edu.cn> Date: Sat, 15 Mar 2025 23:16:16 +0800 Subject: [PATCH 11/27] update --- .../org/apache/iotdb/confignode/manager/ConfigManager.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) 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 253a5a3a43879..18eb632ffaccc 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 @@ -2354,7 +2354,9 @@ public TPipeConfigTransferResp handleTransferConfigPlan(TPipeConfigTransferReq r .setType(req.type) .setBody(req.body) : new TPipeTransferReq(req.version, req.type, req.body)); - return new TPipeConfigTransferResp(result.status).setBody(result.body).setClusterId(result.clusterId); + return new TPipeConfigTransferResp(result.status) + .setBody(result.body) + .setClusterId(result.clusterId); } @Override From 37aaba1b6bdcb4243f4de9cd7c5c92c5b96171b0 Mon Sep 17 00:00:00 2001 From: XNX02 <202100130044@mail.sdu.edu.cn> Date: Sun, 16 Mar 2025 16:43:51 +0800 Subject: [PATCH 12/27] deletedata --- .../iotdb/confignode/manager/ConfigManager.java | 2 +- .../impl/schema/UnsetTemplateProcedure.java | 3 ++- .../task/connection/PipeEventCollector.java | 3 ++- .../common/deletion/PipeDeleteDataNodeEvent.java | 16 +++++++++++++--- .../event/realtime/PipeRealtimeEventFactory.java | 2 +- ...icalDataRegionTsFileAndDeletionExtractor.java | 3 ++- .../impl/DataNodeInternalRPCServiceImpl.java | 16 +++++++++++----- .../executor/ClusterConfigTaskExecutor.java | 3 ++- .../node/pipe/PipeEnrichedDeleteDataNode.java | 2 +- .../node/pipe/PipeEnrichedNonWritePlanNode.java | 6 ++++++ .../scheduler/load/LoadTsFileDispatcherImpl.java | 1 + .../iotdb/db/storageengine/StorageEngine.java | 3 ++- .../db/storageengine/load/LoadTsFileManager.java | 10 ++++++---- .../src/main/thrift/confignode.thrift | 2 +- 14 files changed, 51 insertions(+), 21 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 18eb632ffaccc..37cdd1e013682 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 @@ -2356,7 +2356,7 @@ public TPipeConfigTransferResp handleTransferConfigPlan(TPipeConfigTransferReq r : new TPipeTransferReq(req.version, req.type, req.body)); return new TPipeConfigTransferResp(result.status) .setBody(result.body) - .setClusterId(result.clusterId); + .setCluster(result.clusterId); } @Override diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/UnsetTemplateProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/UnsetTemplateProcedure.java index f7dc52c9e5e8d..a92098ccb090b 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/UnsetTemplateProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/UnsetTemplateProcedure.java @@ -214,7 +214,8 @@ private void unsetTemplate(final ConfigNodeProcedureEnv env) { final TSStatus status = env.getConfigManager() .getClusterSchemaManager() - .unsetSchemaTemplateInBlackList(template.getId(), path, isGeneratedByPipe,originClusterId); + .unsetSchemaTemplateInBlackList( + template.getId(), path, isGeneratedByPipe, originClusterId); if (status.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { setFailure(new ProcedureException(new IoTDBException(status.getMessage(), status.getCode()))); } 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 51221001a58d5..429efe7489837 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 @@ -192,7 +192,8 @@ private void parseAndCollectEvent(final PipeDeleteDataNodeEvent deleteDataEvent) deleteDataEvent.getTablePattern(), deleteDataEvent.getUserName(), deleteDataEvent.isSkipIfNoPrivileges(), - deleteDataEvent.isGeneratedByPipe())) + deleteDataEvent.isGeneratedByPipe(), + deleteDataEvent.getOriginClusterId())) .ifPresent( event -> { hasNoGeneratedEvent = false; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java index f60fafea5d23d..7f28afac31e22 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java @@ -54,7 +54,14 @@ public PipeDeleteDataNodeEvent() { public PipeDeleteDataNodeEvent( final AbstractDeleteDataNode deleteDataNode, final boolean isGeneratedByPipe) { - this(deleteDataNode, null, 0, null, null, null, null, true, isGeneratedByPipe); + this(deleteDataNode, null, 0, null, null, null, null, true, isGeneratedByPipe, null); + } + + public PipeDeleteDataNodeEvent( + final AbstractDeleteDataNode deleteDataNode, + final boolean isGeneratedByPipe, + final String originClusterId) { + this(deleteDataNode, null, 0, null, null, null, null, true, isGeneratedByPipe, originClusterId); } public PipeDeleteDataNodeEvent( @@ -66,7 +73,8 @@ public PipeDeleteDataNodeEvent( final TablePattern tablePattern, final String userName, final boolean skipIfNoPrivileges, - final boolean isGeneratedByPipe) { + final boolean isGeneratedByPipe, + final String originClusterId) { super( pipeName, creationTime, @@ -79,6 +87,7 @@ public PipeDeleteDataNodeEvent( Long.MAX_VALUE); this.isGeneratedByPipe = isGeneratedByPipe; this.deleteDataNode = deleteDataNode; + this.originClusterId = originClusterId; Optional.ofNullable(deleteDataNode) .ifPresent(node -> this.progressIndex = deleteDataNode.getProgressIndex()); } @@ -138,7 +147,8 @@ public EnrichedEvent shallowCopySelfAndBindPipeTaskMetaForProgressReport( tablePattern, userName, skipIfNoPrivileges, - isGeneratedByPipe); + isGeneratedByPipe, + originClusterId); } @Override 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 37361991a8b5b..e543e83838822 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 @@ -125,7 +125,7 @@ public static PipeRealtimeEvent createRealtimeEvent( public static PipeRealtimeEvent createRealtimeEvent( final String dataRegionId, final AbstractDeleteDataNode node) { PipeDeleteDataNodeEvent deleteDataNodeEvent = - new PipeDeleteDataNodeEvent(node, node.isGeneratedByPipe()); + new PipeDeleteDataNodeEvent(node, node.isGeneratedByPipe(), node.getOriginClusterId()); // if using IoTV2, assign a replicateIndex for this event if (DataRegionConsensusImpl.getInstance() instanceof PipeConsensus diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java index 9236807cc8919..8eb051851d33b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java @@ -871,7 +871,8 @@ private Event supplyDeletionEvent(final DeletionResource deletionResource) { tablePattern, userName, skipIfNoPrivileges, - false); + false, + null); if (sloppyPattern || isDbNameCoveredByPattern) { event.skipParsingPattern(); 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 7a7eb6389f974..210f9d3867737 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 @@ -522,6 +522,7 @@ public TLoadResp sendLoadCommand(TLoadCommandReq req) { LoadTsFileScheduler.LoadCommand.values()[req.commandType], req.uuid, req.isSetIsGeneratedByPipe() && req.isGeneratedByPipe, + req.getOriginClusterId(), progressIndex)); } @@ -715,7 +716,8 @@ public TSStatus deleteDataForDeleteSchema(final TDeleteDataForDeleteSchemaReq re req.isSetIsGeneratedByPipe() && req.isIsGeneratedByPipe() ? new PipeEnrichedDeleteDataNode( new DeleteDataNode( - new PlanNodeId(""), pathList, Long.MIN_VALUE, Long.MAX_VALUE)) + new PlanNodeId(""), pathList, Long.MIN_VALUE, Long.MAX_VALUE), + req.getOriginClusterId()) : new DeleteDataNode( new PlanNodeId(""), pathList, Long.MIN_VALUE, Long.MAX_VALUE)) .getStatus()); @@ -742,7 +744,8 @@ public TSStatus deleteTimeSeries(final TDeleteTimeSeriesReq req) throws TExcepti new SchemaRegionId(consensusGroupId.getId()), req.isSetIsGeneratedByPipe() && req.isIsGeneratedByPipe() ? new PipeEnrichedNonWritePlanNode( - new DeleteTimeSeriesNode(new PlanNodeId(""), filteredPatternTree)) + new DeleteTimeSeriesNode(new PlanNodeId(""), filteredPatternTree), + req.getOriginCluster()) : new DeleteTimeSeriesNode(new PlanNodeId(""), filteredPatternTree)) .getStatus(); }); @@ -868,7 +871,8 @@ public TSStatus deactivateTemplate(TDeactivateTemplateReq req) throws TException new SchemaRegionId(consensusGroupId.getId()), req.isSetIsGeneratedByPipe() && req.isIsGeneratedByPipe() ? new PipeEnrichedNonWritePlanNode( - new DeactivateTemplateNode(new PlanNodeId(""), filteredTemplateSetInfo)) + new DeactivateTemplateNode(new PlanNodeId(""), filteredTemplateSetInfo), + req.getOriginClusterId()) : new DeactivateTemplateNode(new PlanNodeId(""), filteredTemplateSetInfo)) .getStatus(); }); @@ -1081,7 +1085,8 @@ public TSStatus deleteViewSchema(TDeleteViewSchemaReq req) { new SchemaRegionId(consensusGroupId.getId()), req.isSetIsGeneratedByPipe() && req.isIsGeneratedByPipe() ? new PipeEnrichedNonWritePlanNode( - new DeleteLogicalViewNode(new PlanNodeId(""), filteredPatternTree)) + new DeleteLogicalViewNode(new PlanNodeId(""), filteredPatternTree), + req.getOriginClusterId()) : new DeleteLogicalViewNode(new PlanNodeId(""), filteredPatternTree)) .getStatus(); }); @@ -1114,7 +1119,8 @@ public TSStatus alterView(TAlterViewReq req) { req.isSetIsGeneratedByPipe() && req.isIsGeneratedByPipe() ? new PipeEnrichedNonWritePlanNode( new AlterLogicalViewNode( - new PlanNodeId(""), schemaRegionRequestMap.get(consensusGroupId))) + new PlanNodeId(""), schemaRegionRequestMap.get(consensusGroupId)), + req.getOriginClusterId()) : new AlterLogicalViewNode( new PlanNodeId(""), schemaRegionRequestMap.get(consensusGroupId))) .getStatus(); 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 8628d31ef5f83..8317e56ae6d29 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 @@ -3298,7 +3298,8 @@ public TPipeTransferResp handleTransferConfigPlan( LOGGER.warn("Failed to handleTransferConfigPlan, status is {}.", pipeConfigTransferResp); } return new TPipeTransferResp(pipeConfigTransferResp.status) - .setBody(pipeConfigTransferResp.body); + .setBody(pipeConfigTransferResp.body) + .setClusterId(pipeConfigTransferResp.cluster); } catch (Exception e) { return new TPipeTransferResp( new TSStatus(TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedDeleteDataNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedDeleteDataNode.java index fc5a4db83bca9..8b7c4b859fda2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedDeleteDataNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedDeleteDataNode.java @@ -218,6 +218,6 @@ public SearchNode merge(List searchNodes) { (SearchNode) ((PipeEnrichedDeleteDataNode) searchNode).getDeleteDataNode()) .collect(Collectors.toList()); return new PipeEnrichedDeleteDataNode( - (DeleteDataNode) deleteDataNode.merge(unrichedDeleteDataNodes)); + (DeleteDataNode) deleteDataNode.merge(unrichedDeleteDataNodes), originClusterId); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedNonWritePlanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedNonWritePlanNode.java index c81565fb7b04f..bda6fe70ccd75 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedNonWritePlanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedNonWritePlanNode.java @@ -67,6 +67,12 @@ public PipeEnrichedNonWritePlanNode(PlanNode nonWritePlanNode) { this.nonWritePlanNode = nonWritePlanNode; } + public PipeEnrichedNonWritePlanNode(PlanNode nonWritePlanNode, String originClusterId) { + super(nonWritePlanNode.getPlanNodeId()); + this.nonWritePlanNode = nonWritePlanNode; + this.originClusterId = originClusterId; + } + public PlanNode getNonWritePlanNode() { return nonWritePlanNode; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileDispatcherImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileDispatcherImpl.java index 76075930351b3..baa3a494dc50c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileDispatcherImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileDispatcherImpl.java @@ -279,6 +279,7 @@ private void dispatchLocally(TLoadCommandReq loadCommandReq) LoadTsFileScheduler.LoadCommand.values()[loadCommandReq.commandType], loadCommandReq.uuid, loadCommandReq.isSetIsGeneratedByPipe() && loadCommandReq.isGeneratedByPipe, + loadCommandReq.getOriginClusterId(), progressIndex); if (!RpcUtils.SUCCESS_STATUS.equals(resultStatus)) { throw new FragmentInstanceDispatchException(resultStatus); 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 f1f188fcb092c..42e31f8f2e11f 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 @@ -978,13 +978,14 @@ public TSStatus executeLoadCommand( LoadTsFileScheduler.LoadCommand loadCommand, String uuid, boolean isGeneratedByPipe, + String originClusterId, ProgressIndex progressIndex) { TSStatus status = new TSStatus(); try { switch (loadCommand) { case EXECUTE: - if (loadTsFileManager.loadAll(uuid, isGeneratedByPipe, progressIndex)) { + if (loadTsFileManager.loadAll(uuid, isGeneratedByPipe, originClusterId, progressIndex)) { status = RpcUtils.SUCCESS_STATUS; } else { status.setCode(TSStatusCode.LOAD_FILE_ERROR.getStatusCode()); 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 a56a02fbe193b..aa3717dfa6a82 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 @@ -273,7 +273,8 @@ private String getNextFolder() throws DiskSpaceInsufficientException { return FOLDER_MANAGER.get().getNextFolder(); } - public boolean loadAll(String uuid, boolean isGeneratedByPipe, ProgressIndex progressIndex) + public boolean loadAll( + String uuid, boolean isGeneratedByPipe, String originClusterId, ProgressIndex progressIndex) throws IOException, LoadFileException { if (!uuid2WriterManager.containsKey(uuid)) { return false; @@ -282,7 +283,7 @@ public boolean loadAll(String uuid, boolean isGeneratedByPipe, ProgressIndex pro final Optional cleanupTask = Optional.of(uuid2CleanupTask.get(uuid)); cleanupTask.ifPresent(CleanupTask::markLoadTaskRunning); try { - uuid2WriterManager.get(uuid).loadAll(isGeneratedByPipe, progressIndex); + uuid2WriterManager.get(uuid).loadAll(isGeneratedByPipe, originClusterId, progressIndex); } finally { cleanupTask.ifPresent(CleanupTask::markLoadTaskNotRunning); } @@ -497,7 +498,8 @@ private void writeDeletion(DataRegion dataRegion, DeletionData deletionData) } } - private void loadAll(boolean isGeneratedByPipe, ProgressIndex progressIndex) + private void loadAll( + boolean isGeneratedByPipe, String originClusterId, ProgressIndex progressIndex) throws IOException, LoadFileException { if (isClosed) { throw new IOException(String.format(MESSAGE_WRITER_MANAGER_HAS_BEEN_CLOSED, taskDir)); @@ -517,7 +519,7 @@ private void loadAll(boolean isGeneratedByPipe, ProgressIndex progressIndex) final DataRegion dataRegion = entry.getKey().getDataRegion(); final TsFileResource tsFileResource = dataPartition2Resource.get(entry.getKey()); endTsFileResource(writer, tsFileResource, progressIndex); - dataRegion.loadNewTsFile(tsFileResource, true, isGeneratedByPipe, null); + dataRegion.loadNewTsFile(tsFileResource, true, isGeneratedByPipe, originClusterId); // Metrics dataRegion diff --git a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift index 57bcb0bf1c3bc..611c765755c08 100644 --- a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift +++ b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift @@ -863,7 +863,7 @@ struct TPipeConfigTransferReq { struct TPipeConfigTransferResp { 1: required common.TSStatus status 2: optional binary body - 3: optional string clusterId + 3: optional string cluster } struct TDeleteTimeSeriesReq { From edee9e86b26bfa54a4681c8026a6a8fccf51a914 Mon Sep 17 00:00:00 2001 From: XNX02 <202100130044@mail.sdu.edu.cn> Date: Sun, 16 Mar 2025 23:25:24 +0800 Subject: [PATCH 13/27] table model --- .../protocol/thrift/IoTDBDataNodeReceiver.java | 2 +- .../plan/relational/planner/RelationPlanner.java | 6 +++--- .../plan/relational/planner/TableLogicalPlanner.java | 2 +- .../plan/relational/planner/TableModelPlanner.java | 5 ++++- .../plan/relational/sql/ast/PipeEnriched.java | 10 +++++++++- .../plan/statement/pipe/PipeEnrichedStatement.java | 2 +- 6 files changed, 19 insertions(+), 8 deletions(-) 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 0cc116f0e0ab2..c20877339e438 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 @@ -1020,7 +1020,7 @@ private TSStatus executeStatementForTableModelWithPermissionCheck( final TSStatus result = Coordinator.getInstance() .executeForTableModel( - shouldMarkAsPipeRequest.get() ? new PipeEnriched(statement) : statement, + shouldMarkAsPipeRequest.get() ? new PipeEnriched(statement,clusterIdFromHandshakeRequest) : statement, relationalSqlParser, SESSION_MANAGER.getCurrSession(), SESSION_MANAGER.requestQueryId(), diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationPlanner.java index 8d1b1576ea60a..73d7320bce323 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationPlanner.java @@ -767,20 +767,20 @@ protected RelationPlan visitPipeEnriched(final PipeEnriched node, final Void con return relationPlan; } else if (relationPlan.getRoot() instanceof InsertNode) { return new RelationPlan( - new PipeEnrichedInsertNode((InsertNode) relationPlan.getRoot()), + new PipeEnrichedInsertNode((InsertNode) relationPlan.getRoot(),node.getOriginClusterId()), analysis.getRootScope(), Collections.emptyList(), outerContext); } else if (relationPlan.getRoot() instanceof RelationalDeleteDataNode) { return new RelationPlan( - new PipeEnrichedDeleteDataNode((RelationalDeleteDataNode) relationPlan.getRoot()), + new PipeEnrichedDeleteDataNode((RelationalDeleteDataNode) relationPlan.getRoot(),node.getOriginClusterId()), analysis.getRootScope(), Collections.emptyList(), outerContext); } return new RelationPlan( - new PipeEnrichedWritePlanNode((WritePlanNode) relationPlan.getRoot()), + new PipeEnrichedWritePlanNode((WritePlanNode) relationPlan.getRoot(),node.getOriginClusterId()), analysis.getRootScope(), Collections.emptyList(), outerContext); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java index b7cac999941be..6652d96ab824f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java @@ -175,7 +175,7 @@ private PlanNode planStatement(final Analysis analysis, Statement statement) { if (innerStatement instanceof CreateOrUpdateDevice) { return new PipeEnrichedWritePlanNode( (WritePlanNode) - planCreateOrUpdateDevice((CreateOrUpdateDevice) innerStatement, analysis)); + planCreateOrUpdateDevice((CreateOrUpdateDevice) innerStatement, analysis),((PipeEnriched)statement).getOriginClusterId()); } if (innerStatement instanceof Update) { return new PipeEnrichedWritePlanNode( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableModelPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableModelPlanner.java index f8d39ddccf0d8..218dde759eb18 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableModelPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableModelPlanner.java @@ -189,7 +189,10 @@ public IScheduler doSchedule( stateMachine, syncInternalServiceClientManager, ClusterPartitionFetcher.getInstance(), - isPipeEnrichedTsFileLoad); + isPipeEnrichedTsFileLoad, + isPipeEnrichedTsFileLoad + ? ((PipeEnriched) statement).getOriginClusterId() + : null); } else { scheduler = new ClusterScheduler( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/PipeEnriched.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/PipeEnriched.java index 0b494c674b7f9..8988b68bef50f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/PipeEnriched.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/PipeEnriched.java @@ -28,9 +28,12 @@ public class PipeEnriched extends Statement { private final Statement innerStatement; - public PipeEnriched(final @NotNull Statement innerstatement) { + private final String originClusterId; + + public PipeEnriched(final @NotNull Statement innerstatement,final String originCluster) { super(innerstatement.getLocation().isPresent() ? innerstatement.getLocation().get() : null); this.innerStatement = innerstatement; + this.originClusterId = originCluster; } @Override @@ -68,4 +71,9 @@ public String toString() { public Statement getInnerStatement() { return innerStatement; } + + public String getOriginClusterId() { + return originClusterId; + } + } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/pipe/PipeEnrichedStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/pipe/PipeEnrichedStatement.java index 77f80a847464c..5a2e0141a1112 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/pipe/PipeEnrichedStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/pipe/PipeEnrichedStatement.java @@ -92,7 +92,7 @@ public List getPaths() { public org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Statement toRelationalStatement( final MPPQueryContext context) { final PipeEnriched pipeEnriched = - new PipeEnriched(innerStatement.toRelationalStatement(context)); + new PipeEnriched(innerStatement.toRelationalStatement(context),originClusterId); if (pipeEnriched.getInnerStatement() instanceof InsertRows) { ((InsertRows) pipeEnriched.getInnerStatement()).setAllowCreateTable(true); } From 1f0c9b560ecdffcd7682a295067511991148f047 Mon Sep 17 00:00:00 2001 From: XNX02 <202100130044@mail.sdu.edu.cn> Date: Mon, 17 Mar 2025 02:43:12 +0800 Subject: [PATCH 14/27] tsfile simple --- .../thrift/IoTDBDataNodeReceiver.java | 4 ++- .../relational/planner/RelationPlanner.java | 9 ++++--- .../planner/TableLogicalPlanner.java | 3 ++- .../relational/planner/TableModelPlanner.java | 4 +-- .../plan/relational/sql/ast/PipeEnriched.java | 3 +-- .../statement/pipe/PipeEnrichedStatement.java | 2 +- .../dataregion/memtable/AbstractMemTable.java | 22 ++++++++++++++++ .../dataregion/memtable/IMemTable.java | 8 ++++++ .../dataregion/memtable/TsFileProcessor.java | 25 ++++++++++++++++++- 9 files changed, 68 insertions(+), 12 deletions(-) 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 c20877339e438..51656f6a925be 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 @@ -1020,7 +1020,9 @@ private TSStatus executeStatementForTableModelWithPermissionCheck( final TSStatus result = Coordinator.getInstance() .executeForTableModel( - shouldMarkAsPipeRequest.get() ? new PipeEnriched(statement,clusterIdFromHandshakeRequest) : statement, + shouldMarkAsPipeRequest.get() + ? new PipeEnriched(statement, clusterIdFromHandshakeRequest) + : statement, relationalSqlParser, SESSION_MANAGER.getCurrSession(), SESSION_MANAGER.requestQueryId(), diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationPlanner.java index 73d7320bce323..30af505775610 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/RelationPlanner.java @@ -767,20 +767,23 @@ protected RelationPlan visitPipeEnriched(final PipeEnriched node, final Void con return relationPlan; } else if (relationPlan.getRoot() instanceof InsertNode) { return new RelationPlan( - new PipeEnrichedInsertNode((InsertNode) relationPlan.getRoot(),node.getOriginClusterId()), + new PipeEnrichedInsertNode( + (InsertNode) relationPlan.getRoot(), node.getOriginClusterId()), analysis.getRootScope(), Collections.emptyList(), outerContext); } else if (relationPlan.getRoot() instanceof RelationalDeleteDataNode) { return new RelationPlan( - new PipeEnrichedDeleteDataNode((RelationalDeleteDataNode) relationPlan.getRoot(),node.getOriginClusterId()), + new PipeEnrichedDeleteDataNode( + (RelationalDeleteDataNode) relationPlan.getRoot(), node.getOriginClusterId()), analysis.getRootScope(), Collections.emptyList(), outerContext); } return new RelationPlan( - new PipeEnrichedWritePlanNode((WritePlanNode) relationPlan.getRoot(),node.getOriginClusterId()), + new PipeEnrichedWritePlanNode( + (WritePlanNode) relationPlan.getRoot(), node.getOriginClusterId()), analysis.getRootScope(), Collections.emptyList(), outerContext); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java index 6652d96ab824f..43e28fd68dabb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java @@ -175,7 +175,8 @@ private PlanNode planStatement(final Analysis analysis, Statement statement) { if (innerStatement instanceof CreateOrUpdateDevice) { return new PipeEnrichedWritePlanNode( (WritePlanNode) - planCreateOrUpdateDevice((CreateOrUpdateDevice) innerStatement, analysis),((PipeEnriched)statement).getOriginClusterId()); + planCreateOrUpdateDevice((CreateOrUpdateDevice) innerStatement, analysis), + ((PipeEnriched) statement).getOriginClusterId()); } if (innerStatement instanceof Update) { return new PipeEnrichedWritePlanNode( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableModelPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableModelPlanner.java index 218dde759eb18..66cd961a08f2d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableModelPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableModelPlanner.java @@ -190,9 +190,7 @@ public IScheduler doSchedule( syncInternalServiceClientManager, ClusterPartitionFetcher.getInstance(), isPipeEnrichedTsFileLoad, - isPipeEnrichedTsFileLoad - ? ((PipeEnriched) statement).getOriginClusterId() - : null); + isPipeEnrichedTsFileLoad ? ((PipeEnriched) statement).getOriginClusterId() : null); } else { scheduler = new ClusterScheduler( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/PipeEnriched.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/PipeEnriched.java index 8988b68bef50f..49366cd10bd69 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/PipeEnriched.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/PipeEnriched.java @@ -30,7 +30,7 @@ public class PipeEnriched extends Statement { private final String originClusterId; - public PipeEnriched(final @NotNull Statement innerstatement,final String originCluster) { + public PipeEnriched(final @NotNull Statement innerstatement, final String originCluster) { super(innerstatement.getLocation().isPresent() ? innerstatement.getLocation().get() : null); this.innerStatement = innerstatement; this.originClusterId = originCluster; @@ -75,5 +75,4 @@ public Statement getInnerStatement() { public String getOriginClusterId() { return originClusterId; } - } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/pipe/PipeEnrichedStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/pipe/PipeEnrichedStatement.java index 5a2e0141a1112..6bb0f643c980e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/pipe/PipeEnrichedStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/pipe/PipeEnrichedStatement.java @@ -92,7 +92,7 @@ public List getPaths() { public org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Statement toRelationalStatement( final MPPQueryContext context) { final PipeEnriched pipeEnriched = - new PipeEnriched(innerStatement.toRelationalStatement(context),originClusterId); + new PipeEnriched(innerStatement.toRelationalStatement(context), originClusterId); if (pipeEnriched.getInnerStatement() instanceof InsertRows) { ((InsertRows) pipeEnriched.getInnerStatement()).setAllowCreateTable(true); } 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 559d06da63cf6..aed1282be749d 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 @@ -124,6 +124,10 @@ public abstract class AbstractMemTable implements IMemTable { private final AtomicBoolean isTotallyGeneratedByPipe = new AtomicBoolean(true); + private final AtomicBoolean isToTallyFromTheSameCluster = new AtomicBoolean(true); + + private String currentOriginClusterId = "Not Set"; + protected AbstractMemTable() { this.database = null; this.dataRegionId = null; @@ -1023,4 +1027,22 @@ public void markAsNotGeneratedByPipe() { public boolean isTotallyGeneratedByPipe() { return this.isTotallyGeneratedByPipe.get(); } + + @Override + public void markAsNotFromTheSameCluster() { + this.isToTallyFromTheSameCluster.set(false); + } + + @Override + public boolean isTotallyFromTheSameCluster() { + return this.isToTallyFromTheSameCluster.get(); + } + + public String getCurrentOriginClusterId() { + return currentOriginClusterId; + } + + public void setCurrentOriginClusterId(String currentOriginClusterId) { + this.currentOriginClusterId = currentOriginClusterId; + } } 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 44098b69a5640..07bf686f15a58 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 @@ -202,4 +202,12 @@ void queryForDeviceRegionScan( void markAsNotGeneratedByPipe(); boolean isTotallyGeneratedByPipe(); + + void markAsNotFromTheSameCluster(); + + boolean isTotallyFromTheSameCluster(); + + String getCurrentOriginClusterId(); + + void setCurrentOriginClusterId(String currentOriginClusterId); } 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 d430e52466348..09bcc067b4607 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 @@ -103,6 +103,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.CopyOnWriteArrayList; @@ -319,6 +320,13 @@ public void insert(InsertRowNode insertRowNode, long[] infoForMetrics) if (!insertRowNode.isGeneratedByPipe()) { workMemTable.markAsNotGeneratedByPipe(); } + if ("Not Set".equals(workMemTable.getCurrentOriginClusterId())) { + workMemTable.setCurrentOriginClusterId(insertRowNode.getOriginClusterId()); + } else if (!Objects.equals( + insertRowNode.getOriginClusterId(), workMemTable.getCurrentOriginClusterId())) { + workMemTable.markAsNotFromTheSameCluster(); + } + PipeInsertionDataNodeListener.getInstance() .listenToInsertNode( dataRegionInfo.getDataRegion().getDataRegionId(), @@ -410,6 +418,12 @@ public void insertRows(InsertRowsNode insertRowsNode, long[] infoForMetrics) if (!insertRowsNode.isGeneratedByPipe()) { workMemTable.markAsNotGeneratedByPipe(); } + if ("Not Set".equals(workMemTable.getCurrentOriginClusterId())) { + workMemTable.setCurrentOriginClusterId(insertRowsNode.getOriginClusterId()); + } else if (!Objects.equals( + insertRowsNode.getOriginClusterId(), workMemTable.getCurrentOriginClusterId())) { + workMemTable.markAsNotFromTheSameCluster(); + } PipeInsertionDataNodeListener.getInstance() .listenToInsertNode( dataRegionInfo.getDataRegion().getDataRegionId(), @@ -577,6 +591,13 @@ public void insertTablet( if (!insertTabletNode.isGeneratedByPipe()) { workMemTable.markAsNotGeneratedByPipe(); } + if ("Not Set".equals(workMemTable.getCurrentOriginClusterId())) { + workMemTable.setCurrentOriginClusterId(insertTabletNode.getOriginClusterId()); + } else if (!Objects.equals( + insertTabletNode.getOriginClusterId(), workMemTable.getCurrentOriginClusterId())) { + workMemTable.markAsNotFromTheSameCluster(); + } + PipeInsertionDataNodeListener.getInstance() .listenToInsertNode( dataRegionInfo.getDataRegion().getDataRegionId(), @@ -1292,7 +1313,9 @@ public Future asyncClose() { tsFileResource, false, tmpMemTable.isTotallyGeneratedByPipe(), - null); + tmpMemTable.isTotallyFromTheSameCluster() + ? tmpMemTable.getCurrentOriginClusterId() + : null); // When invoke closing TsFile after insert data to memTable, we shouldn't flush until invoke // flushing memTable in System module. From 02fbfe94085ab406e417181b50fd28eab6576c05 Mon Sep 17 00:00:00 2001 From: XNX02 <202100130044@mail.sdu.edu.cn> Date: Mon, 17 Mar 2025 16:05:53 +0800 Subject: [PATCH 15/27] update --- .../dataregion/memtable/AbstractMemTable.java | 2 +- .../dataregion/memtable/TsFileProcessor.java | 33 ++++++++++++------- 2 files changed, 22 insertions(+), 13 deletions(-) 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 aed1282be749d..ce22f4f02dbc0 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 @@ -126,7 +126,7 @@ public abstract class AbstractMemTable implements IMemTable { private final AtomicBoolean isToTallyFromTheSameCluster = new AtomicBoolean(true); - private String currentOriginClusterId = "Not Set"; + private String currentOriginClusterId = null; protected AbstractMemTable() { this.database = null; 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 09bcc067b4607..4003c3eacd0a3 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,10 +320,13 @@ public void insert(InsertRowNode insertRowNode, long[] infoForMetrics) if (!insertRowNode.isGeneratedByPipe()) { workMemTable.markAsNotGeneratedByPipe(); } - if ("Not Set".equals(workMemTable.getCurrentOriginClusterId())) { - workMemTable.setCurrentOriginClusterId(insertRowNode.getOriginClusterId()); - } else if (!Objects.equals( - insertRowNode.getOriginClusterId(), workMemTable.getCurrentOriginClusterId())) { + final String originClusterId = + insertRowNode.getOriginClusterId() == null + ? config.getClusterId() + : insertRowNode.getOriginClusterId(); + if (Objects.isNull(workMemTable.getCurrentOriginClusterId())) { + workMemTable.setCurrentOriginClusterId(originClusterId); + } else if (!Objects.equals(originClusterId, workMemTable.getCurrentOriginClusterId())) { workMemTable.markAsNotFromTheSameCluster(); } @@ -418,10 +421,13 @@ public void insertRows(InsertRowsNode insertRowsNode, long[] infoForMetrics) if (!insertRowsNode.isGeneratedByPipe()) { workMemTable.markAsNotGeneratedByPipe(); } - if ("Not Set".equals(workMemTable.getCurrentOriginClusterId())) { - workMemTable.setCurrentOriginClusterId(insertRowsNode.getOriginClusterId()); - } else if (!Objects.equals( - insertRowsNode.getOriginClusterId(), workMemTable.getCurrentOriginClusterId())) { + final String originClusterId = + insertRowsNode.getOriginClusterId() == null + ? config.getClusterId() + : insertRowsNode.getOriginClusterId(); + if (Objects.isNull(workMemTable.getCurrentOriginClusterId())) { + workMemTable.setCurrentOriginClusterId(originClusterId); + } else if (!Objects.equals(originClusterId, workMemTable.getCurrentOriginClusterId())) { workMemTable.markAsNotFromTheSameCluster(); } PipeInsertionDataNodeListener.getInstance() @@ -591,10 +597,13 @@ public void insertTablet( if (!insertTabletNode.isGeneratedByPipe()) { workMemTable.markAsNotGeneratedByPipe(); } - if ("Not Set".equals(workMemTable.getCurrentOriginClusterId())) { - workMemTable.setCurrentOriginClusterId(insertTabletNode.getOriginClusterId()); - } else if (!Objects.equals( - insertTabletNode.getOriginClusterId(), workMemTable.getCurrentOriginClusterId())) { + final String originClusterId = + insertTabletNode.getOriginClusterId() == null + ? config.getClusterId() + : insertTabletNode.getOriginClusterId(); + if (Objects.isNull(workMemTable.getCurrentOriginClusterId())) { + workMemTable.setCurrentOriginClusterId(originClusterId); + } else if (!Objects.equals(originClusterId, workMemTable.getCurrentOriginClusterId())) { workMemTable.markAsNotFromTheSameCluster(); } From 4df0f9da523cf2bee590db94408b733e3045734e Mon Sep 17 00:00:00 2001 From: XNX02 <202100130044@mail.sdu.edu.cn> Date: Mon, 17 Mar 2025 22:35:56 +0800 Subject: [PATCH 16/27] improve --- .../confignode/manager/ProcedureManager.java | 10 ++++--- .../confignode/manager/TriggerManager.java | 3 ++- .../schema/AlterLogicalViewProcedure.java | 17 +++++++++++- .../impl/schema/SetTTLProcedure.java | 16 ----------- .../impl/trigger/CreateTriggerProcedure.java | 27 ++++++++++++++++--- .../thrift/ConfigNodeRPCServiceProcessor.java | 3 ++- .../task/builder/PipeDataNodeTaskBuilder.java | 1 + .../PipeConnectorSubtaskManager.java | 1 - .../sync/IoTDBDataRegionSyncConnector.java | 1 + .../deletion/PipeDeleteDataNodeEvent.java | 14 ++++++---- .../PipeSchemaRegionWritePlanEvent.java | 8 +++++- .../tsfile/PipeTsFileInsertionEvent.java | 14 +++++++--- .../thrift/IoTDBDataNodeReceiver.java | 1 - .../plan/analyze/AnalyzeVisitor.java | 5 +++- .../plan/analyze/load/LoadTsFileAnalyzer.java | 15 ++++++++--- .../plan/planner/TreeModelPlanner.java | 26 +++++++----------- .../analyzer/StatementAnalyzer.java | 4 ++- .../plan/relational/sql/ast/LoadTsFile.java | 9 +++++++ .../scheduler/load/LoadTsFileScheduler.java | 3 ++- .../statement/crud/InsertBaseStatement.java | 16 +++++------ .../statement/crud/LoadTsFileStatement.java | 19 +++++++------ .../LoadTsFileDataTypeConverter.java | 9 ++++--- .../commons/pipe/event/EnrichedEvent.java | 13 ++++++--- .../pipe/event/PipeWritePlanEvent.java | 8 +++--- 24 files changed, 155 insertions(+), 88 deletions(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java index 9b38d8d900e25..78852b2589926 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ProcedureManager.java @@ -414,7 +414,8 @@ public TSStatus alterLogicalView(final TAlterLogicalViewReq req) { new AlterLogicalViewProcedure( queryId, viewPathToSourceMap, - req.isSetIsGeneratedByPipe() && req.isIsGeneratedByPipe()); + req.isSetIsGeneratedByPipe() && req.isIsGeneratedByPipe(), + req.getOriginClusterId()); this.executor.submitProcedure(procedure); } } @@ -1250,9 +1251,12 @@ public TSStatus createRegionGroups( * {@link TSStatusCode#CREATE_TRIGGER_ERROR} otherwise */ public TSStatus createTrigger( - TriggerInformation triggerInformation, Binary jarFile, boolean isGeneratedByPipe) { + TriggerInformation triggerInformation, + Binary jarFile, + boolean isGeneratedByPipe, + String originClusterId) { final CreateTriggerProcedure createTriggerProcedure = - new CreateTriggerProcedure(triggerInformation, jarFile, isGeneratedByPipe); + new CreateTriggerProcedure(triggerInformation, jarFile, isGeneratedByPipe, originClusterId); try { if (jarFile != null && new UpdateProcedurePlan(createTriggerProcedure).getSerializedSize() > planSizeLimit) { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/TriggerManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/TriggerManager.java index eff18c7f2b258..aa2db6203845e 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/TriggerManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/TriggerManager.java @@ -129,7 +129,8 @@ public TSStatus createTrigger(TCreateTriggerReq req) { .createTrigger( triggerInformation, needToSaveJar ? new Binary(req.getJarFile()) : null, - req.isSetIsGeneratedByPipe() && req.isIsGeneratedByPipe()); + req.isSetIsGeneratedByPipe() && req.isIsGeneratedByPipe(), + req.getOriginClusterId()); } public TSStatus dropTrigger(TDropTriggerReq req) { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterLogicalViewProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterLogicalViewProcedure.java index e81df1e558a4f..dee312dcc731f 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterLogicalViewProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterLogicalViewProcedure.java @@ -88,6 +88,18 @@ public AlterLogicalViewProcedure( generatePathPatternTree(); } + public AlterLogicalViewProcedure( + final String queryId, + final Map viewPathToSourceMap, + final boolean isGeneratedByPipe, + final String originClusterId) { + super(isGeneratedByPipe); + this.queryId = queryId; + this.viewPathToSourceMap = viewPathToSourceMap; + this.originClusterId = originClusterId; + generatePathPatternTree(); + } + @Override protected Flow executeFromState( final ConfigNodeProcedureEnv env, final AlterLogicalViewState state) @@ -158,7 +170,10 @@ private void alterLogicalView(final ConfigNodeProcedureEnv env) throws Procedure targetSchemaRegionGroup, CnToDnAsyncRequestType.ALTER_VIEW, (dataNodeLocation, consensusGroupIdList) -> { - TAlterViewReq req = new TAlterViewReq().setIsGeneratedByPipe(isGeneratedByPipe); + TAlterViewReq req = + new TAlterViewReq() + .setIsGeneratedByPipe(isGeneratedByPipe) + .setOriginClusterId(originClusterId); req.setSchemaRegionIdList(consensusGroupIdList); List viewMapBinaryList = new ArrayList<>(); for (TConsensusGroupId consensusGroupId : consensusGroupIdList) { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTTLProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTTLProcedure.java index cc64436f393bb..6442d99890c14 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTTLProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTTLProcedure.java @@ -164,13 +164,6 @@ public void serialize(DataOutputStream stream) throws IOException { : ProcedureType.SET_TTL_PROCEDURE.getTypeCode()); super.serialize(stream); ReadWriteIOUtils.write(plan.serializeToByteBuffer(), stream); - - if (originClusterId == null) { - stream.writeBoolean(false); - } else { - stream.writeBoolean(true); - ReadWriteIOUtils.write(originClusterId, stream); - } } @Override @@ -179,15 +172,6 @@ public void deserialize(ByteBuffer byteBuffer) { try { ReadWriteIOUtils.readInt(byteBuffer); this.plan = (SetTTLPlan) ConfigPhysicalPlan.Factory.create(byteBuffer); - - if (byteBuffer.hasRemaining()) { - boolean hasClusterId = byteBuffer.get() != 0; - if (hasClusterId) { - this.originClusterId = ReadWriteIOUtils.readString(byteBuffer); - } else { - this.originClusterId = null; - } - } } catch (IOException e) { LOGGER.error("IO error when deserialize setTTL plan.", e); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/trigger/CreateTriggerProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/trigger/CreateTriggerProcedure.java index ee06f9a53cb9c..b16b4ba3d775b 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/trigger/CreateTriggerProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/trigger/CreateTriggerProcedure.java @@ -68,6 +68,17 @@ public CreateTriggerProcedure( this.jarFile = jarFile; } + public CreateTriggerProcedure( + final TriggerInformation triggerInformation, + final Binary jarFile, + final boolean isGeneratedByPipe, + final String originClusterId) { + super(isGeneratedByPipe); + this.triggerInformation = triggerInformation; + this.jarFile = jarFile; + this.originClusterId = originClusterId; + } + @Override protected Flow executeFromState( final ConfigNodeProcedureEnv env, final CreateTriggerState state) { @@ -154,7 +165,8 @@ protected Flow executeFromState( isGeneratedByPipe ? new PipeEnrichedPlan( new UpdateTriggerStateInTablePlan( - triggerInformation.getTriggerName(), TTriggerState.ACTIVE)) + triggerInformation.getTriggerName(), TTriggerState.ACTIVE), + originClusterId) : new UpdateTriggerStateInTablePlan( triggerInformation.getTriggerName(), TTriggerState.ACTIVE)); setNextState(CreateTriggerState.CONFIG_NODE_ACTIVE); @@ -208,7 +220,8 @@ protected void rollbackState(ConfigNodeProcedureEnv env, CreateTriggerState stat .write( isGeneratedByPipe ? new PipeEnrichedPlan( - new DeleteTriggerInTablePlan(triggerInformation.getTriggerName())) + new DeleteTriggerInTablePlan(triggerInformation.getTriggerName()), + originClusterId) : new DeleteTriggerInTablePlan(triggerInformation.getTriggerName())); } catch (ConsensusException e) { LOG.warn("Failed in the write API executing the consensus layer due to: ", e); @@ -310,7 +323,8 @@ public boolean equals(Object that) { && thatProc.getCurrentState().equals(this.getCurrentState()) && thatProc.getCycles() == this.getCycles() && thatProc.isGeneratedByPipe == this.isGeneratedByPipe - && thatProc.triggerInformation.equals(this.triggerInformation); + && thatProc.triggerInformation.equals(this.triggerInformation) + && Objects.equals(thatProc.originClusterId, this.originClusterId); } return false; } @@ -318,6 +332,11 @@ public boolean equals(Object that) { @Override public int hashCode() { return Objects.hash( - getProcId(), getCurrentState(), getCycles(), isGeneratedByPipe, triggerInformation); + getProcId(), + getCurrentState(), + getCycles(), + isGeneratedByPipe, + triggerInformation, + originClusterId); } } 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 9865595458577..2dab9bdd2d34c 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 @@ -545,7 +545,8 @@ public TSStatus alterDatabase(final TDatabaseSchema databaseSchema) { public TSStatus deleteDatabase(final TDeleteDatabaseReq tDeleteReq) { return configManager.deleteDatabases( new TDeleteDatabasesReq(Collections.singletonList(tDeleteReq.getPrefixPath())) - .setIsGeneratedByPipe(tDeleteReq.isIsGeneratedByPipe())); + .setIsGeneratedByPipe(tDeleteReq.isIsGeneratedByPipe()) + .setOriginClusterId(tDeleteReq.getOriginClusterId())); } @Override 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 f420ba58d9d0a..17e9a8960517a 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 @@ -96,6 +96,7 @@ public PipeDataNodeTask build() { // We first build the extractor and connector, then build the processor. + // build connector first to get the sink clusterIds final PipeTaskConnectorStage connectorStage; final PipeType pipeType = pipeStaticMeta.getPipeType(); 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 bdf34a7809be9..467c1494bcac6 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 @@ -120,7 +120,6 @@ public synchronized String register( : new UnboundedBlockingPendingQueue<>(new PipeDataRegionEventCounter()); for (int connectorIndex = 0; connectorIndex < connectorNum; connectorIndex++) { - final PipeConnector pipeConnector = isDataRegionConnector ? PipeDataNodeAgent.plugin().dataRegion().reflectConnector(pipeConnectorParameters) 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 2c1f313e55993..56816220a813b 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 @@ -265,6 +265,7 @@ private void doTransferWrapper(final Pair endPo private void doTransfer( final TEndPoint endPoint, final PipeTabletEventPlainBatch batchToTransfer) { final Pair clientAndStatus = clientManager.getClient(endPoint); + final TPipeTransferResp resp; try { final TPipeTransferReq uncompressedReq = batchToTransfer.toTPipeTransferReq(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java index 7f28afac31e22..7a7c33ab58fbe 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java @@ -191,8 +191,11 @@ public boolean mayEventPathsOverlappedWithPattern() { @Override public ByteBuffer serializeToByteBuffer() { final ByteBuffer planBuffer = deleteDataNode.serializeToByteBuffer(); - final ByteBuffer result = ByteBuffer.allocate(Byte.BYTES + planBuffer.limit()); + final ByteBuffer result = + ByteBuffer.allocate( + Byte.BYTES + planBuffer.limit() + computeOriginClusterIdBufferSize(originClusterId)); ReadWriteIOUtils.write(isGeneratedByPipe, result); + ReadWriteIOUtils.write(originClusterId, result); result.put(planBuffer); return result; } @@ -200,6 +203,7 @@ public ByteBuffer serializeToByteBuffer() { @Override public void deserializeFromByteBuffer(final ByteBuffer buffer) { isGeneratedByPipe = ReadWriteIOUtils.readBool(buffer); + originClusterId = ReadWriteIOUtils.readString(buffer); deleteDataNode = (DeleteDataNode) PlanNodeType.deserialize(buffer); progressIndex = deleteDataNode.getProgressIndex(); } @@ -215,8 +219,8 @@ public static PipeDeleteDataNodeEvent deserialize(final ByteBuffer buffer) { @Override public String toString() { return String.format( - "PipDeleteDataNodeEvent{progressIndex=%s, isGeneratedByPipe=%s}", - progressIndex, isGeneratedByPipe) + "PipDeleteDataNodeEvent{progressIndex=%s, isGeneratedByPipe=%s, originClusterId=%s}", + progressIndex, isGeneratedByPipe, originClusterId) + " - " + super.toString(); } @@ -224,8 +228,8 @@ public String toString() { @Override public String coreReportMessage() { return String.format( - "PipeDeleteDataNodeEvent{progressIndex=%s, isGeneratedByPipe=%s}", - progressIndex, isGeneratedByPipe) + "PipeDeleteDataNodeEvent{progressIndex=%s, isGeneratedByPipe=%s, originClusterId=%s}", + progressIndex, isGeneratedByPipe, originClusterId) + " - " + super.coreReportMessage(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java index b12435a91162c..88cd6874a901d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java @@ -104,9 +104,14 @@ public EnrichedEvent shallowCopySelfAndBindPipeTaskMetaForProgressReport( @Override public ByteBuffer serializeToByteBuffer() { final ByteBuffer planBuffer = planNode.serializeToByteBuffer(); - final ByteBuffer result = ByteBuffer.allocate(Byte.BYTES * 2 + planBuffer.limit()); + final ByteBuffer result = + ByteBuffer.allocate( + Byte.BYTES * 2 + + planBuffer.limit() + + computeOriginClusterIdBufferSize(originClusterId)); ReadWriteIOUtils.write(PipeSchemaSerializableEventType.SCHEMA_WRITE_PLAN.getType(), result); ReadWriteIOUtils.write(isGeneratedByPipe, result); + ReadWriteIOUtils.write(originClusterId, result); result.put(planBuffer); return result; } @@ -114,6 +119,7 @@ public ByteBuffer serializeToByteBuffer() { @Override public void deserializeFromByteBuffer(final ByteBuffer buffer) { isGeneratedByPipe = ReadWriteIOUtils.readBool(buffer); + originClusterId = ReadWriteIOUtils.readString(buffer); planNode = PlanNodeType.deserialize(buffer); } 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 3dedb4d42a5eb..057f2d334ba1e 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 @@ -749,8 +749,14 @@ public void close() { @Override public String toString() { return String.format( - "PipeTsFileInsertionEvent{resource=%s, tsFile=%s, isLoaded=%s, isGeneratedByPipe=%s, isClosed=%s, eventParser=%s}", - resource, tsFile, isLoaded, isGeneratedByPipe, isClosed.get(), eventParser) + "PipeTsFileInsertionEvent{resource=%s, tsFile=%s, isLoaded=%s, isGeneratedByPipe=%s, isClosed=%s, eventParser=%s, originClusterId=%s}", + resource, + tsFile, + isLoaded, + isGeneratedByPipe, + isClosed.get(), + eventParser, + originClusterId) + " - " + super.toString(); } @@ -758,8 +764,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, isClosed=%s, originClusterId=%s}", + resource, tsFile, isLoaded, isGeneratedByPipe, isClosed.get(), originClusterId) + " - " + super.coreReportMessage(); } 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 51656f6a925be..56ef9746198d9 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 @@ -691,7 +691,6 @@ private TPipeTransferResp handleTransferSchemaPlan(final PipeTransferPlanNodeReq .alterLogicalViewByPipe((AlterLogicalViewNode) req.getPlanNode())); } final Object statement = PLAN_TO_STATEMENT_VISITOR.process(req.getPlanNode(), null); - return statement instanceof Statement ? new TPipeTransferResp(executeStatementAndClassifyExceptions((Statement) statement)) : new TPipeTransferResp( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java index e0fa5c1c0045b..741371231201f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java @@ -2986,7 +2986,10 @@ public Analysis visitLoadFile(LoadTsFileStatement loadTsFileStatement, MPPQueryC final long startTime = System.nanoTime(); try (final LoadTsFileAnalyzer loadTsFileAnalyzer = new LoadTsFileAnalyzer( - loadTsFileStatement, loadTsFileStatement.isGeneratedByPipe(), context)) { + loadTsFileStatement, + loadTsFileStatement.isGeneratedByPipe(), + loadTsFileStatement.getOriginClusterId(), + context)) { return (Analysis) loadTsFileAnalyzer.analyzeFileByFile(new Analysis()); } catch (final Exception e) { final String exceptionMessage = 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 832d4dbe36999..321c0a0225947 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 @@ -101,6 +101,7 @@ public class LoadTsFileAnalyzer implements AutoCloseable { isTableModelStatement; // Whether the statement itself is table model or not (not the TsFiles) private final String statementString; private final boolean isGeneratedByPipe; + private final String originClusterId; private final List tsFiles; private final List isTableModelTsFile; @@ -119,7 +120,10 @@ public class LoadTsFileAnalyzer implements AutoCloseable { private LoadTsFileTableSchemaCache tableSchemaCache; public LoadTsFileAnalyzer( - LoadTsFileStatement loadTsFileStatement, boolean isGeneratedByPipe, MPPQueryContext context) { + LoadTsFileStatement loadTsFileStatement, + boolean isGeneratedByPipe, + String originClusterId, + MPPQueryContext context) { this.context = context; this.loadTsFileTreeStatement = loadTsFileStatement; @@ -127,6 +131,7 @@ public LoadTsFileAnalyzer( this.isTableModelStatement = false; this.statementString = loadTsFileStatement.toString(); this.isGeneratedByPipe = isGeneratedByPipe; + this.originClusterId = originClusterId; this.tsFiles = loadTsFileStatement.getTsFiles(); this.isTableModelTsFile = new ArrayList<>(Collections.nCopies(this.tsFiles.size(), false)); @@ -140,7 +145,10 @@ public LoadTsFileAnalyzer( } public LoadTsFileAnalyzer( - LoadTsFile loadTsFileTableStatement, boolean isGeneratedByPipe, MPPQueryContext context) { + LoadTsFile loadTsFileTableStatement, + boolean isGeneratedByPipe, + String originClusterId, + MPPQueryContext context) { this.context = context; this.loadTsFileTreeStatement = null; @@ -148,6 +156,7 @@ public LoadTsFileAnalyzer( this.isTableModelStatement = true; this.statementString = loadTsFileTableStatement.toString(); this.isGeneratedByPipe = isGeneratedByPipe; + this.originClusterId = originClusterId; this.tsFiles = loadTsFileTableStatement.getTsFiles(); this.isTableModelTsFile = new ArrayList<>(Collections.nCopies(this.tsFiles.size(), false)); @@ -561,7 +570,7 @@ private void executeTabletConversion(final IAnalysis analysis, final LoadAnalyze } final LoadTsFileDataTypeConverter loadTsFileDataTypeConverter = - new LoadTsFileDataTypeConverter(isGeneratedByPipe); + new LoadTsFileDataTypeConverter(isGeneratedByPipe, originClusterId); for (int i = 0; i < tsFiles.size(); i++) { try { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TreeModelPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TreeModelPlanner.java index c3d75a7328101..e7570cbfb4298 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TreeModelPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TreeModelPlanner.java @@ -134,22 +134,16 @@ public IScheduler doSchedule( instanceof LoadTsFileStatement; if (statement instanceof LoadTsFileStatement || isPipeEnrichedTsFileLoad) { scheduler = - statement instanceof PipeEnrichedStatement - ? new LoadTsFileScheduler( - distributedPlan, - context, - stateMachine, - syncInternalServiceClientManager, - partitionFetcher, - true, - ((PipeEnrichedStatement) statement).getOriginClusterId()) - : new LoadTsFileScheduler( - distributedPlan, - context, - stateMachine, - syncInternalServiceClientManager, - partitionFetcher, - false); + new LoadTsFileScheduler( + distributedPlan, + context, + stateMachine, + syncInternalServiceClientManager, + partitionFetcher, + isPipeEnrichedTsFileLoad, + statement instanceof PipeEnrichedStatement + ? ((PipeEnrichedStatement) statement).getOriginClusterId() + : null); } else { scheduler = diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzer.java index 8619f5b89eb78..c344eeb7e0b72 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzer.java @@ -626,6 +626,7 @@ protected Scope visitPipeEnriched(PipeEnriched node, Optional scope) { // in the analyzer to execute the tsfile-tablet conversion in some cases. if (node.getInnerStatement() instanceof LoadTsFile) { ((LoadTsFile) node.getInnerStatement()).markIsGeneratedByPipe(); + ((LoadTsFile) node.getInnerStatement()).setOriginClusterId(node.getOriginClusterId()); } final Scope ret = node.getInnerStatement().accept(this, scope); @@ -640,7 +641,8 @@ protected Scope visitLoadTsFile(final LoadTsFile node, final Optional sco final long startTime = System.nanoTime(); try (final LoadTsFileAnalyzer loadTsFileAnalyzer = - new LoadTsFileAnalyzer(node, node.isGeneratedByPipe(), queryContext)) { + new LoadTsFileAnalyzer( + node, node.isGeneratedByPipe(), node.getOriginClusterId(), queryContext)) { loadTsFileAnalyzer.analyzeFileByFile(analysis); } catch (final Exception e) { final String exceptionMessage = diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/LoadTsFile.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/LoadTsFile.java index 7414f1bee2b03..a97289b5515ce 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/LoadTsFile.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/LoadTsFile.java @@ -46,6 +46,7 @@ public class LoadTsFile extends Statement { private boolean autoCreateDatabase = true; private boolean verify; private boolean isGeneratedByPipe = false; + private String originClusterId; private final Map loadAttributes; @@ -128,6 +129,14 @@ public boolean isGeneratedByPipe() { return isGeneratedByPipe; } + public void setOriginClusterId(final String originClusterId) { + this.originClusterId = originClusterId; + } + + public String getOriginClusterId() { + return originClusterId; + } + public List getIsTableModel() { return isTableModel; } 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 c05276162f39d..7c8eebc39b47a 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 @@ -428,6 +428,7 @@ private boolean secondPhase( try { loadCommandReq.setIsGeneratedByPipe(isGeneratedByPipe); loadCommandReq.setProgressIndex(assignProgressIndex(tsFileResource)); + loadCommandReq.setOriginClusterId(originClusterId); Future dispatchResultFuture = dispatcher.dispatchCommand(loadCommandReq, allReplicaSets); @@ -582,7 +583,7 @@ private boolean loadLocally(LoadSingleTsFileNode node) throws IoTDBException { private void convertFailedTsFilesToTabletsAndRetry() { final LoadTsFileDataTypeConverter loadTsFileDataTypeConverter = - new LoadTsFileDataTypeConverter(isGeneratedByPipe); + new LoadTsFileDataTypeConverter(isGeneratedByPipe, originClusterId); final Iterator iterator = failedTsFileNodeIndexes.listIterator(); while (iterator.hasNext()) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertBaseStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertBaseStatement.java index 2770080fb4797..d40b0483eaef9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertBaseStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertBaseStatement.java @@ -102,14 +102,6 @@ public abstract class InsertBaseStatement extends Statement { @TableModel protected String databaseName; - public String getOriginClusterId() { - return originClusterId; - } - - public void setOriginClusterId(String originClusterId) { - this.originClusterId = originClusterId; - } - protected String originClusterId; // endregion @@ -624,6 +616,14 @@ public void setWriteToTable(final boolean writeToTable) { } } + public String getOriginClusterId() { + return originClusterId; + } + + public void setOriginClusterId(String originClusterId) { + this.originClusterId = originClusterId; + } + @TableModel public void setDatabaseName(final String databaseName) { this.databaseName = databaseName; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatement.java index 16f6da5939fc8..66b068ed75bf4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/LoadTsFileStatement.java @@ -60,6 +60,7 @@ public class LoadTsFileStatement extends Statement { private boolean convertOnTypeMismatch = true; private boolean autoCreateDatabase = true; private boolean isGeneratedByPipe = false; + private String originClusterId; private Map loadAttributes; @@ -68,16 +69,6 @@ public class LoadTsFileStatement extends Statement { private final List resources; private final List writePointCountList; - public String getOriginClusterId() { - return originClusterId; - } - - public void setOriginClusterId(String originClusterId) { - this.originClusterId = originClusterId; - } - - private String originClusterId; - public LoadTsFileStatement(String filePath) throws FileNotFoundException { this.file = new File(filePath); this.databaseLevel = IoTDBDescriptor.getInstance().getConfig().getDefaultStorageGroupLevel(); @@ -217,6 +208,14 @@ public boolean isGeneratedByPipe() { return isGeneratedByPipe; } + public String getOriginClusterId() { + return originClusterId; + } + + public void setOriginClusterId(String originClusterId) { + this.originClusterId = originClusterId; + } + public List getTsFiles() { return tsFiles; } 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 57e3df75ac037..f310d93593da6 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 @@ -50,6 +50,7 @@ public class LoadTsFileDataTypeConverter { STATEMENT_EXCEPTION_VISITOR = new LoadConvertedInsertTabletStatementExceptionVisitor(); private final boolean isGeneratedByPipe; + private final String originClusterId; private final SqlParser relationalSqlParser = new SqlParser(); private final LoadTableStatementDataTypeConvertExecutionVisitor @@ -59,8 +60,10 @@ public class LoadTsFileDataTypeConverter { treeStatementDataTypeConvertExecutionVisitor = new LoadTreeStatementDataTypeConvertExecutionVisitor(this::executeForTreeModel); - public LoadTsFileDataTypeConverter(final boolean isGeneratedByPipe) { + public LoadTsFileDataTypeConverter( + final boolean isGeneratedByPipe, final String originClusterId) { this.isGeneratedByPipe = isGeneratedByPipe; + this.originClusterId = originClusterId; } public Optional convertForTableModel(final LoadTsFile loadTsFileTableStatement) { @@ -80,7 +83,7 @@ public Optional convertForTableModel(final LoadTsFile loadTsFileTableS private TSStatus executeForTableModel(final Statement statement, final String databaseName) { return Coordinator.getInstance() .executeForTableModel( - isGeneratedByPipe ? new PipeEnrichedStatement(statement) : statement, + isGeneratedByPipe ? new PipeEnrichedStatement(statement, originClusterId) : statement, relationalSqlParser, SESSION_MANAGER.getCurrSession(), SESSION_MANAGER.requestQueryId(), @@ -106,7 +109,7 @@ public Optional convertForTreeModel(final LoadTsFileStatement loadTsFi private TSStatus executeForTreeModel(final Statement statement) { return Coordinator.getInstance() .executeForTreeModel( - isGeneratedByPipe ? new PipeEnrichedStatement(statement) : statement, + isGeneratedByPipe ? new PipeEnrichedStatement(statement, originClusterId) : statement, SESSION_MANAGER.requestQueryId(), SESSION_MANAGER.getSessionInfo(SESSION_MANAGER.getCurrSession()), "", 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 f79a0e6298a76..53d906331be22 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 @@ -29,6 +29,7 @@ import org.apache.iotdb.commons.pipe.datastructure.pattern.TreePattern; import org.apache.iotdb.pipe.api.event.Event; +import org.apache.tsfile.common.conf.TSFileConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -79,8 +80,6 @@ public abstract class EnrichedEvent implements Event { protected String userName; protected boolean skipIfNoPrivileges; - // protected final String originClusterId; - protected EnrichedEvent( final String pipeName, final long creationTime, @@ -103,7 +102,6 @@ protected EnrichedEvent( this.skipIfNoPrivileges = skipIfNoPrivileges; this.startTime = startTime; this.endTime = endTime; - // this.originClusterId = originClusterId; isPatternParsed = (treePattern == null || treePattern.isRoot()) @@ -462,6 +460,15 @@ public boolean isReleased() { return isReleased.get(); } + public int computeOriginClusterIdBufferSize(final String originClusterId) { + if (originClusterId == null) { + return Integer.BYTES; + } else { + byte[] bytes = originClusterId.getBytes(TSFileConfig.STRING_CHARSET); + return Integer.BYTES + bytes.length; + } + } + /** * Used for pipeConsensus. In PipeConsensus, we only need committerKey, commitId and rebootTimes * to uniquely identify an event diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/PipeWritePlanEvent.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/PipeWritePlanEvent.java index e85492305403d..58044ada604d3 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/PipeWritePlanEvent.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/event/PipeWritePlanEvent.java @@ -125,8 +125,8 @@ public boolean mayEventPathsOverlappedWithPattern() { @Override public String toString() { return String.format( - "PipeWritePlanEvent{progressIndex=%s, isGeneratedByPipe=%s}", - progressIndex, isGeneratedByPipe) + "PipeWritePlanEvent{progressIndex=%s, isGeneratedByPipe=%s, originClusterId=%s}", + progressIndex, isGeneratedByPipe, originClusterId) + " - " + super.toString(); } @@ -134,8 +134,8 @@ public String toString() { @Override public String coreReportMessage() { return String.format( - "PipeWritePlanEvent{progressIndex=%s, isGeneratedByPipe=%s}", - progressIndex, isGeneratedByPipe) + "PipeWritePlanEvent{progressIndex=%s, isGeneratedByPipe=%s, originClusterId=%s}", + progressIndex, isGeneratedByPipe, originClusterId) + " - " + super.coreReportMessage(); } From 13eff8b38b10183242d7768a5e469ea837fcdf83 Mon Sep 17 00:00:00 2001 From: XNX02 <202100130044@mail.sdu.edu.cn> Date: Mon, 17 Mar 2025 23:47:42 +0800 Subject: [PATCH 17/27] update --- .../java/org/apache/iotdb/confignode/manager/ConfigManager.java | 2 +- .../execution/config/executor/ClusterConfigTaskExecutor.java | 2 +- .../thrift-confignode/src/main/thrift/confignode.thrift | 2 +- 3 files changed, 3 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 37cdd1e013682..18eb632ffaccc 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 @@ -2356,7 +2356,7 @@ public TPipeConfigTransferResp handleTransferConfigPlan(TPipeConfigTransferReq r : new TPipeTransferReq(req.version, req.type, req.body)); return new TPipeConfigTransferResp(result.status) .setBody(result.body) - .setCluster(result.clusterId); + .setClusterId(result.clusterId); } @Override 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 8317e56ae6d29..02c26a2c8afde 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 @@ -3299,7 +3299,7 @@ public TPipeTransferResp handleTransferConfigPlan( } return new TPipeTransferResp(pipeConfigTransferResp.status) .setBody(pipeConfigTransferResp.body) - .setClusterId(pipeConfigTransferResp.cluster); + .setClusterId(pipeConfigTransferResp.clusterId); } catch (Exception e) { return new TPipeTransferResp( new TSStatus(TSStatusCode.INTERNAL_SERVER_ERROR.getStatusCode()) diff --git a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift index 611c765755c08..57bcb0bf1c3bc 100644 --- a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift +++ b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift @@ -863,7 +863,7 @@ struct TPipeConfigTransferReq { struct TPipeConfigTransferResp { 1: required common.TSStatus status 2: optional binary body - 3: optional string cluster + 3: optional string clusterId } struct TDeleteTimeSeriesReq { From 7b7abbb0248ce76d05ce172247f96f3fc49e1982 Mon Sep 17 00:00:00 2001 From: XNX02 <202100130044@mail.sdu.edu.cn> Date: Wed, 19 Mar 2025 22:48:14 +0800 Subject: [PATCH 18/27] add UT for PipeEnrichedProcedureTest --- .../procedure/impl/StateMachineProcedure.java | 26 ++----- .../schema/AlterLogicalViewProcedure.java | 6 +- .../AbstractAlterOrDropTableProcedure.java | 5 +- .../receiver/PipeEnrichedProcedureTest.java | 70 ++++++++++++++----- .../deletion/PipeDeleteDataNodeEvent.java | 2 - 5 files changed, 65 insertions(+), 44 deletions(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/StateMachineProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/StateMachineProcedure.java index 215e1b5e7fe01..8dfac99b400bc 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/StateMachineProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/StateMachineProcedure.java @@ -25,6 +25,7 @@ import org.apache.iotdb.confignode.procedure.exception.ProcedureYieldException; import org.apache.thrift.annotation.Nullable; +import org.apache.tsfile.utils.ReadWriteIOUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -282,13 +283,7 @@ public void serialize(DataOutputStream stream) throws IOException { for (int state : states) { stream.writeInt(state); } - - if (originClusterId == null) { - stream.writeBoolean(false); - } else { - stream.writeBoolean(true); - stream.writeUTF(originClusterId); - } + ReadWriteIOUtils.write(originClusterId, stream); } @Override @@ -304,18 +299,7 @@ public void deserialize(ByteBuffer byteBuffer) { stateFlow = Flow.NO_MORE_STATE; } } - this.setStateDeserialized(true); - if (byteBuffer.hasRemaining()) { - boolean hasClusterId = byteBuffer.get() != 0; - if (hasClusterId) { - int strLength = byteBuffer.getShort(); - byte[] bytes = new byte[strLength]; - byteBuffer.get(bytes); - originClusterId = new String(bytes); - } else { - originClusterId = null; - } - } + originClusterId = ReadWriteIOUtils.readString(byteBuffer); } /** @@ -331,4 +315,8 @@ public boolean isStateDeserialized() { private void setStateDeserialized(boolean isDeserialized) { this.isStateDeserialized = isDeserialized; } + + public String getOriginClusterId() { + return originClusterId; + } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterLogicalViewProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterLogicalViewProcedure.java index dee312dcc731f..97f531bd791a5 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterLogicalViewProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/AlterLogicalViewProcedure.java @@ -320,7 +320,8 @@ public boolean equals(final Object o) { && Objects.equals(getCycles(), that.getCycles()) && Objects.equals(isGeneratedByPipe, that.isGeneratedByPipe) && Objects.equals(queryId, that.queryId) - && Objects.equals(viewPathToSourceMap, that.viewPathToSourceMap); + && Objects.equals(viewPathToSourceMap, that.viewPathToSourceMap) + && Objects.equals(originClusterId, that.originClusterId); } @Override @@ -331,7 +332,8 @@ public int hashCode() { getCycles(), isGeneratedByPipe, queryId, - viewPathToSourceMap); + viewPathToSourceMap, + originClusterId); } private class AlterLogicalViewRegionTaskExecutor diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/AbstractAlterOrDropTableProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/AbstractAlterOrDropTableProcedure.java index 4fde4030cb21d..1ced07996619e 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/AbstractAlterOrDropTableProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/AbstractAlterOrDropTableProcedure.java @@ -249,11 +249,12 @@ public boolean equals(final Object o) { final AbstractAlterOrDropTableProcedure that = (AbstractAlterOrDropTableProcedure) o; return Objects.equals(database, that.database) && Objects.equals(tableName, that.tableName) - && Objects.equals(queryId, that.queryId); + && Objects.equals(queryId, that.queryId) + && Objects.equals(originClusterId, that.originClusterId); } @Override public int hashCode() { - return Objects.hash(database, tableName, queryId); + return Objects.hash(database, tableName, queryId, originClusterId); } } diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/pipe/receiver/PipeEnrichedProcedureTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/pipe/receiver/PipeEnrichedProcedureTest.java index a0a1dec3092a8..c98902031e9cb 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/pipe/receiver/PipeEnrichedProcedureTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/procedure/impl/pipe/receiver/PipeEnrichedProcedureTest.java @@ -85,7 +85,9 @@ public class PipeEnrichedProcedureTest { public void deleteDatabaseTest() { PublicBAOS byteArrayOutputStream = new PublicBAOS(); DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream); - DeleteDatabaseProcedure p1 = new DeleteDatabaseProcedure(new TDatabaseSchema("root.sg"), true); + DeleteDatabaseProcedure p1 = + new DeleteDatabaseProcedure( + new TDatabaseSchema("root.sg"), true, "a6670472-91a4-4194-9916-08236680b4d8"); try { p1.serialize(outputStream); @@ -109,7 +111,8 @@ public void deleteTimeseriesTest() throws IllegalPathException, IOException { patternTree.appendPathPattern(new PartialPath("root.sg2.*.s1")); patternTree.constructTree(); DeleteTimeSeriesProcedure deleteTimeSeriesProcedure = - new DeleteTimeSeriesProcedure(queryId, patternTree, true); + new DeleteTimeSeriesProcedure( + queryId, patternTree, true, "a6670472-91a4-4194-9916-08236680b4d8"); ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream); @@ -149,7 +152,8 @@ public void deactivateTemplateTest() throws IllegalPathException, IOException { templateSetInfo.put(new PartialPath("root.sg2.**"), Arrays.asList(t2, t1)); DeactivateTemplateProcedure deactivateTemplateProcedure = - new DeactivateTemplateProcedure(queryId, templateSetInfo, true); + new DeactivateTemplateProcedure( + queryId, templateSetInfo, true, "a6670472-91a4-4194-9916-08236680b4d8"); ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream); @@ -176,7 +180,8 @@ public void unsetTemplateTest() throws IllegalPathException, IOException { new CompressionType[] {CompressionType.UNCOMPRESSED, CompressionType.GZIP}); PartialPath path = new PartialPath("root.sg"); UnsetTemplateProcedure unsetTemplateProcedure = - new UnsetTemplateProcedure(queryId, template, path, true); + new UnsetTemplateProcedure( + queryId, template, path, true, "a6670472-91a4-4194-9916-08236680b4d8"); ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream); @@ -193,7 +198,8 @@ public void unsetTemplateTest() throws IllegalPathException, IOException { @Test public void setTemplateTest() throws IOException { SetTemplateProcedure setTemplateProcedure = - new SetTemplateProcedure("1", "t1", "root.sg", true); + new SetTemplateProcedure( + "1", "t1", "root.sg", true, "a6670472-91a4-4194-9916-08236680b4d8"); ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream); @@ -219,7 +225,8 @@ public void alterLogicalViewTest() throws IllegalPathException, IOException { new ConstantViewOperand(TSDataType.BOOLEAN, "true")); } }, - true); + true, + "a6670472-91a4-4194-9916-08236680b4d8"); ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream); @@ -231,6 +238,8 @@ public void alterLogicalViewTest() throws IllegalPathException, IOException { .create(ByteBuffer.wrap(byteArrayOutputStream.toByteArray())); Assert.assertEquals(alterLogicalViewProcedure.getQueryId(), deserializedProcedure.getQueryId()); + Assert.assertEquals( + alterLogicalViewProcedure.getOriginClusterId(), deserializedProcedure.getOriginClusterId()); // Currently skip the "equals" method since "equals" of ViewExpression is not implemented } @@ -240,7 +249,7 @@ public void deleteLogicalViewTest() throws IllegalPathException, IOException { tree.appendFullPath(new PartialPath("root.a.b")); tree.appendFullPath(new PartialPath("root.a.c")); DeleteLogicalViewProcedure deleteLogicalViewProcedure = - new DeleteLogicalViewProcedure("1", tree, true); + new DeleteLogicalViewProcedure("1", tree, true, "a6670472-91a4-4194-9916-08236680b4d8"); ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream); @@ -275,7 +284,11 @@ public void createTriggerTest() throws IllegalPathException { FailureStrategy.OPTIMISTIC, "testMD5test"); CreateTriggerProcedure p1 = - new CreateTriggerProcedure(triggerInformation, new Binary(new byte[] {1, 2, 3}), true); + new CreateTriggerProcedure( + triggerInformation, + new Binary(new byte[] {1, 2, 3}), + true, + "a6670472-91a4-4194-9916-08236680b4d8"); try { p1.serialize(outputStream); @@ -297,7 +310,8 @@ public void dropTriggerTest() { PublicBAOS byteArrayOutputStream = new PublicBAOS(); DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream); - DropTriggerProcedure p1 = new DropTriggerProcedure("test", true); + DropTriggerProcedure p1 = + new DropTriggerProcedure("test", true, "a6670472-91a4-4194-9916-08236680b4d8"); try { p1.serialize(outputStream); @@ -329,7 +343,8 @@ public void authOperationTest() throws AuthException { false, Collections.emptyList()), Collections.emptyList(), - true); + true, + "a6670472-91a4-4194-9916-08236680b4d8"); try { p1.serialize(outputStream); @@ -352,7 +367,8 @@ public void setTTLTest() throws IOException, IllegalPathException { // test1 PartialPath path = new PartialPath("root.test.sg1.group1.group1.**"); SetTTLPlan setTTLPlan = new SetTTLPlan(Arrays.asList(path.getNodes()), 1928300234200L); - SetTTLProcedure proc = new SetTTLProcedure(setTTLPlan, true); + SetTTLProcedure proc = + new SetTTLProcedure(setTTLPlan, true, "a6670472-91a4-4194-9916-08236680b4d8"); proc.serialize(outputStream); ByteBuffer buffer = @@ -365,7 +381,7 @@ public void setTTLTest() throws IOException, IllegalPathException { // test2 path = new PartialPath("root.**"); setTTLPlan = new SetTTLPlan(Arrays.asList(path.getNodes()), -1); - proc = new SetTTLProcedure(setTTLPlan, true); + proc = new SetTTLProcedure(setTTLPlan, true, "a6670472-91a4-4194-9916-08236680b4d8"); proc.serialize(outputStream); buffer = ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); @@ -384,7 +400,7 @@ public void createTableTest() throws IOException { new FieldColumnSchema( "Measurement", TSDataType.DOUBLE, TSEncoding.GORILLA, CompressionType.SNAPPY)); final CreateTableProcedure createTableProcedure = - new CreateTableProcedure("database1", table, true); + new CreateTableProcedure("database1", table, true, "a6670472-91a4-4194-9916-08236680b4d8"); final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); final DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream); @@ -407,6 +423,8 @@ public void createTableTest() throws IOException { deserializedProcedure.getTable().getColumnNum()); Assert.assertEquals( createTableProcedure.getTable().getIdNums(), deserializedProcedure.getTable().getIdNums()); + Assert.assertEquals( + createTableProcedure.getOriginClusterId(), deserializedProcedure.getOriginClusterId()); } @Test @@ -417,7 +435,8 @@ public void addTableColumnTest() throws IOException { "table1", "0", Collections.singletonList(new TagColumnSchema("Id", TSDataType.STRING)), - true); + true, + "a6670472-91a4-4194-9916-08236680b4d8"); final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); final DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream); @@ -435,6 +454,8 @@ public void addTableColumnTest() throws IOException { Assert.assertEquals(addTableColumnProcedure.getDatabase(), deserializedProcedure.getDatabase()); Assert.assertEquals( addTableColumnProcedure.getTableName(), deserializedProcedure.getTableName()); + Assert.assertEquals( + addTableColumnProcedure.getOriginClusterId(), deserializedProcedure.getOriginClusterId()); } @Test @@ -450,7 +471,8 @@ public void setTablePropertiesTest() throws IOException { put("ttl", null); } }, - true); + true, + "a6670472-91a4-4194-9916-08236680b4d8"); final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); final DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream); @@ -471,7 +493,14 @@ public void setTablePropertiesTest() throws IOException { @Test public void renameTableColumnTest() throws IOException { final RenameTableColumnProcedure renameTableColumnProcedure = - new RenameTableColumnProcedure("database1", "table1", "0", "oldName", "newName", true); + new RenameTableColumnProcedure( + "database1", + "table1", + "0", + "oldName", + "newName", + true, + "a6670472-91a4-4194-9916-08236680b4d8"); final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); final DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream); @@ -492,7 +521,8 @@ public void renameTableColumnTest() throws IOException { @Test public void dropTableTest() throws IOException { final DropTableProcedure dropTableProcedure = - new DropTableProcedure("database1", "table1", "0", true); + new DropTableProcedure( + "database1", "table1", "0", true, "a6670472-91a4-4194-9916-08236680b4d8"); final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); final DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream); @@ -512,7 +542,8 @@ public void dropTableTest() throws IOException { @Test public void dropTableColumnTest() throws IOException { final DropTableColumnProcedure dropTableColumnProcedure = - new DropTableColumnProcedure("database1", "table1", "0", "columnName", true); + new DropTableColumnProcedure( + "database1", "table1", "0", "columnName", true, "a6670472-91a4-4194-9916-08236680b4d8"); final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); final DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream); @@ -540,7 +571,8 @@ public void deleteDevicesProcedureTest() throws IOException { new byte[] {0, 1, 2}, new byte[] {0, 1, 2}, new byte[] {0, 1, 2}, - true); + true, + "a6670472-91a4-4194-9916-08236680b4d8"); final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); final DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutputStream); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java index 7a7c33ab58fbe..f370b27d4f339 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java @@ -195,7 +195,6 @@ public ByteBuffer serializeToByteBuffer() { ByteBuffer.allocate( Byte.BYTES + planBuffer.limit() + computeOriginClusterIdBufferSize(originClusterId)); ReadWriteIOUtils.write(isGeneratedByPipe, result); - ReadWriteIOUtils.write(originClusterId, result); result.put(planBuffer); return result; } @@ -203,7 +202,6 @@ public ByteBuffer serializeToByteBuffer() { @Override public void deserializeFromByteBuffer(final ByteBuffer buffer) { isGeneratedByPipe = ReadWriteIOUtils.readBool(buffer); - originClusterId = ReadWriteIOUtils.readString(buffer); deleteDataNode = (DeleteDataNode) PlanNodeType.deserialize(buffer); progressIndex = deleteDataNode.getProgressIndex(); } From 6a3284f1847c61844ae8e054c428d48a39f61530 Mon Sep 17 00:00:00 2001 From: XNX02 <202100130044@mail.sdu.edu.cn> Date: Thu, 20 Mar 2025 01:39:35 +0800 Subject: [PATCH 19/27] update --- .../dataregion/memtable/TsFileProcessor.java | 5 ++++- .../commons/pipe/receiver/IoTDBFileReceiver.java | 11 ++++++++++- .../thrift-datanode/src/main/thrift/datanode.thrift | 3 +-- 3 files changed, 15 insertions(+), 4 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 4003c3eacd0a3..07569c1500f6a 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 @@ -25,6 +25,7 @@ import org.apache.iotdb.commons.exception.MetadataException; import org.apache.iotdb.commons.path.AlignedPath; import org.apache.iotdb.commons.path.IFullPath; +import org.apache.iotdb.commons.pipe.receiver.IoTDBFileReceiver; import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; import org.apache.iotdb.commons.utils.CommonDateTimeUtils; import org.apache.iotdb.commons.utils.TestOnly; @@ -326,7 +327,9 @@ public void insert(InsertRowNode insertRowNode, long[] infoForMetrics) : insertRowNode.getOriginClusterId(); if (Objects.isNull(workMemTable.getCurrentOriginClusterId())) { workMemTable.setCurrentOriginClusterId(originClusterId); - } else if (!Objects.equals(originClusterId, workMemTable.getCurrentOriginClusterId())) { + // Use ClusterIdMap to compare the clusterIds by location + } else if (IoTDBFileReceiver.getClusterIdMap().get(originClusterId) + != IoTDBFileReceiver.getClusterIdMap().get(workMemTable.getCurrentOriginClusterId())) { workMemTable.markAsNotFromTheSameCluster(); } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiver.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiver.java index 53c907eb90b65..92dc035893caf 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiver.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/receiver/IoTDBFileReceiver.java @@ -47,7 +47,9 @@ import java.io.File; import java.io.IOException; import java.io.RandomAccessFile; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; @@ -74,6 +76,8 @@ public abstract class IoTDBFileReceiver implements IoTDBReceiver { protected String username = CONNECTOR_IOTDB_USER_DEFAULT_VALUE; protected String password = CONNECTOR_IOTDB_PASSWORD_DEFAULT_VALUE; + // Used to store the clusterId for location comparison + public static final Map CLUSTER_ID_MAP = new HashMap<>(); protected String clusterIdFromHandshakeRequest; private static final boolean IS_FSYNC_ENABLED = @@ -211,7 +215,7 @@ protected TPipeTransferResp handleTransferHandshakeV2(final PipeTransferHandshak "Receiver id = {}: Handshake failed, response status = {}.", receiverId.get(), status); return new TPipeTransferResp(status); } - + CLUSTER_ID_MAP.putIfAbsent(clusterIdFromConfigNode, clusterIdFromConfigNode); // Reject to handshake if the request does not contain sender's clusterId. clusterIdFromHandshakeRequest = req.getParams().get(PipeTransferHandshakeConstant.HANDSHAKE_KEY_CLUSTER_ID); @@ -223,6 +227,7 @@ protected TPipeTransferResp handleTransferHandshakeV2(final PipeTransferHandshak "Receiver id = {}: Handshake failed, response status = {}.", receiverId.get(), status); return new TPipeTransferResp(status); } + CLUSTER_ID_MAP.putIfAbsent(clusterIdFromHandshakeRequest, clusterIdFromHandshakeRequest); // Reject to handshake if the receiver and sender are from the same cluster. if (Objects.equals(clusterIdFromConfigNode, clusterIdFromHandshakeRequest)) { @@ -315,6 +320,10 @@ protected PipeRequestType getPlanType() { protected abstract TSStatus tryLogin(); + public static Map getClusterIdMap() { + return CLUSTER_ID_MAP; + } + protected final TPipeTransferResp handleTransferFilePiece( final PipeTransferFilePieceReq req, final boolean isRequestThroughAirGap, diff --git a/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift b/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift index c15d612732e05..d79fed83bbe5a 100644 --- a/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift +++ b/iotdb-protocol/thrift-datanode/src/main/thrift/datanode.thrift @@ -390,8 +390,7 @@ struct TLoadCommandReq { 2: required string uuid 3: optional bool isGeneratedByPipe 4: optional binary progressIndex - 5: optional list regionIds - 6: optional string originClusterId + 5: optional string originClusterId } struct TAttributeUpdateReq { From f735818d264859c443ecaab8a0587994cd75b21e Mon Sep 17 00:00:00 2001 From: XNX02 <202100130044@mail.sdu.edu.cn> Date: Thu, 20 Mar 2025 23:08:43 +0800 Subject: [PATCH 20/27] serialize&deserialize --- .../event/PipeConfigRegionWritePlanEvent.java | 20 +++++- .../ConfigRegionListeningQueueTest.java | 7 ++- .../deletion/PipeDeleteDataNodeEvent.java | 4 ++ .../PipeSchemaRegionWritePlanEvent.java | 16 ++++- .../metadata/write/CreateTimeSeriesNode.java | 1 + .../dataregion/memtable/TsFileProcessor.java | 62 +++++++++++-------- .../SchemaRegionListeningQueueTest.java | 5 +- 7 files changed, 83 insertions(+), 32 deletions(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/event/PipeConfigRegionWritePlanEvent.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/event/PipeConfigRegionWritePlanEvent.java index 351b1bb42767d..cb400e8e31382 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/event/PipeConfigRegionWritePlanEvent.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/event/PipeConfigRegionWritePlanEvent.java @@ -117,10 +117,15 @@ public EnrichedEvent shallowCopySelfAndBindPipeTaskMetaForProgressReport( @Override public ByteBuffer serializeToByteBuffer() { final ByteBuffer planBuffer = configPhysicalPlan.serializeToByteBuffer(); - final ByteBuffer result = ByteBuffer.allocate(Byte.BYTES * 2 + planBuffer.limit()); + final ByteBuffer result = + ByteBuffer.allocate( + Byte.BYTES * 2 + + planBuffer.limit() + + computeOriginClusterIdBufferSize(originClusterId)); ReadWriteIOUtils.write(PipeConfigSerializableEventType.CONFIG_WRITE_PLAN.getType(), result); ReadWriteIOUtils.write(isGeneratedByPipe, result); result.put(planBuffer); + ReadWriteIOUtils.write(originClusterId, result); return result; } @@ -128,6 +133,19 @@ public ByteBuffer serializeToByteBuffer() { public void deserializeFromByteBuffer(final ByteBuffer buffer) throws IOException { isGeneratedByPipe = ReadWriteIOUtils.readBool(buffer); configPhysicalPlan = ConfigPhysicalPlan.Factory.create(buffer); + + // There might be an ignoredChildrenSize 0 + if (buffer.hasRemaining()) { + if (buffer.remaining() >= Integer.BYTES) { + buffer.mark(); // Mark current position + if (buffer.getInt() != 0) { + buffer.reset(); + } + } + if (buffer.hasRemaining()) { + originClusterId = ReadWriteIOUtils.readString(buffer); + } + } } /////////////////////////// Object /////////////////////////// diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/ConfigRegionListeningQueueTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/ConfigRegionListeningQueueTest.java index 03038a7a8c2fb..9380a04a558fb 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/ConfigRegionListeningQueueTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/ConfigRegionListeningQueueTest.java @@ -79,7 +79,8 @@ public void testSnapshot() throws TException, IOException, AuthException { "", new HashSet<>(), false, - new ArrayList<>())); + new ArrayList<>()), + "a6670472-91a4-4194-9916-08236680b4d8"); PipeConfigNodeAgent.runtime().listener().tryListenToPlan(plan1, false, null); PipeConfigNodeAgent.runtime().listener().tryListenToPlan(plan2, false, null); @@ -102,7 +103,9 @@ public void testSnapshot() throws TException, IOException, AuthException { Assert.assertEquals( plan2.getInnerPlan(), ((PipeConfigRegionWritePlanEvent) event2).getConfigPhysicalPlan()); Assert.assertTrue(((PipeConfigRegionWritePlanEvent) event2).isGeneratedByPipe()); - + Assert.assertEquals( + "a6670472-91a4-4194-9916-08236680b4d8", + ((PipeConfigRegionWritePlanEvent) event2).getOriginClusterId()); Assert.assertNull(itr.next(0)); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java index f370b27d4f339..57d1be8cf7a39 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java @@ -196,6 +196,7 @@ public ByteBuffer serializeToByteBuffer() { Byte.BYTES + planBuffer.limit() + computeOriginClusterIdBufferSize(originClusterId)); ReadWriteIOUtils.write(isGeneratedByPipe, result); result.put(planBuffer); + ReadWriteIOUtils.write(originClusterId, result); return result; } @@ -204,6 +205,9 @@ public void deserializeFromByteBuffer(final ByteBuffer buffer) { isGeneratedByPipe = ReadWriteIOUtils.readBool(buffer); deleteDataNode = (DeleteDataNode) PlanNodeType.deserialize(buffer); progressIndex = deleteDataNode.getProgressIndex(); + if (buffer.hasRemaining()) { + originClusterId = ReadWriteIOUtils.readString(buffer); + } } public static PipeDeleteDataNodeEvent deserialize(final ByteBuffer buffer) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java index 88cd6874a901d..a7ee6c719f259 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java @@ -111,16 +111,28 @@ public ByteBuffer serializeToByteBuffer() { + computeOriginClusterIdBufferSize(originClusterId)); ReadWriteIOUtils.write(PipeSchemaSerializableEventType.SCHEMA_WRITE_PLAN.getType(), result); ReadWriteIOUtils.write(isGeneratedByPipe, result); - ReadWriteIOUtils.write(originClusterId, result); result.put(planBuffer); + ReadWriteIOUtils.write(originClusterId, result); return result; } @Override public void deserializeFromByteBuffer(final ByteBuffer buffer) { isGeneratedByPipe = ReadWriteIOUtils.readBool(buffer); - originClusterId = ReadWriteIOUtils.readString(buffer); planNode = PlanNodeType.deserialize(buffer); + + // There might be an ignoredChildrenSize 0 + if (buffer.hasRemaining()) { + if (buffer.remaining() >= Integer.BYTES) { + buffer.mark(); // Mark current position + if (buffer.getInt() != 0) { + buffer.reset(); + } + } + if (buffer.hasRemaining()) { + originClusterId = ReadWriteIOUtils.readString(buffer); + } + } } /////////////////////////// Object /////////////////////////// diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metadata/write/CreateTimeSeriesNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metadata/write/CreateTimeSeriesNode.java index 19e49ec4c0b81..08a5f6da18860 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metadata/write/CreateTimeSeriesNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metadata/write/CreateTimeSeriesNode.java @@ -244,6 +244,7 @@ public static CreateTimeSeriesNode deserialize(final ByteBuffer byteBuffer) { } id = ReadWriteIOUtils.readString(byteBuffer); + return new CreateTimeSeriesNode( new PlanNodeId(id), path, dataType, encoding, compressor, props, tags, attributes, alias); } 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 07569c1500f6a..3b78db3e16e85 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,16 +321,18 @@ public void insert(InsertRowNode insertRowNode, long[] infoForMetrics) if (!insertRowNode.isGeneratedByPipe()) { workMemTable.markAsNotGeneratedByPipe(); } - final String originClusterId = - insertRowNode.getOriginClusterId() == null - ? config.getClusterId() - : insertRowNode.getOriginClusterId(); - if (Objects.isNull(workMemTable.getCurrentOriginClusterId())) { - workMemTable.setCurrentOriginClusterId(originClusterId); - // Use ClusterIdMap to compare the clusterIds by location - } else if (IoTDBFileReceiver.getClusterIdMap().get(originClusterId) - != IoTDBFileReceiver.getClusterIdMap().get(workMemTable.getCurrentOriginClusterId())) { - workMemTable.markAsNotFromTheSameCluster(); + if (workMemTable.isTotallyFromTheSameCluster()) { + final String originClusterId = + insertRowNode.getOriginClusterId() == null + ? config.getClusterId() + : insertRowNode.getOriginClusterId(); + if (Objects.isNull(workMemTable.getCurrentOriginClusterId())) { + workMemTable.setCurrentOriginClusterId(originClusterId); + // Use ClusterIdMap to compare the clusterIds by location rather than value + } else if (IoTDBFileReceiver.getClusterIdMap().get(originClusterId) + != IoTDBFileReceiver.getClusterIdMap().get(workMemTable.getCurrentOriginClusterId())) { + workMemTable.markAsNotFromTheSameCluster(); + } } PipeInsertionDataNodeListener.getInstance() @@ -424,14 +426,18 @@ public void insertRows(InsertRowsNode insertRowsNode, long[] infoForMetrics) if (!insertRowsNode.isGeneratedByPipe()) { workMemTable.markAsNotGeneratedByPipe(); } - final String originClusterId = - insertRowsNode.getOriginClusterId() == null - ? config.getClusterId() - : insertRowsNode.getOriginClusterId(); - if (Objects.isNull(workMemTable.getCurrentOriginClusterId())) { - workMemTable.setCurrentOriginClusterId(originClusterId); - } else if (!Objects.equals(originClusterId, workMemTable.getCurrentOriginClusterId())) { - workMemTable.markAsNotFromTheSameCluster(); + if (workMemTable.isTotallyFromTheSameCluster()) { + final String originClusterId = + insertRowsNode.getOriginClusterId() == null + ? config.getClusterId() + : insertRowsNode.getOriginClusterId(); + if (Objects.isNull(workMemTable.getCurrentOriginClusterId())) { + workMemTable.setCurrentOriginClusterId(originClusterId); + // Use ClusterIdMap to compare the clusterIds by location rather than value + } else if (IoTDBFileReceiver.getClusterIdMap().get(originClusterId) + != IoTDBFileReceiver.getClusterIdMap().get(workMemTable.getCurrentOriginClusterId())) { + workMemTable.markAsNotFromTheSameCluster(); + } } PipeInsertionDataNodeListener.getInstance() .listenToInsertNode( @@ -600,14 +606,18 @@ public void insertTablet( if (!insertTabletNode.isGeneratedByPipe()) { workMemTable.markAsNotGeneratedByPipe(); } - final String originClusterId = - insertTabletNode.getOriginClusterId() == null - ? config.getClusterId() - : insertTabletNode.getOriginClusterId(); - if (Objects.isNull(workMemTable.getCurrentOriginClusterId())) { - workMemTable.setCurrentOriginClusterId(originClusterId); - } else if (!Objects.equals(originClusterId, workMemTable.getCurrentOriginClusterId())) { - workMemTable.markAsNotFromTheSameCluster(); + if (workMemTable.isTotallyFromTheSameCluster()) { + final String originClusterId = + insertTabletNode.getOriginClusterId() == null + ? config.getClusterId() + : insertTabletNode.getOriginClusterId(); + if (Objects.isNull(workMemTable.getCurrentOriginClusterId())) { + workMemTable.setCurrentOriginClusterId(originClusterId); + // Use ClusterIdMap to compare the clusterIds by location rather than value + } else if (IoTDBFileReceiver.getClusterIdMap().get(originClusterId) + != IoTDBFileReceiver.getClusterIdMap().get(workMemTable.getCurrentOriginClusterId())) { + workMemTable.markAsNotFromTheSameCluster(); + } } PipeInsertionDataNodeListener.getInstance() diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/SchemaRegionListeningQueueTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/SchemaRegionListeningQueueTest.java index 6f0b8799f35cc..d0fa472507d79 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/SchemaRegionListeningQueueTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/extractor/SchemaRegionListeningQueueTest.java @@ -86,7 +86,7 @@ public void testSnapshot() throws TException, IOException, AuthException, Illega new PipeEnrichedWritePlanNode( new ActivateTemplateNode( new PlanNodeId("ActivateTemplateNode"), new PartialPath("root.sg.d1.s1"), 2, 1)); - + node2.setOriginClusterId("a6670472-91a4-4194-9916-08236680b4d8"); PipeDataNodeAgent.runtime().schemaListener(new SchemaRegionId(0)).tryListenToNode(node1); PipeDataNodeAgent.runtime().schemaListener(new SchemaRegionId(0)).tryListenToNode(node2); @@ -108,6 +108,9 @@ public void testSnapshot() throws TException, IOException, AuthException, Illega Assert.assertEquals( node2.getWritePlanNode(), ((PipeSchemaRegionWritePlanEvent) event2).getPlanNode()); Assert.assertTrue(((PipeSchemaRegionWritePlanEvent) event2).isGeneratedByPipe()); + Assert.assertEquals( + "a6670472-91a4-4194-9916-08236680b4d8", + ((PipeSchemaRegionWritePlanEvent) event2).getOriginClusterId()); Assert.assertNull(itr.next(0)); } From 0c0d90e98d61f241787f721190527ee6aca040ae Mon Sep 17 00:00:00 2001 From: XNX02 <202100130044@mail.sdu.edu.cn> Date: Thu, 20 Mar 2025 23:21:33 +0800 Subject: [PATCH 21/27] update --- .../request/write/pipe/payload/PipeEnrichedPlan.java | 5 +++-- .../consensus/request/ConfigPhysicalPlanSerDeTest.java | 3 ++- .../plan/node/metadata/write/CreateTimeSeriesNode.java | 1 - 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlan.java index 7d7c90e9eb8c7..feefac5e99d4d 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlan.java @@ -93,12 +93,13 @@ public boolean equals(Object obj) { return false; } PipeEnrichedPlan that = (PipeEnrichedPlan) obj; - return innerPlan.equals(that.innerPlan); + return innerPlan.equals(that.innerPlan) + && Objects.equals(originClusterId, that.originClusterId); } @Override public int hashCode() { - return Objects.hash(innerPlan); + return Objects.hash(innerPlan, originClusterId); } @Override diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java index bde18b1e6aefa..067b2afef93fb 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java @@ -1677,7 +1677,8 @@ public void pipeEnrichedPlanTest() throws IOException { .setTTL(Long.MAX_VALUE) .setSchemaReplicationFactor(3) .setDataReplicationFactor(3) - .setTimePartitionInterval(604800))); + .setTimePartitionInterval(604800)), + "a6670472-91a4-4194-9916-08236680b4d8"); Assert.assertEquals(plan, ConfigPhysicalPlan.Factory.create(plan.serializeToByteBuffer())); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metadata/write/CreateTimeSeriesNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metadata/write/CreateTimeSeriesNode.java index 08a5f6da18860..19e49ec4c0b81 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metadata/write/CreateTimeSeriesNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/metadata/write/CreateTimeSeriesNode.java @@ -244,7 +244,6 @@ public static CreateTimeSeriesNode deserialize(final ByteBuffer byteBuffer) { } id = ReadWriteIOUtils.readString(byteBuffer); - return new CreateTimeSeriesNode( new PlanNodeId(id), path, dataType, encoding, compressor, props, tags, attributes, alias); } From af22cab769608a54c602862be7f85625d854cf77 Mon Sep 17 00:00:00 2001 From: XNX02 <202100130044@mail.sdu.edu.cn> Date: Fri, 21 Mar 2025 13:48:06 +0800 Subject: [PATCH 22/27] fix --- .../db/queryengine/plan/analyze/load/LoadTsFileAnalyzer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 7023ef9875c5d..e7c9a6aa004db 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 @@ -390,7 +390,7 @@ private boolean handleSingleMiniFile(final int i) throws FileNotFoundException { final long startTime = System.nanoTime(); try { final LoadTsFileDataTypeConverter loadTsFileDataTypeConverter = - new LoadTsFileDataTypeConverter(isGeneratedByPipe); + new LoadTsFileDataTypeConverter(isGeneratedByPipe, originClusterId); final TSStatus status = isTableModelTsFile.get(i) From 0c4b22cad375d82958dc6e32119346fbccb015da Mon Sep 17 00:00:00 2001 From: XNX02 <202100130044@mail.sdu.edu.cn> Date: Mon, 24 Mar 2025 02:06:25 +0800 Subject: [PATCH 23/27] update --- .../write/pipe/payload/PipeEnrichedPlan.java | 19 ++++--------------- 1 file changed, 4 insertions(+), 15 deletions(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlan.java index feefac5e99d4d..3f5868bcd88fb 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlan.java @@ -22,6 +22,8 @@ import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan; import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType; +import org.apache.tsfile.utils.ReadWriteIOUtils; + import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; @@ -62,26 +64,13 @@ protected void serializeImpl(DataOutputStream stream) throws IOException { ByteBuffer buffer = innerPlan.serializeToByteBuffer(); stream.write(buffer.array(), 0, buffer.limit()); - if (originClusterId == null) { - stream.writeBoolean(false); - } else { - stream.writeBoolean(true); - stream.writeUTF(originClusterId); - } + ReadWriteIOUtils.write(originClusterId, stream); } @Override protected void deserializeImpl(ByteBuffer buffer) throws IOException { innerPlan = ConfigPhysicalPlan.Factory.create(buffer); - - if (buffer.hasRemaining() && buffer.get() == 1) { // Read boolean - int strLength = buffer.getShort(); - byte[] bytes = new byte[strLength]; - buffer.get(bytes); - originClusterId = new String(bytes); - } else { - originClusterId = null; - } + originClusterId = ReadWriteIOUtils.readString(buffer); } @Override From 62b00d4be247f66bef564a489bee362e9c7c954d Mon Sep 17 00:00:00 2001 From: XNX02 <202100130044@mail.sdu.edu.cn> Date: Mon, 24 Mar 2025 02:14:24 +0800 Subject: [PATCH 24/27] update --- .../consensus/request/write/pipe/payload/PipeEnrichedPlan.java | 1 - 1 file changed, 1 deletion(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlan.java index 3f5868bcd88fb..08f5eeb9a9096 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlan.java @@ -63,7 +63,6 @@ protected void serializeImpl(DataOutputStream stream) throws IOException { stream.writeShort(getType().getPlanType()); ByteBuffer buffer = innerPlan.serializeToByteBuffer(); stream.write(buffer.array(), 0, buffer.limit()); - ReadWriteIOUtils.write(originClusterId, stream); } From 4779f7c79daf45362da11990d161c01dc227c1ea Mon Sep 17 00:00:00 2001 From: XNX02 <202100130044@mail.sdu.edu.cn> Date: Wed, 26 Mar 2025 01:23:40 +0800 Subject: [PATCH 25/27] pipeenrichedplanV2 --- .../consensus/request/ConfigPhysicalPlan.java | 4 + .../request/ConfigPhysicalPlanType.java | 1 + .../write/pipe/payload/PipeEnrichedPlan.java | 23 ++--- .../pipe/payload/PipeEnrichedPlanV2.java | 84 +++++++++++++++++++ .../confignode/manager/PermissionManager.java | 4 +- .../ConfigRegionListeningFilter.java | 1 + .../extractor/ConfigRegionListeningQueue.java | 8 +- .../protocol/IoTDBConfigNodeReceiver.java | 4 +- .../manager/schema/ClusterSchemaManager.java | 22 ++--- .../executor/ConfigPlanExecutor.java | 1 + .../schema/DeactivateTemplateProcedure.java | 4 +- .../schema/DeleteLogicalViewProcedure.java | 4 +- .../schema/DeleteTimeSeriesProcedure.java | 4 +- .../impl/schema/SetTTLProcedure.java | 6 +- .../impl/schema/SetTemplateProcedure.java | 4 +- .../schema/table/CreateTableProcedure.java | 4 +- .../schema/table/DeleteDevicesProcedure.java | 4 +- .../table/DropTableColumnProcedure.java | 4 +- .../impl/schema/table/DropTableProcedure.java | 4 +- .../impl/sync/AuthOperationProcedure.java | 4 +- .../impl/trigger/CreateTriggerProcedure.java | 6 +- .../impl/trigger/DropTriggerProcedure.java | 4 +- .../request/ConfigPhysicalPlanSerDeTest.java | 3 +- .../ConfigRegionListeningQueueTest.java | 3 +- 24 files changed, 148 insertions(+), 62 deletions(-) create mode 100644 iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlanV2.java diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlan.java index 081caa9e868fb..7115ba1024f4e 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlan.java @@ -68,6 +68,7 @@ import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteLogicalViewPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteTimeSeriesPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlanV2; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeUnsetSchemaTemplatePlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.plugin.CreatePipePluginPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.plugin.DropPipePluginPlan; @@ -460,6 +461,9 @@ public static ConfigPhysicalPlan create(final ByteBuffer buffer) throws IOExcept case PipeEnriched: plan = new PipeEnrichedPlan(); break; + case PipeEnrichedV2: + plan = new PipeEnrichedPlanV2(); + break; case CreateTopic: plan = new CreateTopicPlan(); break; diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanType.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanType.java index 054881622cdd6..8545bba33c228 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanType.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanType.java @@ -286,6 +286,7 @@ public enum ConfigPhysicalPlanType { PipeSetTTL((short) 1705), PipeCreateTable((short) 1706), PipeDeleteDevices((short) 1707), + PipeEnrichedV2((short) 1708), /** Subscription */ CreateTopic((short) 1800), diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlan.java index 08f5eeb9a9096..33257a6639f5c 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlan.java @@ -22,8 +22,6 @@ import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan; import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType; -import org.apache.tsfile.utils.ReadWriteIOUtils; - import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; @@ -31,9 +29,7 @@ public class PipeEnrichedPlan extends ConfigPhysicalPlan { - private ConfigPhysicalPlan innerPlan; - - private String originClusterId; + protected ConfigPhysicalPlan innerPlan; public PipeEnrichedPlan() { super(ConfigPhysicalPlanType.PipeEnriched); @@ -44,32 +40,24 @@ public PipeEnrichedPlan(ConfigPhysicalPlan innerPlan) { this.innerPlan = innerPlan; } - public PipeEnrichedPlan(ConfigPhysicalPlan innerPlan, String originClusterId) { - super(ConfigPhysicalPlanType.PipeEnriched); - this.innerPlan = innerPlan; - this.originClusterId = originClusterId; + protected PipeEnrichedPlan(ConfigPhysicalPlanType type) { + super(type); } public ConfigPhysicalPlan getInnerPlan() { return innerPlan; } - public String getOriginClusterId() { - return originClusterId; - } - @Override protected void serializeImpl(DataOutputStream stream) throws IOException { stream.writeShort(getType().getPlanType()); ByteBuffer buffer = innerPlan.serializeToByteBuffer(); stream.write(buffer.array(), 0, buffer.limit()); - ReadWriteIOUtils.write(originClusterId, stream); } @Override protected void deserializeImpl(ByteBuffer buffer) throws IOException { innerPlan = ConfigPhysicalPlan.Factory.create(buffer); - originClusterId = ReadWriteIOUtils.readString(buffer); } @Override @@ -81,13 +69,12 @@ public boolean equals(Object obj) { return false; } PipeEnrichedPlan that = (PipeEnrichedPlan) obj; - return innerPlan.equals(that.innerPlan) - && Objects.equals(originClusterId, that.originClusterId); + return innerPlan.equals(that.innerPlan); } @Override public int hashCode() { - return Objects.hash(innerPlan, originClusterId); + return Objects.hash(innerPlan); } @Override diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlanV2.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlanV2.java new file mode 100644 index 0000000000000..6daf994698b8f --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlanV2.java @@ -0,0 +1,84 @@ +/* + * 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.confignode.consensus.request.write.pipe.payload; + +import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan; +import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType; + +import org.apache.tsfile.utils.ReadWriteIOUtils; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Objects; + +public class PipeEnrichedPlanV2 extends PipeEnrichedPlan { + + private String originClusterId; + + public PipeEnrichedPlanV2() { + super(ConfigPhysicalPlanType.PipeEnrichedV2); + } + + public PipeEnrichedPlanV2(ConfigPhysicalPlan innerPlan, String originClusterId) { + super(ConfigPhysicalPlanType.PipeEnrichedV2); + this.innerPlan = innerPlan; + this.originClusterId = originClusterId; + } + + public String getOriginClusterId() { + return originClusterId; + } + + @Override + protected void serializeImpl(DataOutputStream stream) throws IOException { + super.serializeImpl(stream); + ReadWriteIOUtils.write(originClusterId, stream); + } + + @Override + protected void deserializeImpl(ByteBuffer buffer) throws IOException { + super.deserializeImpl(buffer); + originClusterId = ReadWriteIOUtils.readString(buffer); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + PipeEnrichedPlanV2 that = (PipeEnrichedPlanV2) obj; + return innerPlan.equals(that.innerPlan) + && Objects.equals(originClusterId, that.originClusterId); + } + + @Override + public int hashCode() { + return Objects.hash(innerPlan, originClusterId); + } + + @Override + public String toString() { + return "PipeEnrichedPlanV2{" + "innerPlan='" + innerPlan + "'}"; + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/PermissionManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/PermissionManager.java index 8e268dcaaedc0..b1a70d326d4f1 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/PermissionManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/PermissionManager.java @@ -25,7 +25,7 @@ import org.apache.iotdb.commons.auth.entity.PrivilegeUnion; import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType; import org.apache.iotdb.confignode.consensus.request.write.auth.AuthorPlan; -import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlanV2; import org.apache.iotdb.confignode.consensus.response.auth.PermissionInfoResp; import org.apache.iotdb.confignode.manager.consensus.ConsensusManager; import org.apache.iotdb.confignode.persistence.AuthorInfo; @@ -72,7 +72,7 @@ public TSStatus operatePermission( getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan(authorPlan, originClusterIds) + ? new PipeEnrichedPlanV2(authorPlan, originClusterIds) : authorPlan); } else { List allDataNodes = diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/ConfigRegionListeningFilter.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/ConfigRegionListeningFilter.java index 82d23b76350c9..af19619780cf6 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/ConfigRegionListeningFilter.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/ConfigRegionListeningFilter.java @@ -235,6 +235,7 @@ static boolean shouldPlanBeListened(final ConfigPhysicalPlan plan) { // PipeEnriched & UnsetTemplate are not listened directly, // but their inner plan or converted plan are listened. return type.equals(ConfigPhysicalPlanType.PipeEnriched) + || type.equals(ConfigPhysicalPlanType.PipeEnrichedV2) || type.equals(ConfigPhysicalPlanType.UnsetTemplate) || OPTION_PLAN_MAP.values().stream().anyMatch(types -> types.contains(type)); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/ConfigRegionListeningQueue.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/ConfigRegionListeningQueue.java index 0570ab66538c7..543dc0a5feba1 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/ConfigRegionListeningQueue.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/ConfigRegionListeningQueue.java @@ -30,6 +30,7 @@ import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeCreateTablePlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlanV2; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeUnsetSchemaTemplatePlan; import org.apache.iotdb.confignode.consensus.request.write.table.CommitCreateTablePlan; import org.apache.iotdb.confignode.consensus.request.write.template.UnsetSchemaTemplatePlan; @@ -71,10 +72,13 @@ public synchronized void tryListenToPlan( final PipeConfigRegionWritePlanEvent event; switch (plan.getType()) { case PipeEnriched: + tryListenToPlan(((PipeEnrichedPlan) plan).getInnerPlan(), true, null); + return; + case PipeEnrichedV2: tryListenToPlan( - ((PipeEnrichedPlan) plan).getInnerPlan(), + ((PipeEnrichedPlanV2) plan).getInnerPlan(), true, - (((PipeEnrichedPlan) plan).getOriginClusterId())); + (((PipeEnrichedPlanV2) plan).getOriginClusterId())); return; case UnsetTemplate: // Different clusters have different template ids, so we need to diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java index 30eab35ff92ee..732e182e5aa6b 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/receiver/protocol/IoTDBConfigNodeReceiver.java @@ -54,7 +54,7 @@ import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteDevicesPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteLogicalViewPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteTimeSeriesPlan; -import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlanV2; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeUnsetSchemaTemplatePlan; import org.apache.iotdb.confignode.consensus.request.write.table.AbstractTablePlan; import org.apache.iotdb.confignode.consensus.request.write.table.AddTableColumnPlan; @@ -756,7 +756,7 @@ private TSStatus executePlan(final ConfigPhysicalPlan plan) throws ConsensusExce default: return configManager .getConsensusManager() - .write(new PipeEnrichedPlan(plan, clusterIdFromHandshakeRequest)); + .write(new PipeEnrichedPlanV2(plan, clusterIdFromHandshakeRequest)); } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java index 343ce6c13a9b4..46494db776ae5 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java @@ -56,7 +56,7 @@ import org.apache.iotdb.confignode.consensus.request.write.database.SetDataReplicationFactorPlan; import org.apache.iotdb.confignode.consensus.request.write.database.SetSchemaReplicationFactorPlan; import org.apache.iotdb.confignode.consensus.request.write.database.SetTimePartitionIntervalPlan; -import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlanV2; import org.apache.iotdb.confignode.consensus.request.write.table.AddTableColumnPlan; import org.apache.iotdb.confignode.consensus.request.write.table.RenameTableColumnPlan; import org.apache.iotdb.confignode.consensus.request.write.table.SetTableColumnCommentPlan; @@ -183,7 +183,7 @@ public TSStatus setDatabase( getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan(databaseSchemaPlan, originClusterIds) + ? new PipeEnrichedPlanV2(databaseSchemaPlan, originClusterIds) : databaseSchemaPlan); // set ttl if (schema.isSetTTL()) { @@ -267,7 +267,7 @@ public TSStatus alterDatabase( getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan(databaseSchemaPlan, originClusterId) + ? new PipeEnrichedPlanV2(databaseSchemaPlan, originClusterId) : databaseSchemaPlan); PartitionMetrics.bindDatabaseReplicationFactorMetricsWhenUpdate( MetricService.getInstance(), @@ -294,7 +294,7 @@ public TSStatus deleteDatabase( getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan(deleteDatabasePlan, originClusterId) + ? new PipeEnrichedPlanV2(deleteDatabasePlan, originClusterId) : deleteDatabasePlan); } catch (final ConsensusException e) { LOGGER.warn(CONSENSUS_WRITE_ERROR, e); @@ -933,7 +933,7 @@ public TSStatus unsetSchemaTemplateInBlackList( return getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan( + ? new PipeEnrichedPlanV2( new UnsetSchemaTemplatePlan(templateId, path), originClusterId) : new UnsetSchemaTemplatePlan(templateId, path)); } catch (ConsensusException e) { @@ -1043,7 +1043,7 @@ public synchronized TSStatus extendSchemaTemplate( getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan(extendSchemaTemplatePlan, originClusterId) + ? new PipeEnrichedPlanV2(extendSchemaTemplatePlan, originClusterId) : extendSchemaTemplatePlan); } catch (ConsensusException e) { LOGGER.warn(CONSENSUS_WRITE_ERROR, e); @@ -1299,7 +1299,7 @@ public synchronized TSStatus addTableColumn( return getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan(addTableColumnPlan, originClusterId) + ? new PipeEnrichedPlanV2(addTableColumnPlan, originClusterId) : addTableColumnPlan); } catch (final ConsensusException e) { LOGGER.warn(e.getMessage(), e); @@ -1334,7 +1334,7 @@ public synchronized TSStatus renameTableColumn( return getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan(renameTableColumnPlan, originClusterId) + ? new PipeEnrichedPlanV2(renameTableColumnPlan, originClusterId) : renameTableColumnPlan); } catch (final ConsensusException e) { LOGGER.warn(e.getMessage(), e); @@ -1354,7 +1354,7 @@ public synchronized TSStatus setTableProperties( return getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan(setTablePropertiesPlan, originClusterId) + ? new PipeEnrichedPlanV2(setTablePropertiesPlan, originClusterId) : setTablePropertiesPlan); } catch (final ConsensusException e) { LOGGER.warn(e.getMessage(), e); @@ -1374,7 +1374,7 @@ public synchronized TSStatus setTableComment( return getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan(setTableCommentPlan, originClusterId) + ? new PipeEnrichedPlanV2(setTableCommentPlan, originClusterId) : setTableCommentPlan); } catch (final ConsensusException e) { LOGGER.warn(e.getMessage(), e); @@ -1395,7 +1395,7 @@ public synchronized TSStatus setTableColumnComment( return getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan(setTableColumnCommentPlan, originClusterId) + ? new PipeEnrichedPlanV2(setTableColumnCommentPlan, originClusterId) : setTableColumnCommentPlan); } catch (final ConsensusException e) { LOGGER.warn(e.getMessage(), e); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java index 3915ae24a3c21..3eb89753917b0 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java @@ -647,6 +647,7 @@ public TSStatus executeNonQueryPlan(ConfigPhysicalPlan physicalPlan) case ShowPipeV1: return new TSStatus(TSStatusCode.INCOMPATIBLE_VERSION.getStatusCode()); case PipeEnriched: + case PipeEnrichedV2: return executeNonQueryPlan(((PipeEnrichedPlan) physicalPlan).getInnerPlan()); case PipeDeleteTimeSeries: case PipeDeleteLogicalView: diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeactivateTemplateProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeactivateTemplateProcedure.java index 74d8ae3327e18..2ae20d204a2ce 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeactivateTemplateProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeactivateTemplateProcedure.java @@ -32,7 +32,7 @@ import org.apache.iotdb.confignode.client.async.CnToDnInternalServiceAsyncRequestManager; import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeactivateTemplatePlan; -import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlanV2; import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; import org.apache.iotdb.confignode.procedure.exception.ProcedureException; import org.apache.iotdb.confignode.procedure.exception.ProcedureSuspendedException; @@ -274,7 +274,7 @@ private void collectPayload4Pipe(ConfigNodeProcedureEnv env) { .getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan( + ? new PipeEnrichedPlanV2( new PipeDeactivateTemplatePlan(templateSetInfo), originClusterId) : new PipeDeactivateTemplatePlan(templateSetInfo)); } catch (ConsensusException e) { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteLogicalViewProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteLogicalViewProcedure.java index d1d116e5ebbd1..bd151a0b3872f 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteLogicalViewProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteLogicalViewProcedure.java @@ -30,7 +30,7 @@ import org.apache.iotdb.confignode.client.async.CnToDnInternalServiceAsyncRequestManager; import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteLogicalViewPlan; -import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlanV2; import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; import org.apache.iotdb.confignode.procedure.exception.ProcedureException; import org.apache.iotdb.confignode.procedure.exception.ProcedureSuspendedException; @@ -236,7 +236,7 @@ private void collectPayload4Pipe(final ConfigNodeProcedureEnv env) { .getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan( + ? new PipeEnrichedPlanV2( new PipeDeleteLogicalViewPlan(patternTreeBytes), originClusterId) : new PipeDeleteLogicalViewPlan(patternTreeBytes)); } catch (final ConsensusException e) { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java index 2309ab4e71fd5..2f659c35ef422 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/DeleteTimeSeriesProcedure.java @@ -30,7 +30,7 @@ import org.apache.iotdb.confignode.client.async.CnToDnInternalServiceAsyncRequestManager; import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteTimeSeriesPlan; -import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlanV2; import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; import org.apache.iotdb.confignode.procedure.exception.ProcedureException; import org.apache.iotdb.confignode.procedure.exception.ProcedureSuspendedException; @@ -290,7 +290,7 @@ private void collectPayload4Pipe(final ConfigNodeProcedureEnv env) { .getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan( + ? new PipeEnrichedPlanV2( new PipeDeleteTimeSeriesPlan(patternTreeBytes), originClusterId) : new PipeDeleteTimeSeriesPlan(patternTreeBytes)); } catch (final ConsensusException e) { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTTLProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTTLProcedure.java index 6442d99890c14..debbd74b74b05 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTTLProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTTLProcedure.java @@ -29,7 +29,7 @@ import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan; import org.apache.iotdb.confignode.consensus.request.write.database.SetTTLPlan; -import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlanV2; import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; import org.apache.iotdb.confignode.procedure.exception.ProcedureException; import org.apache.iotdb.confignode.procedure.exception.ProcedureSuspendedException; @@ -98,7 +98,9 @@ private void setConfigNodeTTL(ConfigNodeProcedureEnv env) { env.getConfigManager() .getConsensusManager() .write( - isGeneratedByPipe ? new PipeEnrichedPlan(this.plan, originClusterId) : this.plan); + isGeneratedByPipe + ? new PipeEnrichedPlanV2(this.plan, originClusterId) + : this.plan); } catch (ConsensusException e) { LOGGER.warn("Failed in the write API executing the consensus layer due to: ", e); res = new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTemplateProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTemplateProcedure.java index fa1ca49ca7af5..cf05a0d872d40 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTemplateProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/SetTemplateProcedure.java @@ -33,7 +33,7 @@ import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; import org.apache.iotdb.confignode.consensus.request.read.template.CheckTemplateSettablePlan; import org.apache.iotdb.confignode.consensus.request.read.template.GetSchemaTemplatePlan; -import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlanV2; import org.apache.iotdb.confignode.consensus.request.write.template.CommitSetSchemaTemplatePlan; import org.apache.iotdb.confignode.consensus.request.write.template.PreSetSchemaTemplatePlan; import org.apache.iotdb.confignode.consensus.response.template.TemplateInfoResp; @@ -368,7 +368,7 @@ private void commitSetTemplate(final ConfigNodeProcedureEnv env) { .getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan(commitSetSchemaTemplatePlan, originClusterId) + ? new PipeEnrichedPlanV2(commitSetSchemaTemplatePlan, originClusterId) : commitSetSchemaTemplatePlan); } catch (final ConsensusException e) { LOGGER.warn(CONSENSUS_WRITE_ERROR, e); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/CreateTableProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/CreateTableProcedure.java index 75fdc50028ef6..c07ba9ba20333 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/CreateTableProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/CreateTableProcedure.java @@ -23,7 +23,7 @@ import org.apache.iotdb.commons.exception.IoTDBException; import org.apache.iotdb.commons.exception.MetadataException; import org.apache.iotdb.commons.schema.table.TsTable; -import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlanV2; import org.apache.iotdb.confignode.consensus.request.write.table.CommitCreateTablePlan; import org.apache.iotdb.confignode.consensus.request.write.table.PreCreateTablePlan; import org.apache.iotdb.confignode.consensus.request.write.table.RollbackCreateTablePlan; @@ -180,7 +180,7 @@ private void commitCreateTable(final ConfigNodeProcedureEnv env) { final TSStatus status = SchemaUtils.executeInConsensusLayer( isGeneratedByPipe - ? new PipeEnrichedPlan( + ? new PipeEnrichedPlanV2( new CommitCreateTablePlan(database, table.getTableName()), originClusterId) : new CommitCreateTablePlan(database, table.getTableName()), env, diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/DeleteDevicesProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/DeleteDevicesProcedure.java index 39934f9a1674c..beddb86918a63 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/DeleteDevicesProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/DeleteDevicesProcedure.java @@ -29,7 +29,7 @@ import org.apache.iotdb.confignode.client.async.CnToDnInternalServiceAsyncRequestManager; import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteDevicesPlan; -import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlanV2; import org.apache.iotdb.confignode.manager.ClusterManager; import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; import org.apache.iotdb.confignode.procedure.exception.ProcedureException; @@ -302,7 +302,7 @@ private void collectPayload4Pipe(final ConfigNodeProcedureEnv env) { .getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan( + ? new PipeEnrichedPlanV2( new PipeDeleteDevicesPlan( database, tableName, patternBytes, filterBytes, modBytes), originClusterId) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/DropTableColumnProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/DropTableColumnProcedure.java index d50998db38e20..72ecdf8046b34 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/DropTableColumnProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/DropTableColumnProcedure.java @@ -28,7 +28,7 @@ import org.apache.iotdb.confignode.client.async.CnToDnAsyncRequestType; import org.apache.iotdb.confignode.client.async.CnToDnInternalServiceAsyncRequestManager; import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; -import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlanV2; import org.apache.iotdb.confignode.consensus.request.write.table.CommitDeleteColumnPlan; import org.apache.iotdb.confignode.consensus.request.write.table.PreDeleteColumnPlan; import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; @@ -213,7 +213,7 @@ private void dropColumn(final ConfigNodeProcedureEnv env) { final TSStatus status = SchemaUtils.executeInConsensusLayer( isGeneratedByPipe - ? new PipeEnrichedPlan( + ? new PipeEnrichedPlanV2( new CommitDeleteColumnPlan(database, tableName, columnName), originClusterId) : new CommitDeleteColumnPlan(database, tableName, columnName), env, diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/DropTableProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/DropTableProcedure.java index e50c41fc235da..9b4f768af5a3e 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/DropTableProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/schema/table/DropTableProcedure.java @@ -28,7 +28,7 @@ import org.apache.iotdb.confignode.client.async.CnToDnAsyncRequestType; import org.apache.iotdb.confignode.client.async.CnToDnInternalServiceAsyncRequestManager; import org.apache.iotdb.confignode.client.async.handlers.DataNodeAsyncRequestContext; -import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlanV2; import org.apache.iotdb.confignode.consensus.request.write.table.CommitDeleteTablePlan; import org.apache.iotdb.confignode.consensus.request.write.table.PreDeleteTablePlan; import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; @@ -199,7 +199,7 @@ private void dropTable(final ConfigNodeProcedureEnv env) { final TSStatus status = SchemaUtils.executeInConsensusLayer( isGeneratedByPipe - ? new PipeEnrichedPlan( + ? new PipeEnrichedPlanV2( new CommitDeleteTablePlan(database, tableName), originClusterId) : new CommitDeleteTablePlan(database, tableName), env, diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/sync/AuthOperationProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/sync/AuthOperationProcedure.java index 8fe60b2250e40..7bfbd29bcef2d 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/sync/AuthOperationProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/sync/AuthOperationProcedure.java @@ -29,7 +29,7 @@ import org.apache.iotdb.confignode.client.sync.SyncDataNodeClientPool; import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan; import org.apache.iotdb.confignode.consensus.request.write.auth.AuthorPlan; -import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlanV2; import org.apache.iotdb.confignode.procedure.env.ConfigNodeProcedureEnv; import org.apache.iotdb.confignode.procedure.exception.ProcedureException; import org.apache.iotdb.confignode.procedure.impl.node.AbstractNodeProcedure; @@ -160,7 +160,7 @@ private void writePlan(ConfigNodeProcedureEnv env) { res = env.getConfigManager() .getConsensusManager() - .write(isGeneratedByPipe ? new PipeEnrichedPlan(plan, originClusterId) : plan); + .write(isGeneratedByPipe ? new PipeEnrichedPlanV2(plan, originClusterId) : plan); } catch (ConsensusException e) { LOGGER.warn(CONSENSUS_WRITE_ERROR, e); res = new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/trigger/CreateTriggerProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/trigger/CreateTriggerProcedure.java index b16b4ba3d775b..405b00200123a 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/trigger/CreateTriggerProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/trigger/CreateTriggerProcedure.java @@ -22,7 +22,7 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.trigger.TriggerInformation; import org.apache.iotdb.commons.trigger.exception.TriggerManagementException; -import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlanV2; import org.apache.iotdb.confignode.consensus.request.write.trigger.AddTriggerInTablePlan; import org.apache.iotdb.confignode.consensus.request.write.trigger.DeleteTriggerInTablePlan; import org.apache.iotdb.confignode.consensus.request.write.trigger.UpdateTriggerStateInTablePlan; @@ -163,7 +163,7 @@ protected Flow executeFromState( .getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan( + ? new PipeEnrichedPlanV2( new UpdateTriggerStateInTablePlan( triggerInformation.getTriggerName(), TTriggerState.ACTIVE), originClusterId) @@ -219,7 +219,7 @@ protected void rollbackState(ConfigNodeProcedureEnv env, CreateTriggerState stat .getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan( + ? new PipeEnrichedPlanV2( new DeleteTriggerInTablePlan(triggerInformation.getTriggerName()), originClusterId) : new DeleteTriggerInTablePlan(triggerInformation.getTriggerName())); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/trigger/DropTriggerProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/trigger/DropTriggerProcedure.java index a32259d57b470..ab0f773af9e6f 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/trigger/DropTriggerProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/trigger/DropTriggerProcedure.java @@ -20,7 +20,7 @@ package org.apache.iotdb.confignode.procedure.impl.trigger; import org.apache.iotdb.commons.trigger.exception.TriggerManagementException; -import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlanV2; import org.apache.iotdb.confignode.consensus.request.write.trigger.DeleteTriggerInTablePlan; import org.apache.iotdb.confignode.consensus.request.write.trigger.UpdateTriggerStateInTablePlan; import org.apache.iotdb.confignode.persistence.TriggerInfo; @@ -104,7 +104,7 @@ protected Flow executeFromState(ConfigNodeProcedureEnv env, DropTriggerState sta .getConsensusManager() .write( isGeneratedByPipe - ? new PipeEnrichedPlan( + ? new PipeEnrichedPlanV2( new DeleteTriggerInTablePlan(triggerName), originClusterId) : new DeleteTriggerInTablePlan(triggerName)); setNextState(DropTriggerState.CONFIG_NODE_DROPPED); diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java index 067b2afef93fb..05d0c3b1f75f9 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java @@ -97,6 +97,7 @@ import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteLogicalViewPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteTimeSeriesPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlanV2; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeUnsetSchemaTemplatePlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.plugin.CreatePipePluginPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.plugin.DropPipePluginPlan; @@ -1669,7 +1670,7 @@ public void updateClusterIdPlanTest() throws IOException { @Test public void pipeEnrichedPlanTest() throws IOException { final PipeEnrichedPlan plan = - new PipeEnrichedPlan( + new PipeEnrichedPlanV2( new DatabaseSchemaPlan( ConfigPhysicalPlanType.CreateDatabase, new TDatabaseSchema() diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/ConfigRegionListeningQueueTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/ConfigRegionListeningQueueTest.java index 9380a04a558fb..2af3bbf5ccdc1 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/ConfigRegionListeningQueueTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/ConfigRegionListeningQueueTest.java @@ -25,6 +25,7 @@ import org.apache.iotdb.confignode.consensus.request.write.auth.AuthorTreePlan; import org.apache.iotdb.confignode.consensus.request.write.database.DatabaseSchemaPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlanV2; import org.apache.iotdb.confignode.manager.pipe.agent.PipeConfigNodeAgent; import org.apache.iotdb.confignode.manager.pipe.event.PipeConfigRegionWritePlanEvent; import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchema; @@ -70,7 +71,7 @@ public void testSnapshot() throws TException, IOException, AuthException { new DatabaseSchemaPlan( ConfigPhysicalPlanType.CreateDatabase, new TDatabaseSchema("root.test1")); final PipeEnrichedPlan plan2 = - new PipeEnrichedPlan( + new PipeEnrichedPlanV2( new AuthorTreePlan( ConfigPhysicalPlanType.CreateUser, "user0", From 2e3feb72b921d9e42b188c75850b841545b2733f Mon Sep 17 00:00:00 2001 From: XNX02 <202100130044@mail.sdu.edu.cn> Date: Fri, 28 Mar 2025 01:39:58 +0800 Subject: [PATCH 26/27] update --- .../consensus/request/ConfigPhysicalPlan.java | 6 +++--- .../request/ConfigPhysicalPlanType.java | 2 +- ...EnrichedPlan.java => PipeEnrichedPlanV1.java} | 16 ++++++++-------- .../write/pipe/payload/PipeEnrichedPlanV2.java | 2 +- .../extractor/ConfigRegionListeningFilter.java | 4 ++-- .../extractor/ConfigRegionListeningQueue.java | 6 +++--- .../persistence/executor/ConfigPlanExecutor.java | 6 +++--- .../procedure/impl/StateMachineProcedure.java | 4 +++- .../request/ConfigPhysicalPlanSerDeTest.java | 4 ++-- .../ConfigRegionListeningQueueTest.java | 4 ++-- 10 files changed, 28 insertions(+), 26 deletions(-) rename iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/{PipeEnrichedPlan.java => PipeEnrichedPlanV1.java} (82%) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlan.java index 7115ba1024f4e..023056816e9f6 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlan.java @@ -67,7 +67,7 @@ import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteDevicesPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteLogicalViewPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteTimeSeriesPlan; -import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlanV1; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlanV2; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeUnsetSchemaTemplatePlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.plugin.CreatePipePluginPlan; @@ -458,8 +458,8 @@ public static ConfigPhysicalPlan create(final ByteBuffer buffer) throws IOExcept case PipeHandleMetaChange: plan = new PipeHandleMetaChangePlan(); break; - case PipeEnriched: - plan = new PipeEnrichedPlan(); + case PipeEnrichedV1: + plan = new PipeEnrichedPlanV1(); break; case PipeEnrichedV2: plan = new PipeEnrichedPlanV2(); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanType.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanType.java index 8545bba33c228..70486b7676ebd 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanType.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanType.java @@ -278,7 +278,7 @@ public enum ConfigPhysicalPlanType { PipeHandleMetaChange((short) 1601), /** Pipe PayLoad. */ - PipeEnriched((short) 1700), + PipeEnrichedV1((short) 1700), PipeUnsetTemplate((short) 1701), PipeDeleteTimeSeries((short) 1702), PipeDeleteLogicalView((short) 1703), diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlanV1.java similarity index 82% rename from iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlan.java rename to iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlanV1.java index 33257a6639f5c..f70b9a78257ba 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlanV1.java @@ -27,20 +27,20 @@ import java.nio.ByteBuffer; import java.util.Objects; -public class PipeEnrichedPlan extends ConfigPhysicalPlan { +public class PipeEnrichedPlanV1 extends ConfigPhysicalPlan { protected ConfigPhysicalPlan innerPlan; - public PipeEnrichedPlan() { - super(ConfigPhysicalPlanType.PipeEnriched); + public PipeEnrichedPlanV1() { + super(ConfigPhysicalPlanType.PipeEnrichedV1); } - public PipeEnrichedPlan(ConfigPhysicalPlan innerPlan) { - super(ConfigPhysicalPlanType.PipeEnriched); + public PipeEnrichedPlanV1(ConfigPhysicalPlan innerPlan) { + super(ConfigPhysicalPlanType.PipeEnrichedV1); this.innerPlan = innerPlan; } - protected PipeEnrichedPlan(ConfigPhysicalPlanType type) { + protected PipeEnrichedPlanV1(ConfigPhysicalPlanType type) { super(type); } @@ -68,7 +68,7 @@ public boolean equals(Object obj) { if (obj == null || getClass() != obj.getClass()) { return false; } - PipeEnrichedPlan that = (PipeEnrichedPlan) obj; + PipeEnrichedPlanV1 that = (PipeEnrichedPlanV1) obj; return innerPlan.equals(that.innerPlan); } @@ -79,6 +79,6 @@ public int hashCode() { @Override public String toString() { - return "PipeEnrichedPlan{" + "innerPlan='" + innerPlan + "'}"; + return "PipeEnrichedPlanV1{" + "innerPlan='" + innerPlan + "'}"; } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlanV2.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlanV2.java index 6daf994698b8f..25d4e73d30a1b 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlanV2.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/payload/PipeEnrichedPlanV2.java @@ -29,7 +29,7 @@ import java.nio.ByteBuffer; import java.util.Objects; -public class PipeEnrichedPlanV2 extends PipeEnrichedPlan { +public class PipeEnrichedPlanV2 extends PipeEnrichedPlanV1 { private String originClusterId; diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/ConfigRegionListeningFilter.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/ConfigRegionListeningFilter.java index af19619780cf6..c160c51daed58 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/ConfigRegionListeningFilter.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/ConfigRegionListeningFilter.java @@ -232,9 +232,9 @@ static boolean shouldPlanBeListened(final ConfigPhysicalPlan plan) { return false; } - // PipeEnriched & UnsetTemplate are not listened directly, + // PipeEnrichedV1 & UnsetTemplate are not listened directly, // but their inner plan or converted plan are listened. - return type.equals(ConfigPhysicalPlanType.PipeEnriched) + return type.equals(ConfigPhysicalPlanType.PipeEnrichedV1) || type.equals(ConfigPhysicalPlanType.PipeEnrichedV2) || type.equals(ConfigPhysicalPlanType.UnsetTemplate) || OPTION_PLAN_MAP.values().stream().anyMatch(types -> types.contains(type)); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/ConfigRegionListeningQueue.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/ConfigRegionListeningQueue.java index 543dc0a5feba1..1388280d30756 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/ConfigRegionListeningQueue.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/extractor/ConfigRegionListeningQueue.java @@ -29,7 +29,7 @@ import org.apache.iotdb.commons.snapshot.SnapshotProcessor; import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeCreateTablePlan; -import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlanV1; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlanV2; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeUnsetSchemaTemplatePlan; import org.apache.iotdb.confignode.consensus.request.write.table.CommitCreateTablePlan; @@ -71,8 +71,8 @@ public synchronized void tryListenToPlan( if (ConfigRegionListeningFilter.shouldPlanBeListened(plan)) { final PipeConfigRegionWritePlanEvent event; switch (plan.getType()) { - case PipeEnriched: - tryListenToPlan(((PipeEnrichedPlan) plan).getInnerPlan(), true, null); + case PipeEnrichedV1: + tryListenToPlan(((PipeEnrichedPlanV1) plan).getInnerPlan(), true, null); return; case PipeEnrichedV2: tryListenToPlan( diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java index 3eb89753917b0..e127fbd620d59 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java @@ -94,7 +94,7 @@ import org.apache.iotdb.confignode.consensus.request.write.partition.CreateSchemaPartitionPlan; import org.apache.iotdb.confignode.consensus.request.write.partition.RemoveRegionLocationPlan; import org.apache.iotdb.confignode.consensus.request.write.partition.UpdateRegionLocationPlan; -import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlanV1; import org.apache.iotdb.confignode.consensus.request.write.pipe.plugin.CreatePipePluginPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.plugin.DropPipePluginPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.runtime.PipeHandleLeaderChangePlan; @@ -646,9 +646,9 @@ public TSStatus executeNonQueryPlan(ConfigPhysicalPlan physicalPlan) case SetPipeStatusV1: case ShowPipeV1: return new TSStatus(TSStatusCode.INCOMPATIBLE_VERSION.getStatusCode()); - case PipeEnriched: + case PipeEnrichedV1: case PipeEnrichedV2: - return executeNonQueryPlan(((PipeEnrichedPlan) physicalPlan).getInnerPlan()); + return executeNonQueryPlan(((PipeEnrichedPlanV1) physicalPlan).getInnerPlan()); case PipeDeleteTimeSeries: case PipeDeleteLogicalView: case PipeDeactivateTemplate: diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/StateMachineProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/StateMachineProcedure.java index 8dfac99b400bc..063ac40976a58 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/StateMachineProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/StateMachineProcedure.java @@ -299,7 +299,9 @@ public void deserialize(ByteBuffer byteBuffer) { stateFlow = Flow.NO_MORE_STATE; } } - originClusterId = ReadWriteIOUtils.readString(byteBuffer); + if (byteBuffer.hasRemaining()) { + originClusterId = ReadWriteIOUtils.readString(byteBuffer); + } } /** diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java index 05d0c3b1f75f9..7076cddb2645c 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java @@ -96,7 +96,7 @@ import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteDevicesPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteLogicalViewPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeDeleteTimeSeriesPlan; -import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlanV1; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlanV2; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeUnsetSchemaTemplatePlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.plugin.CreatePipePluginPlan; @@ -1669,7 +1669,7 @@ public void updateClusterIdPlanTest() throws IOException { @Test public void pipeEnrichedPlanTest() throws IOException { - final PipeEnrichedPlan plan = + final PipeEnrichedPlanV1 plan = new PipeEnrichedPlanV2( new DatabaseSchemaPlan( ConfigPhysicalPlanType.CreateDatabase, diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/ConfigRegionListeningQueueTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/ConfigRegionListeningQueueTest.java index 2af3bbf5ccdc1..86fc1b572eea3 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/ConfigRegionListeningQueueTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/ConfigRegionListeningQueueTest.java @@ -24,7 +24,7 @@ import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType; import org.apache.iotdb.confignode.consensus.request.write.auth.AuthorTreePlan; import org.apache.iotdb.confignode.consensus.request.write.database.DatabaseSchemaPlan; -import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; +import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlanV1; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlanV2; import org.apache.iotdb.confignode.manager.pipe.agent.PipeConfigNodeAgent; import org.apache.iotdb.confignode.manager.pipe.event.PipeConfigRegionWritePlanEvent; @@ -70,7 +70,7 @@ public void testSnapshot() throws TException, IOException, AuthException { final DatabaseSchemaPlan plan1 = new DatabaseSchemaPlan( ConfigPhysicalPlanType.CreateDatabase, new TDatabaseSchema("root.test1")); - final PipeEnrichedPlan plan2 = + final PipeEnrichedPlanV1 plan2 = new PipeEnrichedPlanV2( new AuthorTreePlan( ConfigPhysicalPlanType.CreateUser, From 2b1df48f02c58f9ede115b05384dab5a925b58b4 Mon Sep 17 00:00:00 2001 From: XNX02 <202100130044@mail.sdu.edu.cn> Date: Fri, 28 Mar 2025 02:25:49 +0800 Subject: [PATCH 27/27] update --- .../plan/node/pipe/PipeEnrichedDeleteDataNode.java | 8 +++++++- .../planner/plan/node/pipe/PipeEnrichedInsertNode.java | 7 ++++++- .../plan/node/pipe/PipeEnrichedNonWritePlanNode.java | 8 +++++++- .../planner/plan/node/pipe/PipeEnrichedWritePlanNode.java | 8 +++++++- 4 files changed, 27 insertions(+), 4 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedDeleteDataNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedDeleteDataNode.java index 8b7c4b859fda2..3eef19da68e37 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedDeleteDataNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedDeleteDataNode.java @@ -34,6 +34,8 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.SearchNode; import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.IWALByteBufferView; +import org.apache.tsfile.utils.ReadWriteIOUtils; + import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; @@ -160,16 +162,20 @@ public R accept(final PlanVisitor visitor, final C context) { protected void serializeAttributes(final ByteBuffer byteBuffer) { PlanNodeType.PIPE_ENRICHED_DELETE_DATA.serialize(byteBuffer); deleteDataNode.serialize(byteBuffer); + ReadWriteIOUtils.write(originClusterId, byteBuffer); } @Override protected void serializeAttributes(final DataOutputStream stream) throws IOException { PlanNodeType.PIPE_ENRICHED_DELETE_DATA.serialize(stream); deleteDataNode.serialize(stream); + ReadWriteIOUtils.write(originClusterId, stream); } public static PipeEnrichedDeleteDataNode deserialize(final ByteBuffer buffer) { - return new PipeEnrichedDeleteDataNode((DeleteDataNode) PlanNodeType.deserialize(buffer)); + return new PipeEnrichedDeleteDataNode( + (DeleteDataNode) PlanNodeType.deserialize(buffer), + buffer.hasRemaining() ? ReadWriteIOUtils.readString(buffer) : null); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedInsertNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedInsertNode.java index eb7f7cf490099..92fd3827ce11d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedInsertNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedInsertNode.java @@ -35,6 +35,7 @@ import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.file.metadata.IDeviceID; +import org.apache.tsfile.utils.ReadWriteIOUtils; import org.apache.tsfile.write.schema.MeasurementSchema; import java.io.DataOutputStream; @@ -244,16 +245,20 @@ public void setSearchIndex(final long searchIndex) { protected void serializeAttributes(final ByteBuffer byteBuffer) { PlanNodeType.PIPE_ENRICHED_INSERT_DATA.serialize(byteBuffer); insertNode.serialize(byteBuffer); + ReadWriteIOUtils.write(originClusterId, byteBuffer); } @Override protected void serializeAttributes(final DataOutputStream stream) throws IOException { PlanNodeType.PIPE_ENRICHED_INSERT_DATA.serialize(stream); insertNode.serialize(stream); + ReadWriteIOUtils.write(originClusterId, stream); } public static PipeEnrichedInsertNode deserialize(final ByteBuffer buffer) { - return new PipeEnrichedInsertNode((InsertNode) PlanNodeType.deserialize(buffer)); + return new PipeEnrichedInsertNode( + (InsertNode) PlanNodeType.deserialize(buffer), + buffer.hasRemaining() ? ReadWriteIOUtils.readString(buffer) : null); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedNonWritePlanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedNonWritePlanNode.java index bda6fe70ccd75..6453262338269 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedNonWritePlanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedNonWritePlanNode.java @@ -33,6 +33,8 @@ import org.apache.iotdb.db.queryengine.plan.statement.IConfigStatement; import org.apache.iotdb.db.queryengine.plan.statement.Statement; +import org.apache.tsfile.utils.ReadWriteIOUtils; + import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; @@ -152,16 +154,20 @@ public R accept(PlanVisitor visitor, C context) { protected void serializeAttributes(ByteBuffer byteBuffer) { PlanNodeType.PIPE_ENRICHED_NON_WRITE.serialize(byteBuffer); nonWritePlanNode.serialize(byteBuffer); + ReadWriteIOUtils.write(originClusterId, byteBuffer); } @Override protected void serializeAttributes(DataOutputStream stream) throws IOException { PlanNodeType.PIPE_ENRICHED_NON_WRITE.serialize(stream); nonWritePlanNode.serialize(stream); + ReadWriteIOUtils.write(originClusterId, stream); } public static PipeEnrichedNonWritePlanNode deserialize(ByteBuffer buffer) { - return new PipeEnrichedNonWritePlanNode(PlanNodeType.deserialize(buffer)); + return new PipeEnrichedNonWritePlanNode( + PlanNodeType.deserialize(buffer), + buffer.hasRemaining() ? ReadWriteIOUtils.readString(buffer) : null); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedWritePlanNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedWritePlanNode.java index 4961948481c81..b7761b8cc4da6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedWritePlanNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/pipe/PipeEnrichedWritePlanNode.java @@ -40,6 +40,8 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.metadata.write.view.CreateLogicalViewNode; import org.apache.iotdb.db.queryengine.plan.statement.Statement; +import org.apache.tsfile.utils.ReadWriteIOUtils; + import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; @@ -168,16 +170,20 @@ public R accept(final PlanVisitor visitor, final C context) { protected void serializeAttributes(final ByteBuffer byteBuffer) { PlanNodeType.PIPE_ENRICHED_WRITE.serialize(byteBuffer); writePlanNode.serialize(byteBuffer); + ReadWriteIOUtils.write(originClusterId, byteBuffer); } @Override protected void serializeAttributes(final DataOutputStream stream) throws IOException { PlanNodeType.PIPE_ENRICHED_WRITE.serialize(stream); writePlanNode.serialize(stream); + ReadWriteIOUtils.write(originClusterId, stream); } public static PipeEnrichedWritePlanNode deserialize(final ByteBuffer buffer) { - return new PipeEnrichedWritePlanNode((WritePlanNode) PlanNodeType.deserialize(buffer)); + return new PipeEnrichedWritePlanNode( + (WritePlanNode) PlanNodeType.deserialize(buffer), + buffer.hasRemaining() ? ReadWriteIOUtils.readString(buffer) : null); } @Override