From 9c8ead4ec02df8c0a94651394b3242df40900297 Mon Sep 17 00:00:00 2001 From: Caideyipi <1336190078@qq.com> Date: Mon, 27 Nov 2023 14:57:38 +0800 Subject: [PATCH 01/11] ConfigNode storage --- .../response/pipe/task/PipeTableResp.java | 2 +- .../pipe/runtime/PipeHeartbeatParser.java | 4 +- .../persistence/pipe/PipeTaskInfo.java | 6 +- .../impl/pipe/task/CreatePipeProcedureV2.java | 24 ++- .../request/ConfigPhysicalPlanSerDeTest.java | 24 ++- .../response/pipe/PipeTableRespTest.java | 37 ++-- .../confignode/persistence/PipeInfoTest.java | 25 ++- .../db/pipe/agent/task/PipeTaskAgent.java | 16 +- .../iotdb/db/pipe/task/PipeBuilder.java | 2 +- .../consensus/index/ProgressIndexType.java | 4 + .../index/impl/SchemaProgressIndex.java | 165 +++++++++++++++++ .../pipe/task/meta/PipeRuntimeMeta.java | 174 +++++++++++++++--- .../task/meta/PipeRuntimeMetaVersion.java | 1 + .../iotdb/commons/pipe/PipeMetaDeSerTest.java | 13 +- 14 files changed, 422 insertions(+), 75 deletions(-) create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SchemaProgressIndex.java diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/pipe/task/PipeTableResp.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/pipe/task/PipeTableResp.java index f2c756de46058..5ad822bbd2795 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/pipe/task/PipeTableResp.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/response/pipe/task/PipeTableResp.java @@ -114,7 +114,7 @@ public TShowPipeResp convertToTShowPipeResp() { .append(e.getMessage()) .append("\n"); } - for (PipeTaskMeta pipeTaskMeta : runtimeMeta.getConsensusGroupId2TaskMetaMap().values()) { + for (PipeTaskMeta pipeTaskMeta : runtimeMeta.getDataRegionId2TaskMetaMap().values()) { for (PipeRuntimeException e : pipeTaskMeta.getExceptionMessages()) { exceptionMessageBuilder .append(DateTimeUtils.convertLongToDate(e.getTimeStamp(), "ms")) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/runtime/PipeHeartbeatParser.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/runtime/PipeHeartbeatParser.java index 2d7cd66f24bf0..1e0d4e08b17de 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/runtime/PipeHeartbeatParser.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/pipe/runtime/PipeHeartbeatParser.java @@ -155,9 +155,9 @@ private void parseHeartbeatAndSaveMetaChangeLocally( } final Map pipeTaskMetaMapOnConfigNode = - pipeMetaOnConfigNode.getRuntimeMeta().getConsensusGroupId2TaskMetaMap(); + pipeMetaOnConfigNode.getRuntimeMeta().getDataRegionId2TaskMetaMap(); final Map pipeTaskMetaMapFromDataNode = - pipeMetaFromDataNode.getRuntimeMeta().getConsensusGroupId2TaskMetaMap(); + pipeMetaFromDataNode.getRuntimeMeta().getDataRegionId2TaskMetaMap(); for (final Map.Entry runtimeMetaOnConfigNode : pipeTaskMetaMapOnConfigNode.entrySet()) { if (runtimeMetaOnConfigNode.getValue().getLeaderDataNodeId() != dataNodeId) { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/pipe/PipeTaskInfo.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/pipe/PipeTaskInfo.java index f6e0003695b11..3cca93d9bd7ff 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/pipe/PipeTaskInfo.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/pipe/PipeTaskInfo.java @@ -359,7 +359,7 @@ private TSStatus handleLeaderChangeInternal(PipeHandleLeaderChangePlan plan) { .forEach( pipeMeta -> { final Map consensusGroupIdToTaskMetaMap = - pipeMeta.getRuntimeMeta().getConsensusGroupId2TaskMetaMap(); + pipeMeta.getRuntimeMeta().getDataRegionId2TaskMetaMap(); if (consensusGroupIdToTaskMetaMap.containsKey(dataRegionGroupId)) { // If the data region leader is -1, it means the data region is @@ -443,7 +443,7 @@ private boolean hasExceptionsInternal(String pipeName) { final AtomicBoolean hasException = new AtomicBoolean(false); runtimeMeta - .getConsensusGroupId2TaskMetaMap() + .getDataRegionId2TaskMetaMap() .values() .forEach( pipeTaskMeta -> { @@ -505,7 +505,7 @@ private void clearExceptionsAndSetIsStoppedByRuntimeExceptionToFalseInternal(Str } runtimeMeta - .getConsensusGroupId2TaskMetaMap() + .getDataRegionId2TaskMetaMap() .values() .forEach( pipeTaskMeta -> { diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/CreatePipeProcedureV2.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/CreatePipeProcedureV2.java index 7739f202f2edf..681a3d76b7ee5 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/CreatePipeProcedureV2.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/pipe/task/CreatePipeProcedureV2.java @@ -100,26 +100,30 @@ protected void executeFromCalculateInfoForTask(ConfigNodeProcedureEnv env) { createPipeRequest.getProcessorAttributes(), createPipeRequest.getConnectorAttributes()); - final Map consensusGroupIdToTaskMetaMap = new HashMap<>(); + final Map dataRegionIdToTaskMetaMap = new HashMap<>(); + final Map schemaRegionIdToTaskMetaMap = new HashMap<>(); + env.getConfigManager() .getLoadManager() .getRegionLeaderMap() .forEach( (regionGroupId, regionLeaderNodeId) -> { - if (regionGroupId.getType().equals(TConsensusGroupType.DataRegion)) { - final String databaseName = - env.getConfigManager() - .getPartitionManager() - .getRegionStorageGroup(regionGroupId); - if (databaseName != null && !databaseName.equals(SchemaConstant.SYSTEM_DATABASE)) { - // Pipe only collect user's data, filter metric database here. - consensusGroupIdToTaskMetaMap.put( + final String databaseName = + env.getConfigManager().getPartitionManager().getRegionStorageGroup(regionGroupId); + if (databaseName != null && !databaseName.equals(SchemaConstant.SYSTEM_DATABASE)) { + TConsensusGroupType type = regionGroupId.getType(); + if (type.equals(TConsensusGroupType.DataRegion)) { + dataRegionIdToTaskMetaMap.put( + regionGroupId, + new PipeTaskMeta(MinimumProgressIndex.INSTANCE, regionLeaderNodeId)); + } else if (type.equals(TConsensusGroupType.SchemaRegion)) { + schemaRegionIdToTaskMetaMap.put( regionGroupId, new PipeTaskMeta(MinimumProgressIndex.INSTANCE, regionLeaderNodeId)); } } }); - pipeRuntimeMeta = new PipeRuntimeMeta(consensusGroupIdToTaskMetaMap); + pipeRuntimeMeta = new PipeRuntimeMeta(dataRegionIdToTaskMetaMap, schemaRegionIdToTaskMetaMap); } @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 a058696fbec67..2921e6458488e 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 @@ -1109,13 +1109,17 @@ public void CreatePipePlanV2Test() throws IOException { extractorAttributes.put("extractor", "org.apache.iotdb.pipe.extractor.DefaultExtractor"); processorAttributes.put("processor", "org.apache.iotdb.pipe.processor.SDTFilterProcessor"); connectorAttributes.put("connector", "org.apache.iotdb.pipe.protocal.ThriftTransporter"); - PipeTaskMeta pipeTaskMeta = new PipeTaskMeta(MinimumProgressIndex.INSTANCE, 1); - Map pipeTasks = new HashMap<>(); - pipeTasks.put(new TConsensusGroupId(DataRegion, 1), pipeTaskMeta); + PipeTaskMeta dataRegionPipeTaskMeta = new PipeTaskMeta(MinimumProgressIndex.INSTANCE, 1); + PipeTaskMeta schemaRegionPipeTaskMeta = new PipeTaskMeta(MinimumProgressIndex.INSTANCE, 1); + Map dataRegionPipeTasks = new HashMap<>(); + Map schemaRegionPipeTasks = new HashMap<>(); + dataRegionPipeTasks.put(new TConsensusGroupId(DataRegion, 1), dataRegionPipeTaskMeta); + schemaRegionPipeTasks.put(new TConsensusGroupId(SchemaRegion, 2), schemaRegionPipeTaskMeta); PipeStaticMeta pipeStaticMeta = new PipeStaticMeta( "testPipe", 121, extractorAttributes, processorAttributes, connectorAttributes); - PipeRuntimeMeta pipeRuntimeMeta = new PipeRuntimeMeta(pipeTasks); + PipeRuntimeMeta pipeRuntimeMeta = + new PipeRuntimeMeta(dataRegionPipeTasks, schemaRegionPipeTasks); CreatePipePlanV2 createPipePlanV2 = new CreatePipePlanV2(pipeStaticMeta, pipeRuntimeMeta); CreatePipePlanV2 createPipePlanV21 = (CreatePipePlanV2) @@ -1229,6 +1233,18 @@ public void pipeHandleMetaChangePlanTest() throws IOException { new PipeTaskMeta( MinimumProgressIndex.INSTANCE, 789)); // TODO: replace with IoTConsensus } + }, + new HashMap() { + { + put( + new TConsensusGroupId(TConsensusGroupType.SchemaRegion, 111), + new PipeTaskMeta( + MinimumProgressIndex.INSTANCE, 222)); // TODO: replace with IoTConsensus + put( + new TConsensusGroupId(TConsensusGroupType.SchemaRegion, 333), + new PipeTaskMeta( + MinimumProgressIndex.INSTANCE, 444)); // TODO: replace with IoTConsensus + } }); pipeMetaList.add(new PipeMeta(pipeStaticMeta, pipeRuntimeMeta)); PipeHandleMetaChangePlan pipeHandleMetaChangePlan1 = new PipeHandleMetaChangePlan(pipeMetaList); diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/response/pipe/PipeTableRespTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/response/pipe/PipeTableRespTest.java index 97967f97da020..7f69946d6151b 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/response/pipe/PipeTableRespTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/response/pipe/PipeTableRespTest.java @@ -37,6 +37,7 @@ import java.util.Map; import static org.apache.iotdb.common.rpc.thrift.TConsensusGroupType.DataRegion; +import static org.apache.iotdb.common.rpc.thrift.TConsensusGroupType.SchemaRegion; public class PipeTableRespTest { @@ -55,13 +56,17 @@ public PipeTableResp constructPipeTableResp() { connectorAttributes.put("host", "127.0.0.1"); connectorAttributes.put("port", "6667"); - PipeTaskMeta pipeTaskMeta = new PipeTaskMeta(MinimumProgressIndex.INSTANCE, 1); - Map pipeTasks = new HashMap<>(); - pipeTasks.put(new TConsensusGroupId(DataRegion, 1), pipeTaskMeta); + PipeTaskMeta dataRegionPipeTaskMeta = new PipeTaskMeta(MinimumProgressIndex.INSTANCE, 1); + PipeTaskMeta schemaRegionPipeTaskMeta = new PipeTaskMeta(MinimumProgressIndex.INSTANCE, 2); + Map dataRegionPipeTasks = new HashMap<>(); + Map schemaRegionPipeTasks = new HashMap<>(); + dataRegionPipeTasks.put(new TConsensusGroupId(DataRegion, 1), dataRegionPipeTaskMeta); + schemaRegionPipeTasks.put(new TConsensusGroupId(SchemaRegion, 2), schemaRegionPipeTaskMeta); PipeStaticMeta pipeStaticMeta = new PipeStaticMeta( "testPipe", 121, extractorAttributes, processorAttributes, connectorAttributes); - PipeRuntimeMeta pipeRuntimeMeta = new PipeRuntimeMeta(pipeTasks); + PipeRuntimeMeta pipeRuntimeMeta = + new PipeRuntimeMeta(dataRegionPipeTasks, schemaRegionPipeTasks); pipeMetaList.add(new PipeMeta(pipeStaticMeta, pipeRuntimeMeta)); // PipeMeta 2 @@ -75,13 +80,17 @@ public PipeTableResp constructPipeTableResp() { connectorAttributes1.put("host", "127.0.0.1"); connectorAttributes1.put("port", "6667"); - PipeTaskMeta pipeTaskMeta1 = new PipeTaskMeta(MinimumProgressIndex.INSTANCE, 1); - Map pipeTasks1 = new HashMap<>(); - pipeTasks1.put(new TConsensusGroupId(DataRegion, 1), pipeTaskMeta1); + PipeTaskMeta dataRegionPipeTaskMeta1 = new PipeTaskMeta(MinimumProgressIndex.INSTANCE, 1); + PipeTaskMeta schemaRegionPipeTaskMeta1 = new PipeTaskMeta(MinimumProgressIndex.INSTANCE, 2); + Map dataRegionPipeTasks1 = new HashMap<>(); + Map schemaRegionPipeTasks1 = new HashMap<>(); + dataRegionPipeTasks1.put(new TConsensusGroupId(DataRegion, 1), dataRegionPipeTaskMeta1); + schemaRegionPipeTasks1.put(new TConsensusGroupId(SchemaRegion, 2), schemaRegionPipeTaskMeta1); PipeStaticMeta pipeStaticMeta1 = new PipeStaticMeta( "testPipe", 121, extractorAttributes1, processorAttributes1, connectorAttributes1); - PipeRuntimeMeta pipeRuntimeMeta1 = new PipeRuntimeMeta(pipeTasks1); + PipeRuntimeMeta pipeRuntimeMeta1 = + new PipeRuntimeMeta(dataRegionPipeTasks1, schemaRegionPipeTasks1); pipeMetaList.add(new PipeMeta(pipeStaticMeta1, pipeRuntimeMeta1)); // PipeMeta 3 @@ -95,13 +104,17 @@ public PipeTableResp constructPipeTableResp() { connectorAttributes2.put("host", "172.30.30.30"); connectorAttributes2.put("port", "6667"); - PipeTaskMeta pipeTaskMeta2 = new PipeTaskMeta(MinimumProgressIndex.INSTANCE, 1); - Map pipeTasks2 = new HashMap<>(); - pipeTasks2.put(new TConsensusGroupId(DataRegion, 1), pipeTaskMeta2); + PipeTaskMeta dataRegionPipeTaskMeta2 = new PipeTaskMeta(MinimumProgressIndex.INSTANCE, 1); + PipeTaskMeta schemaRegionPipeTaskMeta2 = new PipeTaskMeta(MinimumProgressIndex.INSTANCE, 2); + Map dataRegionPipeTasks2 = new HashMap<>(); + Map schemaRegionPipeTasks2 = new HashMap<>(); + dataRegionPipeTasks2.put(new TConsensusGroupId(DataRegion, 1), dataRegionPipeTaskMeta2); + schemaRegionPipeTasks2.put(new TConsensusGroupId(SchemaRegion, 2), schemaRegionPipeTaskMeta2); PipeStaticMeta pipeStaticMeta2 = new PipeStaticMeta( "testPipe", 121, extractorAttributes2, processorAttributes2, connectorAttributes2); - PipeRuntimeMeta pipeRuntimeMeta2 = new PipeRuntimeMeta(pipeTasks2); + PipeRuntimeMeta pipeRuntimeMeta2 = + new PipeRuntimeMeta(dataRegionPipeTasks2, schemaRegionPipeTasks2); pipeMetaList.add(new PipeMeta(pipeStaticMeta2, pipeRuntimeMeta2)); return new PipeTableResp(status, pipeMetaList); diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/PipeInfoTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/PipeInfoTest.java index 5e99bfae1e874..967c4f755939a 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/PipeInfoTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/PipeInfoTest.java @@ -48,6 +48,7 @@ import java.util.Map; import static org.apache.iotdb.common.rpc.thrift.TConsensusGroupType.DataRegion; +import static org.apache.iotdb.common.rpc.thrift.TConsensusGroupType.SchemaRegion; import static org.apache.iotdb.db.utils.constant.TestConstant.BASE_OUTPUT_PATH; public class PipeInfoTest { @@ -86,13 +87,17 @@ public void testSnapshot() throws TException, IOException { connectorAttributes.put("host", "127.0.0.1"); connectorAttributes.put("port", "6667"); - PipeTaskMeta pipeTaskMeta = new PipeTaskMeta(MinimumProgressIndex.INSTANCE, 1); - Map pipeTasks = new HashMap<>(); - pipeTasks.put(new TConsensusGroupId(DataRegion, 1), pipeTaskMeta); + PipeTaskMeta dataRegionPipeTaskMeta = new PipeTaskMeta(MinimumProgressIndex.INSTANCE, 1); + PipeTaskMeta schemaRegionPipeTaskMeta = new PipeTaskMeta(MinimumProgressIndex.INSTANCE, 2); + Map dataRegionPipeTasks = new HashMap<>(); + Map schemaRegionPipeTasks = new HashMap<>(); + dataRegionPipeTasks.put(new TConsensusGroupId(DataRegion, 1), dataRegionPipeTaskMeta); + schemaRegionPipeTasks.put(new TConsensusGroupId(SchemaRegion, 2), schemaRegionPipeTaskMeta); PipeStaticMeta pipeStaticMeta = new PipeStaticMeta( pipeName, 121, extractorAttributes, processorAttributes, connectorAttributes); - PipeRuntimeMeta pipeRuntimeMeta = new PipeRuntimeMeta(pipeTasks); + PipeRuntimeMeta pipeRuntimeMeta = + new PipeRuntimeMeta(dataRegionPipeTasks, schemaRegionPipeTasks); CreatePipePlanV2 createPipePlanV2 = new CreatePipePlanV2(pipeStaticMeta, pipeRuntimeMeta); pipeInfo.getPipeTaskInfo().createPipe(createPipePlanV2); @@ -121,13 +126,17 @@ public void testManagement() { extractorAttributes.put("extractor", "org.apache.iotdb.pipe.extractor.DefaultExtractor"); processorAttributes.put("processor", "org.apache.iotdb.pipe.processor.SDTFilterProcessor"); connectorAttributes.put("connector", "org.apache.iotdb.pipe.protocal.ThriftTransporter"); - PipeTaskMeta pipeTaskMeta = new PipeTaskMeta(MinimumProgressIndex.INSTANCE, 1); - Map pipeTasks = new HashMap<>(); - pipeTasks.put(new TConsensusGroupId(DataRegion, 1), pipeTaskMeta); + PipeTaskMeta dataRegionPipeTaskMeta = new PipeTaskMeta(MinimumProgressIndex.INSTANCE, 1); + PipeTaskMeta schemaRegionPipeTaskMeta = new PipeTaskMeta(MinimumProgressIndex.INSTANCE, 2); + Map dataRegionPipeTasks = new HashMap<>(); + Map schemaRegionPipeTasks = new HashMap<>(); + dataRegionPipeTasks.put(new TConsensusGroupId(DataRegion, 1), dataRegionPipeTaskMeta); + schemaRegionPipeTasks.put(new TConsensusGroupId(SchemaRegion, 2), schemaRegionPipeTaskMeta); PipeStaticMeta pipeStaticMeta = new PipeStaticMeta( pipeName, 121, extractorAttributes, processorAttributes, connectorAttributes); - PipeRuntimeMeta pipeRuntimeMeta = new PipeRuntimeMeta(pipeTasks); + PipeRuntimeMeta pipeRuntimeMeta = + new PipeRuntimeMeta(dataRegionPipeTasks, schemaRegionPipeTasks); CreatePipePlanV2 createPipePlanV2 = new CreatePipePlanV2(pipeStaticMeta, pipeRuntimeMeta); pipeInfo.getPipeTaskInfo().createPipe(createPipePlanV2); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeTaskAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeTaskAgent.java index 1ab5eb7cc8119..20c4429bd1887 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeTaskAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/PipeTaskAgent.java @@ -282,9 +282,9 @@ private void executeSinglePipeRuntimeMetaChanges( @NotNull PipeRuntimeMeta runtimeMetaOnDataNode) { // 1. Handle data region group leader changed first final Map consensusGroupIdToTaskMetaMapFromConfigNode = - runtimeMetaFromConfigNode.getConsensusGroupId2TaskMetaMap(); + runtimeMetaFromConfigNode.getDataRegionId2TaskMetaMap(); final Map consensusGroupIdToTaskMetaMapOnDataNode = - runtimeMetaOnDataNode.getConsensusGroupId2TaskMetaMap(); + runtimeMetaOnDataNode.getDataRegionId2TaskMetaMap(); // 1.1 Iterate over all consensus group ids in config node's pipe runtime meta, decide if we // need to drop and create a new task for each consensus group id @@ -422,7 +422,7 @@ private void stopAllPipesWithCriticalExceptionInternal() { final PipeRuntimeMeta runtimeMeta = pipeMeta.getRuntimeMeta(); runtimeMeta - .getConsensusGroupId2TaskMetaMap() + .getDataRegionId2TaskMetaMap() .values() .forEach( pipeTaskMeta -> { @@ -447,7 +447,7 @@ private void stopAllPipesWithCriticalExceptionInternal() { final PipeRuntimeMeta runtimeMeta = pipeMeta.getRuntimeMeta(); runtimeMeta - .getConsensusGroupId2TaskMetaMap() + .getDataRegionId2TaskMetaMap() .values() .forEach( pipeTaskMeta -> { @@ -482,7 +482,7 @@ private void stopAllPipesWithCriticalExceptionInternal() { if (runtimeMeta.getStatus().get() == PipeStatus.RUNNING) { runtimeMeta - .getConsensusGroupId2TaskMetaMap() + .getDataRegionId2TaskMetaMap() .values() .forEach( pipeTaskMeta -> { @@ -736,7 +736,7 @@ private void startPipe(String pipeName, long creationTime) { // Clear exception messages if started successfully existedPipeMeta .getRuntimeMeta() - .getConsensusGroupId2TaskMetaMap() + .getDataRegionId2TaskMetaMap() .values() .forEach(PipeTaskMeta::clearExceptionMessages); } @@ -831,7 +831,7 @@ private void createPipeTask( pipeMetaKeeper .getPipeMeta(pipeStaticMeta.getPipeName()) .getRuntimeMeta() - .getConsensusGroupId2TaskMetaMap() + .getDataRegionId2TaskMetaMap() .put(consensusGroupId, pipeTaskMeta); } @@ -839,7 +839,7 @@ private void dropPipeTask(TConsensusGroupId dataRegionGroupId, PipeStaticMeta pi pipeMetaKeeper .getPipeMeta(pipeStaticMeta.getPipeName()) .getRuntimeMeta() - .getConsensusGroupId2TaskMetaMap() + .getDataRegionId2TaskMetaMap() .remove(dataRegionGroupId); final PipeTask pipeTask = pipeTaskManager.removePipeTask(pipeStaticMeta, dataRegionGroupId); if (pipeTask != null) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/PipeBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/PipeBuilder.java index 5066de9dbf88a..6abe7975401e8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/PipeBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/PipeBuilder.java @@ -46,7 +46,7 @@ public Map build() { final PipeRuntimeMeta pipeRuntimeMeta = pipeMeta.getRuntimeMeta(); for (Map.Entry consensusGroupIdToPipeTaskMeta : - pipeRuntimeMeta.getConsensusGroupId2TaskMetaMap().entrySet()) { + pipeRuntimeMeta.getDataRegionId2TaskMetaMap().entrySet()) { if (consensusGroupIdToPipeTaskMeta.getValue().getLeaderDataNodeId() == CONFIG.getDataNodeId()) { consensusGroupIdToPipeTaskMap.put( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/ProgressIndexType.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/ProgressIndexType.java index 615ce1336d926..0c0a9cf072f61 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/ProgressIndexType.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/ProgressIndexType.java @@ -23,6 +23,7 @@ import org.apache.iotdb.commons.consensus.index.impl.IoTProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.RecoverProgressIndex; +import org.apache.iotdb.commons.consensus.index.impl.SchemaProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.SimpleProgressIndex; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; @@ -37,6 +38,7 @@ public enum ProgressIndexType { SIMPLE_PROGRESS_INDEX((short) 3), RECOVER_PROGRESS_INDEX((short) 4), HYBRID_PROGRESS_INDEX((short) 5), + SCHEMA_PROGRESS_INDEX((short) 6), ; private final short type; @@ -70,6 +72,8 @@ public static ProgressIndex deserializeFrom(ByteBuffer byteBuffer) { return RecoverProgressIndex.deserializeFrom(byteBuffer); case 5: return HybridProgressIndex.deserializeFrom(byteBuffer); + case 6: + return SchemaProgressIndex.deserializeFrom(byteBuffer); default: throw new UnsupportedOperationException( String.format("Unsupported progress index type %s.", indexType)); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SchemaProgressIndex.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SchemaProgressIndex.java new file mode 100644 index 0000000000000..3f5cc6c7eb40c --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SchemaProgressIndex.java @@ -0,0 +1,165 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.consensus.index.impl; + +import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.commons.consensus.index.ProgressIndexType; +import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +/** + * {@link SchemaProgressIndex} is used only for schema progress recording. It shall not be blended + * or compared to {@link ProgressIndex}es other than {@link SchemaProgressIndex} or {@link + * MinimumProgressIndex}. + */ +public class SchemaProgressIndex implements ProgressIndex { + + private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); + + private int index; + + public SchemaProgressIndex() { + // Empty constructor + } + + public SchemaProgressIndex(int index) { + this.index = index; + } + + @Override + public void serialize(ByteBuffer byteBuffer) { + lock.readLock().lock(); + try { + ProgressIndexType.SCHEMA_PROGRESS_INDEX.serialize(byteBuffer); + + ReadWriteIOUtils.write(index, byteBuffer); + } finally { + lock.readLock().unlock(); + } + } + + @Override + public void serialize(OutputStream stream) throws IOException { + lock.readLock().lock(); + try { + ProgressIndexType.SCHEMA_PROGRESS_INDEX.serialize(stream); + + ReadWriteIOUtils.write(index, stream); + } finally { + lock.readLock().unlock(); + } + } + + @Override + public boolean isAfter(ProgressIndex progressIndex) { + lock.readLock().lock(); + try { + if (progressIndex instanceof MinimumProgressIndex) { + return true; + } + + if (!(progressIndex instanceof SchemaProgressIndex)) { + return false; + } + + final SchemaProgressIndex thisSchemaProgressIndex = this; + final SchemaProgressIndex thatSchemaProgressIndex = (SchemaProgressIndex) progressIndex; + return thatSchemaProgressIndex.index < thisSchemaProgressIndex.index; + } finally { + lock.readLock().unlock(); + } + } + + @Override + public boolean equals(ProgressIndex progressIndex) { + lock.readLock().lock(); + try { + if (!(progressIndex instanceof SchemaProgressIndex)) { + return false; + } + + final SchemaProgressIndex thisSchemaProgressIndex = this; + final SchemaProgressIndex thatSchemaProgressIndex = (SchemaProgressIndex) progressIndex; + return thisSchemaProgressIndex.index == thatSchemaProgressIndex.index; + } finally { + lock.readLock().unlock(); + } + } + + @Override + public boolean equals(Object obj) { + if (obj == null) { + return false; + } + if (this == obj) { + return true; + } + if (!(obj instanceof SchemaProgressIndex)) { + return false; + } + return this.equals((SchemaProgressIndex) obj); + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public ProgressIndex updateToMinimumIsAfterProgressIndex(ProgressIndex progressIndex) { + lock.writeLock().lock(); + try { + if (!(progressIndex instanceof SchemaProgressIndex)) { + return this; + } + + this.index = Math.max(this.index, ((SchemaProgressIndex) progressIndex).index); + return this; + } finally { + lock.writeLock().unlock(); + } + } + + public ProgressIndexType getType() { + return ProgressIndexType.RECOVER_PROGRESS_INDEX; + } + + public static SchemaProgressIndex deserializeFrom(ByteBuffer byteBuffer) { + final SchemaProgressIndex schemaProgressIndex = new SchemaProgressIndex(); + schemaProgressIndex.index = ReadWriteIOUtils.readInt(byteBuffer); + return schemaProgressIndex; + } + + public static SchemaProgressIndex deserializeFrom(InputStream stream) throws IOException { + final SchemaProgressIndex schemaProgressIndex = new SchemaProgressIndex(); + schemaProgressIndex.index = ReadWriteIOUtils.readInt(stream); + return schemaProgressIndex; + } + + @Override + public String toString() { + return "SchemaProgressIndex{" + "index=" + index + '}'; + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMeta.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMeta.java index a773407d979af..3851b748dfffc 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMeta.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMeta.java @@ -21,6 +21,9 @@ import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType; +import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; +import org.apache.iotdb.commons.consensus.index.impl.SchemaProgressIndex; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeConnectorCriticalException; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeCriticalException; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeException; @@ -45,7 +48,23 @@ public class PipeRuntimeMeta { private final AtomicReference status = new AtomicReference<>(PipeStatus.STOPPED); - private final Map consensusGroupId2TaskMetaMap; + // All the progressIndexes of the pipe. + /** + * {@link SchemaProgressIndex} for schema transmission on ConfigNode, always {@link + * MinimumProgressIndex} if the pipe has no relation to schema synchronization. + */ + private final AtomicReference configProgressIndex = + new AtomicReference<>(MinimumProgressIndex.INSTANCE); + /** + * Used to store progress for schema transmission on DataRegion, usually updated since the data + * synchronization is basic function of pipe. + */ + private final Map dataRegionId2TaskMetaMap; + /** + * Used to store {@link SchemaProgressIndex} for schema transmission on SchemaRegion, always + * {@link MinimumProgressIndex} if the pipe has no relation to schema synchronization. + */ + private final Map schemaRegionId2TaskMetaMap; /** * Stores the newest exceptions encountered group by dataNodes. @@ -66,19 +85,27 @@ public class PipeRuntimeMeta { private final AtomicBoolean isStoppedByRuntimeException = new AtomicBoolean(false); public PipeRuntimeMeta() { - consensusGroupId2TaskMetaMap = new ConcurrentHashMap<>(); + dataRegionId2TaskMetaMap = new ConcurrentHashMap<>(); + schemaRegionId2TaskMetaMap = new ConcurrentHashMap<>(); } - public PipeRuntimeMeta(Map consensusGroupId2TaskMetaMap) { - this.consensusGroupId2TaskMetaMap = consensusGroupId2TaskMetaMap; + public PipeRuntimeMeta( + Map dataRegionId2TaskMetaMap, + Map schemaRegionId2TaskMetaMap) { + this.dataRegionId2TaskMetaMap = dataRegionId2TaskMetaMap; + this.schemaRegionId2TaskMetaMap = schemaRegionId2TaskMetaMap; } public AtomicReference getStatus() { return status; } - public Map getConsensusGroupId2TaskMetaMap() { - return consensusGroupId2TaskMetaMap; + public Map getDataRegionId2TaskMetaMap() { + return dataRegionId2TaskMetaMap; + } + + public Map getSchemaRegionId2TaskMetaMap() { + return schemaRegionId2TaskMetaMap; } public Map getDataNodeId2PipeRuntimeExceptionMap() { @@ -111,16 +138,27 @@ public ByteBuffer serialize() throws IOException { } public void serialize(DataOutputStream outputStream) throws IOException { - PipeRuntimeMetaVersion.VERSION_2.serialize(outputStream); + PipeRuntimeMetaVersion.VERSION_3.serialize(outputStream); ReadWriteIOUtils.write(status.get().getType(), outputStream); + configProgressIndex.get().serialize(outputStream); + // Avoid concurrent modification - final Map consensusGroupId2TaskMetaMapView = - new HashMap<>(consensusGroupId2TaskMetaMap); - ReadWriteIOUtils.write(consensusGroupId2TaskMetaMapView.size(), outputStream); + final Map dataRegionId2TaskMetaMapView = + new HashMap<>(dataRegionId2TaskMetaMap); + ReadWriteIOUtils.write(dataRegionId2TaskMetaMapView.size(), outputStream); + for (Map.Entry entry : + dataRegionId2TaskMetaMapView.entrySet()) { + ReadWriteIOUtils.write(entry.getKey().getId(), outputStream); + entry.getValue().serialize(outputStream); + } + + final Map schemaRegionId2TaskMetaMapView = + new HashMap<>(schemaRegionId2TaskMetaMap); + ReadWriteIOUtils.write(schemaRegionId2TaskMetaMapView.size(), outputStream); for (Map.Entry entry : - consensusGroupId2TaskMetaMapView.entrySet()) { + schemaRegionId2TaskMetaMapView.entrySet()) { ReadWriteIOUtils.write(entry.getKey().getId(), outputStream); entry.getValue().serialize(outputStream); } @@ -140,16 +178,27 @@ public void serialize(DataOutputStream outputStream) throws IOException { } public void serialize(FileOutputStream outputStream) throws IOException { - PipeRuntimeMetaVersion.VERSION_2.serialize(outputStream); + PipeRuntimeMetaVersion.VERSION_3.serialize(outputStream); ReadWriteIOUtils.write(status.get().getType(), outputStream); + configProgressIndex.get().serialize(outputStream); + // Avoid concurrent modification - final Map consensusGroupId2TaskMetaMapView = - new HashMap<>(consensusGroupId2TaskMetaMap); - ReadWriteIOUtils.write(consensusGroupId2TaskMetaMapView.size(), outputStream); + final Map dataRegionId2TaskMetaMapView = + new HashMap<>(dataRegionId2TaskMetaMap); + ReadWriteIOUtils.write(dataRegionId2TaskMetaMapView.size(), outputStream); + for (Map.Entry entry : + dataRegionId2TaskMetaMapView.entrySet()) { + ReadWriteIOUtils.write(entry.getKey().getId(), outputStream); + entry.getValue().serialize(outputStream); + } + + final Map schemaRegionId2TaskMetaMapView = + new HashMap<>(schemaRegionId2TaskMetaMap); + ReadWriteIOUtils.write(schemaRegionId2TaskMetaMapView.size(), outputStream); for (Map.Entry entry : - consensusGroupId2TaskMetaMapView.entrySet()) { + schemaRegionId2TaskMetaMapView.entrySet()) { ReadWriteIOUtils.write(entry.getKey().getId(), outputStream); entry.getValue().serialize(outputStream); } @@ -177,6 +226,8 @@ public static PipeRuntimeMeta deserialize(InputStream inputStream) throws IOExce return deserializeVersion1(inputStream, pipeRuntimeVersionByte); case VERSION_2: return deserializeVersion2(inputStream); + case VERSION_3: + return deserializeVersion3(inputStream); default: throw new UnsupportedOperationException( "Unknown pipe runtime meta version: " + pipeRuntimeMetaVersion.getVersion()); @@ -191,7 +242,7 @@ private static PipeRuntimeMeta deserializeVersion1(InputStream inputStream, byte final int size = ReadWriteIOUtils.readInt(inputStream); for (int i = 0; i < size; ++i) { - pipeRuntimeMeta.consensusGroupId2TaskMetaMap.put( + pipeRuntimeMeta.dataRegionId2TaskMetaMap.put( new TConsensusGroupId( TConsensusGroupType.DataRegion, ReadWriteIOUtils.readInt(inputStream)), PipeTaskMeta.deserialize(PipeRuntimeMetaVersion.VERSION_1, inputStream)); @@ -207,12 +258,46 @@ private static PipeRuntimeMeta deserializeVersion2(InputStream inputStream) thro int size = ReadWriteIOUtils.readInt(inputStream); for (int i = 0; i < size; ++i) { - pipeRuntimeMeta.consensusGroupId2TaskMetaMap.put( + pipeRuntimeMeta.dataRegionId2TaskMetaMap.put( + new TConsensusGroupId( + TConsensusGroupType.DataRegion, ReadWriteIOUtils.readInt(inputStream)), + PipeTaskMeta.deserialize(PipeRuntimeMetaVersion.VERSION_2, inputStream)); + } + + size = ReadWriteIOUtils.readInt(inputStream); + for (int i = 0; i < size; ++i) { + pipeRuntimeMeta.dataNodeId2PipeRuntimeExceptionMap.put( + ReadWriteIOUtils.readInt(inputStream), + PipeRuntimeExceptionType.deserializeFrom(PipeRuntimeMetaVersion.VERSION_2, inputStream)); + } + + pipeRuntimeMeta.exceptionsClearTime.set(ReadWriteIOUtils.readLong(inputStream)); + pipeRuntimeMeta.isStoppedByRuntimeException.set(ReadWriteIOUtils.readBool(inputStream)); + + return pipeRuntimeMeta; + } + + private static PipeRuntimeMeta deserializeVersion3(InputStream inputStream) throws IOException { + final PipeRuntimeMeta pipeRuntimeMeta = new PipeRuntimeMeta(); + + pipeRuntimeMeta.status.set(PipeStatus.getPipeStatus(ReadWriteIOUtils.readByte(inputStream))); + + int size = ReadWriteIOUtils.readInt(inputStream); + for (int i = 0; i < size; ++i) { + pipeRuntimeMeta.dataRegionId2TaskMetaMap.put( new TConsensusGroupId( TConsensusGroupType.DataRegion, ReadWriteIOUtils.readInt(inputStream)), PipeTaskMeta.deserialize(PipeRuntimeMetaVersion.VERSION_2, inputStream)); } + size = ReadWriteIOUtils.readInt(inputStream); + for (int i = 0; i < size; ++i) { + pipeRuntimeMeta.schemaRegionId2TaskMetaMap.put( + new TConsensusGroupId( + TConsensusGroupType.SchemaRegion, ReadWriteIOUtils.readInt(inputStream)), + PipeTaskMeta.deserialize(PipeRuntimeMetaVersion.VERSION_2, inputStream)); + } + size = ReadWriteIOUtils.readInt(inputStream); for (int i = 0; i < size; ++i) { pipeRuntimeMeta.dataNodeId2PipeRuntimeExceptionMap.put( @@ -235,6 +320,8 @@ public static PipeRuntimeMeta deserialize(ByteBuffer byteBuffer) { return deserializeVersion1(byteBuffer, pipeRuntimeVersionByte); case VERSION_2: return deserializeVersion2(byteBuffer); + case VERSION_3: + return deserializeVersion3(byteBuffer); default: throw new UnsupportedOperationException( "Unknown pipe runtime meta version: " + pipeRuntimeMetaVersion.getVersion()); @@ -249,7 +336,7 @@ private static PipeRuntimeMeta deserializeVersion1( final int size = ReadWriteIOUtils.readInt(byteBuffer); for (int i = 0; i < size; ++i) { - pipeRuntimeMeta.consensusGroupId2TaskMetaMap.put( + pipeRuntimeMeta.dataRegionId2TaskMetaMap.put( new TConsensusGroupId( TConsensusGroupType.DataRegion, ReadWriteIOUtils.readInt(byteBuffer)), PipeTaskMeta.deserialize(PipeRuntimeMetaVersion.VERSION_1, byteBuffer)); @@ -265,12 +352,46 @@ public static PipeRuntimeMeta deserializeVersion2(ByteBuffer byteBuffer) { int size = ReadWriteIOUtils.readInt(byteBuffer); for (int i = 0; i < size; ++i) { - pipeRuntimeMeta.consensusGroupId2TaskMetaMap.put( + pipeRuntimeMeta.dataRegionId2TaskMetaMap.put( + new TConsensusGroupId( + TConsensusGroupType.DataRegion, ReadWriteIOUtils.readInt(byteBuffer)), + PipeTaskMeta.deserialize(PipeRuntimeMetaVersion.VERSION_2, byteBuffer)); + } + + size = ReadWriteIOUtils.readInt(byteBuffer); + for (int i = 0; i < size; ++i) { + pipeRuntimeMeta.dataNodeId2PipeRuntimeExceptionMap.put( + ReadWriteIOUtils.readInt(byteBuffer), + PipeRuntimeExceptionType.deserializeFrom(PipeRuntimeMetaVersion.VERSION_2, byteBuffer)); + } + + pipeRuntimeMeta.exceptionsClearTime.set(ReadWriteIOUtils.readLong(byteBuffer)); + pipeRuntimeMeta.isStoppedByRuntimeException.set(ReadWriteIOUtils.readBool(byteBuffer)); + + return pipeRuntimeMeta; + } + + public static PipeRuntimeMeta deserializeVersion3(ByteBuffer byteBuffer) { + final PipeRuntimeMeta pipeRuntimeMeta = new PipeRuntimeMeta(); + + pipeRuntimeMeta.status.set(PipeStatus.getPipeStatus(ReadWriteIOUtils.readByte(byteBuffer))); + + int size = ReadWriteIOUtils.readInt(byteBuffer); + for (int i = 0; i < size; ++i) { + pipeRuntimeMeta.dataRegionId2TaskMetaMap.put( new TConsensusGroupId( TConsensusGroupType.DataRegion, ReadWriteIOUtils.readInt(byteBuffer)), PipeTaskMeta.deserialize(PipeRuntimeMetaVersion.VERSION_2, byteBuffer)); } + size = ReadWriteIOUtils.readInt(byteBuffer); + for (int i = 0; i < size; ++i) { + pipeRuntimeMeta.schemaRegionId2TaskMetaMap.put( + new TConsensusGroupId( + TConsensusGroupType.SchemaRegion, ReadWriteIOUtils.readInt(byteBuffer)), + PipeTaskMeta.deserialize(PipeRuntimeMetaVersion.VERSION_2, byteBuffer)); + } + size = ReadWriteIOUtils.readInt(byteBuffer); for (int i = 0; i < size; ++i) { pipeRuntimeMeta.dataNodeId2PipeRuntimeExceptionMap.put( @@ -294,7 +415,8 @@ public boolean equals(Object o) { } PipeRuntimeMeta that = (PipeRuntimeMeta) o; return Objects.equals(status.get().getType(), that.status.get().getType()) - && consensusGroupId2TaskMetaMap.equals(that.consensusGroupId2TaskMetaMap) + && dataRegionId2TaskMetaMap.equals(that.dataRegionId2TaskMetaMap) + && schemaRegionId2TaskMetaMap.equals(that.schemaRegionId2TaskMetaMap) && dataNodeId2PipeRuntimeExceptionMap.equals(that.dataNodeId2PipeRuntimeExceptionMap) && exceptionsClearTime.get() == that.exceptionsClearTime.get() && isStoppedByRuntimeException.get() == that.isStoppedByRuntimeException.get(); @@ -304,7 +426,8 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash( status, - consensusGroupId2TaskMetaMap, + dataRegionId2TaskMetaMap, + schemaRegionId2TaskMetaMap, dataNodeId2PipeRuntimeExceptionMap, exceptionsClearTime.get(), isStoppedByRuntimeException.get()); @@ -315,9 +438,10 @@ public String toString() { return "PipeRuntimeMeta{" + "status=" + status - + ", consensusGroupId2TaskMetaMap=" - + consensusGroupId2TaskMetaMap - + ", dataNodeId2PipeMetaExceptionMap=" + + ", dataRegionId2TaskMetaMap=" + + dataRegionId2TaskMetaMap + + ", schemaRegionId2TaskMetaMap=" + + schemaRegionId2TaskMetaMap + dataNodeId2PipeRuntimeExceptionMap + ", exceptionsClearTime=" + exceptionsClearTime.get() diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMetaVersion.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMetaVersion.java index 4e92d72c77c78..9c703bce2b413 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMetaVersion.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMetaVersion.java @@ -35,6 +35,7 @@ public enum PipeRuntimeMetaVersion { VERSION_1(PipeStatus.RUNNING.getType()), VERSION_2(Byte.MAX_VALUE), + VERSION_3((byte) (Byte.MAX_VALUE - 1)), ; private static final Map VERSION_MAP = new HashMap<>(); diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/PipeMetaDeSerTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/PipeMetaDeSerTest.java index d8bd4a8c0ac2c..b1b9e4de4a5a9 100644 --- a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/PipeMetaDeSerTest.java +++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/PipeMetaDeSerTest.java @@ -25,6 +25,7 @@ import org.apache.iotdb.commons.consensus.index.impl.IoTProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.RecoverProgressIndex; +import org.apache.iotdb.commons.consensus.index.impl.SchemaProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.SimpleProgressIndex; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeConnectorCriticalException; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeCriticalException; @@ -91,6 +92,16 @@ public void test() throws IOException { new PipeTaskMeta( new RecoverProgressIndex(1, new SimpleProgressIndex(1, 9)), 123)); } + }, + new HashMap() { + { + put( + new TConsensusGroupId(TConsensusGroupType.SchemaRegion, 111), + new PipeTaskMeta(MinimumProgressIndex.INSTANCE, 222)); + put( + new TConsensusGroupId(TConsensusGroupType.SchemaRegion, 333), + new PipeTaskMeta(new SchemaProgressIndex(444), 555)); + } }); ByteBuffer runtimeByteBuffer = pipeRuntimeMeta.serialize(); PipeRuntimeMeta pipeRuntimeMeta1 = PipeRuntimeMeta.deserialize(runtimeByteBuffer); @@ -117,7 +128,7 @@ public void test() throws IOException { .getDataNodeId2PipeRuntimeExceptionMap() .put(345, new PipeRuntimeCriticalException("test345")); pipeRuntimeMeta - .getConsensusGroupId2TaskMetaMap() + .getDataRegionId2TaskMetaMap() .get(new TConsensusGroupId(TConsensusGroupType.DataRegion, 456)) .trackExceptionMessage(new PipeRuntimeConnectorCriticalException("test456")); From a49c13656089096572f690734787407ef29234de Mon Sep 17 00:00:00 2001 From: Caideyipi <1336190078@qq.com> Date: Mon, 27 Nov 2023 15:45:04 +0800 Subject: [PATCH 02/11] Update PipeRuntimeMeta.java --- .../apache/iotdb/commons/pipe/task/meta/PipeRuntimeMeta.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMeta.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMeta.java index 3851b748dfffc..42237be5f2aa2 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMeta.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMeta.java @@ -22,6 +22,7 @@ import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType; import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.commons.consensus.index.ProgressIndexType; import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.SchemaProgressIndex; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeConnectorCriticalException; @@ -282,6 +283,8 @@ private static PipeRuntimeMeta deserializeVersion3(InputStream inputStream) thro pipeRuntimeMeta.status.set(PipeStatus.getPipeStatus(ReadWriteIOUtils.readByte(inputStream))); + pipeRuntimeMeta.configProgressIndex.set(ProgressIndexType.deserializeFrom(inputStream)); + int size = ReadWriteIOUtils.readInt(inputStream); for (int i = 0; i < size; ++i) { pipeRuntimeMeta.dataRegionId2TaskMetaMap.put( @@ -376,6 +379,8 @@ public static PipeRuntimeMeta deserializeVersion3(ByteBuffer byteBuffer) { pipeRuntimeMeta.status.set(PipeStatus.getPipeStatus(ReadWriteIOUtils.readByte(byteBuffer))); + pipeRuntimeMeta.configProgressIndex.set(ProgressIndexType.deserializeFrom(byteBuffer)); + int size = ReadWriteIOUtils.readInt(byteBuffer); for (int i = 0; i < size; ++i) { pipeRuntimeMeta.dataRegionId2TaskMetaMap.put( From 7bb04e78a8f7449f2688d16e05a3ff185118dd7e Mon Sep 17 00:00:00 2001 From: Caideyipi <1336190078@qq.com> Date: Mon, 27 Nov 2023 15:59:43 +0800 Subject: [PATCH 03/11] Sonar fix --- .../write/pipe/task/CreatePipePlanV2.java | 28 +++++++++++++++++++ .../write/pipe/task/DropPipePlanV2.java | 23 +++++++++++++++ .../write/pipe/task/SetPipeStatusPlanV2.java | 23 +++++++++++++++ 3 files changed, 74 insertions(+) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/task/CreatePipePlanV2.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/task/CreatePipePlanV2.java index c37f77cdb190a..74e7d05f779ef 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/task/CreatePipePlanV2.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/task/CreatePipePlanV2.java @@ -27,6 +27,7 @@ import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Objects; public class CreatePipePlanV2 extends ConfigPhysicalPlan { @@ -63,4 +64,31 @@ protected void deserializeImpl(ByteBuffer buffer) throws IOException { pipeStaticMeta = PipeStaticMeta.deserialize(buffer); pipeRuntimeMeta = PipeRuntimeMeta.deserialize(buffer); } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + CreatePipePlanV2 that = (CreatePipePlanV2) obj; + return pipeStaticMeta.equals(that.pipeStaticMeta) && pipeRuntimeMeta == that.pipeRuntimeMeta; + } + + @Override + public int hashCode() { + return Objects.hash(pipeStaticMeta, pipeRuntimeMeta); + } + + @Override + public String toString() { + return "CreatePipePlanV2{" + + "pipeStaticMeta='" + + pipeStaticMeta + + "', pipeRuntimeMeta=" + + pipeRuntimeMeta + + "'}"; + } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/task/DropPipePlanV2.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/task/DropPipePlanV2.java index a6570fb11e06a..8397e572f8984 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/task/DropPipePlanV2.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/task/DropPipePlanV2.java @@ -26,6 +26,7 @@ import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Objects; public class DropPipePlanV2 extends ConfigPhysicalPlan { @@ -54,4 +55,26 @@ protected void serializeImpl(DataOutputStream stream) throws IOException { protected void deserializeImpl(ByteBuffer buffer) throws IOException { pipeName = BasicStructureSerDeUtil.readString(buffer); } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + DropPipePlanV2 that = (DropPipePlanV2) obj; + return pipeName.equals(that.pipeName); + } + + @Override + public int hashCode() { + return Objects.hash(pipeName); + } + + @Override + public String toString() { + return "DropPipePlanV2{" + "pipeName='" + pipeName + "'}"; + } } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/task/SetPipeStatusPlanV2.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/task/SetPipeStatusPlanV2.java index d7852046880f3..443e3b26cb3e9 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/task/SetPipeStatusPlanV2.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/task/SetPipeStatusPlanV2.java @@ -27,6 +27,7 @@ import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Objects; public class SetPipeStatusPlanV2 extends ConfigPhysicalPlan { @@ -63,4 +64,26 @@ protected void deserializeImpl(ByteBuffer buffer) throws IOException { pipeName = ReadWriteIOUtils.readString(buffer); status = PipeStatus.getPipeStatus(ReadWriteIOUtils.readByte(buffer)); } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + SetPipeStatusPlanV2 that = (SetPipeStatusPlanV2) obj; + return pipeName.equals(that.pipeName) && status.equals(that.status); + } + + @Override + public int hashCode() { + return Objects.hash(pipeName, status); + } + + @Override + public String toString() { + return "SetPipeStatusPlanV2{" + "pipeName='" + pipeName + "', status=" + status + "'}"; + } } From f52298347c11c1f67f1efb2ea0b1af6a58ed5d62 Mon Sep 17 00:00:00 2001 From: Caideyipi <1336190078@qq.com> Date: Mon, 27 Nov 2023 16:22:43 +0800 Subject: [PATCH 04/11] Update SchemaProgressIndex.java --- .../consensus/index/impl/SchemaProgressIndex.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SchemaProgressIndex.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SchemaProgressIndex.java index 3f5cc6c7eb40c..b9fff71df3f5d 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SchemaProgressIndex.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SchemaProgressIndex.java @@ -34,7 +34,7 @@ * or compared to {@link ProgressIndex}es other than {@link SchemaProgressIndex} or {@link * MinimumProgressIndex}. */ -public class SchemaProgressIndex implements ProgressIndex { +public class SchemaProgressIndex extends ProgressIndex { private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); @@ -119,7 +119,7 @@ public boolean equals(Object obj) { if (!(obj instanceof SchemaProgressIndex)) { return false; } - return this.equals((SchemaProgressIndex) obj); + return this.equals(obj); } @Override @@ -146,6 +146,11 @@ public ProgressIndexType getType() { return ProgressIndexType.RECOVER_PROGRESS_INDEX; } + @Override + public TotalOrderSumTuple getTotalOrderSumTuple() { + return null; + } + public static SchemaProgressIndex deserializeFrom(ByteBuffer byteBuffer) { final SchemaProgressIndex schemaProgressIndex = new SchemaProgressIndex(); schemaProgressIndex.index = ReadWriteIOUtils.readInt(byteBuffer); From c999198eec78c213219204171efefa82a5c992fa Mon Sep 17 00:00:00 2001 From: Caideyipi <1336190078@qq.com> Date: Mon, 27 Nov 2023 20:46:16 +0800 Subject: [PATCH 05/11] Refactor of runtime meta version --- .../write/pipe/task/CreatePipePlanV2.java | 3 +- .../index/impl/HybridProgressIndex.java | 2 +- .../index/impl/IoTProgressIndex.java | 2 +- .../index/impl/RecoverProgressIndex.java | 2 +- .../index/impl/SchemaProgressIndex.java | 6 +- .../index/impl/SimpleProgressIndex.java | 2 +- ...PipeRuntimeConnectorCriticalException.java | 2 +- .../pipe/PipeRuntimeCriticalException.java | 2 +- .../pipe/PipeRuntimeExceptionType.java | 2 +- .../pipe/PipeRuntimeNonCriticalException.java | 2 +- ...peRuntimeOutOfMemoryCriticalException.java | 2 +- .../commons/pipe/task/meta/PipeMeta.java | 19 +- .../pipe/task/meta/PipeRuntimeMeta.java | 211 +++---------- .../task/meta/PipeRuntimeMetaVersion.java | 83 ------ .../pipe/task/meta/PipeStaticMeta.java | 25 +- .../commons/pipe/task/meta/PipeTaskMeta.java | 17 +- .../compatibility/PipeRuntimeMetaVersion.java | 118 ++++++++ .../runtimemeta/FormerPipeRuntimeMeta.java | 43 +++ .../runtimemeta/PipeRuntimeMetaV1.java | 171 +++++++++++ .../runtimemeta/PipeRuntimeMetaV2.java | 276 ++++++++++++++++++ .../pipe/PipeRuntimeExceptionTest.java | 2 +- .../iotdb/commons/pipe/PipeMetaDeSerTest.java | 183 +++++++++++- 22 files changed, 859 insertions(+), 316 deletions(-) delete mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMetaVersion.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/PipeRuntimeMetaVersion.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/FormerPipeRuntimeMeta.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV1.java create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV2.java diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/task/CreatePipePlanV2.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/task/CreatePipePlanV2.java index 74e7d05f779ef..72eb3c2f9df59 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/task/CreatePipePlanV2.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/task/CreatePipePlanV2.java @@ -21,6 +21,7 @@ import org.apache.iotdb.commons.pipe.task.meta.PipeRuntimeMeta; import org.apache.iotdb.commons.pipe.task.meta.PipeStaticMeta; +import org.apache.iotdb.commons.pipe.task.meta.compatibility.PipeRuntimeMetaVersion; import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan; import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType; @@ -62,7 +63,7 @@ protected void serializeImpl(DataOutputStream stream) throws IOException { @Override protected void deserializeImpl(ByteBuffer buffer) throws IOException { pipeStaticMeta = PipeStaticMeta.deserialize(buffer); - pipeRuntimeMeta = PipeRuntimeMeta.deserialize(buffer); + pipeRuntimeMeta = PipeRuntimeMetaVersion.deserializeRuntimeMeta(buffer); } @Override diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/HybridProgressIndex.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/HybridProgressIndex.java index da558e52d6545..620041a514d78 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/HybridProgressIndex.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/HybridProgressIndex.java @@ -156,7 +156,7 @@ public boolean equals(Object obj) { if (!(obj instanceof HybridProgressIndex)) { return false; } - return this.equals((HybridProgressIndex) obj); + return this.equals(obj); } @Override diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/IoTProgressIndex.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/IoTProgressIndex.java index 849a52b6981bf..5a833dd41310e 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/IoTProgressIndex.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/IoTProgressIndex.java @@ -145,7 +145,7 @@ public boolean equals(Object obj) { if (!(obj instanceof IoTProgressIndex)) { return false; } - return this.equals((IoTProgressIndex) obj); + return this.equals(obj); } @Override diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/RecoverProgressIndex.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/RecoverProgressIndex.java index 18e694bc62994..e9fa0df1ae540 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/RecoverProgressIndex.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/RecoverProgressIndex.java @@ -149,7 +149,7 @@ public boolean equals(Object obj) { if (!(obj instanceof RecoverProgressIndex)) { return false; } - return this.equals((RecoverProgressIndex) obj); + return this.equals(obj); } @Override diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SchemaProgressIndex.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SchemaProgressIndex.java index b9fff71df3f5d..58c39da03995b 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SchemaProgressIndex.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SchemaProgressIndex.java @@ -23,6 +23,8 @@ import org.apache.iotdb.commons.consensus.index.ProgressIndexType; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; +import javax.annotation.Nonnull; + import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -73,7 +75,7 @@ public void serialize(OutputStream stream) throws IOException { } @Override - public boolean isAfter(ProgressIndex progressIndex) { + public boolean isAfter(@Nonnull ProgressIndex progressIndex) { lock.readLock().lock(); try { if (progressIndex instanceof MinimumProgressIndex) { @@ -143,7 +145,7 @@ public ProgressIndex updateToMinimumIsAfterProgressIndex(ProgressIndex progressI } public ProgressIndexType getType() { - return ProgressIndexType.RECOVER_PROGRESS_INDEX; + return ProgressIndexType.SCHEMA_PROGRESS_INDEX; } @Override diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SimpleProgressIndex.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SimpleProgressIndex.java index 2984149db584e..3e7aeb6d56483 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SimpleProgressIndex.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SimpleProgressIndex.java @@ -130,7 +130,7 @@ public boolean equals(Object obj) { if (!(obj instanceof SimpleProgressIndex)) { return false; } - return this.equals((SimpleProgressIndex) obj); + return this.equals(obj); } @Override diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeConnectorCriticalException.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeConnectorCriticalException.java index 1325ca55bda11..19644f81be3bd 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeConnectorCriticalException.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeConnectorCriticalException.java @@ -19,7 +19,7 @@ package org.apache.iotdb.commons.exception.pipe; -import org.apache.iotdb.commons.pipe.task.meta.PipeRuntimeMetaVersion; +import org.apache.iotdb.commons.pipe.task.meta.compatibility.PipeRuntimeMetaVersion; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; import java.io.IOException; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeCriticalException.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeCriticalException.java index 0d3c0c2a78690..baba2a52e1c8b 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeCriticalException.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeCriticalException.java @@ -19,7 +19,7 @@ package org.apache.iotdb.commons.exception.pipe; -import org.apache.iotdb.commons.pipe.task.meta.PipeRuntimeMetaVersion; +import org.apache.iotdb.commons.pipe.task.meta.compatibility.PipeRuntimeMetaVersion; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; import java.io.IOException; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeExceptionType.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeExceptionType.java index 287d0d11b06b3..c82ded4c35fe1 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeExceptionType.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeExceptionType.java @@ -19,7 +19,7 @@ package org.apache.iotdb.commons.exception.pipe; -import org.apache.iotdb.commons.pipe.task.meta.PipeRuntimeMetaVersion; +import org.apache.iotdb.commons.pipe.task.meta.compatibility.PipeRuntimeMetaVersion; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; import java.io.IOException; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeNonCriticalException.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeNonCriticalException.java index 6b072a4b6ab75..4c6b2de079d01 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeNonCriticalException.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeNonCriticalException.java @@ -19,7 +19,7 @@ package org.apache.iotdb.commons.exception.pipe; -import org.apache.iotdb.commons.pipe.task.meta.PipeRuntimeMetaVersion; +import org.apache.iotdb.commons.pipe.task.meta.compatibility.PipeRuntimeMetaVersion; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; import java.io.IOException; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeOutOfMemoryCriticalException.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeOutOfMemoryCriticalException.java index b073152a3e558..c9ba275d163e9 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeOutOfMemoryCriticalException.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeOutOfMemoryCriticalException.java @@ -19,7 +19,7 @@ package org.apache.iotdb.commons.exception.pipe; -import org.apache.iotdb.commons.pipe.task.meta.PipeRuntimeMetaVersion; +import org.apache.iotdb.commons.pipe.task.meta.compatibility.PipeRuntimeMetaVersion; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; import java.io.IOException; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeMeta.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeMeta.java index 8b15b0ffa9c9a..ebda7ef9f35e2 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeMeta.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeMeta.java @@ -19,12 +19,15 @@ package org.apache.iotdb.commons.pipe.task.meta; +import org.apache.iotdb.commons.pipe.task.meta.compatibility.PipeRuntimeMetaVersion; import org.apache.iotdb.tsfile.utils.PublicBAOS; +import javax.annotation.Nonnull; + import java.io.DataOutputStream; import java.io.FileInputStream; -import java.io.FileOutputStream; import java.io.IOException; +import java.io.OutputStream; import java.nio.ByteBuffer; import java.util.Objects; @@ -53,25 +56,23 @@ public ByteBuffer serialize() throws IOException { return ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); } - public void serialize(DataOutputStream outputStream) throws IOException { - staticMeta.serialize(outputStream); - runtimeMeta.serialize(outputStream); - } - - public void serialize(FileOutputStream outputStream) throws IOException { + public void serialize(OutputStream outputStream) throws IOException { staticMeta.serialize(outputStream); runtimeMeta.serialize(outputStream); } + @Nonnull public static PipeMeta deserialize(FileInputStream fileInputStream) throws IOException { final PipeStaticMeta staticMeta = PipeStaticMeta.deserialize(fileInputStream); - final PipeRuntimeMeta runtimeMeta = PipeRuntimeMeta.deserialize(fileInputStream); + final PipeRuntimeMeta runtimeMeta = + PipeRuntimeMetaVersion.deserializeRuntimeMeta(fileInputStream); return new PipeMeta(staticMeta, runtimeMeta); } + @Nonnull public static PipeMeta deserialize(ByteBuffer byteBuffer) { final PipeStaticMeta staticMeta = PipeStaticMeta.deserialize(byteBuffer); - final PipeRuntimeMeta runtimeMeta = PipeRuntimeMeta.deserialize(byteBuffer); + final PipeRuntimeMeta runtimeMeta = PipeRuntimeMetaVersion.deserializeRuntimeMeta(byteBuffer); return new PipeMeta(staticMeta, runtimeMeta); } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMeta.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMeta.java index 42237be5f2aa2..843146272dae3 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMeta.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMeta.java @@ -29,13 +29,16 @@ import org.apache.iotdb.commons.exception.pipe.PipeRuntimeCriticalException; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeException; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeExceptionType; +import org.apache.iotdb.commons.pipe.task.meta.compatibility.PipeRuntimeMetaVersion; import org.apache.iotdb.tsfile.utils.PublicBAOS; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; +import javax.annotation.Nonnull; + import java.io.DataOutputStream; -import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import java.nio.ByteBuffer; import java.util.HashMap; import java.util.Map; @@ -47,6 +50,8 @@ public class PipeRuntimeMeta { + /////////////////////////////// Fields /////////////////////////////// + private final AtomicReference status = new AtomicReference<>(PipeStatus.STOPPED); // All the progressIndexes of the pipe. @@ -56,11 +61,13 @@ public class PipeRuntimeMeta { */ private final AtomicReference configProgressIndex = new AtomicReference<>(MinimumProgressIndex.INSTANCE); + /** * Used to store progress for schema transmission on DataRegion, usually updated since the data * synchronization is basic function of pipe. */ - private final Map dataRegionId2TaskMetaMap; + private Map dataRegionId2TaskMetaMap; + /** * Used to store {@link SchemaProgressIndex} for schema transmission on SchemaRegion, always * {@link MinimumProgressIndex} if the pipe has no relation to schema synchronization. @@ -78,13 +85,15 @@ public class PipeRuntimeMeta { *

2. {@link PipeRuntimeConnectorCriticalException}, to record the exception reported by other * pipes sharing the same connector, and will stop the pipe likewise. */ - private final Map dataNodeId2PipeRuntimeExceptionMap = + private Map dataNodeId2PipeRuntimeExceptionMap = new ConcurrentHashMap<>(); private final AtomicLong exceptionsClearTime = new AtomicLong(Long.MIN_VALUE); private final AtomicBoolean isStoppedByRuntimeException = new AtomicBoolean(false); + /////////////////////////////// Initializer /////////////////////////////// + public PipeRuntimeMeta() { dataRegionId2TaskMetaMap = new ConcurrentHashMap<>(); schemaRegionId2TaskMetaMap = new ConcurrentHashMap<>(); @@ -93,14 +102,25 @@ public PipeRuntimeMeta() { public PipeRuntimeMeta( Map dataRegionId2TaskMetaMap, Map schemaRegionId2TaskMetaMap) { - this.dataRegionId2TaskMetaMap = dataRegionId2TaskMetaMap; - this.schemaRegionId2TaskMetaMap = schemaRegionId2TaskMetaMap; + this.dataRegionId2TaskMetaMap = new ConcurrentHashMap<>(dataRegionId2TaskMetaMap); + this.schemaRegionId2TaskMetaMap = new ConcurrentHashMap<>(schemaRegionId2TaskMetaMap); } + /////////////////////////////// Normal getter & setter /////////////////////////////// + public AtomicReference getStatus() { return status; } + public ProgressIndex getConfigProgressIndex() { + return configProgressIndex.get(); + } + + public ProgressIndex updateConfigProgressIndex(ProgressIndex updateIndex) { + return configProgressIndex.updateAndGet( + index -> index.updateToMinimumIsAfterProgressIndex(updateIndex)); + } + public Map getDataRegionId2TaskMetaMap() { return dataRegionId2TaskMetaMap; } @@ -131,6 +151,8 @@ public void setIsStoppedByRuntimeException(boolean isStoppedByRuntimeException) this.isStoppedByRuntimeException.set(isStoppedByRuntimeException); } + /////////////////////////////// Serialization & deserialization /////////////////////////////// + public ByteBuffer serialize() throws IOException { PublicBAOS byteArrayOutputStream = new PublicBAOS(); DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream); @@ -138,47 +160,7 @@ public ByteBuffer serialize() throws IOException { return ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); } - public void serialize(DataOutputStream outputStream) throws IOException { - PipeRuntimeMetaVersion.VERSION_3.serialize(outputStream); - - ReadWriteIOUtils.write(status.get().getType(), outputStream); - - configProgressIndex.get().serialize(outputStream); - - // Avoid concurrent modification - final Map dataRegionId2TaskMetaMapView = - new HashMap<>(dataRegionId2TaskMetaMap); - ReadWriteIOUtils.write(dataRegionId2TaskMetaMapView.size(), outputStream); - for (Map.Entry entry : - dataRegionId2TaskMetaMapView.entrySet()) { - ReadWriteIOUtils.write(entry.getKey().getId(), outputStream); - entry.getValue().serialize(outputStream); - } - - final Map schemaRegionId2TaskMetaMapView = - new HashMap<>(schemaRegionId2TaskMetaMap); - ReadWriteIOUtils.write(schemaRegionId2TaskMetaMapView.size(), outputStream); - for (Map.Entry entry : - schemaRegionId2TaskMetaMapView.entrySet()) { - ReadWriteIOUtils.write(entry.getKey().getId(), outputStream); - entry.getValue().serialize(outputStream); - } - - // Avoid concurrent modification - final Map dataNodeId2PipeRuntimeExceptionMapView = - new HashMap<>(dataNodeId2PipeRuntimeExceptionMap); - ReadWriteIOUtils.write(dataNodeId2PipeRuntimeExceptionMapView.size(), outputStream); - for (Map.Entry entry : - dataNodeId2PipeRuntimeExceptionMapView.entrySet()) { - ReadWriteIOUtils.write(entry.getKey(), outputStream); - entry.getValue().serialize(outputStream); - } - - ReadWriteIOUtils.write(exceptionsClearTime.get(), outputStream); - ReadWriteIOUtils.write(isStoppedByRuntimeException.get(), outputStream); - } - - public void serialize(FileOutputStream outputStream) throws IOException { + public void serialize(OutputStream outputStream) throws IOException { PipeRuntimeMetaVersion.VERSION_3.serialize(outputStream); ReadWriteIOUtils.write(status.get().getType(), outputStream); @@ -218,67 +200,8 @@ public void serialize(FileOutputStream outputStream) throws IOException { ReadWriteIOUtils.write(isStoppedByRuntimeException.get(), outputStream); } + @Nonnull public static PipeRuntimeMeta deserialize(InputStream inputStream) throws IOException { - final byte pipeRuntimeVersionByte = ReadWriteIOUtils.readByte(inputStream); - final PipeRuntimeMetaVersion pipeRuntimeMetaVersion = - PipeRuntimeMetaVersion.deserialize(pipeRuntimeVersionByte); - switch (pipeRuntimeMetaVersion) { - case VERSION_1: - return deserializeVersion1(inputStream, pipeRuntimeVersionByte); - case VERSION_2: - return deserializeVersion2(inputStream); - case VERSION_3: - return deserializeVersion3(inputStream); - default: - throw new UnsupportedOperationException( - "Unknown pipe runtime meta version: " + pipeRuntimeMetaVersion.getVersion()); - } - } - - private static PipeRuntimeMeta deserializeVersion1(InputStream inputStream, byte pipeStatusByte) - throws IOException { - final PipeRuntimeMeta pipeRuntimeMeta = new PipeRuntimeMeta(); - - pipeRuntimeMeta.status.set(PipeStatus.getPipeStatus(pipeStatusByte)); - - final int size = ReadWriteIOUtils.readInt(inputStream); - for (int i = 0; i < size; ++i) { - pipeRuntimeMeta.dataRegionId2TaskMetaMap.put( - new TConsensusGroupId( - TConsensusGroupType.DataRegion, ReadWriteIOUtils.readInt(inputStream)), - PipeTaskMeta.deserialize(PipeRuntimeMetaVersion.VERSION_1, inputStream)); - } - - return pipeRuntimeMeta; - } - - private static PipeRuntimeMeta deserializeVersion2(InputStream inputStream) throws IOException { - final PipeRuntimeMeta pipeRuntimeMeta = new PipeRuntimeMeta(); - - pipeRuntimeMeta.status.set(PipeStatus.getPipeStatus(ReadWriteIOUtils.readByte(inputStream))); - - int size = ReadWriteIOUtils.readInt(inputStream); - for (int i = 0; i < size; ++i) { - pipeRuntimeMeta.dataRegionId2TaskMetaMap.put( - new TConsensusGroupId( - TConsensusGroupType.DataRegion, ReadWriteIOUtils.readInt(inputStream)), - PipeTaskMeta.deserialize(PipeRuntimeMetaVersion.VERSION_2, inputStream)); - } - - size = ReadWriteIOUtils.readInt(inputStream); - for (int i = 0; i < size; ++i) { - pipeRuntimeMeta.dataNodeId2PipeRuntimeExceptionMap.put( - ReadWriteIOUtils.readInt(inputStream), - PipeRuntimeExceptionType.deserializeFrom(PipeRuntimeMetaVersion.VERSION_2, inputStream)); - } - - pipeRuntimeMeta.exceptionsClearTime.set(ReadWriteIOUtils.readLong(inputStream)); - pipeRuntimeMeta.isStoppedByRuntimeException.set(ReadWriteIOUtils.readBool(inputStream)); - - return pipeRuntimeMeta; - } - - private static PipeRuntimeMeta deserializeVersion3(InputStream inputStream) throws IOException { final PipeRuntimeMeta pipeRuntimeMeta = new PipeRuntimeMeta(); pipeRuntimeMeta.status.set(PipeStatus.getPipeStatus(ReadWriteIOUtils.readByte(inputStream))); @@ -314,67 +237,8 @@ private static PipeRuntimeMeta deserializeVersion3(InputStream inputStream) thro return pipeRuntimeMeta; } + @Nonnull public static PipeRuntimeMeta deserialize(ByteBuffer byteBuffer) { - final byte pipeRuntimeVersionByte = ReadWriteIOUtils.readByte(byteBuffer); - final PipeRuntimeMetaVersion pipeRuntimeMetaVersion = - PipeRuntimeMetaVersion.deserialize(pipeRuntimeVersionByte); - switch (pipeRuntimeMetaVersion) { - case VERSION_1: - return deserializeVersion1(byteBuffer, pipeRuntimeVersionByte); - case VERSION_2: - return deserializeVersion2(byteBuffer); - case VERSION_3: - return deserializeVersion3(byteBuffer); - default: - throw new UnsupportedOperationException( - "Unknown pipe runtime meta version: " + pipeRuntimeMetaVersion.getVersion()); - } - } - - private static PipeRuntimeMeta deserializeVersion1( - ByteBuffer byteBuffer, byte pipeRuntimeVersionByte) { - final PipeRuntimeMeta pipeRuntimeMeta = new PipeRuntimeMeta(); - - pipeRuntimeMeta.status.set(PipeStatus.getPipeStatus(pipeRuntimeVersionByte)); - - final int size = ReadWriteIOUtils.readInt(byteBuffer); - for (int i = 0; i < size; ++i) { - pipeRuntimeMeta.dataRegionId2TaskMetaMap.put( - new TConsensusGroupId( - TConsensusGroupType.DataRegion, ReadWriteIOUtils.readInt(byteBuffer)), - PipeTaskMeta.deserialize(PipeRuntimeMetaVersion.VERSION_1, byteBuffer)); - } - - return pipeRuntimeMeta; - } - - public static PipeRuntimeMeta deserializeVersion2(ByteBuffer byteBuffer) { - final PipeRuntimeMeta pipeRuntimeMeta = new PipeRuntimeMeta(); - - pipeRuntimeMeta.status.set(PipeStatus.getPipeStatus(ReadWriteIOUtils.readByte(byteBuffer))); - - int size = ReadWriteIOUtils.readInt(byteBuffer); - for (int i = 0; i < size; ++i) { - pipeRuntimeMeta.dataRegionId2TaskMetaMap.put( - new TConsensusGroupId( - TConsensusGroupType.DataRegion, ReadWriteIOUtils.readInt(byteBuffer)), - PipeTaskMeta.deserialize(PipeRuntimeMetaVersion.VERSION_2, byteBuffer)); - } - - size = ReadWriteIOUtils.readInt(byteBuffer); - for (int i = 0; i < size; ++i) { - pipeRuntimeMeta.dataNodeId2PipeRuntimeExceptionMap.put( - ReadWriteIOUtils.readInt(byteBuffer), - PipeRuntimeExceptionType.deserializeFrom(PipeRuntimeMetaVersion.VERSION_2, byteBuffer)); - } - - pipeRuntimeMeta.exceptionsClearTime.set(ReadWriteIOUtils.readLong(byteBuffer)); - pipeRuntimeMeta.isStoppedByRuntimeException.set(ReadWriteIOUtils.readBool(byteBuffer)); - - return pipeRuntimeMeta; - } - - public static PipeRuntimeMeta deserializeVersion3(ByteBuffer byteBuffer) { final PipeRuntimeMeta pipeRuntimeMeta = new PipeRuntimeMeta(); pipeRuntimeMeta.status.set(PipeStatus.getPipeStatus(ReadWriteIOUtils.readByte(byteBuffer))); @@ -410,6 +274,21 @@ public static PipeRuntimeMeta deserializeVersion3(ByteBuffer byteBuffer) { return pipeRuntimeMeta; } + /////////////////////////////// Compatibility /////////////////////////////// + + // DO NOT CALL IT, unless from the former versions + public void setDataNodeId2PipeRuntimeExceptionMap( + Map dataNodeId2PipeRuntimeExceptionMap) { + this.dataNodeId2PipeRuntimeExceptionMap = dataNodeId2PipeRuntimeExceptionMap; + } + + public void setDataRegionId2TaskMetaMap( + Map dataRegionId2TaskMetaMap) { + this.dataRegionId2TaskMetaMap = dataRegionId2TaskMetaMap; + } + + /////////////////////////////// Object /////////////////////////////// + @Override public boolean equals(Object o) { if (this == o) { diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMetaVersion.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMetaVersion.java deleted file mode 100644 index 9c703bce2b413..0000000000000 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMetaVersion.java +++ /dev/null @@ -1,83 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.commons.pipe.task.meta; - -import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; - -import java.io.DataOutputStream; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.InputStream; -import java.nio.ByteBuffer; -import java.util.HashMap; -import java.util.Map; - -public enum PipeRuntimeMetaVersion { - - // for compatibility use - VERSION_1(PipeStatus.RUNNING.getType()), - - VERSION_2(Byte.MAX_VALUE), - VERSION_3((byte) (Byte.MAX_VALUE - 1)), - ; - - private static final Map VERSION_MAP = new HashMap<>(); - - static { - // for compatibility use - for (final PipeStatus status : PipeStatus.values()) { - VERSION_MAP.put(status.getType(), VERSION_1); - } - - for (final PipeRuntimeMetaVersion version : PipeRuntimeMetaVersion.values()) { - VERSION_MAP.put(version.getVersion(), version); - } - } - - private final byte version; - - PipeRuntimeMetaVersion(byte version) { - this.version = version; - } - - public byte getVersion() { - return version; - } - - public void serialize(FileOutputStream outputStream) throws IOException { - ReadWriteIOUtils.write(version, outputStream); - } - - public void serialize(DataOutputStream outputStream) throws IOException { - ReadWriteIOUtils.write(version, outputStream); - } - - public static PipeRuntimeMetaVersion deserialize(InputStream inputStream) throws IOException { - return deserialize(ReadWriteIOUtils.readByte(inputStream)); - } - - public static PipeRuntimeMetaVersion deserialize(ByteBuffer byteBuffer) { - return deserialize(ReadWriteIOUtils.readByte(byteBuffer)); - } - - public static PipeRuntimeMetaVersion deserialize(byte version) { - return VERSION_MAP.get(version); - } -} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeStaticMeta.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeStaticMeta.java index 9be4ce369d177..5e725f923a249 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeStaticMeta.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeStaticMeta.java @@ -24,9 +24,9 @@ import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; import java.io.DataOutputStream; -import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import java.nio.ByteBuffer; import java.util.HashMap; import java.util.Map; @@ -85,28 +85,7 @@ public ByteBuffer serialize() throws IOException { return ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); } - public void serialize(DataOutputStream outputStream) throws IOException { - ReadWriteIOUtils.write(pipeName, outputStream); - ReadWriteIOUtils.write(creationTime, outputStream); - - ReadWriteIOUtils.write(extractorParameters.getAttribute().size(), outputStream); - for (Map.Entry entry : extractorParameters.getAttribute().entrySet()) { - ReadWriteIOUtils.write(entry.getKey(), outputStream); - ReadWriteIOUtils.write(entry.getValue(), outputStream); - } - ReadWriteIOUtils.write(processorParameters.getAttribute().size(), outputStream); - for (Map.Entry entry : processorParameters.getAttribute().entrySet()) { - ReadWriteIOUtils.write(entry.getKey(), outputStream); - ReadWriteIOUtils.write(entry.getValue(), outputStream); - } - ReadWriteIOUtils.write(connectorParameters.getAttribute().size(), outputStream); - for (Map.Entry entry : connectorParameters.getAttribute().entrySet()) { - ReadWriteIOUtils.write(entry.getKey(), outputStream); - ReadWriteIOUtils.write(entry.getValue(), outputStream); - } - } - - public void serialize(FileOutputStream outputStream) throws IOException { + public void serialize(OutputStream outputStream) throws IOException { ReadWriteIOUtils.write(pipeName, outputStream); ReadWriteIOUtils.write(creationTime, outputStream); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeTaskMeta.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeTaskMeta.java index e5a80f681be6c..0a0529eaeb2b7 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeTaskMeta.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeTaskMeta.java @@ -26,12 +26,12 @@ import org.apache.iotdb.commons.exception.pipe.PipeRuntimeException; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeExceptionType; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeNonCriticalException; +import org.apache.iotdb.commons.pipe.task.meta.compatibility.PipeRuntimeMetaVersion; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; -import java.io.DataOutputStream; -import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Map; @@ -95,18 +95,7 @@ public synchronized void clearExceptionMessages() { exceptionMessages.clear(); } - public synchronized void serialize(DataOutputStream outputStream) throws IOException { - progressIndex.get().serialize(outputStream); - - ReadWriteIOUtils.write(leaderDataNodeId.get(), outputStream); - - ReadWriteIOUtils.write(exceptionMessages.size(), outputStream); - for (final PipeRuntimeException pipeRuntimeException : exceptionMessages.values()) { - pipeRuntimeException.serialize(outputStream); - } - } - - public synchronized void serialize(FileOutputStream outputStream) throws IOException { + public synchronized void serialize(OutputStream outputStream) throws IOException { progressIndex.get().serialize(outputStream); ReadWriteIOUtils.write(leaderDataNodeId.get(), outputStream); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/PipeRuntimeMetaVersion.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/PipeRuntimeMetaVersion.java new file mode 100644 index 0000000000000..98d6109d5faf9 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/PipeRuntimeMetaVersion.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.pipe.task.meta.compatibility; + +import org.apache.iotdb.commons.pipe.task.meta.PipeRuntimeMeta; +import org.apache.iotdb.commons.pipe.task.meta.PipeStatus; +import org.apache.iotdb.commons.pipe.task.meta.compatibility.runtimemeta.PipeRuntimeMetaV1; +import org.apache.iotdb.commons.pipe.task.meta.compatibility.runtimemeta.PipeRuntimeMetaV2; +import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; + +public enum PipeRuntimeMetaVersion { + + // For compatibility use + VERSION_1(PipeStatus.RUNNING.getType()), + + VERSION_2(Byte.MAX_VALUE), + VERSION_3((byte) (Byte.MAX_VALUE - 1)), + ; + + private static final Map VERSION_MAP = new HashMap<>(); + + static { + // For compatibility use + for (final PipeStatus status : PipeStatus.values()) { + VERSION_MAP.put(status.getType(), VERSION_1); + } + + for (final PipeRuntimeMetaVersion version : PipeRuntimeMetaVersion.values()) { + VERSION_MAP.put(version.getVersion(), version); + } + } + + private final byte version; + + PipeRuntimeMetaVersion(byte version) { + this.version = version; + } + + public byte getVersion() { + return version; + } + + public void serialize(OutputStream outputStream) throws IOException { + ReadWriteIOUtils.write(version, outputStream); + } + + public static PipeRuntimeMetaVersion deserialize(InputStream inputStream) throws IOException { + return deserialize(ReadWriteIOUtils.readByte(inputStream)); + } + + public static PipeRuntimeMetaVersion deserialize(ByteBuffer byteBuffer) { + return deserialize(ReadWriteIOUtils.readByte(byteBuffer)); + } + + public static PipeRuntimeMetaVersion deserialize(byte version) { + return VERSION_MAP.get(version); + } + + /////////////////////////////// RuntimeMeta deserializer /////////////////////////////// + + public static PipeRuntimeMeta deserializeRuntimeMeta(InputStream inputStream) throws IOException { + PipeRuntimeMetaVersion pipeRuntimeMetaVersion = + PipeRuntimeMetaVersion.deserialize(ReadWriteIOUtils.readByte(inputStream)); + switch (pipeRuntimeMetaVersion) { + case VERSION_1: + PipeStatus status = PipeStatus.getPipeStatus(pipeRuntimeMetaVersion.getVersion()); + return PipeRuntimeMetaV1.deserialize(inputStream, status).toCurrentPipeRuntimeMetaVersion(); + case VERSION_2: + return PipeRuntimeMetaV2.deserialize(inputStream).toCurrentPipeRuntimeMetaVersion(); + case VERSION_3: + return PipeRuntimeMeta.deserialize(inputStream); + default: + throw new UnsupportedOperationException( + "Unknown pipe runtime meta version: " + pipeRuntimeMetaVersion.getVersion()); + } + } + + public static PipeRuntimeMeta deserializeRuntimeMeta(ByteBuffer byteBuffer) { + PipeRuntimeMetaVersion pipeRuntimeMetaVersion = + PipeRuntimeMetaVersion.deserialize(ReadWriteIOUtils.readByte(byteBuffer)); + switch (pipeRuntimeMetaVersion) { + case VERSION_1: + PipeStatus status = PipeStatus.getPipeStatus(pipeRuntimeMetaVersion.getVersion()); + return PipeRuntimeMetaV1.deserialize(byteBuffer, status).toCurrentPipeRuntimeMetaVersion(); + case VERSION_2: + return PipeRuntimeMetaV2.deserialize(byteBuffer).toCurrentPipeRuntimeMetaVersion(); + case VERSION_3: + return PipeRuntimeMeta.deserialize(byteBuffer); + default: + throw new UnsupportedOperationException( + "Unknown pipe runtime meta version: " + pipeRuntimeMetaVersion.getVersion()); + } + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/FormerPipeRuntimeMeta.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/FormerPipeRuntimeMeta.java new file mode 100644 index 0000000000000..c623e7a1c3eef --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/FormerPipeRuntimeMeta.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.pipe.task.meta.compatibility.runtimemeta; + +import org.apache.iotdb.commons.pipe.task.meta.PipeRuntimeMeta; +import org.apache.iotdb.commons.utils.TestOnly; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; + +public interface FormerPipeRuntimeMeta { + + @TestOnly + ByteBuffer serialize() throws IOException; + + @TestOnly + void serialize(OutputStream outputStream) throws IOException; + + /** + * This method always return the incumbent {@link PipeRuntimeMeta}. When changing to a new + * version, be sure to move the previous one to the packet, grant it a version, make it implement + * this interface, and then update this method in all {@link FormerPipeRuntimeMeta}s. + */ + PipeRuntimeMeta toCurrentPipeRuntimeMetaVersion(); +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV1.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV1.java new file mode 100644 index 0000000000000..1acf8f1578324 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV1.java @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.pipe.task.meta.compatibility.runtimemeta; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType; +import org.apache.iotdb.commons.pipe.task.meta.PipeRuntimeMeta; +import org.apache.iotdb.commons.pipe.task.meta.PipeStatus; +import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta; +import org.apache.iotdb.commons.pipe.task.meta.compatibility.PipeRuntimeMetaVersion; +import org.apache.iotdb.commons.utils.TestOnly; +import org.apache.iotdb.tsfile.utils.PublicBAOS; +import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; + +import javax.annotation.Nonnull; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicReference; + +@Deprecated +public class PipeRuntimeMetaV1 implements FormerPipeRuntimeMeta { + + /////////////////////////////// Fields /////////////////////////////// + + private final AtomicReference status; + private final Map consensusGroupIdToTaskMetaMap; + + /////////////////////////////// Initializer /////////////////////////////// + + public PipeRuntimeMetaV1() { + status = new AtomicReference<>(PipeStatus.STOPPED); + consensusGroupIdToTaskMetaMap = new ConcurrentHashMap<>(); + } + + public PipeRuntimeMetaV1(Map consensusGroupIdToTaskMetaMap) { + status = new AtomicReference<>(PipeStatus.STOPPED); + this.consensusGroupIdToTaskMetaMap = consensusGroupIdToTaskMetaMap; + } + + /////////////////////////////// Normal getter & setter /////////////////////////////// + + @TestOnly + public AtomicReference getStatus() { + return status; + } + + @TestOnly + public Map getConsensusGroupIdToTaskMetaMap() { + return consensusGroupIdToTaskMetaMap; + } + + /////////////////////////////// Serialization & deserialization /////////////////////////////// + + public ByteBuffer serialize() throws IOException { + PublicBAOS byteArrayOutputStream = new PublicBAOS(); + DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream); + serialize(outputStream); + return ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + } + + @Override + public void serialize(OutputStream outputStream) throws IOException { + ReadWriteIOUtils.write(status.get().getType(), outputStream); + + ReadWriteIOUtils.write(consensusGroupIdToTaskMetaMap.size(), outputStream); + for (Map.Entry entry : + consensusGroupIdToTaskMetaMap.entrySet()) { + ReadWriteIOUtils.write(entry.getKey().getId(), outputStream); + entry.getValue().serialize(outputStream); + } + } + + @Nonnull + public static PipeRuntimeMetaV1 deserialize(InputStream inputStream, PipeStatus status) + throws IOException { + final PipeRuntimeMetaV1 pipeRuntimeMeta = new PipeRuntimeMetaV1(); + + pipeRuntimeMeta.status.set(status); + + final int size = ReadWriteIOUtils.readInt(inputStream); + for (int i = 0; i < size; ++i) { + pipeRuntimeMeta.consensusGroupIdToTaskMetaMap.put( + new TConsensusGroupId( + TConsensusGroupType.DataRegion, ReadWriteIOUtils.readInt(inputStream)), + PipeTaskMeta.deserialize(PipeRuntimeMetaVersion.VERSION_1, inputStream)); + } + + return pipeRuntimeMeta; + } + + @Nonnull + public static PipeRuntimeMetaV1 deserialize(ByteBuffer byteBuffer, PipeStatus status) { + final PipeRuntimeMetaV1 pipeRuntimeMeta = new PipeRuntimeMetaV1(); + + pipeRuntimeMeta.status.set(status); + + final int size = ReadWriteIOUtils.readInt(byteBuffer); + for (int i = 0; i < size; ++i) { + pipeRuntimeMeta.consensusGroupIdToTaskMetaMap.put( + new TConsensusGroupId( + TConsensusGroupType.DataRegion, ReadWriteIOUtils.readInt(byteBuffer)), + PipeTaskMeta.deserialize(PipeRuntimeMetaVersion.VERSION_1, byteBuffer)); + } + + return pipeRuntimeMeta; + } + + /////////////////////////////// Compatibility /////////////////////////////// + + @Override + public PipeRuntimeMeta toCurrentPipeRuntimeMetaVersion() { + PipeRuntimeMeta pipeRuntimeMeta = new PipeRuntimeMeta(); + pipeRuntimeMeta.getStatus().set(status.get()); + pipeRuntimeMeta.setDataRegionId2TaskMetaMap(consensusGroupIdToTaskMetaMap); + return pipeRuntimeMeta; + } + + /////////////////////////////// Object /////////////////////////////// + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PipeRuntimeMetaV1 that = (PipeRuntimeMetaV1) o; + return Objects.equals(status.get().getType(), that.status.get().getType()) + && consensusGroupIdToTaskMetaMap.equals(that.consensusGroupIdToTaskMetaMap); + } + + @Override + public int hashCode() { + return Objects.hash(status, consensusGroupIdToTaskMetaMap); + } + + @Override + public String toString() { + return "PipeRuntimeMetaV1{" + + "status=" + + status + + ", consensusGroupIdToTaskMetaMap=" + + consensusGroupIdToTaskMetaMap + + '}'; + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV2.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV2.java new file mode 100644 index 0000000000000..b6a7d01de3830 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV2.java @@ -0,0 +1,276 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.commons.pipe.task.meta.compatibility.runtimemeta; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType; +import org.apache.iotdb.commons.exception.pipe.PipeRuntimeConnectorCriticalException; +import org.apache.iotdb.commons.exception.pipe.PipeRuntimeCriticalException; +import org.apache.iotdb.commons.exception.pipe.PipeRuntimeException; +import org.apache.iotdb.commons.exception.pipe.PipeRuntimeExceptionType; +import org.apache.iotdb.commons.pipe.task.meta.PipeRuntimeMeta; +import org.apache.iotdb.commons.pipe.task.meta.PipeStatus; +import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta; +import org.apache.iotdb.commons.pipe.task.meta.compatibility.PipeRuntimeMetaVersion; +import org.apache.iotdb.commons.utils.TestOnly; +import org.apache.iotdb.tsfile.utils.PublicBAOS; +import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; + +import javax.annotation.Nonnull; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + +@Deprecated +public class PipeRuntimeMetaV2 implements FormerPipeRuntimeMeta { + + /////////////////////////////// Fields /////////////////////////////// + + private final AtomicReference status = new AtomicReference<>(PipeStatus.STOPPED); + + private final Map consensusGroupId2TaskMetaMap; + + /** + * Stores the newest exceptions encountered group by dataNodes. + * + *

The exceptions are all instances of: + * + *

1. {@link PipeRuntimeCriticalException}, to record the failure of pushing pipeMeta, and will + * result in the halt of pipe execution. + * + *

2. {@link PipeRuntimeConnectorCriticalException}, to record the exception reported by other + * pipes sharing the same connector, and will stop the pipe likewise. + */ + private final Map dataNodeId2PipeRuntimeExceptionMap = + new ConcurrentHashMap<>(); + + private final AtomicLong exceptionsClearTime = new AtomicLong(Long.MIN_VALUE); + + private final AtomicBoolean isStoppedByRuntimeException = new AtomicBoolean(false); + + /////////////////////////////// Initializer /////////////////////////////// + + public PipeRuntimeMetaV2() { + consensusGroupId2TaskMetaMap = new ConcurrentHashMap<>(); + } + + public PipeRuntimeMetaV2(Map consensusGroupId2TaskMetaMap) { + this.consensusGroupId2TaskMetaMap = consensusGroupId2TaskMetaMap; + } + + /////////////////////////////// Normal getter & setter /////////////////////////////// + + @TestOnly + public AtomicReference getStatus() { + return status; + } + + @TestOnly + public Map getConsensusGroupId2TaskMetaMap() { + return consensusGroupId2TaskMetaMap; + } + + @TestOnly + public Map getDataNodeId2PipeRuntimeExceptionMap() { + return dataNodeId2PipeRuntimeExceptionMap; + } + + @TestOnly + public long getExceptionsClearTime() { + return exceptionsClearTime.get(); + } + + @TestOnly + public void setExceptionsClearTime(long exceptionsClearTime) { + if (exceptionsClearTime > this.getExceptionsClearTime()) { + this.exceptionsClearTime.set(exceptionsClearTime); + } + } + + @TestOnly + public boolean getIsStoppedByRuntimeException() { + return isStoppedByRuntimeException.get(); + } + + @TestOnly + public void setIsStoppedByRuntimeException(boolean isStoppedByRuntimeException) { + this.isStoppedByRuntimeException.set(isStoppedByRuntimeException); + } + + /////////////////////////////// Serialization & deserialization /////////////////////////////// + + public ByteBuffer serialize() throws IOException { + PublicBAOS byteArrayOutputStream = new PublicBAOS(); + DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream); + serialize(outputStream); + return ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + } + + @Override + public void serialize(OutputStream outputStream) throws IOException { + PipeRuntimeMetaVersion.VERSION_2.serialize(outputStream); + + ReadWriteIOUtils.write(status.get().getType(), outputStream); + + // Avoid concurrent modification + final Map consensusGroupId2TaskMetaMapView = + new HashMap<>(consensusGroupId2TaskMetaMap); + ReadWriteIOUtils.write(consensusGroupId2TaskMetaMapView.size(), outputStream); + for (Map.Entry entry : + consensusGroupId2TaskMetaMapView.entrySet()) { + ReadWriteIOUtils.write(entry.getKey().getId(), outputStream); + entry.getValue().serialize(outputStream); + } + + // Avoid concurrent modification + final Map dataNodeId2PipeRuntimeExceptionMapView = + new HashMap<>(dataNodeId2PipeRuntimeExceptionMap); + ReadWriteIOUtils.write(dataNodeId2PipeRuntimeExceptionMapView.size(), outputStream); + for (Map.Entry entry : + dataNodeId2PipeRuntimeExceptionMapView.entrySet()) { + ReadWriteIOUtils.write(entry.getKey(), outputStream); + entry.getValue().serialize(outputStream); + } + + ReadWriteIOUtils.write(exceptionsClearTime.get(), outputStream); + ReadWriteIOUtils.write(isStoppedByRuntimeException.get(), outputStream); + } + + @Nonnull + public static PipeRuntimeMetaV2 deserialize(InputStream inputStream) throws IOException { + final PipeRuntimeMetaV2 pipeRuntimeMeta = new PipeRuntimeMetaV2(); + + pipeRuntimeMeta.status.set(PipeStatus.getPipeStatus(ReadWriteIOUtils.readByte(inputStream))); + + int size = ReadWriteIOUtils.readInt(inputStream); + for (int i = 0; i < size; ++i) { + pipeRuntimeMeta.consensusGroupId2TaskMetaMap.put( + new TConsensusGroupId( + TConsensusGroupType.DataRegion, ReadWriteIOUtils.readInt(inputStream)), + PipeTaskMeta.deserialize(PipeRuntimeMetaVersion.VERSION_2, inputStream)); + } + + size = ReadWriteIOUtils.readInt(inputStream); + for (int i = 0; i < size; ++i) { + pipeRuntimeMeta.dataNodeId2PipeRuntimeExceptionMap.put( + ReadWriteIOUtils.readInt(inputStream), + PipeRuntimeExceptionType.deserializeFrom(PipeRuntimeMetaVersion.VERSION_2, inputStream)); + } + + pipeRuntimeMeta.exceptionsClearTime.set(ReadWriteIOUtils.readLong(inputStream)); + pipeRuntimeMeta.isStoppedByRuntimeException.set(ReadWriteIOUtils.readBool(inputStream)); + + return pipeRuntimeMeta; + } + + @Nonnull + public static PipeRuntimeMetaV2 deserialize(ByteBuffer byteBuffer) { + final PipeRuntimeMetaV2 pipeRuntimeMeta = new PipeRuntimeMetaV2(); + + pipeRuntimeMeta.status.set(PipeStatus.getPipeStatus(ReadWriteIOUtils.readByte(byteBuffer))); + + int size = ReadWriteIOUtils.readInt(byteBuffer); + for (int i = 0; i < size; ++i) { + pipeRuntimeMeta.consensusGroupId2TaskMetaMap.put( + new TConsensusGroupId( + TConsensusGroupType.DataRegion, ReadWriteIOUtils.readInt(byteBuffer)), + PipeTaskMeta.deserialize(PipeRuntimeMetaVersion.VERSION_2, byteBuffer)); + } + + size = ReadWriteIOUtils.readInt(byteBuffer); + for (int i = 0; i < size; ++i) { + pipeRuntimeMeta.dataNodeId2PipeRuntimeExceptionMap.put( + ReadWriteIOUtils.readInt(byteBuffer), + PipeRuntimeExceptionType.deserializeFrom(PipeRuntimeMetaVersion.VERSION_2, byteBuffer)); + } + + pipeRuntimeMeta.exceptionsClearTime.set(ReadWriteIOUtils.readLong(byteBuffer)); + pipeRuntimeMeta.isStoppedByRuntimeException.set(ReadWriteIOUtils.readBool(byteBuffer)); + + return pipeRuntimeMeta; + } + + /////////////////////////////// Compatibility /////////////////////////////// + + @Override + public PipeRuntimeMeta toCurrentPipeRuntimeMetaVersion() { + PipeRuntimeMeta pipeRuntimeMeta = new PipeRuntimeMeta(); + pipeRuntimeMeta.getStatus().set(status.get()); + pipeRuntimeMeta.setDataNodeId2PipeRuntimeExceptionMap(dataNodeId2PipeRuntimeExceptionMap); + pipeRuntimeMeta.setDataRegionId2TaskMetaMap(consensusGroupId2TaskMetaMap); + pipeRuntimeMeta.setIsStoppedByRuntimeException(isStoppedByRuntimeException.get()); + pipeRuntimeMeta.setExceptionsClearTime(exceptionsClearTime.get()); + return pipeRuntimeMeta; + } + + /////////////////////////////// Object /////////////////////////////// + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PipeRuntimeMetaV2 that = (PipeRuntimeMetaV2) o; + return Objects.equals(status.get().getType(), that.status.get().getType()) + && consensusGroupId2TaskMetaMap.equals(that.consensusGroupId2TaskMetaMap) + && dataNodeId2PipeRuntimeExceptionMap.equals(that.dataNodeId2PipeRuntimeExceptionMap) + && exceptionsClearTime.get() == that.exceptionsClearTime.get() + && isStoppedByRuntimeException.get() == that.isStoppedByRuntimeException.get(); + } + + @Override + public int hashCode() { + return Objects.hash( + status, + consensusGroupId2TaskMetaMap, + dataNodeId2PipeRuntimeExceptionMap, + exceptionsClearTime.get(), + isStoppedByRuntimeException.get()); + } + + @Override + public String toString() { + return "PipeRuntimeMeta{" + + "status=" + + status + + ", consensusGroupId2TaskMetaMap=" + + consensusGroupId2TaskMetaMap + + ", dataNodeId2PipeMetaExceptionMap=" + + dataNodeId2PipeRuntimeExceptionMap + + ", exceptionsClearTime=" + + exceptionsClearTime.get() + + ", isStoppedByRuntimeException=" + + isStoppedByRuntimeException.get() + + "}"; + } +} diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeExceptionTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeExceptionTest.java index c40903cd1cce8..2c8485852ff44 100644 --- a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeExceptionTest.java +++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeExceptionTest.java @@ -19,7 +19,7 @@ package org.apache.iotdb.commons.exception.pipe; -import org.apache.iotdb.commons.pipe.task.meta.PipeRuntimeMetaVersion; +import org.apache.iotdb.commons.pipe.task.meta.compatibility.PipeRuntimeMetaVersion; import org.junit.Assert; import org.junit.Test; diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/PipeMetaDeSerTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/PipeMetaDeSerTest.java index b1b9e4de4a5a9..5b5d5d4b45749 100644 --- a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/PipeMetaDeSerTest.java +++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/PipeMetaDeSerTest.java @@ -29,18 +29,26 @@ import org.apache.iotdb.commons.consensus.index.impl.SimpleProgressIndex; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeConnectorCriticalException; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeCriticalException; +import org.apache.iotdb.commons.exception.pipe.PipeRuntimeException; import org.apache.iotdb.commons.pipe.task.meta.PipeMeta; import org.apache.iotdb.commons.pipe.task.meta.PipeRuntimeMeta; import org.apache.iotdb.commons.pipe.task.meta.PipeStaticMeta; import org.apache.iotdb.commons.pipe.task.meta.PipeStatus; import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta; +import org.apache.iotdb.commons.pipe.task.meta.compatibility.PipeRuntimeMetaVersion; +import org.apache.iotdb.commons.pipe.task.meta.compatibility.runtimemeta.PipeRuntimeMetaV1; +import org.apache.iotdb.commons.pipe.task.meta.compatibility.runtimemeta.PipeRuntimeMetaV2; +import org.apache.iotdb.tsfile.utils.PublicBAOS; import org.junit.Assert; import org.junit.Test; +import java.io.ByteArrayInputStream; +import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; import java.util.HashMap; +import java.util.concurrent.ConcurrentHashMap; public class PipeMetaDeSerTest { @@ -50,18 +58,18 @@ public void test() throws IOException { new PipeStaticMeta( "pipeName", 123L, - new HashMap() { + new HashMap() { { put("extractor-key", "extractor-value"); } }, - new HashMap() { + new HashMap() { { put("processor-key-1", "processor-value-1"); put("processor-key-2", "processor-value-2"); } }, - new HashMap() {}); + new HashMap() {}); ByteBuffer staticByteBuffer = pipeStaticMeta.serialize(); PipeStaticMeta pipeStaticMeta1 = PipeStaticMeta.deserialize(staticByteBuffer); Assert.assertEquals(pipeStaticMeta, pipeStaticMeta1); @@ -73,7 +81,7 @@ public void test() throws IOException { PipeRuntimeMeta pipeRuntimeMeta = new PipeRuntimeMeta( - new HashMap() { + new HashMap() { { put( new TConsensusGroupId(TConsensusGroupType.DataRegion, 123), @@ -93,7 +101,7 @@ public void test() throws IOException { new RecoverProgressIndex(1, new SimpleProgressIndex(1, 9)), 123)); } }, - new HashMap() { + new HashMap() { { put( new TConsensusGroupId(TConsensusGroupType.SchemaRegion, 111), @@ -104,7 +112,8 @@ public void test() throws IOException { } }); ByteBuffer runtimeByteBuffer = pipeRuntimeMeta.serialize(); - PipeRuntimeMeta pipeRuntimeMeta1 = PipeRuntimeMeta.deserialize(runtimeByteBuffer); + PipeRuntimeMeta pipeRuntimeMeta1 = + PipeRuntimeMetaVersion.deserializeRuntimeMeta(runtimeByteBuffer); Assert.assertEquals(pipeRuntimeMeta, pipeRuntimeMeta1); pipeRuntimeMeta.getStatus().set(PipeStatus.RUNNING); @@ -115,7 +124,7 @@ public void test() throws IOException { .put(123, new PipeRuntimeCriticalException("test")); runtimeByteBuffer = pipeRuntimeMeta.serialize(); - pipeRuntimeMeta1 = PipeRuntimeMeta.deserialize(runtimeByteBuffer); + pipeRuntimeMeta1 = PipeRuntimeMetaVersion.deserializeRuntimeMeta(runtimeByteBuffer); Assert.assertEquals(pipeRuntimeMeta, pipeRuntimeMeta1); pipeRuntimeMeta.getStatus().set(PipeStatus.DROPPED); @@ -133,7 +142,7 @@ public void test() throws IOException { .trackExceptionMessage(new PipeRuntimeConnectorCriticalException("test456")); runtimeByteBuffer = pipeRuntimeMeta.serialize(); - pipeRuntimeMeta1 = PipeRuntimeMeta.deserialize(runtimeByteBuffer); + pipeRuntimeMeta1 = PipeRuntimeMetaVersion.deserializeRuntimeMeta(runtimeByteBuffer); Assert.assertEquals(pipeRuntimeMeta, pipeRuntimeMeta1); PipeMeta pipeMeta = new PipeMeta(pipeStaticMeta, pipeRuntimeMeta); @@ -141,4 +150,162 @@ public void test() throws IOException { PipeMeta pipeMeta1 = PipeMeta.deserialize(byteBuffer); Assert.assertEquals(pipeMeta, pipeMeta1); } + + @Test + public void testRuntimeMetaV1ToCurrent() throws IOException { + PipeRuntimeMetaV1 pipeRuntimeMetaV1 = + new PipeRuntimeMetaV1( + new HashMap() { + { + put( + new TConsensusGroupId(TConsensusGroupType.DataRegion, 123), + new PipeTaskMeta(MinimumProgressIndex.INSTANCE, 987)); + put( + new TConsensusGroupId(TConsensusGroupType.DataRegion, 234), + new PipeTaskMeta(new IoTProgressIndex(1, 2L), 789)); + put( + new TConsensusGroupId(TConsensusGroupType.DataRegion, 345), + new PipeTaskMeta(new SimpleProgressIndex(3, 4), 789)); + put( + new TConsensusGroupId(TConsensusGroupType.DataRegion, 456), + new PipeTaskMeta(new HybridProgressIndex(new SimpleProgressIndex(1, 2)), 789)); + put( + new TConsensusGroupId(TConsensusGroupType.DataRegion, 567), + new PipeTaskMeta( + new RecoverProgressIndex(1, new SimpleProgressIndex(1, 9)), 123)); + } + }); + + // Do not set the exceptions here since we do not reserve the old exceptions + pipeRuntimeMetaV1.getStatus().set(PipeStatus.RUNNING); + + // Test byteBuffer + PipeRuntimeMeta pipeRuntimeMeta = + PipeRuntimeMetaVersion.deserializeRuntimeMeta(pipeRuntimeMetaV1.serialize()); + + Assert.assertEquals(pipeRuntimeMetaV1.getStatus().get(), pipeRuntimeMeta.getStatus().get()); + Assert.assertEquals( + pipeRuntimeMetaV1.getConsensusGroupIdToTaskMetaMap(), + pipeRuntimeMeta.getDataRegionId2TaskMetaMap()); + + Assert.assertEquals(MinimumProgressIndex.INSTANCE, pipeRuntimeMeta.getConfigProgressIndex()); + Assert.assertEquals( + new ConcurrentHashMap<>(), pipeRuntimeMeta.getDataNodeId2PipeRuntimeExceptionMap()); + Assert.assertEquals(new ConcurrentHashMap<>(), pipeRuntimeMeta.getSchemaRegionId2TaskMetaMap()); + Assert.assertFalse(pipeRuntimeMeta.getIsStoppedByRuntimeException()); + Assert.assertEquals(Long.MIN_VALUE, pipeRuntimeMeta.getExceptionsClearTime()); + + // Test inputStream + try (PublicBAOS byteArrayOutputStream = new PublicBAOS(); + DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + pipeRuntimeMetaV1.serialize(outputStream); + pipeRuntimeMeta = + PipeRuntimeMetaVersion.deserializeRuntimeMeta( + new ByteArrayInputStream(byteArrayOutputStream.toByteArray())); + + Assert.assertEquals(pipeRuntimeMetaV1.getStatus().get(), pipeRuntimeMeta.getStatus().get()); + Assert.assertEquals( + pipeRuntimeMetaV1.getConsensusGroupIdToTaskMetaMap(), + pipeRuntimeMeta.getDataRegionId2TaskMetaMap()); + + Assert.assertEquals(MinimumProgressIndex.INSTANCE, pipeRuntimeMeta.getConfigProgressIndex()); + Assert.assertEquals( + new ConcurrentHashMap<>(), pipeRuntimeMeta.getDataNodeId2PipeRuntimeExceptionMap()); + Assert.assertEquals( + new ConcurrentHashMap<>(), pipeRuntimeMeta.getSchemaRegionId2TaskMetaMap()); + Assert.assertFalse(pipeRuntimeMeta.getIsStoppedByRuntimeException()); + Assert.assertEquals(Long.MIN_VALUE, pipeRuntimeMeta.getExceptionsClearTime()); + } catch (IOException ignore) { + // Do not fail due to serialization failure + } + } + + @Test + public void testRuntimeMetaV2ToCurrent() throws IOException { + TConsensusGroupId exceptionGroupId = new TConsensusGroupId(TConsensusGroupType.DataRegion, 567); + + PipeRuntimeMetaV2 pipeRuntimeMetaV2 = + new PipeRuntimeMetaV2( + new HashMap() { + { + put( + new TConsensusGroupId(TConsensusGroupType.DataRegion, 123), + new PipeTaskMeta(MinimumProgressIndex.INSTANCE, 987)); + put( + new TConsensusGroupId(TConsensusGroupType.DataRegion, 234), + new PipeTaskMeta(new IoTProgressIndex(1, 2L), 789)); + put( + new TConsensusGroupId(TConsensusGroupType.DataRegion, 345), + new PipeTaskMeta(new SimpleProgressIndex(3, 4), 789)); + put( + new TConsensusGroupId(TConsensusGroupType.DataRegion, 456), + new PipeTaskMeta(new HybridProgressIndex(new SimpleProgressIndex(1, 2)), 789)); + put( + exceptionGroupId, + new PipeTaskMeta( + new RecoverProgressIndex(1, new SimpleProgressIndex(1, 9)), 123)); + } + }); + + PipeRuntimeException testException = new PipeRuntimeCriticalException("No available receivers"); + + pipeRuntimeMetaV2.getStatus().set(PipeStatus.DROPPED); + pipeRuntimeMetaV2.setIsStoppedByRuntimeException(true); + pipeRuntimeMetaV2.setExceptionsClearTime(System.currentTimeMillis()); + pipeRuntimeMetaV2.getDataNodeId2PipeRuntimeExceptionMap().put(1, testException); + pipeRuntimeMetaV2 + .getConsensusGroupId2TaskMetaMap() + .get(exceptionGroupId) + .trackExceptionMessage(testException); + + // Test byteBuffer + PipeRuntimeMeta pipeRuntimeMeta = + PipeRuntimeMetaVersion.deserializeRuntimeMeta(pipeRuntimeMetaV2.serialize()); + + Assert.assertEquals(pipeRuntimeMetaV2.getStatus().get(), pipeRuntimeMeta.getStatus().get()); + Assert.assertEquals( + pipeRuntimeMetaV2.getConsensusGroupId2TaskMetaMap(), + pipeRuntimeMeta.getDataRegionId2TaskMetaMap()); + + Assert.assertEquals( + pipeRuntimeMetaV2.getExceptionsClearTime(), pipeRuntimeMeta.getExceptionsClearTime()); + Assert.assertEquals( + pipeRuntimeMetaV2.getDataNodeId2PipeRuntimeExceptionMap(), + pipeRuntimeMeta.getDataNodeId2PipeRuntimeExceptionMap()); + Assert.assertEquals( + pipeRuntimeMetaV2.getIsStoppedByRuntimeException(), + pipeRuntimeMeta.getIsStoppedByRuntimeException()); + + Assert.assertEquals(MinimumProgressIndex.INSTANCE, pipeRuntimeMeta.getConfigProgressIndex()); + Assert.assertEquals(new ConcurrentHashMap<>(), pipeRuntimeMeta.getSchemaRegionId2TaskMetaMap()); + + // Test inputStream + try (PublicBAOS byteArrayOutputStream = new PublicBAOS(); + DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + pipeRuntimeMetaV2.serialize(outputStream); + pipeRuntimeMeta = + PipeRuntimeMetaVersion.deserializeRuntimeMeta( + new ByteArrayInputStream(byteArrayOutputStream.toByteArray())); + + Assert.assertEquals(pipeRuntimeMetaV2.getStatus().get(), pipeRuntimeMeta.getStatus().get()); + Assert.assertEquals( + pipeRuntimeMetaV2.getConsensusGroupId2TaskMetaMap(), + pipeRuntimeMeta.getDataRegionId2TaskMetaMap()); + + Assert.assertEquals( + pipeRuntimeMetaV2.getExceptionsClearTime(), pipeRuntimeMeta.getExceptionsClearTime()); + Assert.assertEquals( + pipeRuntimeMetaV2.getDataNodeId2PipeRuntimeExceptionMap(), + pipeRuntimeMeta.getDataNodeId2PipeRuntimeExceptionMap()); + Assert.assertEquals( + pipeRuntimeMetaV2.getIsStoppedByRuntimeException(), + pipeRuntimeMeta.getIsStoppedByRuntimeException()); + + Assert.assertEquals(MinimumProgressIndex.INSTANCE, pipeRuntimeMeta.getConfigProgressIndex()); + Assert.assertEquals( + new ConcurrentHashMap<>(), pipeRuntimeMeta.getSchemaRegionId2TaskMetaMap()); + } catch (IOException ignore) { + // Do not fail due to serialization failure + } + } } From 2457143246b6e54ff92b82c932e00f3d491c14d9 Mon Sep 17 00:00:00 2001 From: Caideyipi <1336190078@qq.com> Date: Mon, 27 Nov 2023 20:54:09 +0800 Subject: [PATCH 06/11] Deleted nonnull --- .../apache/iotdb/commons/pipe/task/meta/PipeRuntimeMeta.java | 4 +--- .../meta/compatibility/runtimemeta/PipeRuntimeMetaV1.java | 2 -- .../meta/compatibility/runtimemeta/PipeRuntimeMetaV2.java | 2 -- 3 files changed, 1 insertion(+), 7 deletions(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMeta.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMeta.java index 843146272dae3..60e1cdb9a5497 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMeta.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMeta.java @@ -200,7 +200,6 @@ public void serialize(OutputStream outputStream) throws IOException { ReadWriteIOUtils.write(isStoppedByRuntimeException.get(), outputStream); } - @Nonnull public static PipeRuntimeMeta deserialize(InputStream inputStream) throws IOException { final PipeRuntimeMeta pipeRuntimeMeta = new PipeRuntimeMeta(); @@ -237,7 +236,6 @@ public static PipeRuntimeMeta deserialize(InputStream inputStream) throws IOExce return pipeRuntimeMeta; } - @Nonnull public static PipeRuntimeMeta deserialize(ByteBuffer byteBuffer) { final PipeRuntimeMeta pipeRuntimeMeta = new PipeRuntimeMeta(); @@ -276,7 +274,7 @@ public static PipeRuntimeMeta deserialize(ByteBuffer byteBuffer) { /////////////////////////////// Compatibility /////////////////////////////// - // DO NOT CALL IT, unless from the former versions + // DO NOT CALL it directly, unless from the previous versions public void setDataNodeId2PipeRuntimeExceptionMap( Map dataNodeId2PipeRuntimeExceptionMap) { this.dataNodeId2PipeRuntimeExceptionMap = dataNodeId2PipeRuntimeExceptionMap; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV1.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV1.java index 1acf8f1578324..adda2be19fede 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV1.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV1.java @@ -94,7 +94,6 @@ public void serialize(OutputStream outputStream) throws IOException { } } - @Nonnull public static PipeRuntimeMetaV1 deserialize(InputStream inputStream, PipeStatus status) throws IOException { final PipeRuntimeMetaV1 pipeRuntimeMeta = new PipeRuntimeMetaV1(); @@ -112,7 +111,6 @@ public static PipeRuntimeMetaV1 deserialize(InputStream inputStream, PipeStatus return pipeRuntimeMeta; } - @Nonnull public static PipeRuntimeMetaV1 deserialize(ByteBuffer byteBuffer, PipeStatus status) { final PipeRuntimeMetaV1 pipeRuntimeMeta = new PipeRuntimeMetaV1(); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV2.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV2.java index b6a7d01de3830..1edbe2535e9d8 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV2.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV2.java @@ -163,7 +163,6 @@ public void serialize(OutputStream outputStream) throws IOException { ReadWriteIOUtils.write(isStoppedByRuntimeException.get(), outputStream); } - @Nonnull public static PipeRuntimeMetaV2 deserialize(InputStream inputStream) throws IOException { final PipeRuntimeMetaV2 pipeRuntimeMeta = new PipeRuntimeMetaV2(); @@ -190,7 +189,6 @@ public static PipeRuntimeMetaV2 deserialize(InputStream inputStream) throws IOEx return pipeRuntimeMeta; } - @Nonnull public static PipeRuntimeMetaV2 deserialize(ByteBuffer byteBuffer) { final PipeRuntimeMetaV2 pipeRuntimeMeta = new PipeRuntimeMetaV2(); From 44d8d60a1022578f1b3ea8669968e8f98da1b382 Mon Sep 17 00:00:00 2001 From: Caideyipi <1336190078@qq.com> Date: Tue, 28 Nov 2023 09:46:16 +0800 Subject: [PATCH 07/11] refactor --- .../consensus/request/write/pipe/task/CreatePipePlanV2.java | 2 +- .../exception/pipe/PipeRuntimeConnectorCriticalException.java | 2 +- .../commons/exception/pipe/PipeRuntimeCriticalException.java | 2 +- .../commons/exception/pipe/PipeRuntimeExceptionType.java | 2 +- .../exception/pipe/PipeRuntimeNonCriticalException.java | 2 +- .../pipe/PipeRuntimeOutOfMemoryCriticalException.java | 2 +- .../org/apache/iotdb/commons/pipe/task/meta/PipeMeta.java | 2 +- .../apache/iotdb/commons/pipe/task/meta/PipeRuntimeMeta.java | 4 +--- .../org/apache/iotdb/commons/pipe/task/meta/PipeTaskMeta.java | 2 +- .../meta/compatibility/runtimemeta/FormerPipeRuntimeMeta.java | 4 ++-- .../meta/compatibility/runtimemeta/PipeRuntimeMetaV1.java | 3 --- .../meta/compatibility/runtimemeta/PipeRuntimeMetaV2.java | 3 --- .../{ => runtimemeta}/PipeRuntimeMetaVersion.java | 4 +--- .../commons/exception/pipe/PipeRuntimeExceptionTest.java | 2 +- .../java/org/apache/iotdb/commons/pipe/PipeMetaDeSerTest.java | 2 +- 15 files changed, 14 insertions(+), 24 deletions(-) rename iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/{ => runtimemeta}/PipeRuntimeMetaVersion.java (94%) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/task/CreatePipePlanV2.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/task/CreatePipePlanV2.java index 72eb3c2f9df59..74f749c3db493 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/task/CreatePipePlanV2.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/pipe/task/CreatePipePlanV2.java @@ -21,7 +21,7 @@ import org.apache.iotdb.commons.pipe.task.meta.PipeRuntimeMeta; import org.apache.iotdb.commons.pipe.task.meta.PipeStaticMeta; -import org.apache.iotdb.commons.pipe.task.meta.compatibility.PipeRuntimeMetaVersion; +import org.apache.iotdb.commons.pipe.task.meta.compatibility.runtimemeta.PipeRuntimeMetaVersion; import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan; import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeConnectorCriticalException.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeConnectorCriticalException.java index 19644f81be3bd..dbdc287702992 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeConnectorCriticalException.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeConnectorCriticalException.java @@ -19,7 +19,7 @@ package org.apache.iotdb.commons.exception.pipe; -import org.apache.iotdb.commons.pipe.task.meta.compatibility.PipeRuntimeMetaVersion; +import org.apache.iotdb.commons.pipe.task.meta.compatibility.runtimemeta.PipeRuntimeMetaVersion; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; import java.io.IOException; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeCriticalException.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeCriticalException.java index baba2a52e1c8b..0939e9bee62c6 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeCriticalException.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeCriticalException.java @@ -19,7 +19,7 @@ package org.apache.iotdb.commons.exception.pipe; -import org.apache.iotdb.commons.pipe.task.meta.compatibility.PipeRuntimeMetaVersion; +import org.apache.iotdb.commons.pipe.task.meta.compatibility.runtimemeta.PipeRuntimeMetaVersion; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; import java.io.IOException; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeExceptionType.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeExceptionType.java index c82ded4c35fe1..35ab856abc1ee 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeExceptionType.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeExceptionType.java @@ -19,7 +19,7 @@ package org.apache.iotdb.commons.exception.pipe; -import org.apache.iotdb.commons.pipe.task.meta.compatibility.PipeRuntimeMetaVersion; +import org.apache.iotdb.commons.pipe.task.meta.compatibility.runtimemeta.PipeRuntimeMetaVersion; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; import java.io.IOException; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeNonCriticalException.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeNonCriticalException.java index 4c6b2de079d01..b6d6920aa9491 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeNonCriticalException.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeNonCriticalException.java @@ -19,7 +19,7 @@ package org.apache.iotdb.commons.exception.pipe; -import org.apache.iotdb.commons.pipe.task.meta.compatibility.PipeRuntimeMetaVersion; +import org.apache.iotdb.commons.pipe.task.meta.compatibility.runtimemeta.PipeRuntimeMetaVersion; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; import java.io.IOException; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeOutOfMemoryCriticalException.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeOutOfMemoryCriticalException.java index c9ba275d163e9..d1d5c17437e8d 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeOutOfMemoryCriticalException.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeOutOfMemoryCriticalException.java @@ -19,7 +19,7 @@ package org.apache.iotdb.commons.exception.pipe; -import org.apache.iotdb.commons.pipe.task.meta.compatibility.PipeRuntimeMetaVersion; +import org.apache.iotdb.commons.pipe.task.meta.compatibility.runtimemeta.PipeRuntimeMetaVersion; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; import java.io.IOException; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeMeta.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeMeta.java index ebda7ef9f35e2..d39c06141c506 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeMeta.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeMeta.java @@ -19,7 +19,7 @@ package org.apache.iotdb.commons.pipe.task.meta; -import org.apache.iotdb.commons.pipe.task.meta.compatibility.PipeRuntimeMetaVersion; +import org.apache.iotdb.commons.pipe.task.meta.compatibility.runtimemeta.PipeRuntimeMetaVersion; import org.apache.iotdb.tsfile.utils.PublicBAOS; import javax.annotation.Nonnull; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMeta.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMeta.java index 60e1cdb9a5497..2c41c221a60fc 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMeta.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMeta.java @@ -29,12 +29,10 @@ import org.apache.iotdb.commons.exception.pipe.PipeRuntimeCriticalException; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeException; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeExceptionType; -import org.apache.iotdb.commons.pipe.task.meta.compatibility.PipeRuntimeMetaVersion; +import org.apache.iotdb.commons.pipe.task.meta.compatibility.runtimemeta.PipeRuntimeMetaVersion; import org.apache.iotdb.tsfile.utils.PublicBAOS; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; -import javax.annotation.Nonnull; - import java.io.DataOutputStream; import java.io.IOException; import java.io.InputStream; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeTaskMeta.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeTaskMeta.java index 0a0529eaeb2b7..2b8ed06e0de2f 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeTaskMeta.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeTaskMeta.java @@ -26,7 +26,7 @@ import org.apache.iotdb.commons.exception.pipe.PipeRuntimeException; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeExceptionType; import org.apache.iotdb.commons.exception.pipe.PipeRuntimeNonCriticalException; -import org.apache.iotdb.commons.pipe.task.meta.compatibility.PipeRuntimeMetaVersion; +import org.apache.iotdb.commons.pipe.task.meta.compatibility.runtimemeta.PipeRuntimeMetaVersion; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; import java.io.IOException; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/FormerPipeRuntimeMeta.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/FormerPipeRuntimeMeta.java index c623e7a1c3eef..de68d50656959 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/FormerPipeRuntimeMeta.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/FormerPipeRuntimeMeta.java @@ -35,9 +35,9 @@ public interface FormerPipeRuntimeMeta { void serialize(OutputStream outputStream) throws IOException; /** - * This method always return the incumbent {@link PipeRuntimeMeta}. When changing to a new + * This method always return the incumbent {@link PipeRuntimeMeta}. When switching to a new * version, be sure to move the previous one to the packet, grant it a version, make it implement - * this interface, and then update this method in all {@link FormerPipeRuntimeMeta}s. + * this interface, and then update this very method in all {@link FormerPipeRuntimeMeta}s. */ PipeRuntimeMeta toCurrentPipeRuntimeMetaVersion(); } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV1.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV1.java index adda2be19fede..76ae5ccf197d6 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV1.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV1.java @@ -24,13 +24,10 @@ import org.apache.iotdb.commons.pipe.task.meta.PipeRuntimeMeta; import org.apache.iotdb.commons.pipe.task.meta.PipeStatus; import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta; -import org.apache.iotdb.commons.pipe.task.meta.compatibility.PipeRuntimeMetaVersion; import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.tsfile.utils.PublicBAOS; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; -import javax.annotation.Nonnull; - import java.io.DataOutputStream; import java.io.IOException; import java.io.InputStream; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV2.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV2.java index 1edbe2535e9d8..a354d528a3d64 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV2.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV2.java @@ -28,13 +28,10 @@ import org.apache.iotdb.commons.pipe.task.meta.PipeRuntimeMeta; import org.apache.iotdb.commons.pipe.task.meta.PipeStatus; import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta; -import org.apache.iotdb.commons.pipe.task.meta.compatibility.PipeRuntimeMetaVersion; import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.tsfile.utils.PublicBAOS; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; -import javax.annotation.Nonnull; - import java.io.DataOutputStream; import java.io.IOException; import java.io.InputStream; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/PipeRuntimeMetaVersion.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaVersion.java similarity index 94% rename from iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/PipeRuntimeMetaVersion.java rename to iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaVersion.java index 98d6109d5faf9..85d1f5c5f4af8 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/PipeRuntimeMetaVersion.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaVersion.java @@ -17,12 +17,10 @@ * under the License. */ -package org.apache.iotdb.commons.pipe.task.meta.compatibility; +package org.apache.iotdb.commons.pipe.task.meta.compatibility.runtimemeta; import org.apache.iotdb.commons.pipe.task.meta.PipeRuntimeMeta; import org.apache.iotdb.commons.pipe.task.meta.PipeStatus; -import org.apache.iotdb.commons.pipe.task.meta.compatibility.runtimemeta.PipeRuntimeMetaV1; -import org.apache.iotdb.commons.pipe.task.meta.compatibility.runtimemeta.PipeRuntimeMetaV2; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; import java.io.IOException; diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeExceptionTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeExceptionTest.java index 2c8485852ff44..4fa0ae0cf900b 100644 --- a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeExceptionTest.java +++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/exception/pipe/PipeRuntimeExceptionTest.java @@ -19,7 +19,7 @@ package org.apache.iotdb.commons.exception.pipe; -import org.apache.iotdb.commons.pipe.task.meta.compatibility.PipeRuntimeMetaVersion; +import org.apache.iotdb.commons.pipe.task.meta.compatibility.runtimemeta.PipeRuntimeMetaVersion; import org.junit.Assert; import org.junit.Test; diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/PipeMetaDeSerTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/PipeMetaDeSerTest.java index 5b5d5d4b45749..24b8f00b43d70 100644 --- a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/PipeMetaDeSerTest.java +++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/pipe/PipeMetaDeSerTest.java @@ -35,9 +35,9 @@ import org.apache.iotdb.commons.pipe.task.meta.PipeStaticMeta; import org.apache.iotdb.commons.pipe.task.meta.PipeStatus; import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta; -import org.apache.iotdb.commons.pipe.task.meta.compatibility.PipeRuntimeMetaVersion; import org.apache.iotdb.commons.pipe.task.meta.compatibility.runtimemeta.PipeRuntimeMetaV1; import org.apache.iotdb.commons.pipe.task.meta.compatibility.runtimemeta.PipeRuntimeMetaV2; +import org.apache.iotdb.commons.pipe.task.meta.compatibility.runtimemeta.PipeRuntimeMetaVersion; import org.apache.iotdb.tsfile.utils.PublicBAOS; import org.junit.Assert; From f5714ea8662fc53a43ec9ed3acd6eeb69a1b4cf8 Mon Sep 17 00:00:00 2001 From: Caideyipi <1336190078@qq.com> Date: Tue, 28 Nov 2023 11:22:07 +0800 Subject: [PATCH 08/11] Update FormerPipeRuntimeMeta.java --- .../runtimemeta/FormerPipeRuntimeMeta.java | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/FormerPipeRuntimeMeta.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/FormerPipeRuntimeMeta.java index de68d50656959..0e278501b2820 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/FormerPipeRuntimeMeta.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/FormerPipeRuntimeMeta.java @@ -26,7 +26,15 @@ import java.io.OutputStream; import java.nio.ByteBuffer; -public interface FormerPipeRuntimeMeta { +/** + * The interface for all outdated PipeRuntimeMetas. With this, we are able to intuitively test the + * compatibility between the previous ones and the current one, ease the transformation of old + * snapShot to incumbent class instance, and avoid the dilation of the deserialization method + * numbers emerging in the working process. + * + *

The current version of {@link PipeRuntimeMeta} is PipeRuntimeMetaV2_2. + */ +interface FormerPipeRuntimeMeta { @TestOnly ByteBuffer serialize() throws IOException; @@ -37,7 +45,7 @@ public interface FormerPipeRuntimeMeta { /** * This method always return the incumbent {@link PipeRuntimeMeta}. When switching to a new * version, be sure to move the previous one to the packet, grant it a version, make it implement - * this interface, and then update this very method in all {@link FormerPipeRuntimeMeta}s. + * this interface, and then update this method in all {@link FormerPipeRuntimeMeta}s. */ PipeRuntimeMeta toCurrentPipeRuntimeMetaVersion(); } From cf8c9da345de435d007a107dfbc53c553efeb2e8 Mon Sep 17 00:00:00 2001 From: Caideyipi <1336190078@qq.com> Date: Tue, 28 Nov 2023 11:29:06 +0800 Subject: [PATCH 09/11] Display fix --- .../iotdb/commons/pipe/task/meta/PipeRuntimeMeta.java | 5 +++-- .../meta/compatibility/runtimemeta/PipeRuntimeMetaV1.java | 4 ++-- .../meta/compatibility/runtimemeta/PipeRuntimeMetaV2.java | 8 ++++---- 3 files changed, 9 insertions(+), 8 deletions(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMeta.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMeta.java index 2c41c221a60fc..45e19b86bad5e 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMeta.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/PipeRuntimeMeta.java @@ -305,7 +305,7 @@ public boolean equals(Object o) { @Override public int hashCode() { return Objects.hash( - status, + status.get(), dataRegionId2TaskMetaMap, schemaRegionId2TaskMetaMap, dataNodeId2PipeRuntimeExceptionMap, @@ -317,11 +317,12 @@ public int hashCode() { public String toString() { return "PipeRuntimeMeta{" + "status=" - + status + + status.get() + ", dataRegionId2TaskMetaMap=" + dataRegionId2TaskMetaMap + ", schemaRegionId2TaskMetaMap=" + schemaRegionId2TaskMetaMap + + ", dataNodeId2PipeRuntimeExceptionMap=" + dataNodeId2PipeRuntimeExceptionMap + ", exceptionsClearTime=" + exceptionsClearTime.get() diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV1.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV1.java index 76ae5ccf197d6..0e7e7e4f3f441 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV1.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV1.java @@ -151,14 +151,14 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hash(status, consensusGroupIdToTaskMetaMap); + return Objects.hash(status.get(), consensusGroupIdToTaskMetaMap); } @Override public String toString() { return "PipeRuntimeMetaV1{" + "status=" - + status + + status.get() + ", consensusGroupIdToTaskMetaMap=" + consensusGroupIdToTaskMetaMap + '}'; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV2.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV2.java index a354d528a3d64..7a717bccac761 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV2.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/PipeRuntimeMetaV2.java @@ -246,7 +246,7 @@ public boolean equals(Object o) { @Override public int hashCode() { return Objects.hash( - status, + status.get(), consensusGroupId2TaskMetaMap, dataNodeId2PipeRuntimeExceptionMap, exceptionsClearTime.get(), @@ -255,12 +255,12 @@ public int hashCode() { @Override public String toString() { - return "PipeRuntimeMeta{" + return "PipeRuntimeMetaV2{" + "status=" - + status + + status.get() + ", consensusGroupId2TaskMetaMap=" + consensusGroupId2TaskMetaMap - + ", dataNodeId2PipeMetaExceptionMap=" + + ", dataNodeId2PipeRuntimeExceptionMap=" + dataNodeId2PipeRuntimeExceptionMap + ", exceptionsClearTime=" + exceptionsClearTime.get() From 5c6dc47d54d50af4539927a10dc47ac62fa507f0 Mon Sep 17 00:00:00 2001 From: Caideyipi <1336190078@qq.com> Date: Tue, 28 Nov 2023 11:39:03 +0800 Subject: [PATCH 10/11] Update FormerPipeRuntimeMeta.java --- .../meta/compatibility/runtimemeta/FormerPipeRuntimeMeta.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/FormerPipeRuntimeMeta.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/FormerPipeRuntimeMeta.java index 0e278501b2820..f7119d86dabda 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/FormerPipeRuntimeMeta.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/task/meta/compatibility/runtimemeta/FormerPipeRuntimeMeta.java @@ -32,7 +32,7 @@ * snapShot to incumbent class instance, and avoid the dilation of the deserialization method * numbers emerging in the working process. * - *

The current version of {@link PipeRuntimeMeta} is PipeRuntimeMetaV2_2. + *

The current version of {@link PipeRuntimeMeta} is PipeRuntimeMetaV3. */ interface FormerPipeRuntimeMeta { From 15cc1bbe48753a2c18b2ff01d1bcd47e1ce6fd92 Mon Sep 17 00:00:00 2001 From: Caideyipi <1336190078@qq.com> Date: Wed, 29 Nov 2023 09:48:44 +0800 Subject: [PATCH 11/11] Revert the change of progress index equals method --- .../iotdb/commons/consensus/index/impl/HybridProgressIndex.java | 2 +- .../iotdb/commons/consensus/index/impl/IoTProgressIndex.java | 2 +- .../commons/consensus/index/impl/RecoverProgressIndex.java | 2 +- .../iotdb/commons/consensus/index/impl/SchemaProgressIndex.java | 2 +- .../iotdb/commons/consensus/index/impl/SimpleProgressIndex.java | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/HybridProgressIndex.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/HybridProgressIndex.java index 620041a514d78..da558e52d6545 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/HybridProgressIndex.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/HybridProgressIndex.java @@ -156,7 +156,7 @@ public boolean equals(Object obj) { if (!(obj instanceof HybridProgressIndex)) { return false; } - return this.equals(obj); + return this.equals((HybridProgressIndex) obj); } @Override diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/IoTProgressIndex.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/IoTProgressIndex.java index 5a833dd41310e..849a52b6981bf 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/IoTProgressIndex.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/IoTProgressIndex.java @@ -145,7 +145,7 @@ public boolean equals(Object obj) { if (!(obj instanceof IoTProgressIndex)) { return false; } - return this.equals(obj); + return this.equals((IoTProgressIndex) obj); } @Override diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/RecoverProgressIndex.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/RecoverProgressIndex.java index e9fa0df1ae540..18e694bc62994 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/RecoverProgressIndex.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/RecoverProgressIndex.java @@ -149,7 +149,7 @@ public boolean equals(Object obj) { if (!(obj instanceof RecoverProgressIndex)) { return false; } - return this.equals(obj); + return this.equals((RecoverProgressIndex) obj); } @Override diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SchemaProgressIndex.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SchemaProgressIndex.java index 58c39da03995b..b2f19923358f6 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SchemaProgressIndex.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SchemaProgressIndex.java @@ -121,7 +121,7 @@ public boolean equals(Object obj) { if (!(obj instanceof SchemaProgressIndex)) { return false; } - return this.equals(obj); + return this.equals((SchemaProgressIndex) obj); } @Override diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SimpleProgressIndex.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SimpleProgressIndex.java index 3e7aeb6d56483..2984149db584e 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SimpleProgressIndex.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/consensus/index/impl/SimpleProgressIndex.java @@ -130,7 +130,7 @@ public boolean equals(Object obj) { if (!(obj instanceof SimpleProgressIndex)) { return false; } - return this.equals(obj); + return this.equals((SimpleProgressIndex) obj); } @Override