From 79596ec5c6d0179f92e3d1cf7e3cbb2acba95d08 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Wed, 24 Jul 2024 17:17:07 +0800 Subject: [PATCH 01/85] complete deletion support for pipeconsensus, todo: deletion GC and historical collection --- .../db/consensus/DataRegionConsensusImpl.java | 2 + .../dataregion/DataExecutionVisitor.java | 11 +- .../PipeConsensusAsyncConnector.java | 2 +- .../PipeConsensusSyncConnector.java | 2 +- .../PipeConsensusTabletBatchEventHandler.java | 2 +- ...ConsensusTabletInsertNodeEventHandler.java | 2 +- ...eConsensusTabletInsertionEventHandler.java | 2 +- ...eConsensusTsFileInsertionEventHandler.java | 2 +- .../consensus/deletion/DeletionResource.java | 125 +++++++++++ .../deletion/DeletionResourceManager.java | 184 +++++++++++++++++ .../PipeConsensusConnectorMetrics.java | 2 +- .../PipeConsensusReceiverMetrics.java | 2 +- .../PipeSchemaRegionWritePlanEvent.java | 25 +++ .../realtime/PipeRealtimeEventFactory.java | 12 +- .../pipeconsensus/PipeConsensusReceiver.java | 2 +- .../storageengine/dataregion/DataRegion.java | 19 +- .../dataregion/DataRegionTest.java | 195 ++++++++++++++---- 17 files changed, 525 insertions(+), 66 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java rename iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/{ => metric}/PipeConsensusConnectorMetrics.java (99%) rename iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/{ => metric}/PipeConsensusReceiverMetrics.java (99%) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/DataRegionConsensusImpl.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/DataRegionConsensusImpl.java index 515a9f54ddca..01702113b214 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/DataRegionConsensusImpl.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/DataRegionConsensusImpl.java @@ -40,6 +40,7 @@ import org.apache.iotdb.db.pipe.consensus.ConsensusPipeDataNodeDispatcher; import org.apache.iotdb.db.pipe.consensus.ConsensusPipeDataNodeRuntimeAgentGuardian; import org.apache.iotdb.db.pipe.consensus.ProgressIndexDataNodeManager; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; import org.apache.iotdb.db.storageengine.StorageEngine; import org.apache.iotdb.db.storageengine.dataregion.DataRegion; @@ -81,6 +82,7 @@ private static class DataRegionConsensusImplHolder { static { reinitializeStatics(); PipeDataNodeAgent.receiver().pipeConsensus().initConsensusInRuntime(); + DeletionResourceManager.build(); } private static void reinitializeStatics() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/statemachine/dataregion/DataExecutionVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/statemachine/dataregion/DataExecutionVisitor.java index efd9859588ce..6830c33a0ebd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/statemachine/dataregion/DataExecutionVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/consensus/statemachine/dataregion/DataExecutionVisitor.java @@ -27,7 +27,6 @@ import org.apache.iotdb.db.exception.WriteProcessException; import org.apache.iotdb.db.exception.WriteProcessRejectException; import org.apache.iotdb.db.exception.query.OutOfTTLException; -import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.listener.PipeInsertionDataNodeListener; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.pipe.PipeEnrichedDeleteDataNode; @@ -230,18 +229,12 @@ public TSStatus visitDeleteData(DeleteDataNode node, DataRegion dataRegion) { "now try to delete directly, databasePath: {}, deletePath:{}", databaseToDelete.getFullPath(), path.getFullPath()); - dataRegion.deleteDataDirectly( - databaseToDelete, - node.getDeleteStartTime(), - node.getDeleteEndTime(), - node.getSearchIndex()); + dataRegion.deleteDataDirectly(databaseToDelete, node); } else { - dataRegion.deleteByDevice( - path, node.getDeleteStartTime(), node.getDeleteEndTime(), node.getSearchIndex()); + dataRegion.deleteByDevice(path, node); } } dataRegion.insertSeparatorToWAL(); - PipeInsertionDataNodeListener.getInstance().listenToDeleteData(node); return StatusUtils.OK; } catch (IOException | IllegalPathException e) { LOGGER.error("Error in executing plan node: {}", node, e); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java index a61156a98ed0..8bccf12b5c1a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java @@ -45,7 +45,7 @@ import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.builder.PipeConsensusAsyncBatchReqBuilder; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletBinaryReq; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletInsertNodeReq; -import org.apache.iotdb.db.pipe.consensus.PipeConsensusConnectorMetrics; +import org.apache.iotdb.db.pipe.consensus.metric.PipeConsensusConnectorMetrics; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java index ad7ae47b302b..616d46f61ef7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java @@ -40,7 +40,7 @@ import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFilePieceWithModReq; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFileSealReq; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFileSealWithModReq; -import org.apache.iotdb.db.pipe.consensus.PipeConsensusConnectorMetrics; +import org.apache.iotdb.db.pipe.consensus.metric.PipeConsensusConnectorMetrics; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTabletBatchEventHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTabletBatchEventHandler.java index de9c7bec37d3..5c92e9286305 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTabletBatchEventHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTabletBatchEventHandler.java @@ -27,7 +27,7 @@ import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferResp; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.PipeConsensusAsyncConnector; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.builder.PipeConsensusAsyncBatchReqBuilder; -import org.apache.iotdb.db.pipe.consensus.PipeConsensusConnectorMetrics; +import org.apache.iotdb.db.pipe.consensus.metric.PipeConsensusConnectorMetrics; import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.pipe.api.exception.PipeException; import org.apache.iotdb.rpc.TSStatusCode; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTabletInsertNodeEventHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTabletInsertNodeEventHandler.java index cdd56d72cce5..731e0af06deb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTabletInsertNodeEventHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTabletInsertNodeEventHandler.java @@ -23,7 +23,7 @@ import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq; import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferResp; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.PipeConsensusAsyncConnector; -import org.apache.iotdb.db.pipe.consensus.PipeConsensusConnectorMetrics; +import org.apache.iotdb.db.pipe.consensus.metric.PipeConsensusConnectorMetrics; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; import org.apache.thrift.TException; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTabletInsertionEventHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTabletInsertionEventHandler.java index ae6a1e5334d2..347cd67d8c62 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTabletInsertionEventHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTabletInsertionEventHandler.java @@ -26,7 +26,7 @@ import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferResp; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.PipeConsensusAsyncConnector; import org.apache.iotdb.db.pipe.connector.protocol.thrift.async.handler.PipeTransferTabletInsertionEventHandler; -import org.apache.iotdb.db.pipe.consensus.PipeConsensusConnectorMetrics; +import org.apache.iotdb.db.pipe.consensus.metric.PipeConsensusConnectorMetrics; import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; import org.apache.iotdb.pipe.api.exception.PipeException; import org.apache.iotdb.rpc.TSStatusCode; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTsFileInsertionEventHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTsFileInsertionEventHandler.java index 2989323bb071..95e48c6b74a3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTsFileInsertionEventHandler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/handler/PipeConsensusTsFileInsertionEventHandler.java @@ -31,7 +31,7 @@ import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFilePieceWithModReq; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFileSealReq; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFileSealWithModReq; -import org.apache.iotdb.db.pipe.consensus.PipeConsensusConnectorMetrics; +import org.apache.iotdb.db.pipe.consensus.metric.PipeConsensusConnectorMetrics; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.rpc.TSStatusCode; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java new file mode 100644 index 000000000000..de9c9b4bd0bb --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.consensus.deletion; + +import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; + +import java.nio.ByteBuffer; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; + +/** + * DeletionResource is designed for PipeConsensus to manage the lifecycle of all deletion operations + * including realtime deletion and historical deletion. In order to be compatible with user pipe + * framework, PipeConsensus will use {@link PipeSchemaRegionWritePlanEvent} + */ +public class DeletionResource { + private final Consumer persistFunc; + private final Consumer removeFunc; + private final AtomicLong latestUpdateTime; + private PipeSchemaRegionWritePlanEvent deletionEvent; + + public DeletionResource( + PipeSchemaRegionWritePlanEvent deletionEvent, + Consumer persistFunc, + Consumer removeFunc) { + this.deletionEvent = deletionEvent; + this.persistFunc = persistFunc; + this.removeFunc = removeFunc; + latestUpdateTime = new AtomicLong(System.currentTimeMillis()); + } + + /** + * This method is invoked when DeletionResource is deleted by DeleteResourceManager. In this + * method, we release the reference of deletionEvent to resolve circular references between + * deletionResource and deletionEvent so that GC can reclaim them. + */ + public void releaseSelf() { + deletionEvent = null; + } + + public void persistSelf() { + persistFunc.accept(this); + } + + public void removeSelf() { + removeFunc.accept(this); + } + + public void increaseReferenceCount() { + deletionEvent.increaseReferenceCount(DeletionResource.class.getSimpleName()); + updateLatestUpdateTime(); + } + + public void decreaseReferenceCount() { + deletionEvent.decreaseReferenceCount(DeletionResource.class.getSimpleName(), false); + } + + public long getReferenceCount() { + return deletionEvent.getReferenceCount(); + } + + public long getLatestUpdateTime() { + return latestUpdateTime.get(); + } + + public ByteBuffer serialize() { + return deletionEvent.serializeToByteBuffer(); + } + + public static DeletionResource deserialize( + final ByteBuffer buffer, + final Consumer persistFunc, + Consumer removeFunc) { + PipeSchemaRegionWritePlanEvent event = new PipeSchemaRegionWritePlanEvent(); + event.deserializeFromByteBuffer(buffer); + return new DeletionResource(event, persistFunc, removeFunc); + } + + private void updateLatestUpdateTime() { + latestUpdateTime.set(System.currentTimeMillis()); + } + + @Override + public String toString() { + return String.format( + "DeletionResource[%s]{referenceCount=%s, latestUpdateTime=%s}", + deletionEvent, getReferenceCount(), getLatestUpdateTime()); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final DeletionResource otherEvent = (DeletionResource) o; + return Objects.equals(deletionEvent, otherEvent.deletionEvent) + && latestUpdateTime.get() == otherEvent.latestUpdateTime.get(); + } + + @Override + public int hashCode() { + return Objects.hash(deletionEvent, latestUpdateTime); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java new file mode 100644 index 000000000000..885360b3a9b6 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.consensus.deletion; + +import org.apache.iotdb.commons.utils.FileUtils; +import org.apache.iotdb.consensus.pipe.PipeConsensus; +import org.apache.iotdb.db.consensus.DataRegionConsensusImpl; +import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; +import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; + +import com.google.common.collect.ImmutableList; +import org.apache.tsfile.utils.ReadWriteIOUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.DataOutputStream; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.stream.Stream; + +public class DeletionResourceManager { + private static final Logger LOGGER = LoggerFactory.getLogger(DeletionResourceManager.class); + private static final long DELETION_TIME_TO_LIVE_IN_MS = 1000L; + private static final long CHECK_DELETION_DURATION_IN_MS = 1000L * 5; + private static final String DELETION_CHECKER_TASK_ID = "pipe_consensus_deletion_checker"; + private static final String DELETION_FILE_SUFFIX = ".deletion"; + // TODO: read it from conf + private final File storageDir = new File("tmp"); + private final List deletionResources = new CopyOnWriteArrayList<>(); + + public DeletionResourceManager() throws IOException { + initAndRecover(); + // Register scheduled deletion check task. + PipeDataNodeAgent.runtime() + .registerPeriodicalJob( + DELETION_CHECKER_TASK_ID, this::checkAndCleanDeletions, CHECK_DELETION_DURATION_IN_MS); + } + + private void initAndRecover() throws IOException { + if (!storageDir.exists()) { + // Init + if (!storageDir.mkdirs()) { + LOGGER.warn("Unable to create pipeConsensus deletion dir at {}", storageDir); + throw new IOException( + String.format("Unable to create pipeConsensus deletion dir at %s", storageDir)); + } + } + try (Stream pathStream = Files.walk(Paths.get(storageDir.getPath()), 1)) { + Path[] deletionPaths = + pathStream + .filter(Files::isRegularFile) + .filter(path -> path.endsWith(DELETION_FILE_SUFFIX)) + .toArray(Path[]::new); + ByteBuffer readBuffer; + for (Path path : deletionPaths) { + readBuffer = ByteBuffer.wrap(Files.readAllBytes(path)); + deletionResources.add( + DeletionResource.deserialize(readBuffer, this::persist, this::removeDeletionResource)); + } + } + } + + public void registerDeletionResource(PipeSchemaRegionWritePlanEvent event) { + DeletionResource deletionResource = + new DeletionResource(event, this::persist, this::removeDeletionResource); + event.setDeletionResource(deletionResource); + this.deletionResources.add(deletionResource); + } + + public List getAllDeletionResources() { + return deletionResources.stream().collect(ImmutableList.toImmutableList()); + } + + public void persist(final DeletionResource deletionResource) { + File deletionFile = + new File( + storageDir, String.format("%d%s", deletionResource.hashCode(), DELETION_FILE_SUFFIX)); + if (deletionFile.exists()) { + LOGGER.warn("Deletion file {} already exists, delete it.", deletionFile); + FileUtils.deleteFileOrDirectory(deletionFile); + } + + try (FileOutputStream fileOutputStream = new FileOutputStream(deletionFile)) { + try (DataOutputStream dataOutputStream = new DataOutputStream(fileOutputStream)) { + final ByteBuffer byteBuffer = deletionResource.serialize(); + ReadWriteIOUtils.write(byteBuffer, dataOutputStream); + } finally { + try { + fileOutputStream.flush(); + fileOutputStream.getFD().sync(); + } catch (IOException ignore) { + // ignore sync exception + } + } + } catch (IOException e) { + // log error and ignore exception + LOGGER.error( + "Failed to persist deletion resource {}, may cause inconsistency during replication", + deletionResource, + e); + } + } + + private void removeDeletionResource(DeletionResource deletionResource) { + // TODO: 参考 WAL 的水位线机制,攒批之类的,参数也可以参考 WAL + // TODO: 删除文件也可以通过考虑利用 progressIndex 来删 + // TODO: 需要考虑删除后的 index 维护,恢复之类的 + } + + private void checkAndCleanDeletions() { + final ImmutableList toBeCleaned = + deletionResources.stream() + .filter(deletionResource -> deletionResource.getReferenceCount() == 0) + .collect(ImmutableList.toImmutableList()); + + toBeCleaned.forEach( + deletionResource -> { + // Clean disk + File deletionFile = + new File( + storageDir, + String.format("%d%s", deletionResource.hashCode(), DELETION_FILE_SUFFIX)); + if (deletionFile.exists()) { + FileUtils.deleteFileOrDirectory(deletionFile); + } + // Clean memory + deletionResources.remove(deletionResource); + deletionResource.releaseSelf(); + }); + } + + //////////////////////////// singleton //////////////////////////// + private static class DeletionResourceManagerHolder { + private static DeletionResourceManager INSTANCE; + + private DeletionResourceManagerHolder() {} + + public static void build() throws IOException { + if (INSTANCE == null) { + INSTANCE = new DeletionResourceManager(); + } + } + } + + public static DeletionResourceManager getInstance() { + return DeletionResourceManager.DeletionResourceManagerHolder.INSTANCE; + } + + // Only when consensus protocol is PipeConsensus, will this class be initialized. + public static void build() { + if (DataRegionConsensusImpl.getInstance() instanceof PipeConsensus) { + try { + DeletionResourceManagerHolder.build(); + } catch (IOException e) { + LOGGER.error("Failed to initialize DeletionResourceManager", e); + throw new RuntimeException(e); + } + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/PipeConsensusConnectorMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/metric/PipeConsensusConnectorMetrics.java similarity index 99% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/PipeConsensusConnectorMetrics.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/metric/PipeConsensusConnectorMetrics.java index 23ab4d410c49..3faf7b987a43 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/PipeConsensusConnectorMetrics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/metric/PipeConsensusConnectorMetrics.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.db.pipe.consensus; +package org.apache.iotdb.db.pipe.consensus.metric; import org.apache.iotdb.commons.service.metric.enums.Metric; import org.apache.iotdb.commons.service.metric.enums.Tag; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/PipeConsensusReceiverMetrics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/metric/PipeConsensusReceiverMetrics.java similarity index 99% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/PipeConsensusReceiverMetrics.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/metric/PipeConsensusReceiverMetrics.java index 58d587d8b8d2..0c2d6ef55399 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/PipeConsensusReceiverMetrics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/metric/PipeConsensusReceiverMetrics.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iotdb.db.pipe.consensus; +package org.apache.iotdb.db.pipe.consensus.metric; import org.apache.iotdb.commons.service.metric.enums.Metric; import org.apache.iotdb.commons.service.metric.enums.Tag; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java index d6eefe086e55..459e113595d1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java @@ -23,6 +23,7 @@ import org.apache.iotdb.commons.pipe.event.PipeWritePlanEvent; import org.apache.iotdb.commons.pipe.pattern.PipePattern; import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeType; @@ -33,6 +34,7 @@ public class PipeSchemaRegionWritePlanEvent extends PipeWritePlanEvent { private PlanNode planNode; + private DeletionResource deletionResource; public PipeSchemaRegionWritePlanEvent() { // Used for deserialization @@ -58,6 +60,29 @@ public PlanNode getPlanNode() { return planNode; } + public void setDeletionResource(DeletionResource deletionResource) { + this.deletionResource = deletionResource; + } + + @Override + public boolean internallyIncreaseResourceReferenceCount(String holderMessage) { + if (deletionResource != null) { + deletionResource.persistSelf(); + } + return true; + } + + @Override + public boolean internallyDecreaseResourceReferenceCount(String holderMessage) { + if (deletionResource != null) { + deletionResource.removeSelf(); + // Resolve circular reference to let GC reclaim them all. + deletionResource.releaseSelf(); + deletionResource = null; + } + return true; + } + @Override public EnrichedEvent shallowCopySelfAndBindPipeTaskMetaForProgressReport( final String pipeName, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java index 94793b74bc15..b328b762d5c6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.pipe.event.realtime; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; @@ -29,9 +30,13 @@ import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALEntryHandler; +import java.util.Optional; + public class PipeRealtimeEventFactory { private static final TsFileEpochManager TS_FILE_EPOCH_MANAGER = new TsFileEpochManager(); + private static final DeletionResourceManager DELETION_RESOURCE_MANAGER = + DeletionResourceManager.getInstance(); public static PipeRealtimeEvent createRealtimeEvent( TsFileResource resource, boolean isLoaded, boolean isGeneratedByPipe) { @@ -59,8 +64,11 @@ public static PipeRealtimeEvent createRealtimeEvent( } public static PipeRealtimeEvent createRealtimeEvent(DeleteDataNode node) { - return new PipeRealtimeEvent( - new PipeSchemaRegionWritePlanEvent(node, node.isGeneratedByPipe()), null, null, null); + PipeSchemaRegionWritePlanEvent deletionEvent = + new PipeSchemaRegionWritePlanEvent(node, node.isGeneratedByPipe()); + Optional.ofNullable(DELETION_RESOURCE_MANAGER) + .ifPresent(mgr -> mgr.registerDeletionResource(deletionEvent)); + return new PipeRealtimeEvent(deletionEvent, null, null, null); } private PipeRealtimeEventFactory() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java index 2480ac702be2..b5f01e52b5ae 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java @@ -47,7 +47,7 @@ import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFilePieceReq; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFileSealReq; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFileSealWithModReq; -import org.apache.iotdb.db.pipe.consensus.PipeConsensusReceiverMetrics; +import org.apache.iotdb.db.pipe.consensus.metric.PipeConsensusReceiverMetrics; import org.apache.iotdb.db.pipe.event.common.tsfile.TsFileInsertionPointCounter; import org.apache.iotdb.db.queryengine.execution.load.LoadTsFileManager; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java index 1396ab501f44..fdcf6e5e2549 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java @@ -1472,6 +1472,7 @@ private String getNewTsFileName(long time, long version, int mergeCnt, int unseq * @param tsFileProcessor tsfile processor */ public void syncCloseOneTsFileProcessor(boolean sequence, TsFileProcessor tsFileProcessor) { + // TODO: writelock? synchronized (closeStorageGroupCondition) { try { asyncCloseOneTsFileProcessor(sequence, tsFileProcessor); @@ -2036,17 +2037,17 @@ private void getTwoKindsOfTsFiles( /** * @param pattern Must be a pattern start with a precise device path - * @param startTime - * @param endTime - * @param searchIndex + * @param node * @throws IOException */ - public void deleteByDevice(PartialPath pattern, long startTime, long endTime, long searchIndex) - throws IOException { + public void deleteByDevice(PartialPath pattern, DeleteDataNode node) throws IOException { if (SettleService.getINSTANCE().getFilesToBeSettledCount().get() != 0) { throw new IOException( "Delete failed. " + "Please do not delete until the old files settled."); } + final long startTime = node.getDeleteStartTime(); + final long endTime = node.getDeleteEndTime(); + final long searchIndex = node.getSearchIndex(); // TODO: how to avoid partial deletion? // FIXME: notice that if we may remove a SGProcessor out of memory, we need to close all opened // mod files in mergingModification, sequenceFileList, and unsequenceFileList @@ -2077,6 +2078,7 @@ public void deleteByDevice(PartialPath pattern, long startTime, long endTime, lo // deviceMatchInfo contains the DeviceId means this device matched the pattern Set deviceMatchInfo = new HashSet<>(); deleteDataInFiles(unsealedTsFileResource, deletion, devicePaths, deviceMatchInfo); + PipeInsertionDataNodeListener.getInstance().listenToDeleteData(node); writeUnlock(); hasReleasedLock = true; @@ -2090,8 +2092,10 @@ public void deleteByDevice(PartialPath pattern, long startTime, long endTime, lo } } - public void deleteDataDirectly( - PartialPath pathToDelete, long startTime, long endTime, long searchIndex) throws IOException { + public void deleteDataDirectly(PartialPath pathToDelete, DeleteDataNode node) throws IOException { + final long startTime = node.getDeleteStartTime(); + final long endTime = node.getDeleteEndTime(); + final long searchIndex = node.getSearchIndex(); logger.info( "{} will delete data files directly for deleting data between {} and {}", databaseName + "-" + dataRegionId, @@ -2117,6 +2121,7 @@ public void deleteDataDirectly( List unsealedTsFileResource = new ArrayList<>(); getTwoKindsOfTsFiles(sealedTsFileResource, unsealedTsFileResource, startTime, endTime); deleteDataDirectlyInFile(unsealedTsFileResource, pathToDelete, startTime, endTime); + PipeInsertionDataNodeListener.getInstance().listenToDeleteData(node); writeUnlock(); releasedLock = true; deleteDataDirectlyInFile(sealedTsFileResource, pathToDelete, startTime, endTime); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/DataRegionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/DataRegionTest.java index 492e66f6b923..0bd1d0adc50e 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/DataRegionTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/DataRegionTest.java @@ -37,6 +37,7 @@ import org.apache.iotdb.db.queryengine.common.QueryId; import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertRowsNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertTabletNode; @@ -186,7 +187,10 @@ record = new TSRecord(j, deviceId); CompressionType.UNCOMPRESSED, Collections.emptyMap())); - dataRegion.deleteByDevice(new PartialPath(deviceId, measurementId), 0, 15L, -1); + PartialPath path = new PartialPath(deviceId, measurementId); + DeleteDataNode deleteDataNode1 = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 0, 15L); + dataRegion.deleteByDevice(new PartialPath(deviceId, measurementId), deleteDataNode1); List tsfileResourcesForQuery = new ArrayList<>(); for (TsFileProcessor tsfileProcessor : dataRegion.getWorkUnsequenceTsFileProcessors()) { @@ -1307,11 +1311,18 @@ public void testDeleteDataNotInFile() dataRegion.syncCloseAllWorkingTsFileProcessors(); } + PartialPath path = new PartialPath("root.vehicle.d2.s0"); + DeleteDataNode deleteDataNode1 = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 150); + deleteDataNode1.setSearchIndex(0); // delete root.vehicle.d2.s0 data in the second file - dataRegion.deleteByDevice(new PartialPath("root.vehicle.d2.s0"), 50, 150, 0); + dataRegion.deleteByDevice(new PartialPath("root.vehicle.d2.s0"), deleteDataNode1); + DeleteDataNode deleteDataNode2 = + new DeleteDataNode(new PlanNodeId("2"), Collections.singletonList(path), 150, 450); + deleteDataNode2.setSearchIndex(0); // delete root.vehicle.d2.s0 data in the third file - dataRegion.deleteByDevice(new PartialPath("root.vehicle.d2.s0"), 150, 450, 0); + dataRegion.deleteByDevice(new PartialPath("root.vehicle.d2.s0"), deleteDataNode2); for (int i = 0; i < dataRegion.getSequenceFileList().size(); i++) { TsFileResource resource = dataRegion.getSequenceFileList().get(i); @@ -1356,11 +1367,19 @@ public void testDeleteDataNotInFlushingMemtable() TsFileProcessor tsFileProcessor = tsFileResource.getProcessor(); tsFileProcessor.getFlushingMemTable().addLast(tsFileProcessor.getWorkMemTable()); + PartialPath path = new PartialPath("root.vehicle.d2.s0"); + DeleteDataNode deleteDataNode1 = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 70); + deleteDataNode1.setSearchIndex(0); // delete data which is in memtable - dataRegion.deleteByDevice(new PartialPath("root.vehicle.d2.s0"), 50, 70, 0); + dataRegion.deleteByDevice(path, deleteDataNode1); + PartialPath path2 = new PartialPath("root.vehicle.d200.s0"); + DeleteDataNode deleteDataNode2 = + new DeleteDataNode(new PlanNodeId("2"), Collections.singletonList(path2), 50, 70); + deleteDataNode2.setSearchIndex(0); // delete data which is not in memtable - dataRegion.deleteByDevice(new PartialPath("root.vehicle.d200.s0"), 50, 70, 0); + dataRegion.deleteByDevice(path2, deleteDataNode2); dataRegion.syncCloseAllWorkingTsFileProcessors(); Assert.assertFalse(tsFileResource.getModFile().exists()); @@ -1378,14 +1397,31 @@ public void testDeleteDataInSeqFlushingMemtable() TsFileProcessor tsFileProcessor = tsFileResource.getProcessor(); tsFileProcessor.getFlushingMemTable().addLast(tsFileProcessor.getWorkMemTable()); + PartialPath path = new PartialPath("root.vehicle.d0.s0"); + DeleteDataNode deleteDataNode1 = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 99); + deleteDataNode1.setSearchIndex(0); + PartialPath path2 = new PartialPath("root.vehicle.d200.s0"); + DeleteDataNode deleteDataNode2 = + new DeleteDataNode(new PlanNodeId("2"), Collections.singletonList(path2), 50, 70); + deleteDataNode2.setSearchIndex(0); // delete data which is not in flushing memtable - dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 50, 99, 0); - dataRegion.deleteByDevice(new PartialPath("root.vehicle.d200.s0"), 50, 70, 0); - + dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), deleteDataNode1); + dataRegion.deleteByDevice(new PartialPath("root.vehicle.d200.s0"), deleteDataNode2); + + DeleteDataNode deleteDataNode3 = + new DeleteDataNode(new PlanNodeId("3"), Collections.singletonList(path), 50, 100); + deleteDataNode3.setSearchIndex(0); + DeleteDataNode deleteDataNode4 = + new DeleteDataNode(new PlanNodeId("4"), Collections.singletonList(path), 50, 150); + deleteDataNode4.setSearchIndex(0); + DeleteDataNode deleteDataNode5 = + new DeleteDataNode(new PlanNodeId("5"), Collections.singletonList(path), 100, 190); + deleteDataNode5.setSearchIndex(0); // delete data which is in flushing memtable - dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 50, 100, 0); - dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 50, 150, 0); - dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 100, 190, 0); + dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), deleteDataNode3); + dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), deleteDataNode4); + dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), deleteDataNode4); dataRegion.syncCloseAllWorkingTsFileProcessors(); Assert.assertTrue(tsFileResource.getModFile().exists()); @@ -1402,14 +1438,31 @@ public void testDeleteDataInUnSeqFlushingMemtable() } TsFileResource tsFileResource = dataRegion.getTsFileManager().getTsFileList(true).get(0); + PartialPath path = new PartialPath("root.vehicle.d0.s0"); + DeleteDataNode deleteDataNode1 = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 99); + deleteDataNode1.setSearchIndex(0); + PartialPath path2 = new PartialPath("root.vehicle.d200.s0"); + DeleteDataNode deleteDataNode2 = + new DeleteDataNode(new PlanNodeId("2"), Collections.singletonList(path2), 50, 70); + deleteDataNode2.setSearchIndex(0); // delete data which is not in work memtable - dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 50, 99, 0); - dataRegion.deleteByDevice(new PartialPath("root.vehicle.d200.s0"), 50, 70, 0); - + dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), deleteDataNode1); + dataRegion.deleteByDevice(new PartialPath("root.vehicle.d200.s0"), deleteDataNode2); + + DeleteDataNode deleteDataNode3 = + new DeleteDataNode(new PlanNodeId("3"), Collections.singletonList(path), 50, 100); + deleteDataNode3.setSearchIndex(0); + DeleteDataNode deleteDataNode4 = + new DeleteDataNode(new PlanNodeId("4"), Collections.singletonList(path), 50, 150); + deleteDataNode4.setSearchIndex(0); + DeleteDataNode deleteDataNode5 = + new DeleteDataNode(new PlanNodeId("5"), Collections.singletonList(path), 100, 190); + deleteDataNode5.setSearchIndex(0); // delete data which is in work memtable - dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 50, 100, 0); - dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 50, 150, 0); - dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 100, 190, 0); + dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), deleteDataNode3); + dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), deleteDataNode4); + dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), deleteDataNode5); dataRegion.syncCloseAllWorkingTsFileProcessors(); Assert.assertFalse(tsFileResource.getModFile().exists()); @@ -1420,12 +1473,19 @@ public void testDeleteDataInUnSeqFlushingMemtable() record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j))); dataRegion.insert(buildInsertRowNodeByTSRecord(record)); } + + DeleteDataNode deleteDataNode6 = + new DeleteDataNode(new PlanNodeId("6"), Collections.singletonList(path), 200, 299); + deleteDataNode6.setSearchIndex(0); // delete data which is not in work memtable - dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 200, 299, 0); - dataRegion.deleteByDevice(new PartialPath("root.vehicle.d200.s0"), 50, 70, 0); + dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), deleteDataNode6); + dataRegion.deleteByDevice(new PartialPath("root.vehicle.d200.s0"), deleteDataNode2); + DeleteDataNode deleteDataNode7 = + new DeleteDataNode(new PlanNodeId("7"), Collections.singletonList(path), 80, 85); + deleteDataNode7.setSearchIndex(0); // delete data which is in work memtable - dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 80, 85, 0); + dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), deleteDataNode7); Assert.assertFalse(tsFileResource.getModFile().exists()); @@ -1433,15 +1493,30 @@ public void testDeleteDataInUnSeqFlushingMemtable() TsFileProcessor tsFileProcessor = tsFileResource.getProcessor(); tsFileProcessor.getFlushingMemTable().addLast(tsFileProcessor.getWorkMemTable()); + DeleteDataNode deleteDataNode8 = + new DeleteDataNode(new PlanNodeId("8"), Collections.singletonList(path), 0, 49); + deleteDataNode8.setSearchIndex(0); + DeleteDataNode deleteDataNode9 = + new DeleteDataNode(new PlanNodeId("9"), Collections.singletonList(path), 100, 200); + deleteDataNode9.setSearchIndex(0); // delete data which is not in flushing memtable - dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 0, 49, 0); - dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 100, 200, 0); - dataRegion.deleteByDevice(new PartialPath("root.vehicle.d200.s0"), 50, 70, 0); - + dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), deleteDataNode8); + dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), deleteDataNode9); + dataRegion.deleteByDevice(new PartialPath("root.vehicle.d200.s0"), deleteDataNode2); + + DeleteDataNode deleteDataNode10 = + new DeleteDataNode(new PlanNodeId("10"), Collections.singletonList(path2), 25, 50); + deleteDataNode10.setSearchIndex(0); + DeleteDataNode deleteDataNode11 = + new DeleteDataNode(new PlanNodeId("11"), Collections.singletonList(path2), 50, 80); + deleteDataNode11.setSearchIndex(0); + DeleteDataNode deleteDataNode12 = + new DeleteDataNode(new PlanNodeId("12"), Collections.singletonList(path2), 99, 150); + deleteDataNode12.setSearchIndex(0); // delete data which is in flushing memtable - dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 25, 50, 0); - dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 50, 80, 0); - dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 99, 150, 0); + dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), deleteDataNode10); + dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), deleteDataNode11); + dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), deleteDataNode12); dataRegion.syncCloseAllWorkingTsFileProcessors(); Assert.assertTrue(tsFileResource.getModFile().exists()); @@ -1463,12 +1538,24 @@ public void testDeleteDataInSeqWorkingMemtable() } TsFileResource tsFileResource = dataRegion.getTsFileManager().getTsFileList(true).get(0); + PartialPath path = new PartialPath("root.vehicle.d0.s0"); + DeleteDataNode deleteDataNode1 = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 99); + deleteDataNode1.setSearchIndex(0); + PartialPath path2 = new PartialPath("root.vehicle.d200.s0"); + DeleteDataNode deleteDataNode2 = + new DeleteDataNode(new PlanNodeId("2"), Collections.singletonList(path2), 50, 70); + deleteDataNode2.setSearchIndex(0); // delete data which is not in working memtable - dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 50, 99, 0); - dataRegion.deleteByDevice(new PartialPath("root.vehicle.d200.s0"), 50, 70, 0); + dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), deleteDataNode1); + dataRegion.deleteByDevice(new PartialPath("root.vehicle.d200.s0"), deleteDataNode2); + PartialPath path3 = new PartialPath("root.vehicle.d199.*"); + DeleteDataNode deleteDataNode3 = + new DeleteDataNode(new PlanNodeId("3"), Collections.singletonList(path3), 50, 500); + deleteDataNode3.setSearchIndex(0); // delete data which is in working memtable - dataRegion.deleteByDevice(new PartialPath("root.vehicle.d199.*"), 50, 500, 0); + dataRegion.deleteByDevice(new PartialPath("root.vehicle.d199.*"), deleteDataNode3); dataRegion.syncCloseAllWorkingTsFileProcessors(); Assert.assertFalse(tsFileResource.getModFile().exists()); @@ -1486,8 +1573,12 @@ public void testFlushingEmptyMemtable() } TsFileResource tsFileResource = dataRegion.getTsFileManager().getTsFileList(true).get(0); + PartialPath path = new PartialPath("root.vehicle.d0.s0"); + DeleteDataNode deleteDataNode = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 100, 200); + deleteDataNode.setSearchIndex(0); // delete all data which is in flushing memtable - dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), 100, 200, 0); + dataRegion.deleteByDevice(new PartialPath("root.vehicle.d0.s0"), deleteDataNode); dataRegion.syncCloseAllWorkingTsFileProcessors(); Assert.assertFalse(tsFileResource.getTsFile().exists()); @@ -1523,19 +1614,29 @@ public void testDeleteDataDirectlySeqWriteModsOrDeleteFiles() TsFileResource tsFileResource = dataRegion.getTsFileManager().getTsFileList(true).get(0); // delete data in work mem, no mods. - dataRegion.deleteDataDirectly(new PartialPath("root.vehicle.d0.**"), 50, 100, 0); + PartialPath path = new PartialPath("root.vehicle.d0.**"); + DeleteDataNode deleteDataNode1 = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 100); + deleteDataNode1.setSearchIndex(0); + dataRegion.deleteDataDirectly(path, deleteDataNode1); Assert.assertTrue(tsFileResource.getTsFile().exists()); Assert.assertFalse(tsFileResource.getModFile().exists()); dataRegion.syncCloseAllWorkingTsFileProcessors(); // delete data in closed file, but time not match - dataRegion.deleteDataDirectly(new PartialPath("root.vehicle.d0.**"), 100, 120, 0); + DeleteDataNode deleteDataNode2 = + new DeleteDataNode(new PlanNodeId("2"), Collections.singletonList(path), 100, 120); + deleteDataNode2.setSearchIndex(0); + dataRegion.deleteDataDirectly(new PartialPath("root.vehicle.d0.**"), deleteDataNode2); Assert.assertTrue(tsFileResource.getTsFile().exists()); Assert.assertTrue(tsFileResource.getModFile().exists()); // delete data in closed file, and time all match - dataRegion.deleteDataDirectly(new PartialPath("root.vehicle.d0.**"), 100, 199, 0); + DeleteDataNode deleteDataNode3 = + new DeleteDataNode(new PlanNodeId("3"), Collections.singletonList(path), 100, 199); + deleteDataNode3.setSearchIndex(0); + dataRegion.deleteDataDirectly(new PartialPath("root.vehicle.d0.**"), deleteDataNode3); Assert.assertFalse(tsFileResource.getTsFile().exists()); Assert.assertFalse(tsFileResource.getModFile().exists()); } @@ -1565,9 +1666,16 @@ public void testDeleteDataDirectlyUnseqWriteModsOrDeleteFiles() Assert.assertTrue(tsFileResourceUnSeq.getTsFile().exists()); // already closed, will have a mods file. - dataRegion.deleteDataDirectly(new PartialPath("root.vehicle.d0.**"), 40, 60, 0); + PartialPath path = new PartialPath("root.vehicle.d0.**"); + DeleteDataNode deleteDataNode1 = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 40, 60); + deleteDataNode1.setSearchIndex(0); + dataRegion.deleteDataDirectly(new PartialPath("root.vehicle.d0.**"), deleteDataNode1); // not close yet, just delete in memory. - dataRegion.deleteDataDirectly(new PartialPath("root.vehicle.d0.**"), 140, 160, 0); + DeleteDataNode deleteDataNode2 = + new DeleteDataNode(new PlanNodeId("2"), Collections.singletonList(path), 140, 160); + deleteDataNode2.setSearchIndex(0); + dataRegion.deleteDataDirectly(new PartialPath("root.vehicle.d0.**"), deleteDataNode2); // delete data in mem table, there is no mods Assert.assertTrue(tsFileResourceSeq.getTsFile().exists()); @@ -1576,14 +1684,23 @@ public void testDeleteDataDirectlyUnseqWriteModsOrDeleteFiles() Assert.assertFalse(tsFileResourceUnSeq.getModFile().exists()); dataRegion.syncCloseAllWorkingTsFileProcessors(); - dataRegion.deleteDataDirectly(new PartialPath("root.vehicle.d0.**"), 40, 80, 0); + DeleteDataNode deleteDataNode3 = + new DeleteDataNode(new PlanNodeId("3"), Collections.singletonList(path), 40, 80); + deleteDataNode3.setSearchIndex(0); + dataRegion.deleteDataDirectly(new PartialPath("root.vehicle.d0.**"), deleteDataNode3); Assert.assertTrue(tsFileResourceUnSeq.getTsFile().exists()); Assert.assertTrue(tsFileResourceUnSeq.getModFile().exists()); // seq file and unseq file have data file and mod file now, // this deletion will remove data file and mod file. - dataRegion.deleteDataDirectly(new PartialPath("root.vehicle.d0.**"), 30, 100, 0); - dataRegion.deleteDataDirectly(new PartialPath("root.vehicle.d0.**"), 100, 199, 0); + DeleteDataNode deleteDataNode4 = + new DeleteDataNode(new PlanNodeId("4"), Collections.singletonList(path), 30, 100); + deleteDataNode4.setSearchIndex(0); + DeleteDataNode deleteDataNode5 = + new DeleteDataNode(new PlanNodeId("5"), Collections.singletonList(path), 100, 199); + deleteDataNode5.setSearchIndex(0); + dataRegion.deleteDataDirectly(new PartialPath("root.vehicle.d0.**"), deleteDataNode4); + dataRegion.deleteDataDirectly(new PartialPath("root.vehicle.d0.**"), deleteDataNode5); Assert.assertFalse(tsFileResourceSeq.getTsFile().exists()); Assert.assertFalse(tsFileResourceUnSeq.getTsFile().exists()); From ac53ff6c818c587b1cd004aa1f1a2b4e4cb943b5 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Wed, 24 Jul 2024 18:02:02 +0800 Subject: [PATCH 02/85] support transfer and receive of deletion --- .../PipeConsensusAsyncConnector.java | 10 +- .../PipeConsensusSyncConnector.java | 76 ++++++++++- .../request/PipeConsensusPlanNodeReq.java | 126 ++++++++++++++++++ .../PipeSchemaRegionWritePlanEvent.java | 4 + .../pipeconsensus/PipeConsensusReceiver.java | 37 ++++- .../plan/node/write/DeleteDataNode.java | 12 ++ .../planner/plan/node/write/InsertNode.java | 3 +- .../planner/plan/node/write/SearchNode.java | 5 +- .../request/PipeConsensusRequestType.java | 1 + 9 files changed, 254 insertions(+), 20 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusPlanNodeReq.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java index 8bccf12b5c1a..c99df7b2d223 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java @@ -47,6 +47,7 @@ import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletInsertNodeReq; import org.apache.iotdb.db.pipe.consensus.metric.PipeConsensusConnectorMetrics; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; +import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; @@ -392,13 +393,16 @@ public void transfer(Event event) throws Exception { syncTransferQueuedEventsIfNecessary(); transferBatchedEventsIfNecessary(); + // Transfer deletion + if (event instanceof PipeSchemaRegionWritePlanEvent) { + retryConnector.transfer(event); + return; + } + if (!(event instanceof PipeHeartbeatEvent)) { LOGGER.warn( "PipeConsensusAsyncConnector does not support transferring generic event: {}.", event); - return; } - - retryConnector.transfer(event); } /** Try its best to commit data in order. Flush can also be a trigger to transfer batched data. */ diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java index 616d46f61ef7..43d5bdbd324a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java @@ -34,6 +34,7 @@ import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusBatchTransferResp; import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferResp; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.builder.PipeConsensusSyncBatchReqBuilder; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusPlanNodeReq; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletBinaryReq; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletInsertNodeReq; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFilePieceReq; @@ -41,7 +42,7 @@ import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFileSealReq; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFileSealWithModReq; import org.apache.iotdb.db.pipe.consensus.metric.PipeConsensusConnectorMetrics; -import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; +import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode; @@ -177,10 +178,8 @@ public void transfer(Event event) throws Exception { doTransfer(); } - if (!(event instanceof PipeHeartbeatEvent)) { - LOGGER.warn( - "PipeConsensusSyncConnector does not support transferring generic event: {}.", event); - } + // Only deletion event will be passed here. + doTransferWrapper((PipeSchemaRegionWritePlanEvent) event); } private void doTransfer() { @@ -221,6 +220,73 @@ private void doTransfer() { } } + private void doTransferWrapper( + final PipeSchemaRegionWritePlanEvent pipeSchemaRegionWritePlanEvent) throws PipeException { + try { + // We increase the reference count for this event to determine if the event may be released. + if (!pipeSchemaRegionWritePlanEvent.increaseReferenceCount( + PipeConsensusSyncConnector.class.getName())) { + return; + } + doTransfer(pipeSchemaRegionWritePlanEvent); + } finally { + pipeSchemaRegionWritePlanEvent.decreaseReferenceCount( + PipeConsensusSyncConnector.class.getName(), false); + } + } + + private void doTransfer(final PipeSchemaRegionWritePlanEvent pipeSchemaRegionWritePlanEvent) + throws PipeException { + final ProgressIndex progressIndex; + final TPipeConsensusTransferResp resp; + TCommitId tCommitId = + new TCommitId( + pipeSchemaRegionWritePlanEvent.getCommitId(), + pipeSchemaRegionWritePlanEvent.getRebootTimes()); + TConsensusGroupId tConsensusGroupId = + new TConsensusGroupId(TConsensusGroupType.DataRegion, consensusGroupId); + + try (final SyncPipeConsensusServiceClient syncPipeConsensusServiceClient = + syncRetryClientManager.borrowClient(getFollowerUrl())) { + progressIndex = pipeSchemaRegionWritePlanEvent.getProgressIndex(); + resp = + syncPipeConsensusServiceClient.pipeConsensusTransfer( + PipeConsensusPlanNodeReq.toTPipeConsensusTransferReq( + pipeSchemaRegionWritePlanEvent.getPlanNode(), + tCommitId, + tConsensusGroupId, + progressIndex, + thisDataNodeId)); + } catch (Exception e) { + throw new PipeConnectionException( + String.format( + PIPE_CONSENSUS_SYNC_CONNECTION_FAILED_FORMAT, + getFollowerUrl().getIp(), + getFollowerUrl().getPort(), + TABLET_INSERTION_NODE_SCENARIO, + e.getMessage()), + e); + } + + final TSStatus status = resp.getStatus(); + // Only handle the failed statuses to avoid string format performance overhead + if (resp.getStatus().getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode() + && resp.getStatus().getCode() != TSStatusCode.REDIRECTION_RECOMMEND.getStatusCode()) { + receiverStatusHandler.handle( + status, + String.format( + "PipeConsensus transfer DeletionEvent %s error, result status %s.", + pipeSchemaRegionWritePlanEvent.getDeletionResource(), status), + pipeSchemaRegionWritePlanEvent.getPlanNode().toString()); + } + + if (LOGGER.isDebugEnabled()) { + LOGGER.debug( + "Successfully transferred deletion event {}.", + pipeSchemaRegionWritePlanEvent.getDeletionResource()); + } + } + private void doTransferWrapper( final PipeInsertNodeTabletInsertionEvent pipeInsertNodeTabletInsertionEvent) throws PipeException { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusPlanNodeReq.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusPlanNodeReq.java new file mode 100644 index 000000000000..55c7923c3bdb --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusPlanNodeReq.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request; + +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; +import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.commons.pipe.connector.payload.pipeconsensus.request.PipeConsensusRequestType; +import org.apache.iotdb.commons.pipe.connector.payload.pipeconsensus.request.PipeConsensusRequestVersion; +import org.apache.iotdb.consensus.pipe.thrift.TCommitId; +import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeType; + +import org.apache.tsfile.utils.PublicBAOS; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Objects; + +public class PipeConsensusPlanNodeReq extends TPipeConsensusTransferReq { + private static final Logger LOGGER = LoggerFactory.getLogger(PipeConsensusPlanNodeReq.class); + private transient PlanNode planNode; + + private PipeConsensusPlanNodeReq() { + // Do nothing + } + + public PlanNode getPlanNode() { + return planNode; + } + + /////////////////////////////// Thrift /////////////////////////////// + + public static PipeConsensusPlanNodeReq toTPipeConsensusTransferReq( + PlanNode planNode, + TCommitId commitId, + TConsensusGroupId consensusGroupId, + ProgressIndex progressIndex, + int thisDataNodeId) { + final PipeConsensusPlanNodeReq req = new PipeConsensusPlanNodeReq(); + + req.planNode = planNode; + + req.commitId = commitId; + req.consensusGroupId = consensusGroupId; + req.dataNodeId = thisDataNodeId; + req.version = PipeConsensusRequestVersion.VERSION_1.getVersion(); + req.type = PipeConsensusRequestType.TRANSFER_DELETION.getType(); + req.body = planNode.serializeToByteBuffer(); + + try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); + final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + progressIndex.serialize(outputStream); + req.progressIndex = + ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + } catch (IOException e) { + LOGGER.warn("Failed to serialize progress index {}", progressIndex, e); + } + + return req; + } + + public static PipeConsensusPlanNodeReq fromTPipeConsensusTransferReq( + TPipeConsensusTransferReq transferReq) { + final PipeConsensusPlanNodeReq planNodeReq = new PipeConsensusPlanNodeReq(); + + planNodeReq.planNode = PlanNodeType.deserialize(transferReq.body); + + planNodeReq.version = transferReq.version; + planNodeReq.type = transferReq.type; + planNodeReq.body = transferReq.body; + planNodeReq.commitId = transferReq.commitId; + planNodeReq.dataNodeId = transferReq.dataNodeId; + planNodeReq.consensusGroupId = transferReq.consensusGroupId; + planNodeReq.progressIndex = transferReq.progressIndex; + + return planNodeReq; + } + + /////////////////////////////// Object /////////////////////////////// + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + PipeConsensusPlanNodeReq that = (PipeConsensusPlanNodeReq) obj; + return planNode.equals(that.planNode) + && version == that.version + && type == that.type + && Objects.equals(body, that.body) + && Objects.equals(commitId, that.commitId) + && Objects.equals(consensusGroupId, that.consensusGroupId) + && Objects.equals(progressIndex, that.progressIndex) + && Objects.equals(dataNodeId, that.dataNodeId); + } + + @Override + public int hashCode() { + return Objects.hash( + planNode, version, type, body, commitId, consensusGroupId, dataNodeId, progressIndex); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java index 459e113595d1..14fd657ff041 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java @@ -60,6 +60,10 @@ public PlanNode getPlanNode() { return planNode; } + public DeletionResource getDeletionResource() { + return deletionResource; + } + public void setDeletionResource(DeletionResource deletionResource) { this.deletionResource = deletionResource; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java index b5f01e52b5ae..c6566cc2ccdc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java @@ -42,6 +42,7 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.exception.DiskSpaceInsufficientException; import org.apache.iotdb.db.exception.LoadFileException; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusPlanNodeReq; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletBinaryReq; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletInsertNodeReq; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFilePieceReq; @@ -50,6 +51,7 @@ import org.apache.iotdb.db.pipe.consensus.metric.PipeConsensusReceiverMetrics; import org.apache.iotdb.db.pipe.event.common.tsfile.TsFileInsertionPointCounter; import org.apache.iotdb.db.queryengine.execution.load.LoadTsFileManager; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode; import org.apache.iotdb.db.storageengine.StorageEngine; import org.apache.iotdb.db.storageengine.dataregion.DataRegion; @@ -169,6 +171,7 @@ public TPipeConsensusTransferResp receive(final TPipeConsensusTransferReq req) { // TODO: check memory when logging WAL(in further version) resp = requestExecutor.onRequest(req, false, true); break; + case TRANSFER_DELETION: case TRANSFER_TABLET_BINARY: case TRANSFER_TABLET_INSERT_NODE: // TODO: support batch transfer(in further version) @@ -248,6 +251,9 @@ private TPipeConsensusTransferResp loadEvent(final TPipeConsensusTransferReq req case TRANSFER_TABLET_BINARY: return handleTransferTabletBinary( PipeConsensusTabletBinaryReq.fromTPipeConsensusTransferReq(req)); + case TRANSFER_DELETION: + return handleTransferDeletion( + PipeConsensusPlanNodeReq.fromTPipeConsensusTransferReq(req)); case TRANSFER_TS_FILE_PIECE: return handleTransferFilePiece( PipeConsensusTsFilePieceReq.fromTPipeConsensusTransferReq(req), true); @@ -286,8 +292,6 @@ private TPipeConsensusTransferResp loadEvent(final TPipeConsensusTransferReq req private TPipeConsensusTransferResp handleTransferTabletInsertNode( final PipeConsensusTabletInsertNodeReq req) throws ConsensusGroupNotExistException { - LOGGER.info( - "PipeConsensus-PipeName-{}: starting to receive tablet insertNode", consensusPipeName); PipeConsensusServerImpl impl = Optional.ofNullable(pipeConsensus.getImpl(consensusGroupId)) .orElseThrow(() -> new ConsensusGroupNotExistException(consensusGroupId)); @@ -300,7 +304,6 @@ private TPipeConsensusTransferResp handleTransferTabletInsertNode( private TPipeConsensusTransferResp handleTransferTabletBinary( final PipeConsensusTabletBinaryReq req) throws ConsensusGroupNotExistException { - LOGGER.info("PipeConsensus-PipeName-{}: starting to receive tablet binary", consensusPipeName); PipeConsensusServerImpl impl = Optional.ofNullable(pipeConsensus.getImpl(consensusGroupId)) .orElseThrow(() -> new ConsensusGroupNotExistException(consensusGroupId)); @@ -311,9 +314,24 @@ private TPipeConsensusTransferResp handleTransferTabletBinary( return new TPipeConsensusTransferResp(impl.writeOnFollowerReplica(insertNode)); } + private TPipeConsensusTransferResp handleTransferDeletion(final PipeConsensusPlanNodeReq req) + throws ConsensusGroupNotExistException { + PipeConsensusServerImpl impl = + Optional.ofNullable(pipeConsensus.getImpl(consensusGroupId)) + .orElseThrow(() -> new ConsensusGroupNotExistException(consensusGroupId)); + final DeleteDataNode planNode = (DeleteDataNode) req.getPlanNode(); + planNode.markAsGeneratedByRemoteConsensusLeader(); + planNode.setProgressIndex( + ProgressIndexType.deserializeFrom(ByteBuffer.wrap(req.getProgressIndex()))); + return new TPipeConsensusTransferResp(impl.writeOnFollowerReplica(planNode)); + } + private TPipeConsensusTransferResp handleTransferFilePiece( final PipeConsensusTransferFilePieceReq req, final boolean isSingleFile) { - LOGGER.info("PipeConsensus-PipeName-{}: starting to receive tsFile pieces", consensusPipeName); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug( + "PipeConsensus-PipeName-{}: starting to receive tsFile pieces", consensusPipeName); + } long startBorrowTsFileWriterNanos = System.nanoTime(); PipeConsensusTsFileWriter tsFileWriter = pipeConsensusTsFileWriterPool.borrowCorrespondingWriter(req.getCommitId()); @@ -376,7 +394,9 @@ private TPipeConsensusTransferResp handleTransferFilePiece( } private TPipeConsensusTransferResp handleTransferFileSeal(final PipeConsensusTsFileSealReq req) { - LOGGER.info("PipeConsensus-PipeName-{}: starting to receive tsFile seal", consensusPipeName); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("PipeConsensus-PipeName-{}: starting to receive tsFile seal", consensusPipeName); + } long startBorrowTsFileWriterNanos = System.nanoTime(); PipeConsensusTsFileWriter tsFileWriter = pipeConsensusTsFileWriterPool.borrowCorrespondingWriter(req.getCommitId()); @@ -477,8 +497,11 @@ private TPipeConsensusTransferResp handleTransferFileSeal(final PipeConsensusTsF private TPipeConsensusTransferResp handleTransferFileSealWithMods( final PipeConsensusTsFileSealWithModReq req) { - LOGGER.info( - "PipeConsensus-PipeName-{}: starting to receive tsFile seal with mods", consensusPipeName); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug( + "PipeConsensus-PipeName-{}: starting to receive tsFile seal with mods", + consensusPipeName); + } long startBorrowTsFileWriterNanos = System.nanoTime(); PipeConsensusTsFileWriter tsFileWriter = pipeConsensusTsFileWriterPool.borrowCorrespondingWriter(req.getCommitId()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java index 54f20728de2a..5fa7cf876853 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.queryengine.plan.planner.plan.node.write; import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet; +import org.apache.iotdb.commons.consensus.index.ProgressIndex; import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.partition.DataPartition; import org.apache.iotdb.commons.path.PartialPath; @@ -63,6 +64,7 @@ public class DeleteDataNode extends SearchNode implements WALEntryValue { private final long deleteEndTime; private TRegionReplicaSet regionReplicaSet; + private ProgressIndex progressIndex; public DeleteDataNode( PlanNodeId id, List pathList, long deleteStartTime, long deleteEndTime) { @@ -97,6 +99,16 @@ public long getDeleteEndTime() { return deleteEndTime; } + @Override + public ProgressIndex getProgressIndex() { + return progressIndex; + } + + @Override + public void setProgressIndex(ProgressIndex progressIndex) { + this.progressIndex = progressIndex; + } + @Override public List getChildren() { return new ArrayList<>(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertNode.java index 161b6ac9d2e8..45a135d0458b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertNode.java @@ -20,7 +20,6 @@ package org.apache.iotdb.db.queryengine.plan.planner.plan.node.write; import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet; -import org.apache.iotdb.commons.consensus.index.ComparableConsensusRequest; import org.apache.iotdb.commons.consensus.index.ProgressIndex; import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.utils.TestOnly; @@ -44,7 +43,7 @@ import java.util.Arrays; import java.util.Objects; -public abstract class InsertNode extends SearchNode implements ComparableConsensusRequest { +public abstract class InsertNode extends SearchNode { private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/SearchNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/SearchNode.java index c5172534298e..28c2c2ee542f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/SearchNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/SearchNode.java @@ -19,13 +19,12 @@ package org.apache.iotdb.db.queryengine.plan.planner.plan.node.write; +import org.apache.iotdb.commons.consensus.index.ComparableConsensusRequest; import org.apache.iotdb.consensus.iot.log.ConsensusReqReader; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.WritePlanNode; -import static org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode.NO_CONSENSUS_INDEX; - -public abstract class SearchNode extends WritePlanNode { +public abstract class SearchNode extends WritePlanNode implements ComparableConsensusRequest { /** this insert node doesn't need to participate in iot consensus */ public static final long NO_CONSENSUS_INDEX = ConsensusReqReader.DEFAULT_SEARCH_INDEX; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/pipeconsensus/request/PipeConsensusRequestType.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/pipeconsensus/request/PipeConsensusRequestType.java index 3ac3e8f2685f..6f046da4e1cd 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/pipeconsensus/request/PipeConsensusRequestType.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/connector/payload/pipeconsensus/request/PipeConsensusRequestType.java @@ -34,6 +34,7 @@ public enum PipeConsensusRequestType { TRANSFER_TABLET_BINARY((short) 104), TRANSFER_TS_FILE_PIECE_WITH_MOD((short) 105), TRANSFER_TS_FILE_SEAL_WITH_MOD((short) 106), + TRANSFER_DELETION((short) 107), // Note: temporarily PipeConsensus only support data region. But we put this class in `node-common` // to reserve the scalability From 1271ce51bcdd850fd9e1843c6328b8acc395bd0e Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Wed, 24 Jul 2024 18:25:15 +0800 Subject: [PATCH 03/85] fix ut fail --- ...sSpaceCompactionWithFastPerformerTest.java | 55 ++++++++++++++----- ...eCompactionWithReadPointPerformerTest.java | 55 ++++++++++++++----- ...nerSeqCompactionWithFastPerformerTest.java | 18 +++++- ...qCompactionWithReadChunkPerformerTest.java | 18 +++++- .../settle/SettleRequestHandlerTest.java | 10 +++- 5 files changed, 118 insertions(+), 38 deletions(-) diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithFastPerformerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithFastPerformerTest.java index c2920db6050e..b89380a62943 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithFastPerformerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithFastPerformerTest.java @@ -26,6 +26,8 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.exception.StorageEngineException; import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; import org.apache.iotdb.db.storageengine.dataregion.DataRegion; import org.apache.iotdb.db.storageengine.dataregion.compaction.AbstractCompactionTest; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.impl.FastCompactionPerformer; @@ -580,6 +582,17 @@ public void testOneDeletionDuringCompaction() throws Exception { createFiles(2, 1, 5, 100, 450, 20450, 0, 0, true, false); vsgp.getTsFileResourceManager().addAll(seqResources, true); vsgp.getTsFileResourceManager().addAll(unseqResources, false); + PartialPath path = + new PartialPath( + COMPACTION_TEST_SG + + PATH_SEPARATOR + + "d" + + (TsFileGeneratorUtils.getAlignDeviceOffset() + 3) + + PATH_SEPARATOR + + "s0"); + DeleteDataNode deleteDataNode = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 0, 1000); + deleteDataNode.setSearchIndex(0); vsgp.deleteByDevice( new PartialPath( COMPACTION_TEST_SG @@ -588,9 +601,7 @@ public void testOneDeletionDuringCompaction() throws Exception { + (TsFileGeneratorUtils.getAlignDeviceOffset() + 3) + PATH_SEPARATOR + "s0"), - 0, - 1000, - 0); + deleteDataNode); CrossSpaceCompactionTask task = new CrossSpaceCompactionTask( @@ -605,6 +616,9 @@ public void testOneDeletionDuringCompaction() throws Exception { seqResources.forEach(f -> f.setStatus(TsFileResourceStatus.COMPACTING)); unseqResources.forEach(f -> f.setStatus(TsFileResourceStatus.COMPACTING)); // delete data in source file during compaction + DeleteDataNode deleteDataNode2 = + new DeleteDataNode(new PlanNodeId("2"), Collections.singletonList(path), 0, 1200); + deleteDataNode2.setSearchIndex(0); vsgp.deleteByDevice( new PartialPath( COMPACTION_TEST_SG @@ -613,9 +627,7 @@ public void testOneDeletionDuringCompaction() throws Exception { + (TsFileGeneratorUtils.getAlignDeviceOffset() + 3) + PATH_SEPARATOR + "s0"), - 0, - 1200, - 0); + deleteDataNode2); for (int i = 0; i < seqResources.size(); i++) { TsFileResource resource = seqResources.get(i); resource.resetModFile(); @@ -699,6 +711,18 @@ public void testSeveralDeletionsDuringCompaction() throws Exception { createFiles(2, 1, 5, 100, 450, 20450, 0, 0, true, false); vsgp.getTsFileResourceManager().addAll(seqResources, true); vsgp.getTsFileResourceManager().addAll(unseqResources, false); + + PartialPath path = + new PartialPath( + COMPACTION_TEST_SG + + PATH_SEPARATOR + + "d" + + (TsFileGeneratorUtils.getAlignDeviceOffset() + 3) + + PATH_SEPARATOR + + "s0"); + DeleteDataNode deleteDataNode = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 0, 1000); + deleteDataNode.setSearchIndex(0); vsgp.deleteByDevice( new PartialPath( COMPACTION_TEST_SG @@ -707,9 +731,7 @@ public void testSeveralDeletionsDuringCompaction() throws Exception { + (TsFileGeneratorUtils.getAlignDeviceOffset() + 3) + PATH_SEPARATOR + "s0"), - 0, - 1000, - 0); + deleteDataNode); CrossSpaceCompactionTask task = new CrossSpaceCompactionTask( @@ -724,6 +746,9 @@ public void testSeveralDeletionsDuringCompaction() throws Exception { seqResources.forEach(f -> f.setStatus(TsFileResourceStatus.COMPACTING)); unseqResources.forEach(f -> f.setStatus(TsFileResourceStatus.COMPACTING)); // delete data in source file during compaction + DeleteDataNode deleteDataNode2 = + new DeleteDataNode(new PlanNodeId("2"), Collections.singletonList(path), 0, 1200); + deleteDataNode2.setSearchIndex(0); vsgp.deleteByDevice( new PartialPath( COMPACTION_TEST_SG @@ -732,9 +757,11 @@ public void testSeveralDeletionsDuringCompaction() throws Exception { + (TsFileGeneratorUtils.getAlignDeviceOffset() + 3) + PATH_SEPARATOR + "s0"), - 0, - 1200, - 0); + deleteDataNode2); + + DeleteDataNode deleteDataNode3 = + new DeleteDataNode(new PlanNodeId("3"), Collections.singletonList(path), 0, 1800); + deleteDataNode3.setSearchIndex(0); vsgp.deleteByDevice( new PartialPath( COMPACTION_TEST_SG @@ -743,9 +770,7 @@ public void testSeveralDeletionsDuringCompaction() throws Exception { + (TsFileGeneratorUtils.getAlignDeviceOffset() + 3) + PATH_SEPARATOR + "s0"), - 0, - 1800, - 0); + deleteDataNode3); for (int i = 0; i < seqResources.size(); i++) { TsFileResource resource = seqResources.get(i); resource.resetModFile(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithReadPointPerformerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithReadPointPerformerTest.java index 54f4dfbbb79f..1c202c65b21f 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithReadPointPerformerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithReadPointPerformerTest.java @@ -26,6 +26,8 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.exception.StorageEngineException; import org.apache.iotdb.db.queryengine.execution.fragment.FragmentInstanceContext; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; import org.apache.iotdb.db.storageengine.dataregion.DataRegion; import org.apache.iotdb.db.storageengine.dataregion.compaction.AbstractCompactionTest; import org.apache.iotdb.db.storageengine.dataregion.compaction.execute.performer.impl.ReadPointCompactionPerformer; @@ -575,6 +577,17 @@ public void testOneDeletionDuringCompaction() throws Exception { createFiles(2, 1, 5, 100, 450, 20450, 0, 0, true, false); vsgp.getTsFileResourceManager().addAll(seqResources, true); vsgp.getTsFileResourceManager().addAll(unseqResources, false); + PartialPath path = + new PartialPath( + COMPACTION_TEST_SG + + PATH_SEPARATOR + + "d" + + (TsFileGeneratorUtils.getAlignDeviceOffset() + 3) + + PATH_SEPARATOR + + "s0"); + DeleteDataNode deleteDataNode = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 0, 1000); + deleteDataNode.setSearchIndex(0); vsgp.deleteByDevice( new PartialPath( COMPACTION_TEST_SG @@ -583,9 +596,7 @@ public void testOneDeletionDuringCompaction() throws Exception { + (TsFileGeneratorUtils.getAlignDeviceOffset() + 3) + PATH_SEPARATOR + "s0"), - 0, - 1000, - 0); + deleteDataNode); CrossSpaceCompactionTask task = new CrossSpaceCompactionTask( @@ -600,6 +611,9 @@ public void testOneDeletionDuringCompaction() throws Exception { seqResources.forEach(f -> f.setStatus(TsFileResourceStatus.COMPACTING)); unseqResources.forEach(f -> f.setStatus(TsFileResourceStatus.COMPACTING)); // delete data in source file during compaction + DeleteDataNode deleteDataNode2 = + new DeleteDataNode(new PlanNodeId("2"), Collections.singletonList(path), 0, 1200); + deleteDataNode2.setSearchIndex(0); vsgp.deleteByDevice( new PartialPath( COMPACTION_TEST_SG @@ -608,9 +622,7 @@ public void testOneDeletionDuringCompaction() throws Exception { + (TsFileGeneratorUtils.getAlignDeviceOffset() + 3) + PATH_SEPARATOR + "s0"), - 0, - 1200, - 0); + deleteDataNode2); for (int i = 0; i < seqResources.size(); i++) { TsFileResource resource = seqResources.get(i); resource.resetModFile(); @@ -694,6 +706,18 @@ public void testSeveralDeletionsDuringCompaction() throws Exception { createFiles(2, 1, 5, 100, 450, 20450, 0, 0, true, false); vsgp.getTsFileResourceManager().addAll(seqResources, true); vsgp.getTsFileResourceManager().addAll(unseqResources, false); + + PartialPath path = + new PartialPath( + COMPACTION_TEST_SG + + PATH_SEPARATOR + + "d" + + (TsFileGeneratorUtils.getAlignDeviceOffset() + 3) + + PATH_SEPARATOR + + "s0"); + DeleteDataNode deleteDataNode = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 0, 1000); + deleteDataNode.setSearchIndex(0); vsgp.deleteByDevice( new PartialPath( COMPACTION_TEST_SG @@ -702,9 +726,7 @@ public void testSeveralDeletionsDuringCompaction() throws Exception { + (TsFileGeneratorUtils.getAlignDeviceOffset() + 3) + PATH_SEPARATOR + "s0"), - 0, - 1000, - 0); + deleteDataNode); CrossSpaceCompactionTask task = new CrossSpaceCompactionTask( @@ -719,6 +741,9 @@ public void testSeveralDeletionsDuringCompaction() throws Exception { seqResources.forEach(f -> f.setStatus(TsFileResourceStatus.COMPACTING)); unseqResources.forEach(f -> f.setStatus(TsFileResourceStatus.COMPACTING)); // delete data in source file during compaction + DeleteDataNode deleteDataNode2 = + new DeleteDataNode(new PlanNodeId("2"), Collections.singletonList(path), 0, 1200); + deleteDataNode2.setSearchIndex(0); vsgp.deleteByDevice( new PartialPath( COMPACTION_TEST_SG @@ -727,9 +752,11 @@ public void testSeveralDeletionsDuringCompaction() throws Exception { + (TsFileGeneratorUtils.getAlignDeviceOffset() + 3) + PATH_SEPARATOR + "s0"), - 0, - 1200, - 0); + deleteDataNode2); + + DeleteDataNode deleteDataNode3 = + new DeleteDataNode(new PlanNodeId("3"), Collections.singletonList(path), 0, 1800); + deleteDataNode3.setSearchIndex(0); vsgp.deleteByDevice( new PartialPath( COMPACTION_TEST_SG @@ -738,9 +765,7 @@ public void testSeveralDeletionsDuringCompaction() throws Exception { + (TsFileGeneratorUtils.getAlignDeviceOffset() + 3) + PATH_SEPARATOR + "s0"), - 0, - 1800, - 0); + deleteDataNode3); for (int i = 0; i < seqResources.size(); i++) { TsFileResource resource = seqResources.get(i); resource.resetModFile(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/InnerSeqCompactionWithFastPerformerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/InnerSeqCompactionWithFastPerformerTest.java index e7b032da2f20..b864d16e7c24 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/InnerSeqCompactionWithFastPerformerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/InnerSeqCompactionWithFastPerformerTest.java @@ -24,6 +24,8 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.exception.DataRegionException; import org.apache.iotdb.db.exception.StorageEngineException; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; import org.apache.iotdb.db.storageengine.buffer.BloomFilterCache; import org.apache.iotdb.db.storageengine.buffer.ChunkCache; import org.apache.iotdb.db.storageengine.buffer.TimeSeriesMetadataCache; @@ -1139,7 +1141,11 @@ public void testCompactionWithDeletionsDuringCompactions() COMPACTION_TEST_SG); vsgp.getTsFileResourceManager().addAll(sourceResources, true); // delete data before compaction - vsgp.deleteByDevice(new PartialPath(fullPaths[0]), 0, 1000, 0); + PartialPath path = new PartialPath(fullPaths[0]); + DeleteDataNode deleteDataNode = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 0, 1000); + deleteDataNode.setSearchIndex(0); + vsgp.deleteByDevice(new PartialPath(fullPaths[0]), deleteDataNode); ICompactionPerformer performer = new FastCompactionPerformer(false); InnerSpaceCompactionTask task = @@ -1150,8 +1156,14 @@ public void testCompactionWithDeletionsDuringCompactions() // set the source files to COMPACTING manually to simulate the concurrent scenario sourceResources.forEach(f -> f.setStatus(TsFileResourceStatus.COMPACTING)); // delete data during compaction - vsgp.deleteByDevice(new PartialPath(fullPaths[0]), 0, 1200, 0); - vsgp.deleteByDevice(new PartialPath(fullPaths[0]), 0, 1800, 0); + DeleteDataNode deleteDataNode2 = + new DeleteDataNode(new PlanNodeId("2"), Collections.singletonList(path), 0, 1200); + deleteDataNode2.setSearchIndex(0); + DeleteDataNode deleteDataNode3 = + new DeleteDataNode(new PlanNodeId("3"), Collections.singletonList(path), 0, 1800); + deleteDataNode3.setSearchIndex(0); + vsgp.deleteByDevice(new PartialPath(fullPaths[0]), deleteDataNode2); + vsgp.deleteByDevice(new PartialPath(fullPaths[0]), deleteDataNode3); for (int i = 0; i < sourceResources.size() - 1; i++) { TsFileResource resource = sourceResources.get(i); resource.resetModFile(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/InnerSeqCompactionWithReadChunkPerformerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/InnerSeqCompactionWithReadChunkPerformerTest.java index fef89b402a62..5a0606a27217 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/InnerSeqCompactionWithReadChunkPerformerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/InnerSeqCompactionWithReadChunkPerformerTest.java @@ -24,6 +24,8 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.exception.DataRegionException; import org.apache.iotdb.db.exception.StorageEngineException; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; import org.apache.iotdb.db.storageengine.buffer.BloomFilterCache; import org.apache.iotdb.db.storageengine.buffer.ChunkCache; import org.apache.iotdb.db.storageengine.buffer.TimeSeriesMetadataCache; @@ -1104,7 +1106,11 @@ public void testCompactionWithDeletionsDuringCompactions() COMPACTION_TEST_SG); vsgp.getTsFileResourceManager().addAll(sourceResources, true); // delete data before compaction - vsgp.deleteByDevice(new PartialPath(fullPaths[0]), 0, 1000, 0); + PartialPath path = new PartialPath(fullPaths[0]); + DeleteDataNode deleteDataNode = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 0, 1000); + deleteDataNode.setSearchIndex(0); + vsgp.deleteByDevice(new PartialPath(fullPaths[0]), deleteDataNode); InnerSpaceCompactionTask task = new InnerSpaceCompactionTask( @@ -1117,8 +1123,14 @@ public void testCompactionWithDeletionsDuringCompactions() task.setSourceFilesToCompactionCandidate(); sourceResources.forEach(f -> f.setStatus(TsFileResourceStatus.COMPACTING)); // delete data during compaction - vsgp.deleteByDevice(new PartialPath(fullPaths[0]), 0, 1200, 0); - vsgp.deleteByDevice(new PartialPath(fullPaths[0]), 0, 1800, 0); + DeleteDataNode deleteDataNode2 = + new DeleteDataNode(new PlanNodeId("2"), Collections.singletonList(path), 0, 1200); + deleteDataNode2.setSearchIndex(0); + DeleteDataNode deleteDataNode3 = + new DeleteDataNode(new PlanNodeId("3"), Collections.singletonList(path), 0, 1800); + deleteDataNode3.setSearchIndex(0); + vsgp.deleteByDevice(new PartialPath(fullPaths[0]), deleteDataNode2); + vsgp.deleteByDevice(new PartialPath(fullPaths[0]), deleteDataNode3); for (int i = 0; i < sourceResources.size() - 1; i++) { TsFileResource resource = sourceResources.get(i); resource.resetModFile(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/settle/SettleRequestHandlerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/settle/SettleRequestHandlerTest.java index 176ecd078405..dd0e2e8ea14d 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/settle/SettleRequestHandlerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/settle/SettleRequestHandlerTest.java @@ -31,6 +31,8 @@ import org.apache.iotdb.db.exception.StorageEngineException; import org.apache.iotdb.db.exception.TsFileProcessorException; import org.apache.iotdb.db.exception.WriteProcessException; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; import org.apache.iotdb.db.storageengine.StorageEngine; import org.apache.iotdb.db.storageengine.dataregion.DataRegion; import org.apache.iotdb.db.storageengine.dataregion.DataRegionTest; @@ -52,6 +54,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.List; public class SettleRequestHandlerTest { @@ -147,8 +150,11 @@ record = new TSRecord(timestamp, deviceId); } dataRegion.syncCloseAllWorkingTsFileProcessors(); if (i != 2) { - dataRegion.deleteByDevice( - new PartialPath(deviceId, measurementId), 3L * i + 1, 3L * i + 1, -1); + PartialPath path = new PartialPath(deviceId, measurementId); + DeleteDataNode deleteDataNode = + new DeleteDataNode( + new PlanNodeId("1"), Collections.singletonList(path), 3L * i + 1, 3L * i + 1); + dataRegion.deleteByDevice(new PartialPath(deviceId, measurementId), deleteDataNode); } } } From d876131fd79263bcd6b5dd822d5b559cbc733fbc Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Fri, 30 Aug 2024 09:34:33 +0800 Subject: [PATCH 04/85] minor fix WAL close time --- .../iotdb/db/storageengine/dataregion/wal/buffer/WALBuffer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALBuffer.java index 742824a0fabe..30e1cdd05d70 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALBuffer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALBuffer.java @@ -669,6 +669,7 @@ public boolean waitForFlush(long time, TimeUnit unit) throws InterruptedExceptio @Override public void close() { // first waiting serialize and sync tasks finished, then release all resources + isClosed = true; if (serializeThread != null) { // add close signal WALEntry to notify serializeThread try { @@ -677,7 +678,6 @@ public void close() { logger.error("Fail to put CLOSE_SIGNAL to walEntries.", e); Thread.currentThread().interrupt(); } - isClosed = true; shutdownThread(serializeThread, ThreadName.WAL_SERIALIZE); } if (syncBufferThread != null) { From b0aecb8a37b770f81acef27ba928583e46fdf2ce Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Fri, 30 Aug 2024 09:34:59 +0800 Subject: [PATCH 05/85] prepare for deletion ahead log --- .../consensus/deletion/DeletionResource.java | 28 ++- .../deletion/DeletionResourceManager.java | 188 ++++++++++-------- .../PipeSchemaRegionWritePlanEvent.java | 9 +- .../index/impl/SimpleProgressIndex.java | 36 ++-- 4 files changed, 138 insertions(+), 123 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java index de9c9b4bd0bb..4229c3f37d66 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java @@ -19,7 +19,9 @@ package org.apache.iotdb.db.pipe.consensus.deletion; +import org.apache.iotdb.commons.consensus.index.ProgressIndex; import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; import java.nio.ByteBuffer; import java.util.Objects; @@ -32,18 +34,14 @@ * framework, PipeConsensus will use {@link PipeSchemaRegionWritePlanEvent} */ public class DeletionResource { - private final Consumer persistFunc; - private final Consumer removeFunc; + private final Consumer removeHook; private final AtomicLong latestUpdateTime; private PipeSchemaRegionWritePlanEvent deletionEvent; public DeletionResource( - PipeSchemaRegionWritePlanEvent deletionEvent, - Consumer persistFunc, - Consumer removeFunc) { + PipeSchemaRegionWritePlanEvent deletionEvent, Consumer removeHook) { this.deletionEvent = deletionEvent; - this.persistFunc = persistFunc; - this.removeFunc = removeFunc; + this.removeHook = removeHook; latestUpdateTime = new AtomicLong(System.currentTimeMillis()); } @@ -56,12 +54,8 @@ public void releaseSelf() { deletionEvent = null; } - public void persistSelf() { - persistFunc.accept(this); - } - public void removeSelf() { - removeFunc.accept(this); + removeHook.accept(this); } public void increaseReferenceCount() { @@ -81,17 +75,19 @@ public long getLatestUpdateTime() { return latestUpdateTime.get(); } + public ProgressIndex getProgressIndex() { + return ((DeleteDataNode) deletionEvent.getPlanNode()).getProgressIndex(); + } + public ByteBuffer serialize() { return deletionEvent.serializeToByteBuffer(); } public static DeletionResource deserialize( - final ByteBuffer buffer, - final Consumer persistFunc, - Consumer removeFunc) { + final ByteBuffer buffer, final Consumer removeHook) { PipeSchemaRegionWritePlanEvent event = new PipeSchemaRegionWritePlanEvent(); event.deserializeFromByteBuffer(buffer); - return new DeletionResource(event, persistFunc, removeFunc); + return new DeletionResource(event, removeHook); } private void updateLatestUpdateTime() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java index 885360b3a9b6..ae3471f74f9f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java @@ -19,45 +19,57 @@ package org.apache.iotdb.db.pipe.consensus.deletion; +import org.apache.iotdb.commons.consensus.index.impl.RecoverProgressIndex; +import org.apache.iotdb.commons.consensus.index.impl.SimpleProgressIndex; import org.apache.iotdb.commons.utils.FileUtils; import org.apache.iotdb.consensus.pipe.PipeConsensus; +import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.consensus.DataRegionConsensusImpl; -import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; +import org.apache.iotdb.db.pipe.consensus.deletion.persist.DeletionBuffer; +import org.apache.iotdb.db.pipe.consensus.deletion.recover.DeletionReader; import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import com.google.common.collect.ImmutableList; -import org.apache.tsfile.utils.ReadWriteIOUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.DataOutputStream; import java.io.File; -import java.io.FileOutputStream; import java.io.IOException; -import java.nio.ByteBuffer; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import java.util.stream.Stream; -public class DeletionResourceManager { +public class DeletionResourceManager implements AutoCloseable { private static final Logger LOGGER = LoggerFactory.getLogger(DeletionResourceManager.class); - private static final long DELETION_TIME_TO_LIVE_IN_MS = 1000L; - private static final long CHECK_DELETION_DURATION_IN_MS = 1000L * 5; - private static final String DELETION_CHECKER_TASK_ID = "pipe_consensus_deletion_checker"; - private static final String DELETION_FILE_SUFFIX = ".deletion"; + public static final String DELETION_FILE_SUFFIX = ".deletion"; + public static final String MAGIC_VERSION_V1 = "DELETION_V1"; + private static final String REBOOT_TIME = "REBOOT_TIME"; + private static final String MEM_TABLE_FLUSH_ORDER = "MEM_TABLE_FLUSH_ORDER"; + private static final String DELETION_FILE_NAME_PATTERN = + String.format( + "^_(?<%s>\\d+)_(?<%s>\\d+)\\%s$", + REBOOT_TIME, MEM_TABLE_FLUSH_ORDER, DELETION_FILE_SUFFIX); + private final String dataRegionId; + private final DeletionBuffer deletionBuffer; + // TODO: read it from conf - private final File storageDir = new File("tmp"); + private final File storageDir; private final List deletionResources = new CopyOnWriteArrayList<>(); - public DeletionResourceManager() throws IOException { + public DeletionResourceManager(String dataRegionId) throws IOException { + this.dataRegionId = dataRegionId; + this.storageDir = new File("tmp" + File.separator + dataRegionId); + this.deletionBuffer = new DeletionBuffer(dataRegionId, storageDir.getAbsolutePath()); initAndRecover(); - // Register scheduled deletion check task. - PipeDataNodeAgent.runtime() - .registerPeriodicalJob( - DELETION_CHECKER_TASK_ID, this::checkAndCleanDeletions, CHECK_DELETION_DURATION_IN_MS); + // Only after initAndRecover can we start serialize and sync new deletions. + this.deletionBuffer.start(); } private void initAndRecover() throws IOException { @@ -73,112 +85,118 @@ private void initAndRecover() throws IOException { Path[] deletionPaths = pathStream .filter(Files::isRegularFile) - .filter(path -> path.endsWith(DELETION_FILE_SUFFIX)) + .filter(path -> path.getFileName().toString().matches(DELETION_FILE_NAME_PATTERN)) .toArray(Path[]::new); - ByteBuffer readBuffer; + for (Path path : deletionPaths) { - readBuffer = ByteBuffer.wrap(Files.readAllBytes(path)); - deletionResources.add( - DeletionResource.deserialize(readBuffer, this::persist, this::removeDeletionResource)); + try (DeletionReader deletionReader = + new DeletionReader(path.toFile(), this::removeDeletionResource)) { + deletionResources.addAll(deletionReader.readAllDeletions()); + } } } } + @Override + public void close() throws Exception { + this.deletionBuffer.close(); + this.deletionResources.clear(); + } + public void registerDeletionResource(PipeSchemaRegionWritePlanEvent event) { - DeletionResource deletionResource = - new DeletionResource(event, this::persist, this::removeDeletionResource); + DeletionResource deletionResource = new DeletionResource(event, this::removeDeletionResource); event.setDeletionResource(deletionResource); this.deletionResources.add(deletionResource); + deletionBuffer.registerDeletionResource(deletionResource); } public List getAllDeletionResources() { return deletionResources.stream().collect(ImmutableList.toImmutableList()); } - public void persist(final DeletionResource deletionResource) { - File deletionFile = - new File( - storageDir, String.format("%d%s", deletionResource.hashCode(), DELETION_FILE_SUFFIX)); - if (deletionFile.exists()) { - LOGGER.warn("Deletion file {} already exists, delete it.", deletionFile); - FileUtils.deleteFileOrDirectory(deletionFile); - } + /** + * This is a hook function, which will be automatically invoked when deletionResource's reference + * count returns to 0. + */ + private void removeDeletionResource(DeletionResource deletionResource) { + // Clean memory + deletionResources.remove(deletionResource); + // Clean disk + int thisDataNodeId = IoTDBDescriptor.getInstance().getConfig().getDataNodeId(); + SimpleProgressIndex currentProgressIndex = + ((RecoverProgressIndex) deletionResource.getProgressIndex()) + .getDataNodeId2LocalIndex() + .get(thisDataNodeId); - try (FileOutputStream fileOutputStream = new FileOutputStream(deletionFile)) { - try (DataOutputStream dataOutputStream = new DataOutputStream(fileOutputStream)) { - final ByteBuffer byteBuffer = deletionResource.serialize(); - ReadWriteIOUtils.write(byteBuffer, dataOutputStream); - } finally { - try { - fileOutputStream.flush(); - fileOutputStream.getFD().sync(); - } catch (IOException ignore) { - // ignore sync exception - } + try (Stream pathStream = Files.walk(Paths.get(storageDir.getPath()), 1)) { + Path[] deletionPaths = + pathStream + .filter(Files::isRegularFile) + .filter(path -> path.getFileName().toString().matches(DELETION_FILE_NAME_PATTERN)) + .filter( + path -> + isCurrentFileCanBeDeleted( + path.getFileName().toString(), currentProgressIndex)) + .toArray(Path[]::new); + for (Path path : deletionPaths) { + FileUtils.deleteFileOrDirectory(path.toFile()); } } catch (IOException e) { - // log error and ignore exception - LOGGER.error( - "Failed to persist deletion resource {}, may cause inconsistency during replication", - deletionResource, - e); + LOGGER.warn( + "DeletionManager-{} failed to delete file in {} dir, please manually check!", + dataRegionId, + storageDir); } } - private void removeDeletionResource(DeletionResource deletionResource) { - // TODO: 参考 WAL 的水位线机制,攒批之类的,参数也可以参考 WAL - // TODO: 删除文件也可以通过考虑利用 progressIndex 来删 - // TODO: 需要考虑删除后的 index 维护,恢复之类的 - } - - private void checkAndCleanDeletions() { - final ImmutableList toBeCleaned = - deletionResources.stream() - .filter(deletionResource -> deletionResource.getReferenceCount() == 0) - .collect(ImmutableList.toImmutableList()); - - toBeCleaned.forEach( - deletionResource -> { - // Clean disk - File deletionFile = - new File( - storageDir, - String.format("%d%s", deletionResource.hashCode(), DELETION_FILE_SUFFIX)); - if (deletionFile.exists()) { - FileUtils.deleteFileOrDirectory(deletionFile); - } - // Clean memory - deletionResources.remove(deletionResource); - deletionResource.releaseSelf(); - }); + private boolean isCurrentFileCanBeDeleted( + String fileName, SimpleProgressIndex currentProgressIndex) { + int curRebootTimes = currentProgressIndex.getRebootTimes(); + long curMemTableFlushOrderId = currentProgressIndex.getMemTableFlushOrderId(); + + Pattern pattern = Pattern.compile(DELETION_FILE_NAME_PATTERN); + Matcher matcher = pattern.matcher(fileName); + // Definitely match. Because upper caller has filtered fileNames. + if (matcher.matches()) { + int fileRebootTimes = Integer.parseInt(matcher.group(REBOOT_TIME)); + long fileMemTableFlushOrderId = Long.parseLong(matcher.group(MEM_TABLE_FLUSH_ORDER)); + return fileRebootTimes == curRebootTimes + ? fileMemTableFlushOrderId < curMemTableFlushOrderId + : fileRebootTimes < curMemTableFlushOrderId; + } + return false; } //////////////////////////// singleton //////////////////////////// private static class DeletionResourceManagerHolder { - private static DeletionResourceManager INSTANCE; + private static Map CONSENSU_GROUP_ID_2_INSTANCE_MAP; private DeletionResourceManagerHolder() {} - public static void build() throws IOException { - if (INSTANCE == null) { - INSTANCE = new DeletionResourceManager(); + public static void build() { + if (CONSENSU_GROUP_ID_2_INSTANCE_MAP == null) { + CONSENSU_GROUP_ID_2_INSTANCE_MAP = new ConcurrentHashMap<>(); } } } - public static DeletionResourceManager getInstance() { - return DeletionResourceManager.DeletionResourceManagerHolder.INSTANCE; + public static DeletionResourceManager getInstance(String groupId) { + return DeletionResourceManagerHolder.CONSENSU_GROUP_ID_2_INSTANCE_MAP.computeIfAbsent( + groupId, + key -> { + try { + return new DeletionResourceManager(groupId); + } catch (IOException e) { + LOGGER.error("Failed to initialize DeletionResourceManager", e); + throw new RuntimeException(e); + } + }); } // Only when consensus protocol is PipeConsensus, will this class be initialized. public static void build() { if (DataRegionConsensusImpl.getInstance() instanceof PipeConsensus) { - try { - DeletionResourceManagerHolder.build(); - } catch (IOException e) { - LOGGER.error("Failed to initialize DeletionResourceManager", e); - throw new RuntimeException(e); - } + DeletionResourceManagerHolder.build(); } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java index 14fd657ff041..f70491e5273a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java @@ -68,17 +68,10 @@ public void setDeletionResource(DeletionResource deletionResource) { this.deletionResource = deletionResource; } - @Override - public boolean internallyIncreaseResourceReferenceCount(String holderMessage) { - if (deletionResource != null) { - deletionResource.persistSelf(); - } - return true; - } - @Override public boolean internallyDecreaseResourceReferenceCount(String holderMessage) { if (deletionResource != null) { + // Trigger hook function. deletionResource.removeSelf(); // Resolve circular reference to let GC reclaim them all. deletionResource.releaseSelf(); 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 677585ce91c9..4e8122016b56 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 @@ -37,11 +37,11 @@ public class SimpleProgressIndex extends ProgressIndex { private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); private final int rebootTimes; - private final long memtableFlushOrderId; + private final long memTableFlushOrderId; public SimpleProgressIndex(int rebootTimes, long memtableFlushOrderId) { this.rebootTimes = rebootTimes; - this.memtableFlushOrderId = memtableFlushOrderId; + this.memTableFlushOrderId = memtableFlushOrderId; } @Override @@ -51,7 +51,7 @@ public void serialize(ByteBuffer byteBuffer) { ProgressIndexType.SIMPLE_PROGRESS_INDEX.serialize(byteBuffer); ReadWriteIOUtils.write(rebootTimes, byteBuffer); - ReadWriteIOUtils.write(memtableFlushOrderId, byteBuffer); + ReadWriteIOUtils.write(memTableFlushOrderId, byteBuffer); } finally { lock.readLock().unlock(); } @@ -64,7 +64,7 @@ public void serialize(OutputStream stream) throws IOException { ProgressIndexType.SIMPLE_PROGRESS_INDEX.serialize(stream); ReadWriteIOUtils.write(rebootTimes, stream); - ReadWriteIOUtils.write(memtableFlushOrderId, stream); + ReadWriteIOUtils.write(memTableFlushOrderId, stream); } finally { lock.readLock().unlock(); } @@ -95,8 +95,8 @@ public boolean isAfter(@Nonnull ProgressIndex progressIndex) { return false; } // thisSimpleProgressIndex.rebootTimes == thatSimpleProgressIndex.rebootTimes - return thisSimpleProgressIndex.memtableFlushOrderId - > thatSimpleProgressIndex.memtableFlushOrderId; + return thisSimpleProgressIndex.memTableFlushOrderId + > thatSimpleProgressIndex.memTableFlushOrderId; } finally { lock.readLock().unlock(); } @@ -113,8 +113,8 @@ public boolean equals(ProgressIndex progressIndex) { final SimpleProgressIndex thisSimpleProgressIndex = this; final SimpleProgressIndex thatSimpleProgressIndex = (SimpleProgressIndex) progressIndex; return thisSimpleProgressIndex.rebootTimes == thatSimpleProgressIndex.rebootTimes - && thisSimpleProgressIndex.memtableFlushOrderId - == thatSimpleProgressIndex.memtableFlushOrderId; + && thisSimpleProgressIndex.memTableFlushOrderId + == thatSimpleProgressIndex.memTableFlushOrderId; } finally { lock.readLock().unlock(); } @@ -156,12 +156,12 @@ public ProgressIndex updateToMinimumEqualOrIsAfterProgressIndex(ProgressIndex pr return progressIndex; } // thisSimpleProgressIndex.rebootTimes == thatSimpleProgressIndex.rebootTimes - if (thisSimpleProgressIndex.memtableFlushOrderId - > thatSimpleProgressIndex.memtableFlushOrderId) { + if (thisSimpleProgressIndex.memTableFlushOrderId + > thatSimpleProgressIndex.memTableFlushOrderId) { return this; } - if (thisSimpleProgressIndex.memtableFlushOrderId - < thatSimpleProgressIndex.memtableFlushOrderId) { + if (thisSimpleProgressIndex.memTableFlushOrderId + < thatSimpleProgressIndex.memTableFlushOrderId) { return progressIndex; } // thisSimpleProgressIndex.memtableFlushOrderId == @@ -178,7 +178,7 @@ public ProgressIndexType getType() { @Override public TotalOrderSumTuple getTotalOrderSumTuple() { - return new TotalOrderSumTuple(memtableFlushOrderId, (long) rebootTimes); + return new TotalOrderSumTuple(memTableFlushOrderId, (long) rebootTimes); } public static SimpleProgressIndex deserializeFrom(ByteBuffer byteBuffer) { @@ -193,13 +193,21 @@ public static SimpleProgressIndex deserializeFrom(InputStream stream) throws IOE return new SimpleProgressIndex(rebootTimes, memtableFlushOrderId); } + public int getRebootTimes() { + return rebootTimes; + } + + public long getMemTableFlushOrderId() { + return memTableFlushOrderId; + } + @Override public String toString() { return "SimpleProgressIndex{" + "rebootTimes=" + rebootTimes + ", memtableFlushOrderId=" - + memtableFlushOrderId + + memTableFlushOrderId + '}'; } } From 35d893d1251747a7f4916ccd5f664cb3276fcdce Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Fri, 30 Aug 2024 09:35:28 +0800 Subject: [PATCH 06/85] sync, serialize and recover for deletion ahead log --- .../deletion/persist/DeletionBuffer.java | 353 ++++++++++++++++++ .../deletion/recover/DeletionReader.java | 74 ++++ .../iotdb/commons/concurrent/ThreadName.java | 3 + 3 files changed, 430 insertions(+) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java new file mode 100644 index 000000000000..f263a504d01f --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java @@ -0,0 +1,353 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.consensus.deletion.persist; + +import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory; +import org.apache.iotdb.commons.concurrent.ThreadName; +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.RecoverProgressIndex; +import org.apache.iotdb.commons.consensus.index.impl.SimpleProgressIndex; +import org.apache.iotdb.db.conf.IoTDBConfig; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; +import org.apache.iotdb.db.utils.MmapUtil; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.charset.StandardCharsets; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +public class DeletionBuffer implements AutoCloseable { + private static final Logger LOGGER = LoggerFactory.getLogger(DeletionBuffer.class); + private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig(); + // TODO: make it deletion own + private static final int ONE_THIRD_WAL_BUFFER_SIZE = config.getWalBufferSize() / 3; + private static final double FSYNC_BUFFER_RATIO = 0.95; + private static final int QUEUE_CAPACITY = config.getWalBufferQueueCapacity(); + + // whether close method is called + private volatile boolean isClosed = false; + // DeletionResources + private final BlockingQueue deletionResources = + new ArrayBlockingQueue<>(QUEUE_CAPACITY); + // lock to provide synchronization for double buffers mechanism, protecting buffers status + private final Lock buffersLock = new ReentrantLock(); + // condition to guarantee correctness of switching buffers + private final Condition idleBufferReadyCondition = buffersLock.newCondition(); + private final String groupId; + + // region these variables should be protected by buffersLock + /** two buffers switch between three statuses (there is always 1 buffer working). */ + // buffer in working status, only updated by serializeThread + // it's safe to use volatile here to make this reference thread-safe. + @SuppressWarnings("squid:S3077") + private volatile ByteBuffer workingBuffer; + + // buffer in idle status + // it's safe to use volatile here to make this reference thread-safe. + @SuppressWarnings("squid:S3077") + private volatile ByteBuffer idleBuffer; + + // buffer in syncing status, serializeThread makes sure no more writes to syncingBuffer + // it's safe to use volatile here to make this reference thread-safe. + @SuppressWarnings("squid:S3077") + private volatile ByteBuffer syncingBuffer; + + // single thread to serialize WALEntry to workingBuffer + private final ExecutorService serializeThread; + // single thread to sync syncingBuffer to disk + private final ExecutorService syncBufferThread; + // directory to store .deletion files + private final String baseDirectory; + + public DeletionBuffer(String groupId, String baseDirectory) { + this.groupId = groupId; + this.baseDirectory = baseDirectory; + allocateBuffers(); + serializeThread = + IoTDBThreadPoolFactory.newSingleThreadExecutor( + ThreadName.PIPE_CONSENSUS_DELETION_SERIALIZE.getName() + "(group-" + groupId + ")"); + syncBufferThread = + IoTDBThreadPoolFactory.newSingleThreadExecutor( + ThreadName.PIPE_CONSENSUS_DELETION_SYNC.getName() + "(group-" + groupId + ")"); + } + + public void start() { + // start receiving serialize tasks + serializeThread.submit(new SerializeTask()); + } + + public void registerDeletionResource(DeletionResource deletionResource) { + if (isClosed) { + LOGGER.error( + "Fail to register DeletionResource into deletionBuffer-{} because this buffer is closed.", + groupId); + return; + } + deletionResources.add(deletionResource); + } + + private void allocateBuffers() { + try { + workingBuffer = ByteBuffer.allocateDirect(ONE_THIRD_WAL_BUFFER_SIZE); + idleBuffer = ByteBuffer.allocateDirect(ONE_THIRD_WAL_BUFFER_SIZE); + } catch (OutOfMemoryError e) { + LOGGER.error( + "Fail to allocate deletionBuffer-group-{}'s buffer because out of memory.", groupId, e); + close(); + throw e; + } + } + + /** Notice: this method only called when buffer is exhausted by SerializeTask. */ + private void syncWorkingBuffer(ProgressIndex maxProgressIndexInCurrentBatch, int deletionNum) { + switchWorkingBufferToFlushing(); + try { + syncBufferThread.submit(new SyncBufferTask(maxProgressIndexInCurrentBatch, deletionNum)); + } catch (IOException e) { + LOGGER.warn( + "Failed to submit syncBufferTask, May because file open error and cause data inconsistency. Please check your file system. ", + e); + } + } + + // only called by serializeThread + private void switchWorkingBufferToFlushing() { + buffersLock.lock(); + try { + while (idleBuffer == null) { + idleBufferReadyCondition.await(); + } + syncingBuffer = workingBuffer; + workingBuffer = idleBuffer; + workingBuffer.clear(); + idleBuffer = null; + } catch (InterruptedException e) { + LOGGER.warn("Interrupted When waiting for available working buffer."); + Thread.currentThread().interrupt(); + } finally { + buffersLock.unlock(); + } + } + + private class SerializeTask implements Runnable { + // Total size of this batch. + private int totalSize = 0; + // Deletion num of this batch. + private int deletionNum = 0; + // Max progressIndex among this batch. Used by SyncTask for naming .deletion file. + private ProgressIndex maxProgressIndexInCurrentBatch = MinimumProgressIndex.INSTANCE; + + @Override + public void run() { + try { + serialize(); + } finally { + if (!isClosed) { + serializeThread.submit(new SerializeTask()); + } + } + } + + private void serialize() { + // For first deletion we use blocking take() method. + try { + DeletionResource firstDeletionResource = deletionResources.take(); + // For first serialization, we don't need to judge whether working buffer is exhausted. + // Because a single DeleteDataNode can't exceed size of working buffer. + serializeToWorkingBuffer(firstDeletionResource); + maxProgressIndexInCurrentBatch.updateToMinimumEqualOrIsAfterProgressIndex( + firstDeletionResource.getProgressIndex()); + } catch (InterruptedException e) { + LOGGER.warn( + "Interrupted when waiting for taking DeletionResource from blocking queue to serialize."); + Thread.currentThread().interrupt(); + } + + // For further deletion, we use non-blocking poll() method to persist existing deletion of + // current batch in time. + while (totalSize < ONE_THIRD_WAL_BUFFER_SIZE * FSYNC_BUFFER_RATIO) { + DeletionResource deletionResource = null; + try { + // TODO: add deletion timeout to config + deletionResource = + deletionResources.poll(config.getWalSyncModeFsyncDelayInMs(), TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + LOGGER.warn( + "Interrupted when waiting for taking WALEntry from blocking queue to serialize."); + Thread.currentThread().interrupt(); + } + // If timeout, flush deletions to disk. + if (deletionResource == null) { + break; + } + // Serialize deletion + while (!serializeToWorkingBuffer(deletionResource)) { + // If working buffer is exhausted, submit a syncTask to consume current batch and switch + // buffer to start a new batch. + syncWorkingBuffer(maxProgressIndexInCurrentBatch, deletionNum); + // Reset maxProgressIndex and deletionNum for new batch. + maxProgressIndexInCurrentBatch = MinimumProgressIndex.INSTANCE; + deletionNum = 0; + } + // Update max progressIndex + maxProgressIndexInCurrentBatch.updateToMinimumEqualOrIsAfterProgressIndex( + deletionResource.getProgressIndex()); + } + // Persist deletions; Defensive programming here, just in case. + if (totalSize > 0) { + syncWorkingBuffer(maxProgressIndexInCurrentBatch, deletionNum); + } + } + + /** + * Serialize deletionResource to working buffer. Return true if serialize successfully, false + * otherwise. + */ + private boolean serializeToWorkingBuffer(DeletionResource deletionResource) { + ByteBuffer buffer = deletionResource.serialize(); + // if working buffer doesn't have enough space + if (buffer.position() > workingBuffer.remaining()) { + return false; + } + workingBuffer.put(buffer); + totalSize += buffer.position(); + deletionNum++; + return true; + } + } + + // only called by syncBufferThread + private void switchSyncingBufferToIdle() { + buffersLock.lock(); + try { + // No need to judge whether idleBuffer is null because syncingBuffer is not null + // and there is only one buffer can be null between syncingBuffer and idleBuffer + idleBuffer = syncingBuffer; + syncingBuffer = null; + idleBufferReadyCondition.signalAll(); + } finally { + buffersLock.unlock(); + } + } + + private class SyncBufferTask implements Runnable { + private final int deletionNum; + private final File logFile; + private final FileOutputStream logStream; + private final FileChannel logChannel; + + public SyncBufferTask(ProgressIndex maxProgressIndexInCurrentBatch, int deletionNum) + throws IOException { + this.deletionNum = deletionNum; + // PipeConsensus ensures that deleteDataNodes use recoverProgressIndex. + RecoverProgressIndex maxProgressIndex = (RecoverProgressIndex) maxProgressIndexInCurrentBatch; + int thisDataNodeId = IoTDBDescriptor.getInstance().getConfig().getDataNodeId(); + SimpleProgressIndex progressIndex = + maxProgressIndex.getDataNodeId2LocalIndex().get(thisDataNodeId); + // Deletion file name format: "_{rebootTimes}_{memTableFlushOrderId}.deletion" + this.logFile = + new File( + baseDirectory, + String.format( + "_%d_%d%s", + progressIndex.getRebootTimes(), + progressIndex.getMemTableFlushOrderId(), + DeletionResourceManager.DELETION_FILE_SUFFIX)); + this.logStream = new FileOutputStream(logFile, true); + this.logChannel = logStream.getChannel(); + // Create file && write magic string + if (!logFile.exists() || logFile.length() == 0) { + this.logChannel.write( + ByteBuffer.wrap( + DeletionResourceManager.MAGIC_VERSION_V1.getBytes(StandardCharsets.UTF_8))); + } + } + + @Override + public void run() { + // Sync deletion to disk. + workingBuffer.flip(); + try { + // Write metaData. + ByteBuffer metaData = ByteBuffer.allocate(4); + metaData.putInt(deletionNum); + this.logChannel.write(metaData); + // Write deletions. + this.logChannel.write(workingBuffer); + } catch (IOException e) { + LOGGER.warn( + "Deletion persist: Cannot write to {}, may cause data inconsistency.", logFile, e); + } finally { + switchSyncingBufferToIdle(); + } + // Close resource. + try { + this.logChannel.close(); + this.logStream.close(); + } catch (IOException e) { + LOGGER.warn("Failed to close deletion writing resource when writing to {}.", logFile, e); + } + } + } + + @Override + public void close() { + isClosed = true; + // Force sync existing data in memory to disk. + // first waiting serialize and sync tasks finished, then release all resources + if (serializeThread != null) { + shutdownThread(serializeThread, ThreadName.PIPE_CONSENSUS_DELETION_SERIALIZE); + } + if (syncBufferThread != null) { + shutdownThread(syncBufferThread, ThreadName.PIPE_CONSENSUS_DELETION_SYNC); + } + + MmapUtil.clean(workingBuffer); + MmapUtil.clean(workingBuffer); + MmapUtil.clean(syncingBuffer); + } + + private void shutdownThread(ExecutorService thread, ThreadName threadName) { + thread.shutdown(); + try { + if (!thread.awaitTermination(30, TimeUnit.SECONDS)) { + LOGGER.warn("Waiting thread {} to be terminated is timeout", threadName.getName()); + } + } catch (InterruptedException e) { + LOGGER.warn("Thread {} still doesn't exit after 30s", threadName.getName()); + Thread.currentThread().interrupt(); + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java new file mode 100644 index 000000000000..e3ad133e694a --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.consensus.deletion.recover; + +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.util.ArrayList; +import java.util.List; +import java.util.function.Consumer; + +public class DeletionReader implements Closeable { + private static final Logger LOGGER = LoggerFactory.getLogger(DeletionReader.class); + private final Consumer removeHook; + private final FileInputStream fileInputStream; + private final FileChannel fileChannel; + + public DeletionReader(File logFile, Consumer removeHook) throws IOException { + this.fileInputStream = new FileInputStream(logFile); + this.fileChannel = fileInputStream.getChannel(); + this.removeHook = removeHook; + } + + public List readAllDeletions() throws IOException { + // Read metaData + ByteBuffer intBuffer = ByteBuffer.allocate(4); + fileChannel.read(intBuffer); + intBuffer.flip(); // 切换到读模式 + int deletionNum = intBuffer.getInt(); + + // Read deletions + long remainingBytes = fileChannel.size() - fileChannel.position(); + ByteBuffer byteBuffer = ByteBuffer.allocate((int) remainingBytes); + fileChannel.read(byteBuffer); + byteBuffer.flip(); + + List deletions = new ArrayList<>(); + for (int i = 0; i < deletionNum; i++) { + deletions.add(DeletionResource.deserialize(byteBuffer, removeHook)); + } + return deletions; + } + + @Override + public void close() throws IOException { + this.fileInputStream.close(); + this.fileChannel.close(); + } +} diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java index 00701d163e69..3e3821fc3343 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java @@ -102,6 +102,9 @@ public enum ThreadName { PIPE_CONSENSUS_RPC_SERVICE("PipeConsensusRPC-Service"), PIPE_CONSENSUS_RPC_PROCESSOR("PipeConsensusRPC-Processor"), ASYNC_DATANODE_PIPE_CONSENSUS_CLIENT_POOL("AsyncDataNodePipeConsensusServiceClientPool"), + PIPE_CONSENSUS_DELETION_SERIALIZE("WAL-Serialize"), + PIPE_CONSENSUS_DELETION_SYNC("WAL-Sync"), + PIPE_CONSENSUS_DELETION_DELETE("WAL-Delete"), // -------------------------- IoTConsensus -------------------------- IOT_CONSENSUS_RPC_SERVICE("IoTConsensusRPC-Service"), From 17aaa74cee51f9782b917a06cd68308b1fa930f6 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Fri, 30 Aug 2024 09:36:02 +0800 Subject: [PATCH 07/85] remove useless --- .../java/org/apache/iotdb/commons/concurrent/ThreadName.java | 1 - 1 file changed, 1 deletion(-) diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java index 3e3821fc3343..a73e0756625a 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ThreadName.java @@ -104,7 +104,6 @@ public enum ThreadName { ASYNC_DATANODE_PIPE_CONSENSUS_CLIENT_POOL("AsyncDataNodePipeConsensusServiceClientPool"), PIPE_CONSENSUS_DELETION_SERIALIZE("WAL-Serialize"), PIPE_CONSENSUS_DELETION_SYNC("WAL-Sync"), - PIPE_CONSENSUS_DELETION_DELETE("WAL-Delete"), // -------------------------- IoTConsensus -------------------------- IOT_CONSENSUS_RPC_SERVICE("IoTConsensusRPC-Service"), From 464191be0f5ed51911b35bdb53183dbebb1f52dc Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Fri, 30 Aug 2024 09:55:09 +0800 Subject: [PATCH 08/85] fix --- .../ProgressIndexDataNodeManager.java | 48 +++++++++---------- .../deletion/DeletionResourceManager.java | 46 ++++++++++-------- .../deletion/persist/DeletionBuffer.java | 13 +++-- .../realtime/PipeRealtimeEventFactory.java | 6 +-- .../PipeInsertionDataNodeListener.java | 5 +- .../storageengine/dataregion/DataRegion.java | 9 ++-- 6 files changed, 65 insertions(+), 62 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ProgressIndexDataNodeManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ProgressIndexDataNodeManager.java index 2f50365f07b4..c492bef3bd73 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ProgressIndexDataNodeManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ProgressIndexDataNodeManager.java @@ -50,6 +50,30 @@ public ProgressIndexDataNodeManager() { recoverMaxProgressIndexFromDataRegion(); } + public static ProgressIndex extractLocalSimpleProgressIndex(ProgressIndex progressIndex) { + if (progressIndex instanceof RecoverProgressIndex) { + final Map dataNodeId2LocalIndex = + ((RecoverProgressIndex) progressIndex).getDataNodeId2LocalIndex(); + return dataNodeId2LocalIndex.containsKey(DATA_NODE_ID) + ? dataNodeId2LocalIndex.get(DATA_NODE_ID) + : MinimumProgressIndex.INSTANCE; + } else if (progressIndex instanceof HybridProgressIndex) { + final Map type2Index = + ((HybridProgressIndex) progressIndex).getType2Index(); + if (!type2Index.containsKey(ProgressIndexType.RECOVER_PROGRESS_INDEX.getType())) { + return MinimumProgressIndex.INSTANCE; + } + final Map dataNodeId2LocalIndex = + ((RecoverProgressIndex) + type2Index.get(ProgressIndexType.RECOVER_PROGRESS_INDEX.getType())) + .getDataNodeId2LocalIndex(); + return dataNodeId2LocalIndex.containsKey(DATA_NODE_ID) + ? dataNodeId2LocalIndex.get(DATA_NODE_ID) + : MinimumProgressIndex.INSTANCE; + } + return MinimumProgressIndex.INSTANCE; + } + private void recoverMaxProgressIndexFromDataRegion() { StorageEngine.getInstance() .getAllDataRegionIds() @@ -86,30 +110,6 @@ private void recoverMaxProgressIndexFromDataRegion() { // TODO: update deletion progress index } - private ProgressIndex extractLocalSimpleProgressIndex(ProgressIndex progressIndex) { - if (progressIndex instanceof RecoverProgressIndex) { - final Map dataNodeId2LocalIndex = - ((RecoverProgressIndex) progressIndex).getDataNodeId2LocalIndex(); - return dataNodeId2LocalIndex.containsKey(DATA_NODE_ID) - ? dataNodeId2LocalIndex.get(DATA_NODE_ID) - : MinimumProgressIndex.INSTANCE; - } else if (progressIndex instanceof HybridProgressIndex) { - final Map type2Index = - ((HybridProgressIndex) progressIndex).getType2Index(); - if (!type2Index.containsKey(ProgressIndexType.RECOVER_PROGRESS_INDEX.getType())) { - return MinimumProgressIndex.INSTANCE; - } - final Map dataNodeId2LocalIndex = - ((RecoverProgressIndex) - type2Index.get(ProgressIndexType.RECOVER_PROGRESS_INDEX.getType())) - .getDataNodeId2LocalIndex(); - return dataNodeId2LocalIndex.containsKey(DATA_NODE_ID) - ? dataNodeId2LocalIndex.get(DATA_NODE_ID) - : MinimumProgressIndex.INSTANCE; - } - return MinimumProgressIndex.INSTANCE; - } - @Override public ProgressIndex getProgressIndex(ConsensusPipeName consensusPipeName) { return PipeDataNodeAgent.task() diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java index ae3471f74f9f..45fa572decab 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java @@ -19,12 +19,12 @@ package org.apache.iotdb.db.pipe.consensus.deletion; -import org.apache.iotdb.commons.consensus.index.impl.RecoverProgressIndex; +import org.apache.iotdb.commons.consensus.index.ProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.SimpleProgressIndex; import org.apache.iotdb.commons.utils.FileUtils; import org.apache.iotdb.consensus.pipe.PipeConsensus; -import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.consensus.DataRegionConsensusImpl; +import org.apache.iotdb.db.pipe.consensus.ProgressIndexDataNodeManager; import org.apache.iotdb.db.pipe.consensus.deletion.persist.DeletionBuffer; import org.apache.iotdb.db.pipe.consensus.deletion.recover.DeletionReader; import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; @@ -122,11 +122,9 @@ private void removeDeletionResource(DeletionResource deletionResource) { // Clean memory deletionResources.remove(deletionResource); // Clean disk - int thisDataNodeId = IoTDBDescriptor.getInstance().getConfig().getDataNodeId(); - SimpleProgressIndex currentProgressIndex = - ((RecoverProgressIndex) deletionResource.getProgressIndex()) - .getDataNodeId2LocalIndex() - .get(thisDataNodeId); + ProgressIndex currentProgressIndex = + ProgressIndexDataNodeManager.extractLocalSimpleProgressIndex( + deletionResource.getProgressIndex()); try (Stream pathStream = Files.walk(Paths.get(storageDir.getPath()), 1)) { Path[] deletionPaths = @@ -149,20 +147,22 @@ private void removeDeletionResource(DeletionResource deletionResource) { } } - private boolean isCurrentFileCanBeDeleted( - String fileName, SimpleProgressIndex currentProgressIndex) { - int curRebootTimes = currentProgressIndex.getRebootTimes(); - long curMemTableFlushOrderId = currentProgressIndex.getMemTableFlushOrderId(); - - Pattern pattern = Pattern.compile(DELETION_FILE_NAME_PATTERN); - Matcher matcher = pattern.matcher(fileName); - // Definitely match. Because upper caller has filtered fileNames. - if (matcher.matches()) { - int fileRebootTimes = Integer.parseInt(matcher.group(REBOOT_TIME)); - long fileMemTableFlushOrderId = Long.parseLong(matcher.group(MEM_TABLE_FLUSH_ORDER)); - return fileRebootTimes == curRebootTimes - ? fileMemTableFlushOrderId < curMemTableFlushOrderId - : fileRebootTimes < curMemTableFlushOrderId; + private boolean isCurrentFileCanBeDeleted(String fileName, ProgressIndex currentProgressIndex) { + if (currentProgressIndex instanceof SimpleProgressIndex) { + SimpleProgressIndex simpleProgressIndex = (SimpleProgressIndex) currentProgressIndex; + int curRebootTimes = simpleProgressIndex.getRebootTimes(); + long curMemTableFlushOrderId = simpleProgressIndex.getMemTableFlushOrderId(); + + Pattern pattern = Pattern.compile(DELETION_FILE_NAME_PATTERN); + Matcher matcher = pattern.matcher(fileName); + // Definitely match. Because upper caller has filtered fileNames. + if (matcher.matches()) { + int fileRebootTimes = Integer.parseInt(matcher.group(REBOOT_TIME)); + long fileMemTableFlushOrderId = Long.parseLong(matcher.group(MEM_TABLE_FLUSH_ORDER)); + return fileRebootTimes == curRebootTimes + ? fileMemTableFlushOrderId < curMemTableFlushOrderId + : fileRebootTimes < curMemTableFlushOrderId; + } } return false; } @@ -181,6 +181,10 @@ public static void build() { } public static DeletionResourceManager getInstance(String groupId) { + // If consensusImpl is not PipeConsensus. + if (DeletionResourceManagerHolder.CONSENSU_GROUP_ID_2_INSTANCE_MAP == null) { + return null; + } return DeletionResourceManagerHolder.CONSENSU_GROUP_ID_2_INSTANCE_MAP.computeIfAbsent( groupId, key -> { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java index f263a504d01f..720094ec6d73 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java @@ -23,10 +23,10 @@ import org.apache.iotdb.commons.concurrent.ThreadName; 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.RecoverProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.SimpleProgressIndex; import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.pipe.consensus.ProgressIndexDataNodeManager; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; import org.apache.iotdb.db.utils.MmapUtil; @@ -272,10 +272,13 @@ public SyncBufferTask(ProgressIndex maxProgressIndexInCurrentBatch, int deletion throws IOException { this.deletionNum = deletionNum; // PipeConsensus ensures that deleteDataNodes use recoverProgressIndex. - RecoverProgressIndex maxProgressIndex = (RecoverProgressIndex) maxProgressIndexInCurrentBatch; - int thisDataNodeId = IoTDBDescriptor.getInstance().getConfig().getDataNodeId(); - SimpleProgressIndex progressIndex = - maxProgressIndex.getDataNodeId2LocalIndex().get(thisDataNodeId); + ProgressIndex curProgressIndex = + ProgressIndexDataNodeManager.extractLocalSimpleProgressIndex( + maxProgressIndexInCurrentBatch); + if (!(curProgressIndex instanceof SimpleProgressIndex)) { + throw new IOException("Invalid deletion progress index: " + curProgressIndex); + } + SimpleProgressIndex progressIndex = (SimpleProgressIndex) curProgressIndex; // Deletion file name format: "_{rebootTimes}_{memTableFlushOrderId}.deletion" this.logFile = new File( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java index b328b762d5c6..8747529a8fde 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java @@ -35,8 +35,6 @@ public class PipeRealtimeEventFactory { private static final TsFileEpochManager TS_FILE_EPOCH_MANAGER = new TsFileEpochManager(); - private static final DeletionResourceManager DELETION_RESOURCE_MANAGER = - DeletionResourceManager.getInstance(); public static PipeRealtimeEvent createRealtimeEvent( TsFileResource resource, boolean isLoaded, boolean isGeneratedByPipe) { @@ -63,10 +61,10 @@ public static PipeRealtimeEvent createRealtimeEvent( new PipeHeartbeatEvent(dataRegionId, shouldPrintMessage), null, null, null); } - public static PipeRealtimeEvent createRealtimeEvent(DeleteDataNode node) { + public static PipeRealtimeEvent createRealtimeEvent(DeleteDataNode node, String regionId) { PipeSchemaRegionWritePlanEvent deletionEvent = new PipeSchemaRegionWritePlanEvent(node, node.isGeneratedByPipe()); - Optional.ofNullable(DELETION_RESOURCE_MANAGER) + Optional.ofNullable(DeletionResourceManager.getInstance(regionId)) .ifPresent(mgr -> mgr.registerDeletionResource(deletionEvent)); return new PipeRealtimeEvent(deletionEvent, null, null, null); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java index 3eb118701a78..371cc379eae3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java @@ -141,9 +141,10 @@ public void listenToHeartbeat(boolean shouldPrintMessage) { PipeRealtimeEventFactory.createRealtimeEvent(key, shouldPrintMessage))); } - public void listenToDeleteData(DeleteDataNode node) { + public void listenToDeleteData(DeleteDataNode node, String regionId) { dataRegionId2Assigner.forEach( - (key, value) -> value.publishToAssign(PipeRealtimeEventFactory.createRealtimeEvent(node))); + (key, value) -> + value.publishToAssign(PipeRealtimeEventFactory.createRealtimeEvent(node, regionId))); } /////////////////////////////// singleton /////////////////////////////// diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java index fdcf6e5e2549..276a00b9263f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java @@ -2078,7 +2078,7 @@ public void deleteByDevice(PartialPath pattern, DeleteDataNode node) throws IOEx // deviceMatchInfo contains the DeviceId means this device matched the pattern Set deviceMatchInfo = new HashSet<>(); deleteDataInFiles(unsealedTsFileResource, deletion, devicePaths, deviceMatchInfo); - PipeInsertionDataNodeListener.getInstance().listenToDeleteData(node); + PipeInsertionDataNodeListener.getInstance().listenToDeleteData(node, dataRegionId); writeUnlock(); hasReleasedLock = true; @@ -2121,7 +2121,7 @@ public void deleteDataDirectly(PartialPath pathToDelete, DeleteDataNode node) th List unsealedTsFileResource = new ArrayList<>(); getTwoKindsOfTsFiles(sealedTsFileResource, unsealedTsFileResource, startTime, endTime); deleteDataDirectlyInFile(unsealedTsFileResource, pathToDelete, startTime, endTime); - PipeInsertionDataNodeListener.getInstance().listenToDeleteData(node); + PipeInsertionDataNodeListener.getInstance().listenToDeleteData(node, dataRegionId); writeUnlock(); releasedLock = true; deleteDataDirectlyInFile(sealedTsFileResource, pathToDelete, startTime, endTime); @@ -2159,7 +2159,6 @@ private List logDeletionInWAL( // but it's still necessary to write to the WAL, so that iotconsensus can synchronize the delete // operation to other nodes. if (walFlushListeners.isEmpty()) { - // TODO: IoTConsensusV2 deletion support getWALNode() .ifPresent( walNode -> @@ -3477,9 +3476,7 @@ public void insertTablets(InsertMultiTabletsNode insertMultiTabletsNode) } } - /** - * @return the disk space occupied by this data region, unit is MB - */ + /** @return the disk space occupied by this data region, unit is MB */ public long countRegionDiskSize() { AtomicLong diskSize = new AtomicLong(0); TierManager.getInstance() From e66ed1076ce3a2cd7b08b599092906275641dd3f Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Fri, 30 Aug 2024 09:58:41 +0800 Subject: [PATCH 09/85] fix --- .../db/pipe/consensus/deletion/recover/DeletionReader.java | 2 +- .../apache/iotdb/db/storageengine/dataregion/DataRegion.java | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java index e3ad133e694a..392e0381acf8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java @@ -50,7 +50,7 @@ public List readAllDeletions() throws IOException { // Read metaData ByteBuffer intBuffer = ByteBuffer.allocate(4); fileChannel.read(intBuffer); - intBuffer.flip(); // 切换到读模式 + intBuffer.flip(); int deletionNum = intBuffer.getInt(); // Read deletions diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java index 276a00b9263f..994db070b6d7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java @@ -3476,7 +3476,9 @@ public void insertTablets(InsertMultiTabletsNode insertMultiTabletsNode) } } - /** @return the disk space occupied by this data region, unit is MB */ + /** + * @return the disk space occupied by this data region, unit is MB + */ public long countRegionDiskSize() { AtomicLong diskSize = new AtomicLong(0); TierManager.getInstance() From 8d32e06722a2bf960ca8c3faff6637073b3123cd Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Fri, 30 Aug 2024 12:38:18 +0800 Subject: [PATCH 10/85] historical extraction for deletion --- .../consensus/deletion/DeletionResource.java | 4 + .../deletion/persist/DeletionBuffer.java | 2 +- .../dataregion/IoTDBDataRegionExtractor.java | 4 +- ...DataRegionTsFileAndDeletionExtractor.java} | 330 +++++++++++------- 4 files changed, 208 insertions(+), 132 deletions(-) rename iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/{PipeHistoricalDataRegionTsFileExtractor.java => PipeHistoricalDataRegionTsFileAndDeletionExtractor.java} (73%) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java index 4229c3f37d66..c60a3a3a1443 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java @@ -79,6 +79,10 @@ public ProgressIndex getProgressIndex() { return ((DeleteDataNode) deletionEvent.getPlanNode()).getProgressIndex(); } + public PipeSchemaRegionWritePlanEvent getDeletionEvent() { + return deletionEvent; + } + public ByteBuffer serialize() { return deletionEvent.serializeToByteBuffer(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java index 720094ec6d73..5ba849972938 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java @@ -104,7 +104,7 @@ public DeletionBuffer(String groupId, String baseDirectory) { } public void start() { - // start receiving serialize tasks + // Start serialize and sync pipeline. serializeThread.submit(new SerializeTask()); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java index 0b447ffe7a4f..bc451c4c361a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/IoTDBDataRegionExtractor.java @@ -28,7 +28,7 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; import org.apache.iotdb.db.pipe.extractor.dataregion.historical.PipeHistoricalDataRegionExtractor; -import org.apache.iotdb.db.pipe.extractor.dataregion.historical.PipeHistoricalDataRegionTsFileExtractor; +import org.apache.iotdb.db.pipe.extractor.dataregion.historical.PipeHistoricalDataRegionTsFileAndDeletionExtractor; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionExtractor; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionHeartbeatExtractor; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionHybridExtractor; @@ -236,7 +236,7 @@ private void validatePattern(final PipePattern pattern) { private void constructHistoricalExtractor() { // Enable historical extractor by default - historicalExtractor = new PipeHistoricalDataRegionTsFileExtractor(); + historicalExtractor = new PipeHistoricalDataRegionTsFileAndDeletionExtractor(); } private void constructRealtimeExtractor(final PipeParameters parameters) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java similarity index 73% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java index bbf34892e42e..360a669b5586 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java @@ -29,6 +29,9 @@ import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskExtractorRuntimeEnvironment; import org.apache.iotdb.commons.pipe.pattern.PipePattern; import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; +import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.db.pipe.event.common.terminate.PipeTerminateEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.pipe.extractor.dataregion.DataRegionListeningFilter; @@ -86,10 +89,11 @@ import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.SOURCE_MODS_ENABLE_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.SOURCE_START_TIME_KEY; -public class PipeHistoricalDataRegionTsFileExtractor implements PipeHistoricalDataRegionExtractor { +public class PipeHistoricalDataRegionTsFileAndDeletionExtractor + implements PipeHistoricalDataRegionExtractor { private static final Logger LOGGER = - LoggerFactory.getLogger(PipeHistoricalDataRegionTsFileExtractor.class); + LoggerFactory.getLogger(PipeHistoricalDataRegionTsFileAndDeletionExtractor.class); private static final Map DATA_REGION_ID_TO_PIPE_FLUSHED_TIME_MAP = new HashMap<>(); private static final long PIPE_MIN_FLUSH_INTERVAL_IN_MS = 2000; @@ -120,7 +124,10 @@ public class PipeHistoricalDataRegionTsFileExtractor implements PipeHistoricalDa private boolean shouldTerminatePipeOnAllHistoricalEventsConsumed; private boolean isTerminateSignalSent = false; - private Queue pendingQueue; + private boolean isTsFileEventAllConsumed = false; + + private Queue pendingTsFileQueue; + private Queue pendingDeletionQueue; @Override public void validate(final PipeParameterValidator validator) { @@ -367,6 +374,129 @@ private void flushDataRegionAllTsFiles() { } } + private void extractDeletions(final DeletionResourceManager deletionResourceManager) { + LOGGER.info("Pipe {}@{}: start to extract deletions", pipeName, dataRegionId); + List resourceList = deletionResourceManager.getAllDeletionResources(); + final int originalDeletionCount = resourceList.size(); + resourceList = + resourceList.stream() + .filter(this::mayDeletionUnprocessed) + .sorted((o1, o2) -> o1.getProgressIndex().isAfter(o2.getProgressIndex()) ? 1 : -1) + .collect(Collectors.toList()); + pendingDeletionQueue = new ArrayDeque<>(resourceList); + LOGGER.info( + "Pipe {}@{}: finish to extract deletions, extract deletions count {}/{}", + pipeName, + dataRegionId, + resourceList.size(), + originalDeletionCount); + } + + private void flushTsFilesForExtraction( + DataRegion dataRegion, final long startHistoricalExtractionTime) { + LOGGER.info("Pipe {}@{}: start to flush data region", pipeName, dataRegionId); + synchronized (DATA_REGION_ID_TO_PIPE_FLUSHED_TIME_MAP) { + final long lastFlushedByPipeTime = DATA_REGION_ID_TO_PIPE_FLUSHED_TIME_MAP.get(dataRegionId); + if (System.currentTimeMillis() - lastFlushedByPipeTime >= PIPE_MIN_FLUSH_INTERVAL_IN_MS) { + dataRegion.syncCloseAllWorkingTsFileProcessors(); + DATA_REGION_ID_TO_PIPE_FLUSHED_TIME_MAP.replace(dataRegionId, System.currentTimeMillis()); + LOGGER.info( + "Pipe {}@{}: finish to flush data region, took {} ms", + pipeName, + dataRegionId, + System.currentTimeMillis() - startHistoricalExtractionTime); + } else { + LOGGER.info( + "Pipe {}@{}: skip to flush data region, last flushed time {} ms ago", + pipeName, + dataRegionId, + System.currentTimeMillis() - lastFlushedByPipeTime); + } + } + } + + private void extractTsFiles( + final DataRegion dataRegion, final long startHistoricalExtractionTime) { + final TsFileManager tsFileManager = dataRegion.getTsFileManager(); + tsFileManager.readLock(); + try { + final int originalSequenceTsFileCount = tsFileManager.size(true); + final int originalUnsequenceTsFileCount = tsFileManager.size(false); + final List resourceList = + new ArrayList<>(originalSequenceTsFileCount + originalUnsequenceTsFileCount); + LOGGER.info( + "Pipe {}@{}: start to extract historical TsFile, original sequence file count {}, " + + "original unsequence file count {}, start progress index {}", + pipeName, + dataRegionId, + originalSequenceTsFileCount, + originalUnsequenceTsFileCount, + startIndex); + + final Collection sequenceTsFileResources = + tsFileManager.getTsFileList(true).stream() + .filter( + resource -> + // Some resource may not be closed due to the control of + // PIPE_MIN_FLUSH_INTERVAL_IN_MS. We simply ignore them. + !resource.isClosed() + || mayTsFileContainUnprocessedData(resource) + && isTsFileResourceOverlappedWithTimeRange(resource) + && isTsFileGeneratedAfterExtractionTimeLowerBound(resource) + && mayTsFileResourceOverlappedWithPattern(resource)) + .collect(Collectors.toList()); + resourceList.addAll(sequenceTsFileResources); + + final Collection unsequenceTsFileResources = + tsFileManager.getTsFileList(false).stream() + .filter( + resource -> + // Some resource may not be closed due to the control of + // PIPE_MIN_FLUSH_INTERVAL_IN_MS. We simply ignore them. + !resource.isClosed() + || mayTsFileContainUnprocessedData(resource) + && isTsFileResourceOverlappedWithTimeRange(resource) + && isTsFileGeneratedAfterExtractionTimeLowerBound(resource) + && mayTsFileResourceOverlappedWithPattern(resource)) + .collect(Collectors.toList()); + resourceList.addAll(unsequenceTsFileResources); + + resourceList.forEach( + resource -> { + // Pin the resource, in case the file is removed by compaction or anything. + // Will unpin it after the PipeTsFileInsertionEvent is created and pinned. + try { + PipeDataNodeResourceManager.tsfile() + .pinTsFileResource(resource, shouldTransferModFile); + } catch (final IOException e) { + LOGGER.warn("Pipe: failed to pin TsFileResource {}", resource.getTsFilePath()); + } + }); + + resourceList.sort( + (o1, o2) -> + startIndex instanceof TimeWindowStateProgressIndex + ? Long.compare(o1.getFileStartTime(), o2.getFileStartTime()) + : o1.getMaxProgressIndex().topologicalCompareTo(o2.getMaxProgressIndex())); + pendingTsFileQueue = new ArrayDeque<>(resourceList); + + LOGGER.info( + "Pipe {}@{}: finish to extract historical TsFile, extracted sequence file count {}/{}, " + + "extracted unsequence file count {}/{}, extracted file count {}/{}, took {} ms", + pipeName, + dataRegionId, + sequenceTsFileResources.size(), + originalSequenceTsFileCount, + unsequenceTsFileResources.size(), + originalUnsequenceTsFileCount, + resourceList.size(), + originalSequenceTsFileCount + originalUnsequenceTsFileCount, + System.currentTimeMillis() - startHistoricalExtractionTime); + } finally { + tsFileManager.readUnlock(); + } + } + @Override public synchronized void start() { if (!shouldExtractInsertion) { @@ -376,117 +506,37 @@ public synchronized void start() { final DataRegion dataRegion = StorageEngine.getInstance().getDataRegion(new DataRegionId(dataRegionId)); if (Objects.isNull(dataRegion)) { - pendingQueue = new ArrayDeque<>(); + pendingTsFileQueue = new ArrayDeque<>(); + pendingDeletionQueue = new ArrayDeque<>(); return; } - dataRegion.writeLock("Pipe: start to extract historical TsFile"); + dataRegion.writeLock( + "Pipe: start to extract historical TsFile and Deletion(if uses pipeConsensus)"); + // Extract deletions + DeletionResourceManager deletionResourceManager = + DeletionResourceManager.getInstance(String.valueOf(dataRegionId)); + if (deletionResourceManager == null) { + // If not uses pipeConsensus + pendingDeletionQueue = new ArrayDeque<>(); + } else { + extractDeletions(deletionResourceManager); + } + // Flush TsFiles final long startHistoricalExtractionTime = System.currentTimeMillis(); try { - LOGGER.info("Pipe {}@{}: start to flush data region", pipeName, dataRegionId); - synchronized (DATA_REGION_ID_TO_PIPE_FLUSHED_TIME_MAP) { - final long lastFlushedByPipeTime = - DATA_REGION_ID_TO_PIPE_FLUSHED_TIME_MAP.get(dataRegionId); - if (System.currentTimeMillis() - lastFlushedByPipeTime >= PIPE_MIN_FLUSH_INTERVAL_IN_MS) { - dataRegion.syncCloseAllWorkingTsFileProcessors(); - DATA_REGION_ID_TO_PIPE_FLUSHED_TIME_MAP.replace(dataRegionId, System.currentTimeMillis()); - LOGGER.info( - "Pipe {}@{}: finish to flush data region, took {} ms", - pipeName, - dataRegionId, - System.currentTimeMillis() - startHistoricalExtractionTime); - } else { - LOGGER.info( - "Pipe {}@{}: skip to flush data region, last flushed time {} ms ago", - pipeName, - dataRegionId, - System.currentTimeMillis() - lastFlushedByPipeTime); - } - } - - final TsFileManager tsFileManager = dataRegion.getTsFileManager(); - tsFileManager.readLock(); - try { - final int originalSequenceTsFileCount = tsFileManager.size(true); - final int originalUnsequenceTsFileCount = tsFileManager.size(false); - final List resourceList = - new ArrayList<>(originalSequenceTsFileCount + originalUnsequenceTsFileCount); - LOGGER.info( - "Pipe {}@{}: start to extract historical TsFile, original sequence file count {}, " - + "original unsequence file count {}, start progress index {}", - pipeName, - dataRegionId, - originalSequenceTsFileCount, - originalUnsequenceTsFileCount, - startIndex); - - final Collection sequenceTsFileResources = - tsFileManager.getTsFileList(true).stream() - .filter( - resource -> - // Some resource may not be closed due to the control of - // PIPE_MIN_FLUSH_INTERVAL_IN_MS. We simply ignore them. - !resource.isClosed() - || mayTsFileContainUnprocessedData(resource) - && isTsFileResourceOverlappedWithTimeRange(resource) - && isTsFileGeneratedAfterExtractionTimeLowerBound(resource) - && mayTsFileResourceOverlappedWithPattern(resource)) - .collect(Collectors.toList()); - resourceList.addAll(sequenceTsFileResources); - - final Collection unsequenceTsFileResources = - tsFileManager.getTsFileList(false).stream() - .filter( - resource -> - // Some resource may not be closed due to the control of - // PIPE_MIN_FLUSH_INTERVAL_IN_MS. We simply ignore them. - !resource.isClosed() - || mayTsFileContainUnprocessedData(resource) - && isTsFileResourceOverlappedWithTimeRange(resource) - && isTsFileGeneratedAfterExtractionTimeLowerBound(resource) - && mayTsFileResourceOverlappedWithPattern(resource)) - .collect(Collectors.toList()); - resourceList.addAll(unsequenceTsFileResources); - - resourceList.forEach( - resource -> { - // Pin the resource, in case the file is removed by compaction or anything. - // Will unpin it after the PipeTsFileInsertionEvent is created and pinned. - try { - PipeDataNodeResourceManager.tsfile() - .pinTsFileResource(resource, shouldTransferModFile); - } catch (final IOException e) { - LOGGER.warn("Pipe: failed to pin TsFileResource {}", resource.getTsFilePath()); - } - }); - - resourceList.sort( - (o1, o2) -> - startIndex instanceof TimeWindowStateProgressIndex - ? Long.compare(o1.getFileStartTime(), o2.getFileStartTime()) - : o1.getMaxProgressIndex().topologicalCompareTo(o2.getMaxProgressIndex())); - pendingQueue = new ArrayDeque<>(resourceList); - - LOGGER.info( - "Pipe {}@{}: finish to extract historical TsFile, extracted sequence file count {}/{}, " - + "extracted unsequence file count {}/{}, extracted file count {}/{}, took {} ms", - pipeName, - dataRegionId, - sequenceTsFileResources.size(), - originalSequenceTsFileCount, - unsequenceTsFileResources.size(), - originalUnsequenceTsFileCount, - resourceList.size(), - originalSequenceTsFileCount + originalUnsequenceTsFileCount, - System.currentTimeMillis() - startHistoricalExtractionTime); - } finally { - tsFileManager.readUnlock(); - } + flushTsFilesForExtraction(dataRegion, startHistoricalExtractionTime); + // Extract TsFiles + extractTsFiles(dataRegion, startHistoricalExtractionTime); } finally { dataRegion.writeUnlock(); } } + private boolean mayDeletionUnprocessed(final DeletionResource resource) { + return !startIndex.isAfter(resource.getProgressIndex()); + } + private boolean mayTsFileContainUnprocessedData(final TsFileResource resource) { if (startIndex instanceof TimeWindowStateProgressIndex) { // The resource is closed thus the TsFileResource#getFileEndTime() is safe to use @@ -566,22 +616,7 @@ private boolean isTsFileGeneratedAfterExtractionTimeLowerBound(final TsFileResou } } - @Override - public synchronized Event supply() { - if (Objects.isNull(pendingQueue)) { - return null; - } - - final TsFileResource resource = pendingQueue.poll(); - if (resource == null) { - isTerminateSignalSent = true; - final PipeTerminateEvent terminateEvent = - new PipeTerminateEvent(pipeName, creationTime, pipeTaskMeta, dataRegionId); - terminateEvent.increaseReferenceCount( - PipeHistoricalDataRegionTsFileExtractor.class.getName()); - return terminateEvent; - } - + private Event supplyTsFileEvent(TsFileResource resource) { final PipeTsFileInsertionEvent event = new PipeTsFileInsertionEvent( resource, @@ -602,7 +637,8 @@ public synchronized Event supply() { event.skipParsingTime(); } - event.increaseReferenceCount(PipeHistoricalDataRegionTsFileExtractor.class.getName()); + event.increaseReferenceCount( + PipeHistoricalDataRegionTsFileAndDeletionExtractor.class.getName()); try { PipeDataNodeResourceManager.tsfile().unpinTsFileResource(resource); } catch (final IOException e) { @@ -612,28 +648,64 @@ public synchronized Event supply() { dataRegionId, resource.getTsFilePath()); } + return event; + } + private Event supplyDeletionEvent() { + final DeletionResource deletionResource = pendingDeletionQueue.poll(); + if (deletionResource == null) { + isTerminateSignalSent = true; + final PipeTerminateEvent terminateEvent = + new PipeTerminateEvent(pipeName, creationTime, pipeTaskMeta, dataRegionId); + terminateEvent.increaseReferenceCount( + PipeHistoricalDataRegionTsFileAndDeletionExtractor.class.getName()); + return terminateEvent; + } + PipeSchemaRegionWritePlanEvent event = deletionResource.getDeletionEvent(); + event.increaseReferenceCount( + PipeHistoricalDataRegionTsFileAndDeletionExtractor.class.getName()); return event; } + @Override + public synchronized Event supply() { + if (Objects.isNull(pendingTsFileQueue) && Objects.isNull(pendingDeletionQueue)) { + return null; + } + + // Consume tsFile first + if (!isTsFileEventAllConsumed) { + final TsFileResource resource = pendingTsFileQueue.poll(); + if (resource == null) { + isTsFileEventAllConsumed = true; + return supplyDeletionEvent(); + } + return supplyTsFileEvent(resource); + } else { + // Consume deletions + return supplyDeletionEvent(); + } + } + @Override public synchronized boolean hasConsumedAll() { - // If the pendingQueue is null when the function is called, it implies that the extractor only + // If the pendingQueues are null when the function is called, it implies that the extractor only // extracts deletion thus the historical event has nothing to consume. - return Objects.isNull(pendingQueue) - || pendingQueue.isEmpty() + return (Objects.isNull(pendingTsFileQueue) && Objects.isNull(pendingDeletionQueue)) + || pendingTsFileQueue.isEmpty() + && pendingDeletionQueue.isEmpty() && (!shouldTerminatePipeOnAllHistoricalEventsConsumed || isTerminateSignalSent); } @Override public int getPendingQueueSize() { - return Objects.nonNull(pendingQueue) ? pendingQueue.size() : 0; + return Objects.nonNull(pendingTsFileQueue) ? pendingTsFileQueue.size() : 0; } @Override public synchronized void close() { - if (Objects.nonNull(pendingQueue)) { - pendingQueue.forEach( + if (Objects.nonNull(pendingTsFileQueue)) { + pendingTsFileQueue.forEach( resource -> { try { PipeDataNodeResourceManager.tsfile().unpinTsFileResource(resource); @@ -645,8 +717,8 @@ public synchronized void close() { resource.getTsFilePath()); } }); - pendingQueue.clear(); - pendingQueue = null; + pendingTsFileQueue.clear(); + pendingTsFileQueue = null; } } } From 5b85c76a5846d228d73060398ad95a1d59af34c9 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Sat, 31 Aug 2024 15:40:09 +0800 Subject: [PATCH 11/85] add config of deletion dirs --- .../org/apache/iotdb/db/conf/IoTDBConfig.java | 21 +++++++++++++++++++ .../apache/iotdb/db/conf/IoTDBDescriptor.java | 4 ++++ .../deletion/DeletionResourceManager.java | 9 +++++--- .../conf/iotdb-system.properties.template | 14 +++++++++++++ .../iotdb/commons/conf/IoTDBConstant.java | 1 + 5 files changed, 46 insertions(+), 3 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java index 52b80dc01bc1..3eb4da5487f1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java @@ -68,7 +68,10 @@ import java.util.concurrent.TimeUnit; import java.util.regex.Pattern; +import static org.apache.iotdb.commons.conf.IoTDBConstant.CONSENSUS_FOLDER_NAME; +import static org.apache.iotdb.commons.conf.IoTDBConstant.DELETION_FOLDER_NAME; import static org.apache.iotdb.commons.conf.IoTDBConstant.OBJECT_STORAGE_DIR; +import static org.apache.iotdb.commons.conf.IoTDBConstant.PIPE_FOLDER_NAME; import static org.apache.tsfile.common.constant.TsFileConstant.PATH_SEPARATOR; public class IoTDBConfig { @@ -1128,6 +1131,15 @@ public class IoTDBConfig { private String[] pipeConsensusReceiverFileDirs = new String[0]; + private String pipeConsensusDeletionFileDir = + systemDir + + File.separator + + PIPE_FOLDER_NAME + + File.separator + + CONSENSUS_FOLDER_NAME + + File.separator + + DELETION_FOLDER_NAME; + /** Resource control */ private boolean quotaEnable = false; @@ -1282,6 +1294,7 @@ private void formulateFolders() { systemDir = addDataHomeDir(systemDir); schemaDir = addDataHomeDir(schemaDir); consensusDir = addDataHomeDir(consensusDir); + pipeConsensusDeletionFileDir = addDataHomeDir(pipeConsensusDeletionFileDir); dataRegionConsensusDir = addDataHomeDir(dataRegionConsensusDir); ratisDataRegionSnapshotDir = addDataHomeDir(ratisDataRegionSnapshotDir); schemaRegionConsensusDir = addDataHomeDir(schemaRegionConsensusDir); @@ -1470,6 +1483,14 @@ public void setSystemDir(String systemDir) { this.systemDir = systemDir; } + public String getPipeConsensusDeletionFileDir() { + return pipeConsensusDeletionFileDir; + } + + public void setPipeConsensusDeletionFileDir(String pipeConsensusDeletionFileDir) { + this.pipeConsensusDeletionFileDir = pipeConsensusDeletionFileDir; + } + public String[] getLoadTsFileDirs() { return this.loadTsFileDirs; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java index b131acede4f9..4dcc3b46cfa8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java @@ -2279,6 +2279,10 @@ private void loadPipeProps(Properties properties) { .split(",")) .filter(dir -> !dir.isEmpty()) .toArray(String[]::new)); + + conf.setPipeConsensusDeletionFileDir( + properties.getProperty( + "pipe_consensus_deletion_file_dir", conf.getPipeConsensusDeletionFileDir())); } private void loadCQProps(Properties properties) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java index 45fa572decab..7a4d9786e8a6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java @@ -23,6 +23,7 @@ import org.apache.iotdb.commons.consensus.index.impl.SimpleProgressIndex; import org.apache.iotdb.commons.utils.FileUtils; import org.apache.iotdb.consensus.pipe.PipeConsensus; +import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.consensus.DataRegionConsensusImpl; import org.apache.iotdb.db.pipe.consensus.ProgressIndexDataNodeManager; import org.apache.iotdb.db.pipe.consensus.deletion.persist.DeletionBuffer; @@ -58,14 +59,16 @@ public class DeletionResourceManager implements AutoCloseable { REBOOT_TIME, MEM_TABLE_FLUSH_ORDER, DELETION_FILE_SUFFIX); private final String dataRegionId; private final DeletionBuffer deletionBuffer; - - // TODO: read it from conf private final File storageDir; private final List deletionResources = new CopyOnWriteArrayList<>(); public DeletionResourceManager(String dataRegionId) throws IOException { this.dataRegionId = dataRegionId; - this.storageDir = new File("tmp" + File.separator + dataRegionId); + this.storageDir = + new File( + IoTDBDescriptor.getInstance().getConfig().getPipeConsensusDeletionFileDir() + + File.separator + + dataRegionId); this.deletionBuffer = new DeletionBuffer(dataRegionId, storageDir.getAbsolutePath()); initAndRecover(); // Only after initAndRecover can we start serialize and sync new deletions. diff --git a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template index 279aa063b859..98316ba9c89b 100644 --- a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template +++ b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template @@ -332,6 +332,20 @@ dn_pipe_receiver_file_dirs=data/datanode/system/pipe/receiver # If its prefix is "/", then the path is absolute. Otherwise, it is relative. pipe_consensus_receiver_file_dirs=data/datanode/system/pipe/consensus/receiver +# pipe_consensus_deletion_file_dir +# If this property is unset, system will save the data in the default relative path directory under the IoTDB folder(i.e., %IOTDB_HOME%/${dn_system_dir}/pipe/consensus/deletion). +# If it is absolute, system will save the data in the exact location it points to. +# If it is relative, system will save the data in the relative path directory it indicates under the IoTDB folder. +# If there are more than one directory, please separate them by commas ",". +# Note: If pipe_consensus_deletion_file_dir is assigned an empty string(i.e.,zero-size), it will be handled as a relative path. +# effectiveMode: restart +# For windows platform +# If its prefix is a drive specifier followed by "\\", or if its prefix is "\\\\", then the path is absolute. Otherwise, it is relative. +# pipe_consensus_deletion_file_dir=data\\datanode\\system\\pipe\\consensus\\deletion +# For Linux platform +# If its prefix is "/", then the path is absolute. Otherwise, it is relative. +pipe_consensus_deletion_file_dir=data/datanode/system/pipe/consensus/deletion + #################### ### Metric Configuration #################### diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java index 023e91144917..9bf280c14053 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java @@ -250,6 +250,7 @@ private IoTDBConstant() {} public static final String TRIGGER_FOLDER_NAME = "trigger"; public static final String PIPE_FOLDER_NAME = "pipe"; public static final String TMP_FOLDER_NAME = "tmp"; + public static final String DELETION_FOLDER_NAME = "deletion"; public static final String MQTT_FOLDER_NAME = "mqtt"; public static final String WAL_FOLDER_NAME = "wal"; From eff35073ba807b59854643e5a41118e9ec1c96b3 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Sun, 1 Sep 2024 00:15:55 +0800 Subject: [PATCH 12/85] add base ut --- .../deletion/DeletionResourceManager.java | 38 ++++- .../deletion/persist/DeletionBuffer.java | 12 +- .../pipe/consensus/DeletionResourceTest.java | 144 ++++++++++++++++++ 3 files changed, 183 insertions(+), 11 deletions(-) create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java index 7a4d9786e8a6..968a414be6a6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java @@ -22,6 +22,7 @@ import org.apache.iotdb.commons.consensus.index.ProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.SimpleProgressIndex; import org.apache.iotdb.commons.utils.FileUtils; +import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.consensus.pipe.PipeConsensus; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.consensus.DataRegionConsensusImpl; @@ -51,18 +52,18 @@ public class DeletionResourceManager implements AutoCloseable { private static final Logger LOGGER = LoggerFactory.getLogger(DeletionResourceManager.class); public static final String DELETION_FILE_SUFFIX = ".deletion"; public static final String MAGIC_VERSION_V1 = "DELETION_V1"; - private static final String REBOOT_TIME = "REBOOT_TIME"; - private static final String MEM_TABLE_FLUSH_ORDER = "MEM_TABLE_FLUSH_ORDER"; + private static final String REBOOT_TIME = "rebootTime"; + private static final String MEM_TABLE_FLUSH_ORDER = "memTableFlushOrderId"; private static final String DELETION_FILE_NAME_PATTERN = String.format( - "^_(?<%s>\\d+)_(?<%s>\\d+)\\%s$", + "^_(?<%s>\\d+)-(?<%s>\\d+)\\%s$", REBOOT_TIME, MEM_TABLE_FLUSH_ORDER, DELETION_FILE_SUFFIX); private final String dataRegionId; private final DeletionBuffer deletionBuffer; private final File storageDir; private final List deletionResources = new CopyOnWriteArrayList<>(); - public DeletionResourceManager(String dataRegionId) throws IOException { + private DeletionResourceManager(String dataRegionId) throws IOException { this.dataRegionId = dataRegionId; this.storageDir = new File( @@ -163,8 +164,8 @@ private boolean isCurrentFileCanBeDeleted(String fileName, ProgressIndex current int fileRebootTimes = Integer.parseInt(matcher.group(REBOOT_TIME)); long fileMemTableFlushOrderId = Long.parseLong(matcher.group(MEM_TABLE_FLUSH_ORDER)); return fileRebootTimes == curRebootTimes - ? fileMemTableFlushOrderId < curMemTableFlushOrderId - : fileRebootTimes < curMemTableFlushOrderId; + ? fileMemTableFlushOrderId <= curMemTableFlushOrderId + : fileRebootTimes < curRebootTimes; } } return false; @@ -206,4 +207,29 @@ public static void build() { DeletionResourceManagerHolder.build(); } } + + @TestOnly + public static void buildForTest() { + DeletionResourceManagerHolder.build(); + } + + @TestOnly + public void recoverForTest() { + try (Stream pathStream = Files.walk(Paths.get(storageDir.getPath()), 1)) { + Path[] deletionPaths = + pathStream + .filter(Files::isRegularFile) + .filter(path -> path.getFileName().toString().matches(DELETION_FILE_NAME_PATTERN)) + .toArray(Path[]::new); + + for (Path path : deletionPaths) { + try (DeletionReader deletionReader = + new DeletionReader(path.toFile(), this::removeDeletionResource)) { + deletionResources.addAll(deletionReader.readAllDeletions()); + } + } + } catch (IOException e) { + LOGGER.error("Failed to recover DeletionResourceManager", e); + } + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java index 5ba849972938..995b71b8ca63 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java @@ -187,8 +187,9 @@ private void serialize() { // For first serialization, we don't need to judge whether working buffer is exhausted. // Because a single DeleteDataNode can't exceed size of working buffer. serializeToWorkingBuffer(firstDeletionResource); - maxProgressIndexInCurrentBatch.updateToMinimumEqualOrIsAfterProgressIndex( - firstDeletionResource.getProgressIndex()); + maxProgressIndexInCurrentBatch = + maxProgressIndexInCurrentBatch.updateToMinimumEqualOrIsAfterProgressIndex( + firstDeletionResource.getProgressIndex()); } catch (InterruptedException e) { LOGGER.warn( "Interrupted when waiting for taking DeletionResource from blocking queue to serialize."); @@ -222,8 +223,9 @@ private void serialize() { deletionNum = 0; } // Update max progressIndex - maxProgressIndexInCurrentBatch.updateToMinimumEqualOrIsAfterProgressIndex( - deletionResource.getProgressIndex()); + maxProgressIndexInCurrentBatch = + maxProgressIndexInCurrentBatch.updateToMinimumEqualOrIsAfterProgressIndex( + deletionResource.getProgressIndex()); } // Persist deletions; Defensive programming here, just in case. if (totalSize > 0) { @@ -284,7 +286,7 @@ public SyncBufferTask(ProgressIndex maxProgressIndexInCurrentBatch, int deletion new File( baseDirectory, String.format( - "_%d_%d%s", + "_%d-%d%s", progressIndex.getRebootTimes(), progressIndex.getMemTableFlushOrderId(), DeletionResourceManager.DELETION_FILE_SUFFIX)); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java new file mode 100644 index 000000000000..e4fe0b0b4cbd --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.consensus; + +import org.apache.iotdb.commons.consensus.index.impl.RecoverProgressIndex; +import org.apache.iotdb.commons.consensus.index.impl.SimpleProgressIndex; +import org.apache.iotdb.commons.exception.IllegalPathException; +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.utils.FileUtils; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; +import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Collections; +import java.util.stream.Stream; + +public class DeletionResourceTest { + private static final String FAKE_DATE_REGION_ID = "1"; + private static final String DELETION_BASE_DIR = + IoTDBDescriptor.getInstance().getConfig().getPipeConsensusDeletionFileDir(); + private static final String BASE_PATH = DELETION_BASE_DIR + File.separator + FAKE_DATE_REGION_ID; + private static final int THIS_DATANODE_ID = + IoTDBDescriptor.getInstance().getConfig().getDataNodeId(); + private DeletionResourceManager deletionResourceManager; + + @Before + public void setUp() throws Exception { + DeletionResourceManager.buildForTest(); + deletionResourceManager = DeletionResourceManager.getInstance(FAKE_DATE_REGION_ID); + } + + @After + public void tearDown() throws Exception { + deletionResourceManager.close(); + try (Stream pathStream = Files.walk(Paths.get(BASE_PATH), 1)) { + for (Path path : pathStream.toArray(Path[]::new)) { + FileUtils.deleteFileOrDirectory(path.toFile()); + } + } + } + + @Test + public void testCreateBaseDir() { + File baseDir = new File(DELETION_BASE_DIR); + File dataRegionDir = new File(baseDir + File.separator + FAKE_DATE_REGION_ID); + Assert.assertTrue(baseDir.exists()); + Assert.assertTrue(dataRegionDir.exists()); + } + + @Test + public void testAddBatchDeletionResource() + throws IllegalPathException, InterruptedException, IOException { + int deletionCount = 10; + int rebootTimes = 0; + PartialPath path = new PartialPath("root.vehicle.d2.s0"); + for (int i = 0; i < deletionCount; i++) { + DeleteDataNode deleteDataNode = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 150); + deleteDataNode.setProgressIndex( + new RecoverProgressIndex(THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, i))); + PipeSchemaRegionWritePlanEvent deletionEvent = + new PipeSchemaRegionWritePlanEvent(deleteDataNode, true); + deletionResourceManager.registerDeletionResource(deletionEvent); + } + // Sleep to wait deletion being persisted + Thread.sleep(1000); + Stream paths = Files.list(Paths.get(BASE_PATH)); + Assert.assertTrue(paths.anyMatch(Files::isRegularFile)); + } + + @Test + public void testAddDeletionResourceTimeout() + throws IllegalPathException, InterruptedException, IOException { + int rebootTimes = 0; + PartialPath path = new PartialPath("root.vehicle.d2.s0"); + DeleteDataNode deleteDataNode = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 150); + deleteDataNode.setProgressIndex( + new RecoverProgressIndex(THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, 1))); + PipeSchemaRegionWritePlanEvent deletionEvent = + new PipeSchemaRegionWritePlanEvent(deleteDataNode, true); + // Only register one deletionResource + deletionResourceManager.registerDeletionResource(deletionEvent); + // Sleep to wait deletion being persisted + Thread.sleep(1000); + Stream paths = Files.list(Paths.get(BASE_PATH)); + Assert.assertTrue(paths.anyMatch(Files::isRegularFile)); + } + + @Test + public void testDeletionRemove() throws IllegalPathException, InterruptedException, IOException { + // new a deletion + int rebootTimes = 0; + PartialPath path = new PartialPath("root.vehicle.d2.s0"); + DeleteDataNode deleteDataNode = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 150); + deleteDataNode.setProgressIndex( + new RecoverProgressIndex(THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, 1))); + PipeSchemaRegionWritePlanEvent deletionEvent = + new PipeSchemaRegionWritePlanEvent(deleteDataNode, true); + // Only register one deletionResource + deletionResourceManager.registerDeletionResource(deletionEvent); + deletionEvent.increaseReferenceCount("test"); + // Sleep to wait deletion being persisted + Thread.sleep(1000); + Stream paths = Files.list(Paths.get(BASE_PATH)); + Assert.assertTrue(paths.anyMatch(Files::isRegularFile)); + // Remove deletion + deletionEvent.decreaseReferenceCount("test", false); + // Sleep to wait deletion being removed + Thread.sleep(1000); + Stream newPaths = Files.list(Paths.get(BASE_PATH)); + Assert.assertFalse(newPaths.anyMatch(Files::isRegularFile)); + } +} From 47a335d7dff10e249c2cbe3ab3e8ae6d7d3e97f3 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Sun, 1 Sep 2024 15:50:23 +0800 Subject: [PATCH 13/85] fix deletion read --- .../deletion/persist/DeletionBuffer.java | 6 ++++-- .../deletion/recover/DeletionReader.java | 19 ++++++++++++++++++- 2 files changed, 22 insertions(+), 3 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java index 995b71b8ca63..11403586c5e3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java @@ -243,7 +243,7 @@ private boolean serializeToWorkingBuffer(DeletionResource deletionResource) { if (buffer.position() > workingBuffer.remaining()) { return false; } - workingBuffer.put(buffer); + workingBuffer.put(buffer.array()); totalSize += buffer.position(); deletionNum++; return true; @@ -308,9 +308,11 @@ public void run() { // Write metaData. ByteBuffer metaData = ByteBuffer.allocate(4); metaData.putInt(deletionNum); + metaData.flip(); this.logChannel.write(metaData); // Write deletions. - this.logChannel.write(workingBuffer); + syncingBuffer.flip(); + this.logChannel.write(syncingBuffer); } catch (IOException e) { LOGGER.warn( "Deletion persist: Cannot write to {}, may cause data inconsistency.", logFile, e); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java index 392e0381acf8..766604731411 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.pipe.consensus.deletion.recover; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -30,28 +31,41 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.FileChannel; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; import java.util.function.Consumer; public class DeletionReader implements Closeable { private static final Logger LOGGER = LoggerFactory.getLogger(DeletionReader.class); + private static final int MAGIC_STRING_BYTES_SIZE = + DeletionResourceManager.MAGIC_VERSION_V1.getBytes(StandardCharsets.UTF_8).length; private final Consumer removeHook; + private final File logFile; private final FileInputStream fileInputStream; private final FileChannel fileChannel; public DeletionReader(File logFile, Consumer removeHook) throws IOException { + this.logFile = logFile; this.fileInputStream = new FileInputStream(logFile); this.fileChannel = fileInputStream.getChannel(); this.removeHook = removeHook; } public List readAllDeletions() throws IOException { + // Read magic string + ByteBuffer magicStringBuffer = ByteBuffer.allocate(MAGIC_STRING_BYTES_SIZE); + fileChannel.read(magicStringBuffer); + magicStringBuffer.flip(); + String magicVersion = new String(magicStringBuffer.array(), StandardCharsets.UTF_8); + LOGGER.debug("Read deletion file-{} magic version: {}", logFile, magicVersion); + // Read metaData ByteBuffer intBuffer = ByteBuffer.allocate(4); fileChannel.read(intBuffer); intBuffer.flip(); int deletionNum = intBuffer.getInt(); + LOGGER.debug("Read deletion file-{} contains {} deletions", logFile, deletionNum); // Read deletions long remainingBytes = fileChannel.size() - fileChannel.position(); @@ -61,7 +75,10 @@ public List readAllDeletions() throws IOException { List deletions = new ArrayList<>(); for (int i = 0; i < deletionNum; i++) { - deletions.add(DeletionResource.deserialize(byteBuffer, removeHook)); + DeletionResource deletionResource = DeletionResource.deserialize(byteBuffer, removeHook); + deletions.add(deletionResource); + LOGGER.debug("Read deletion: {} from file {}", deletionResource, logFile); + System.out.println("read deletion" + i + " :" + deletionResource); } return deletions; } From 56f5c5551b9b4eed1ce2e14078fcf43061814c7c Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Sun, 1 Sep 2024 15:50:37 +0800 Subject: [PATCH 14/85] fix deserialize logic --- .../event/common/schema/PipeSchemaRegionWritePlanEvent.java | 1 + .../plan/planner/plan/node/write/DeleteDataNode.java | 3 +++ 2 files changed, 4 insertions(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java index f70491e5273a..4a9a58a807e1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java @@ -104,6 +104,7 @@ public ByteBuffer serializeToByteBuffer() { @Override public void deserializeFromByteBuffer(final ByteBuffer buffer) { + byte ignoredEventType = ReadWriteIOUtils.readByte(buffer); isGeneratedByPipe = ReadWriteIOUtils.readBool(buffer); planNode = PlanNodeType.deserialize(buffer); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java index 5fa7cf876853..d9c731113c76 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java @@ -232,6 +232,9 @@ public static DeleteDataNode deserialize(ByteBuffer byteBuffer) { long deleteEndTime = ReadWriteIOUtils.readLong(byteBuffer); PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer); + + // DeleteDataNode has no child + int ignoredChildrenSize = ReadWriteIOUtils.readInt(byteBuffer); return new DeleteDataNode(planNodeId, pathList, deleteStartTime, deleteEndTime); } From 4c45c33135533d4797b209a976f19c58b86069ed Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Sun, 1 Sep 2024 15:50:48 +0800 Subject: [PATCH 15/85] add recover ut --- .../pipe/consensus/DeletionRecoverTest.java | 90 +++++++++++++++++++ 1 file changed, 90 insertions(+) create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java new file mode 100644 index 000000000000..64ffacf5d9b5 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.consensus; + +import org.apache.iotdb.commons.consensus.index.impl.RecoverProgressIndex; +import org.apache.iotdb.commons.consensus.index.impl.SimpleProgressIndex; +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.utils.FileUtils; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; +import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Collections; +import java.util.stream.Stream; + +public class DeletionRecoverTest { + private static final String FAKE_DATE_REGION_ID = "1"; + private static final int THIS_DATANODE_ID = + IoTDBDescriptor.getInstance().getConfig().getDataNodeId(); + private static final String DELETION_BASE_DIR = + IoTDBDescriptor.getInstance().getConfig().getPipeConsensusDeletionFileDir(); + private static final String BASE_PATH = DELETION_BASE_DIR + File.separator + FAKE_DATE_REGION_ID; + private final int deletionCount = 10; + private DeletionResourceManager deletionResourceManager; + + @Before + public void setUp() throws Exception { + try (Stream pathStream = Files.walk(Paths.get(BASE_PATH), 1)) { + for (Path path : pathStream.toArray(Path[]::new)) { + FileUtils.deleteFileOrDirectory(path.toFile()); + } + } + DeletionResourceManager.buildForTest(); + deletionResourceManager = DeletionResourceManager.getInstance(FAKE_DATE_REGION_ID); + // Create some deletion files + int rebootTimes = 0; + PartialPath path = new PartialPath("root.vehicle.d2.s0"); + for (int i = 0; i < deletionCount; i++) { + DeleteDataNode deleteDataNode = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 150); + deleteDataNode.setProgressIndex( + new RecoverProgressIndex(THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, i))); + PipeSchemaRegionWritePlanEvent deletionEvent = + new PipeSchemaRegionWritePlanEvent(deleteDataNode, true); + deletionResourceManager.registerDeletionResource(deletionEvent); + } + // Manually close for further test + deletionResourceManager.close(); + } + + @After + public void tearDown() throws Exception { + deletionResourceManager.close(); + } + + @Test + public void testDeletionRecover() throws Exception { + Assert.assertEquals(0, deletionResourceManager.getAllDeletionResources().size()); + deletionResourceManager.recoverForTest(); + Assert.assertEquals(deletionCount, deletionResourceManager.getAllDeletionResources().size()); + } +} From ee9c40a814183af4b64eb4698fc8973f87e9a631 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Sun, 1 Sep 2024 16:34:32 +0800 Subject: [PATCH 16/85] remove duplicate --- .../iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java | 1 - 1 file changed, 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java index 766604731411..e3d7ffd187bd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java @@ -78,7 +78,6 @@ public List readAllDeletions() throws IOException { DeletionResource deletionResource = DeletionResource.deserialize(byteBuffer, removeHook); deletions.add(deletionResource); LOGGER.debug("Read deletion: {} from file {}", deletionResource, logFile); - System.out.println("read deletion" + i + " :" + deletionResource); } return deletions; } From d115cdcd9fc5d7ec5def2851d675b0a17ec6ed90 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Sun, 1 Sep 2024 16:35:42 +0800 Subject: [PATCH 17/85] remove ut --- .../pipe/consensus/DeletionRecoverTest.java | 90 ----------- .../pipe/consensus/DeletionResourceTest.java | 144 ------------------ 2 files changed, 234 deletions(-) delete mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java delete mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java deleted file mode 100644 index 64ffacf5d9b5..000000000000 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java +++ /dev/null @@ -1,90 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.db.pipe.consensus; - -import org.apache.iotdb.commons.consensus.index.impl.RecoverProgressIndex; -import org.apache.iotdb.commons.consensus.index.impl.SimpleProgressIndex; -import org.apache.iotdb.commons.path.PartialPath; -import org.apache.iotdb.commons.utils.FileUtils; -import org.apache.iotdb.db.conf.IoTDBDescriptor; -import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; -import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; -import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; -import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; - -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.io.File; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.Collections; -import java.util.stream.Stream; - -public class DeletionRecoverTest { - private static final String FAKE_DATE_REGION_ID = "1"; - private static final int THIS_DATANODE_ID = - IoTDBDescriptor.getInstance().getConfig().getDataNodeId(); - private static final String DELETION_BASE_DIR = - IoTDBDescriptor.getInstance().getConfig().getPipeConsensusDeletionFileDir(); - private static final String BASE_PATH = DELETION_BASE_DIR + File.separator + FAKE_DATE_REGION_ID; - private final int deletionCount = 10; - private DeletionResourceManager deletionResourceManager; - - @Before - public void setUp() throws Exception { - try (Stream pathStream = Files.walk(Paths.get(BASE_PATH), 1)) { - for (Path path : pathStream.toArray(Path[]::new)) { - FileUtils.deleteFileOrDirectory(path.toFile()); - } - } - DeletionResourceManager.buildForTest(); - deletionResourceManager = DeletionResourceManager.getInstance(FAKE_DATE_REGION_ID); - // Create some deletion files - int rebootTimes = 0; - PartialPath path = new PartialPath("root.vehicle.d2.s0"); - for (int i = 0; i < deletionCount; i++) { - DeleteDataNode deleteDataNode = - new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 150); - deleteDataNode.setProgressIndex( - new RecoverProgressIndex(THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, i))); - PipeSchemaRegionWritePlanEvent deletionEvent = - new PipeSchemaRegionWritePlanEvent(deleteDataNode, true); - deletionResourceManager.registerDeletionResource(deletionEvent); - } - // Manually close for further test - deletionResourceManager.close(); - } - - @After - public void tearDown() throws Exception { - deletionResourceManager.close(); - } - - @Test - public void testDeletionRecover() throws Exception { - Assert.assertEquals(0, deletionResourceManager.getAllDeletionResources().size()); - deletionResourceManager.recoverForTest(); - Assert.assertEquals(deletionCount, deletionResourceManager.getAllDeletionResources().size()); - } -} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java deleted file mode 100644 index e4fe0b0b4cbd..000000000000 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java +++ /dev/null @@ -1,144 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.db.pipe.consensus; - -import org.apache.iotdb.commons.consensus.index.impl.RecoverProgressIndex; -import org.apache.iotdb.commons.consensus.index.impl.SimpleProgressIndex; -import org.apache.iotdb.commons.exception.IllegalPathException; -import org.apache.iotdb.commons.path.PartialPath; -import org.apache.iotdb.commons.utils.FileUtils; -import org.apache.iotdb.db.conf.IoTDBDescriptor; -import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; -import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; -import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; -import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; - -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.Collections; -import java.util.stream.Stream; - -public class DeletionResourceTest { - private static final String FAKE_DATE_REGION_ID = "1"; - private static final String DELETION_BASE_DIR = - IoTDBDescriptor.getInstance().getConfig().getPipeConsensusDeletionFileDir(); - private static final String BASE_PATH = DELETION_BASE_DIR + File.separator + FAKE_DATE_REGION_ID; - private static final int THIS_DATANODE_ID = - IoTDBDescriptor.getInstance().getConfig().getDataNodeId(); - private DeletionResourceManager deletionResourceManager; - - @Before - public void setUp() throws Exception { - DeletionResourceManager.buildForTest(); - deletionResourceManager = DeletionResourceManager.getInstance(FAKE_DATE_REGION_ID); - } - - @After - public void tearDown() throws Exception { - deletionResourceManager.close(); - try (Stream pathStream = Files.walk(Paths.get(BASE_PATH), 1)) { - for (Path path : pathStream.toArray(Path[]::new)) { - FileUtils.deleteFileOrDirectory(path.toFile()); - } - } - } - - @Test - public void testCreateBaseDir() { - File baseDir = new File(DELETION_BASE_DIR); - File dataRegionDir = new File(baseDir + File.separator + FAKE_DATE_REGION_ID); - Assert.assertTrue(baseDir.exists()); - Assert.assertTrue(dataRegionDir.exists()); - } - - @Test - public void testAddBatchDeletionResource() - throws IllegalPathException, InterruptedException, IOException { - int deletionCount = 10; - int rebootTimes = 0; - PartialPath path = new PartialPath("root.vehicle.d2.s0"); - for (int i = 0; i < deletionCount; i++) { - DeleteDataNode deleteDataNode = - new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 150); - deleteDataNode.setProgressIndex( - new RecoverProgressIndex(THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, i))); - PipeSchemaRegionWritePlanEvent deletionEvent = - new PipeSchemaRegionWritePlanEvent(deleteDataNode, true); - deletionResourceManager.registerDeletionResource(deletionEvent); - } - // Sleep to wait deletion being persisted - Thread.sleep(1000); - Stream paths = Files.list(Paths.get(BASE_PATH)); - Assert.assertTrue(paths.anyMatch(Files::isRegularFile)); - } - - @Test - public void testAddDeletionResourceTimeout() - throws IllegalPathException, InterruptedException, IOException { - int rebootTimes = 0; - PartialPath path = new PartialPath("root.vehicle.d2.s0"); - DeleteDataNode deleteDataNode = - new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 150); - deleteDataNode.setProgressIndex( - new RecoverProgressIndex(THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, 1))); - PipeSchemaRegionWritePlanEvent deletionEvent = - new PipeSchemaRegionWritePlanEvent(deleteDataNode, true); - // Only register one deletionResource - deletionResourceManager.registerDeletionResource(deletionEvent); - // Sleep to wait deletion being persisted - Thread.sleep(1000); - Stream paths = Files.list(Paths.get(BASE_PATH)); - Assert.assertTrue(paths.anyMatch(Files::isRegularFile)); - } - - @Test - public void testDeletionRemove() throws IllegalPathException, InterruptedException, IOException { - // new a deletion - int rebootTimes = 0; - PartialPath path = new PartialPath("root.vehicle.d2.s0"); - DeleteDataNode deleteDataNode = - new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 150); - deleteDataNode.setProgressIndex( - new RecoverProgressIndex(THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, 1))); - PipeSchemaRegionWritePlanEvent deletionEvent = - new PipeSchemaRegionWritePlanEvent(deleteDataNode, true); - // Only register one deletionResource - deletionResourceManager.registerDeletionResource(deletionEvent); - deletionEvent.increaseReferenceCount("test"); - // Sleep to wait deletion being persisted - Thread.sleep(1000); - Stream paths = Files.list(Paths.get(BASE_PATH)); - Assert.assertTrue(paths.anyMatch(Files::isRegularFile)); - // Remove deletion - deletionEvent.decreaseReferenceCount("test", false); - // Sleep to wait deletion being removed - Thread.sleep(1000); - Stream newPaths = Files.list(Paths.get(BASE_PATH)); - Assert.assertFalse(newPaths.anyMatch(Files::isRegularFile)); - } -} From bc14898d565689934fe3c2777b67eef3c6595014 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Sun, 1 Sep 2024 23:52:29 +0800 Subject: [PATCH 18/85] fix conflict --- .../realtime/PipeRealtimeEventFactory.java | 5 +- ...lDataRegionTsFileAndDeletionExtractor.java | 199 +++++++++--------- .../pipeconsensus/PipeConsensusReceiver.java | 1 - .../storageengine/dataregion/DataRegion.java | 3 +- .../dataregion/DataRegionTest.java | 47 ++--- ...sSpaceCompactionWithFastPerformerTest.java | 8 +- ...eCompactionWithReadPointPerformerTest.java | 8 +- ...nerSeqCompactionWithFastPerformerTest.java | 2 +- ...qCompactionWithReadChunkPerformerTest.java | 2 +- .../settle/SettleRequestHandlerTest.java | 3 +- 10 files changed, 135 insertions(+), 143 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java index 80e5b7d3b70b..3e6430b8f969 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java @@ -19,8 +19,8 @@ package org.apache.iotdb.db.pipe.event.realtime; -import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; import org.apache.iotdb.commons.pipe.event.ProgressReportEvent; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; @@ -64,7 +64,8 @@ public static PipeRealtimeEvent createRealtimeEvent( new PipeHeartbeatEvent(dataRegionId, shouldPrintMessage), null, null, null); } - public static PipeRealtimeEvent createRealtimeEvent(final DeleteDataNode node, final String regionId) { + public static PipeRealtimeEvent createRealtimeEvent( + final DeleteDataNode node, final String regionId) { PipeSchemaRegionWritePlanEvent deletionEvent = new PipeSchemaRegionWritePlanEvent(node, node.isGeneratedByPipe()); Optional.ofNullable(DeletionResourceManager.getInstance(regionId)) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java index dd750b65eb61..2cfda02dfc89 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java @@ -373,23 +373,23 @@ private void flushDataRegionAllTsFiles() { } } - private void extractDeletions(final DeletionResourceManager deletionResourceManager) { - LOGGER.info("Pipe {}@{}: start to extract deletions", pipeName, dataRegionId); - List resourceList = deletionResourceManager.getAllDeletionResources(); - final int originalDeletionCount = resourceList.size(); - resourceList = - resourceList.stream() - .filter(this::mayDeletionUnprocessed) - .sorted((o1, o2) -> o1.getProgressIndex().isAfter(o2.getProgressIndex()) ? 1 : -1) - .collect(Collectors.toList()); - pendingDeletionQueue = new ArrayDeque<>(resourceList); - LOGGER.info( - "Pipe {}@{}: finish to extract deletions, extract deletions count {}/{}", - pipeName, - dataRegionId, - resourceList.size(), - originalDeletionCount); - } + private void extractDeletions(final DeletionResourceManager deletionResourceManager) { + LOGGER.info("Pipe {}@{}: start to extract deletions", pipeName, dataRegionId); + List resourceList = deletionResourceManager.getAllDeletionResources(); + final int originalDeletionCount = resourceList.size(); + resourceList = + resourceList.stream() + .filter(this::mayDeletionUnprocessed) + .sorted((o1, o2) -> o1.getProgressIndex().isAfter(o2.getProgressIndex()) ? 1 : -1) + .collect(Collectors.toList()); + pendingDeletionQueue = new ArrayDeque<>(resourceList); + LOGGER.info( + "Pipe {}@{}: finish to extract deletions, extract deletions count {}/{}", + pipeName, + dataRegionId, + resourceList.size(), + originalDeletionCount); + } private void flushTsFilesForExtraction( DataRegion dataRegion, final long startHistoricalExtractionTime) { @@ -499,9 +499,9 @@ && mayTsFileResourceOverlappedWithPattern(resource)) @Override public synchronized void start() { - if (!StorageEngine.getInstance().isReadyForNonReadWriteFunctions()) { - return; - } + if (!StorageEngine.getInstance().isReadyForNonReadWriteFunctions()) { + return; + } if (!shouldExtractInsertion) { return; @@ -620,93 +620,94 @@ private boolean isTsFileGeneratedAfterExtractionTimeLowerBound(final TsFileResou } } - private Event supplyTsFileEvent(TsFileResource resource) { - final PipeTsFileInsertionEvent event = - new PipeTsFileInsertionEvent( - resource, - shouldTransferModFile, - false, - false, - true, - pipeName, - creationTime, - pipeTaskMeta, - pipePattern, - historicalDataExtractionStartTime, - historicalDataExtractionEndTime); - if (sloppyPattern || isDbNameCoveredByPattern) { - event.skipParsingPattern(); - } + private Event supplyTsFileEvent(TsFileResource resource) { + final PipeTsFileInsertionEvent event = + new PipeTsFileInsertionEvent( + resource, + shouldTransferModFile, + false, + false, + true, + pipeName, + creationTime, + pipeTaskMeta, + pipePattern, + historicalDataExtractionStartTime, + historicalDataExtractionEndTime); + if (sloppyPattern || isDbNameCoveredByPattern) { + event.skipParsingPattern(); + } - if (sloppyTimeRange || isTsFileResourceCoveredByTimeRange(resource)) { - event.skipParsingTime(); - } + if (sloppyTimeRange || isTsFileResourceCoveredByTimeRange(resource)) { + event.skipParsingTime(); + } - try { - final boolean isReferenceCountIncreased = - event.increaseReferenceCount(PipeHistoricalDataRegionTsFileExtractor.class.getName()); - if (!isReferenceCountIncreased) { - LOGGER.warn( - "Pipe {}@{}: failed to increase reference count for historical event {}, will discard it", - pipeName, - dataRegionId, - event); - } - return isReferenceCountIncreased ? event : null; - } finally { - try { - PipeDataNodeResourceManager.tsfile().unpinTsFileResource(resource); - } catch (final IOException e) { - LOGGER.warn( - "Pipe {}@{}: failed to unpin TsFileResource after creating event, original path: {}", - pipeName, - dataRegionId, - resource.getTsFilePath()); - } - } + try { + final boolean isReferenceCountIncreased = + event.increaseReferenceCount( + PipeHistoricalDataRegionTsFileAndDeletionExtractor.class.getName()); + if (!isReferenceCountIncreased) { + LOGGER.warn( + "Pipe {}@{}: failed to increase reference count for historical event {}, will discard it", + pipeName, + dataRegionId, + event); + } + return isReferenceCountIncreased ? event : null; + } finally { + try { + PipeDataNodeResourceManager.tsfile().unpinTsFileResource(resource); + } catch (final IOException e) { + LOGGER.warn( + "Pipe {}@{}: failed to unpin TsFileResource after creating event, original path: {}", + pipeName, + dataRegionId, + resource.getTsFilePath()); + } } + } - private Event supplyDeletionEvent() { - final DeletionResource deletionResource = pendingDeletionQueue.poll(); - if (deletionResource == null) { - final PipeTerminateEvent terminateEvent = - new PipeTerminateEvent(pipeName, creationTime, pipeTaskMeta, dataRegionId); - if (!terminateEvent.increaseReferenceCount( - PipeHistoricalDataRegionTsFileExtractor.class.getName())) { - LOGGER.warn( - "Pipe {}@{}: failed to increase reference count for terminate event, will resend it", - pipeName, - dataRegionId); - return null; - } - isTerminateSignalSent = true; - return terminateEvent; - } - PipeSchemaRegionWritePlanEvent event = deletionResource.getDeletionEvent(); - event.increaseReferenceCount( - PipeHistoricalDataRegionTsFileAndDeletionExtractor.class.getName()); - return event; + private Event supplyDeletionEvent() { + final DeletionResource deletionResource = pendingDeletionQueue.poll(); + if (deletionResource == null) { + final PipeTerminateEvent terminateEvent = + new PipeTerminateEvent(pipeName, creationTime, pipeTaskMeta, dataRegionId); + if (!terminateEvent.increaseReferenceCount( + PipeHistoricalDataRegionTsFileAndDeletionExtractor.class.getName())) { + LOGGER.warn( + "Pipe {}@{}: failed to increase reference count for terminate event, will resend it", + pipeName, + dataRegionId); + return null; + } + isTerminateSignalSent = true; + return terminateEvent; } + PipeSchemaRegionWritePlanEvent event = deletionResource.getDeletionEvent(); + event.increaseReferenceCount( + PipeHistoricalDataRegionTsFileAndDeletionExtractor.class.getName()); + return event; + } - @Override - public synchronized Event supply() { - if (Objects.isNull(pendingTsFileQueue) && Objects.isNull(pendingDeletionQueue)) { - return null; - } + @Override + public synchronized Event supply() { + if (Objects.isNull(pendingTsFileQueue) && Objects.isNull(pendingDeletionQueue)) { + return null; + } - // Consume tsFile first - if (!isTsFileEventAllConsumed) { - final TsFileResource resource = pendingTsFileQueue.poll(); - if (resource == null) { - isTsFileEventAllConsumed = true; - return supplyDeletionEvent(); - } - return supplyTsFileEvent(resource); - } else { - // Consume deletions - return supplyDeletionEvent(); - } + // Consume tsFile first + if (!isTsFileEventAllConsumed) { + final TsFileResource resource = pendingTsFileQueue.poll(); + if (resource == null) { + isTsFileEventAllConsumed = true; + return supplyDeletionEvent(); + } + return supplyTsFileEvent(resource); + } else { + // Consume deletions + return supplyDeletionEvent(); } + } @Override public synchronized boolean hasConsumedAll() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java index 3dfbf8252cca..779fdfd47373 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java @@ -50,7 +50,6 @@ import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFileSealWithModReq; import org.apache.iotdb.db.pipe.consensus.metric.PipeConsensusReceiverMetrics; import org.apache.iotdb.db.pipe.event.common.tsfile.TsFileInsertionPointCounter; -import org.apache.iotdb.db.queryengine.execution.load.LoadTsFileManager; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode; import org.apache.iotdb.db.storageengine.StorageEngine; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java index a495a8ce837d..de6fe214270f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java @@ -2323,7 +2323,8 @@ public void deleteByDevice(MeasurementPath pattern, DeleteDataNode node) throws } } - public void deleteDataDirectly(MeasurementPath pathToDelete, DeleteDataNode node) throws IOException { + public void deleteDataDirectly(MeasurementPath pathToDelete, DeleteDataNode node) + throws IOException { final long startTime = node.getDeleteStartTime(); final long endTime = node.getDeleteEndTime(); final long searchIndex = node.getSearchIndex(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/DataRegionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/DataRegionTest.java index 9a605171ef42..01c4d7a4cf3b 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/DataRegionTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/DataRegionTest.java @@ -219,7 +219,7 @@ record = new TSRecord(j, deviceId); CompressionType.UNCOMPRESSED, Collections.emptyMap())); - PartialPath path = new PartialPath(deviceId, measurementId); + MeasurementPath path = new MeasurementPath(deviceId, measurementId); DeleteDataNode deleteDataNode1 = new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 0, 15L); dataRegion.deleteByDevice(new MeasurementPath(deviceId, measurementId), deleteDataNode1); @@ -452,9 +452,7 @@ public void testIoTDBTabletWriteAndSyncClose() @Test public void testIoTDBTabletWriteAndDeleteDataRegion() - throws QueryProcessException, - IllegalPathException, - WriteProcessException, + throws QueryProcessException, IllegalPathException, WriteProcessException, TsFileProcessorException { String[] measurements = new String[2]; measurements[0] = "s0"; @@ -1049,11 +1047,8 @@ public void testDisableSeparateDataForInsertTablet3() @Test public void testInsertUnSequenceRows() - throws IllegalPathException, - WriteProcessRejectException, - QueryProcessException, - DataRegionException, - TsFileProcessorException { + throws IllegalPathException, WriteProcessRejectException, QueryProcessException, + DataRegionException, TsFileProcessorException { int defaultAvgSeriesPointNumberThreshold = config.getAvgSeriesPointNumberThreshold(); config.setAvgSeriesPointNumberThreshold(2); DataRegion dataRegion1 = new DummyDataRegion(systemDir, "root.Rows"); @@ -1090,12 +1085,8 @@ tmpDeviceId, new MeasurementSchema(measurementId, TSDataType.INT32))), @Test public void testSmallReportProportionInsertRow() - throws WriteProcessException, - QueryProcessException, - IllegalPathException, - IOException, - DataRegionException, - TsFileProcessorException { + throws WriteProcessException, QueryProcessException, IllegalPathException, IOException, + DataRegionException, TsFileProcessorException { double defaultValue = config.getWriteMemoryVariationReportProportion(); config.setWriteMemoryVariationReportProportion(0); DataRegion dataRegion1 = new DummyDataRegion(systemDir, "root.ln22"); @@ -1408,7 +1399,7 @@ public void testDeleteDataNotInFile() dataRegion.syncCloseAllWorkingTsFileProcessors(); } - PartialPath path = new PartialPath("root.vehicle.d2.s0"); + MeasurementPath path = new MeasurementPath("root.vehicle.d2.s0"); DeleteDataNode deleteDataNode1 = new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 150); deleteDataNode1.setSearchIndex(0); @@ -1464,14 +1455,14 @@ public void testDeleteDataNotInFlushingMemtable() TsFileProcessor tsFileProcessor = tsFileResource.getProcessor(); tsFileProcessor.getFlushingMemTable().addLast(tsFileProcessor.getWorkMemTable()); - PartialPath path = new PartialPath("root.vehicle.d2.s0"); + MeasurementPath path = new MeasurementPath("root.vehicle.d2.s0"); DeleteDataNode deleteDataNode1 = new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 70); deleteDataNode1.setSearchIndex(0); // delete data which is in memtable dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d2.s0"), deleteDataNode1); - PartialPath path2 = new PartialPath("root.vehicle.d200.s0"); + MeasurementPath path2 = new MeasurementPath("root.vehicle.d200.s0"); DeleteDataNode deleteDataNode2 = new DeleteDataNode(new PlanNodeId("2"), Collections.singletonList(path2), 50, 70); deleteDataNode2.setSearchIndex(0); @@ -1494,11 +1485,11 @@ public void testDeleteDataInSeqFlushingMemtable() TsFileProcessor tsFileProcessor = tsFileResource.getProcessor(); tsFileProcessor.getFlushingMemTable().addLast(tsFileProcessor.getWorkMemTable()); - PartialPath path = new PartialPath("root.vehicle.d0.s0"); + MeasurementPath path = new MeasurementPath("root.vehicle.d0.s0"); DeleteDataNode deleteDataNode1 = new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 99); deleteDataNode1.setSearchIndex(0); - PartialPath path2 = new PartialPath("root.vehicle.d200.s0"); + MeasurementPath path2 = new MeasurementPath("root.vehicle.d200.s0"); DeleteDataNode deleteDataNode2 = new DeleteDataNode(new PlanNodeId("2"), Collections.singletonList(path2), 50, 70); deleteDataNode2.setSearchIndex(0); @@ -1535,11 +1526,11 @@ public void testDeleteDataInUnSeqFlushingMemtable() } TsFileResource tsFileResource = dataRegion.getTsFileManager().getTsFileList(true).get(0); - PartialPath path = new PartialPath("root.vehicle.d0.s0"); + MeasurementPath path = new MeasurementPath("root.vehicle.d0.s0"); DeleteDataNode deleteDataNode1 = new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 99); deleteDataNode1.setSearchIndex(0); - PartialPath path2 = new PartialPath("root.vehicle.d200.s0"); + MeasurementPath path2 = new MeasurementPath("root.vehicle.d200.s0"); DeleteDataNode deleteDataNode2 = new DeleteDataNode(new PlanNodeId("2"), Collections.singletonList(path2), 50, 70); deleteDataNode2.setSearchIndex(0); @@ -1635,11 +1626,11 @@ public void testDeleteDataInSeqWorkingMemtable() } TsFileResource tsFileResource = dataRegion.getTsFileManager().getTsFileList(true).get(0); - PartialPath path = new PartialPath("root.vehicle.d0.s0"); + MeasurementPath path = new MeasurementPath("root.vehicle.d0.s0"); DeleteDataNode deleteDataNode1 = new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 99); deleteDataNode1.setSearchIndex(0); - PartialPath path2 = new PartialPath("root.vehicle.d200.s0"); + MeasurementPath path2 = new MeasurementPath("root.vehicle.d200.s0"); DeleteDataNode deleteDataNode2 = new DeleteDataNode(new PlanNodeId("2"), Collections.singletonList(path2), 50, 70); deleteDataNode2.setSearchIndex(0); @@ -1647,7 +1638,7 @@ public void testDeleteDataInSeqWorkingMemtable() dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode1); dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d200.s0"), deleteDataNode2); - PartialPath path3 = new PartialPath("root.vehicle.d199.*"); + MeasurementPath path3 = new MeasurementPath("root.vehicle.d199.*"); DeleteDataNode deleteDataNode3 = new DeleteDataNode(new PlanNodeId("3"), Collections.singletonList(path3), 50, 500); deleteDataNode3.setSearchIndex(0); @@ -1672,7 +1663,7 @@ public void testFlushingEmptyMemtable() } TsFileResource tsFileResource = dataRegion.getTsFileManager().getTsFileList(true).get(0); - PartialPath path = new PartialPath("root.vehicle.d0.s0"); + MeasurementPath path = new MeasurementPath("root.vehicle.d0.s0"); DeleteDataNode deleteDataNode = new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 100, 200); deleteDataNode.setSearchIndex(0); @@ -1713,7 +1704,7 @@ public void testDeleteDataDirectlySeqWriteModsOrDeleteFiles() TsFileResource tsFileResource = dataRegion.getTsFileManager().getTsFileList(true).get(0); // delete data in work mem, no mods. - PartialPath path = new PartialPath("root.vehicle.d0.**"); + MeasurementPath path = new MeasurementPath("root.vehicle.d0.**"); DeleteDataNode deleteDataNode1 = new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 100); deleteDataNode1.setSearchIndex(0); @@ -1765,7 +1756,7 @@ public void testDeleteDataDirectlyUnseqWriteModsOrDeleteFiles() Assert.assertTrue(tsFileResourceUnSeq.getTsFile().exists()); // already closed, will have a mods file. - PartialPath path = new PartialPath("root.vehicle.d0.**"); + MeasurementPath path = new MeasurementPath("root.vehicle.d0.**"); DeleteDataNode deleteDataNode1 = new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 40, 60); deleteDataNode1.setSearchIndex(0); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithFastPerformerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithFastPerformerTest.java index e444360c5312..d309497f3462 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithFastPerformerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithFastPerformerTest.java @@ -588,8 +588,8 @@ public void testOneDeletionDuringCompaction() throws Exception { createFiles(2, 1, 5, 100, 450, 20450, 0, 0, true, false); vsgp.getTsFileResourceManager().addAll(seqResources, true); vsgp.getTsFileResourceManager().addAll(unseqResources, false); - PartialPath path = - new PartialPath( + MeasurementPath path = + new MeasurementPath( COMPACTION_TEST_SG + PATH_SEPARATOR + "d" @@ -718,8 +718,8 @@ public void testSeveralDeletionsDuringCompaction() throws Exception { vsgp.getTsFileResourceManager().addAll(seqResources, true); vsgp.getTsFileResourceManager().addAll(unseqResources, false); - PartialPath path = - new PartialPath( + MeasurementPath path = + new MeasurementPath( COMPACTION_TEST_SG + PATH_SEPARATOR + "d" diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithReadPointPerformerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithReadPointPerformerTest.java index 057dfd3ca56a..b86920f4c2f2 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithReadPointPerformerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithReadPointPerformerTest.java @@ -583,8 +583,8 @@ public void testOneDeletionDuringCompaction() throws Exception { createFiles(2, 1, 5, 100, 450, 20450, 0, 0, true, false); vsgp.getTsFileResourceManager().addAll(seqResources, true); vsgp.getTsFileResourceManager().addAll(unseqResources, false); - PartialPath path = - new PartialPath( + MeasurementPath path = + new MeasurementPath( COMPACTION_TEST_SG + PATH_SEPARATOR + "d" @@ -713,8 +713,8 @@ public void testSeveralDeletionsDuringCompaction() throws Exception { vsgp.getTsFileResourceManager().addAll(seqResources, true); vsgp.getTsFileResourceManager().addAll(unseqResources, false); - PartialPath path = - new PartialPath( + MeasurementPath path = + new MeasurementPath( COMPACTION_TEST_SG + PATH_SEPARATOR + "d" diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/InnerSeqCompactionWithFastPerformerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/InnerSeqCompactionWithFastPerformerTest.java index 09fd68b0295c..2a62081f4e3d 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/InnerSeqCompactionWithFastPerformerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/InnerSeqCompactionWithFastPerformerTest.java @@ -1141,7 +1141,7 @@ public void testCompactionWithDeletionsDuringCompactions() COMPACTION_TEST_SG); vsgp.getTsFileResourceManager().addAll(sourceResources, true); // delete data before compaction - PartialPath path = new PartialPath(fullPaths[0]); + MeasurementPath path = new MeasurementPath(fullPaths[0]); DeleteDataNode deleteDataNode = new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 0, 1000); deleteDataNode.setSearchIndex(0); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/InnerSeqCompactionWithReadChunkPerformerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/InnerSeqCompactionWithReadChunkPerformerTest.java index c062c85491e9..e8b79b224261 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/InnerSeqCompactionWithReadChunkPerformerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/InnerSeqCompactionWithReadChunkPerformerTest.java @@ -1106,7 +1106,7 @@ public void testCompactionWithDeletionsDuringCompactions() COMPACTION_TEST_SG); vsgp.getTsFileResourceManager().addAll(sourceResources, true); // delete data before compaction - PartialPath path = new PartialPath(fullPaths[0]); + MeasurementPath path = new MeasurementPath(fullPaths[0]); DeleteDataNode deleteDataNode = new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 0, 1000); deleteDataNode.setSearchIndex(0); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/settle/SettleRequestHandlerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/settle/SettleRequestHandlerTest.java index f4ca3a2198a0..51e4a72079da 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/settle/SettleRequestHandlerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/settle/SettleRequestHandlerTest.java @@ -25,7 +25,6 @@ import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.exception.StartupException; import org.apache.iotdb.commons.path.MeasurementPath; -import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.exception.DataRegionException; @@ -151,7 +150,7 @@ record = new TSRecord(timestamp, deviceId); } dataRegion.syncCloseAllWorkingTsFileProcessors(); if (i != 2) { - PartialPath path = new PartialPath(deviceId, measurementId); + MeasurementPath path = new MeasurementPath(deviceId, measurementId); DeleteDataNode deleteDataNode = new DeleteDataNode( new PlanNodeId("1"), Collections.singletonList(path), 3L * i + 1, 3L * i + 1); From 13bbbc0c01ee277fd6a70fa6064b8bd298e848ff Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Mon, 2 Sep 2024 00:13:16 +0800 Subject: [PATCH 19/85] spotless --- .../dataregion/DataRegionTest.java | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/DataRegionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/DataRegionTest.java index 01c4d7a4cf3b..d7ca89f7637e 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/DataRegionTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/DataRegionTest.java @@ -452,7 +452,9 @@ public void testIoTDBTabletWriteAndSyncClose() @Test public void testIoTDBTabletWriteAndDeleteDataRegion() - throws QueryProcessException, IllegalPathException, WriteProcessException, + throws QueryProcessException, + IllegalPathException, + WriteProcessException, TsFileProcessorException { String[] measurements = new String[2]; measurements[0] = "s0"; @@ -1047,8 +1049,11 @@ public void testDisableSeparateDataForInsertTablet3() @Test public void testInsertUnSequenceRows() - throws IllegalPathException, WriteProcessRejectException, QueryProcessException, - DataRegionException, TsFileProcessorException { + throws IllegalPathException, + WriteProcessRejectException, + QueryProcessException, + DataRegionException, + TsFileProcessorException { int defaultAvgSeriesPointNumberThreshold = config.getAvgSeriesPointNumberThreshold(); config.setAvgSeriesPointNumberThreshold(2); DataRegion dataRegion1 = new DummyDataRegion(systemDir, "root.Rows"); @@ -1085,8 +1090,12 @@ tmpDeviceId, new MeasurementSchema(measurementId, TSDataType.INT32))), @Test public void testSmallReportProportionInsertRow() - throws WriteProcessException, QueryProcessException, IllegalPathException, IOException, - DataRegionException, TsFileProcessorException { + throws WriteProcessException, + QueryProcessException, + IllegalPathException, + IOException, + DataRegionException, + TsFileProcessorException { double defaultValue = config.getWriteMemoryVariationReportProportion(); config.setWriteMemoryVariationReportProportion(0); DataRegion dataRegion1 = new DummyDataRegion(systemDir, "root.ln22"); From f57b8a9da62da8a18410ff1ef89160011015072a Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Mon, 2 Sep 2024 10:30:00 +0800 Subject: [PATCH 20/85] avoid Concurrent issue --- .../db/pipe/consensus/deletion/DeletionResourceManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java index 968a414be6a6..232be03f4ca8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java @@ -122,7 +122,7 @@ public List getAllDeletionResources() { * This is a hook function, which will be automatically invoked when deletionResource's reference * count returns to 0. */ - private void removeDeletionResource(DeletionResource deletionResource) { + private synchronized void removeDeletionResource(DeletionResource deletionResource) { // Clean memory deletionResources.remove(deletionResource); // Clean disk From 464c1364b95a90cdb6d800133d64fe912962d97d Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Mon, 2 Sep 2024 10:44:47 +0800 Subject: [PATCH 21/85] fix concurrent issue --- .../deletion/DeletionResourceManager.java | 65 ++++++++++++++----- .../pipeconsensus/PipeConsensusReceiver.java | 1 - 2 files changed, 47 insertions(+), 19 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java index 232be03f4ca8..5f02afb9b912 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java @@ -44,6 +44,9 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.stream.Stream; @@ -62,6 +65,10 @@ public class DeletionResourceManager implements AutoCloseable { private final DeletionBuffer deletionBuffer; private final File storageDir; private final List deletionResources = new CopyOnWriteArrayList<>(); + private final Lock recoverLock = new ReentrantLock(); + // condition to guarantee correctness of switching buffers + private final Condition recoveryReadyCondition = recoverLock.newCondition(); + private boolean hasCompletedRecovery = false; private DeletionResourceManager(String dataRegionId) throws IOException { this.dataRegionId = dataRegionId; @@ -77,27 +84,34 @@ private DeletionResourceManager(String dataRegionId) throws IOException { } private void initAndRecover() throws IOException { - if (!storageDir.exists()) { - // Init - if (!storageDir.mkdirs()) { - LOGGER.warn("Unable to create pipeConsensus deletion dir at {}", storageDir); - throw new IOException( - String.format("Unable to create pipeConsensus deletion dir at %s", storageDir)); + recoverLock.lock(); + try { + if (!storageDir.exists()) { + // Init + if (!storageDir.mkdirs()) { + LOGGER.warn("Unable to create pipeConsensus deletion dir at {}", storageDir); + throw new IOException( + String.format("Unable to create pipeConsensus deletion dir at %s", storageDir)); + } } - } - try (Stream pathStream = Files.walk(Paths.get(storageDir.getPath()), 1)) { - Path[] deletionPaths = - pathStream - .filter(Files::isRegularFile) - .filter(path -> path.getFileName().toString().matches(DELETION_FILE_NAME_PATTERN)) - .toArray(Path[]::new); + try (Stream pathStream = Files.walk(Paths.get(storageDir.getPath()), 1)) { + Path[] deletionPaths = + pathStream + .filter(Files::isRegularFile) + .filter(path -> path.getFileName().toString().matches(DELETION_FILE_NAME_PATTERN)) + .toArray(Path[]::new); - for (Path path : deletionPaths) { - try (DeletionReader deletionReader = - new DeletionReader(path.toFile(), this::removeDeletionResource)) { - deletionResources.addAll(deletionReader.readAllDeletions()); + for (Path path : deletionPaths) { + try (DeletionReader deletionReader = + new DeletionReader(path.toFile(), this::removeDeletionResource)) { + deletionResources.addAll(deletionReader.readAllDeletions()); + } } + this.hasCompletedRecovery = true; + recoveryReadyCondition.signalAll(); } + } finally { + recoverLock.unlock(); } } @@ -115,7 +129,22 @@ public void registerDeletionResource(PipeSchemaRegionWritePlanEvent event) { } public List getAllDeletionResources() { - return deletionResources.stream().collect(ImmutableList.toImmutableList()); + recoverLock.lock(); + try { + if (!hasCompletedRecovery) { + recoveryReadyCondition.await(); + } + return deletionResources.stream().collect(ImmutableList.toImmutableList()); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOGGER.warn( + "DeletionManager-{}: current waiting is interrupted. May because current application is down. ", + dataRegionId, + e); + return deletionResources.stream().collect(ImmutableList.toImmutableList()); + } finally { + recoverLock.unlock(); + } } /** diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java index 779fdfd47373..1f5505f5c8a8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java @@ -1463,7 +1463,6 @@ private TPipeConsensusTransferResp onRequest( consensusPipeName, tCommitId.getCommitIndex(), e); - Thread.currentThread().interrupt(); // Avoid infinite loop when RPC thread is killed by OS return new TPipeConsensusTransferResp( RpcUtils.getStatus( From 3f629f159ef142014ec1e59590aead2c7f76a6ee Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Mon, 2 Sep 2024 10:45:12 +0800 Subject: [PATCH 22/85] fix --- .../db/pipe/consensus/deletion/DeletionResourceManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java index 5f02afb9b912..585a686d1f3f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java @@ -107,7 +107,7 @@ private void initAndRecover() throws IOException { deletionResources.addAll(deletionReader.readAllDeletions()); } } - this.hasCompletedRecovery = true; + hasCompletedRecovery = true; recoveryReadyCondition.signalAll(); } } finally { From 1468eab3cbb01ffeb59adc5b2184116a6a7a3999 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Mon, 2 Sep 2024 10:46:08 +0800 Subject: [PATCH 23/85] fix --- .../db/pipe/consensus/deletion/DeletionResourceManager.java | 1 - 1 file changed, 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java index 585a686d1f3f..da3bb507fa36 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java @@ -66,7 +66,6 @@ public class DeletionResourceManager implements AutoCloseable { private final File storageDir; private final List deletionResources = new CopyOnWriteArrayList<>(); private final Lock recoverLock = new ReentrantLock(); - // condition to guarantee correctness of switching buffers private final Condition recoveryReadyCondition = recoverLock.newCondition(); private boolean hasCompletedRecovery = false; From 8a47b93da2d765cf505a878a2d9745031da35ba6 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Mon, 2 Sep 2024 14:41:07 +0800 Subject: [PATCH 24/85] deletion close --- .../deletion/DeletionResourceManager.java | 26 +++++++++++++++++-- .../deletion/persist/DeletionBuffer.java | 9 +++++++ .../iotdb/db/service/IoTDBShutdownHook.java | 8 +++++- 3 files changed, 40 insertions(+), 3 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java index da3bb507fa36..81b963e0eb5b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java @@ -115,9 +115,21 @@ private void initAndRecover() throws IOException { } @Override - public void close() throws Exception { - this.deletionBuffer.close(); + public void close() { this.deletionResources.clear(); + this.deletionBuffer.close(); + waitUntilFlushAllDeletions(); + } + + private void waitUntilFlushAllDeletions() { + while (!deletionBuffer.isAllDeletionFlushed()) { + try { + Thread.sleep(50); + } catch (InterruptedException e) { + LOGGER.error("Interrupted when waiting for all deletions flushed."); + Thread.currentThread().interrupt(); + } + } } public void registerDeletionResource(PipeSchemaRegionWritePlanEvent event) { @@ -236,6 +248,16 @@ public static void build() { } } + public static void exit() { + if (DeletionResourceManagerHolder.CONSENSU_GROUP_ID_2_INSTANCE_MAP == null) { + return; + } + DeletionResourceManagerHolder.CONSENSU_GROUP_ID_2_INSTANCE_MAP.forEach( + (groupId, resourceManager) -> { + resourceManager.close(); + }); + } + @TestOnly public static void buildForTest() { DeletionResourceManagerHolder.build(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java index 11403586c5e3..f27d3e550480 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java @@ -118,6 +118,15 @@ public void registerDeletionResource(DeletionResource deletionResource) { deletionResources.add(deletionResource); } + public boolean isAllDeletionFlushed() { + buffersLock.lock(); + try { + return deletionResources.isEmpty() && workingBuffer.position() == 0 && syncingBuffer == null; + } finally { + buffersLock.unlock(); + } + } + private void allocateBuffers() { try { workingBuffer = ByteBuffer.allocateDirect(ONE_THIRD_WAL_BUFFER_SIZE); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/IoTDBShutdownHook.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/IoTDBShutdownHook.java index 8ef6bbdc4308..06b194b7d409 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/IoTDBShutdownHook.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/IoTDBShutdownHook.java @@ -29,6 +29,7 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.consensus.DataRegionConsensusImpl; import org.apache.iotdb.db.consensus.SchemaRegionConsensusImpl; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; import org.apache.iotdb.db.protocol.client.ConfigNodeClient; import org.apache.iotdb.db.protocol.client.ConfigNodeClientManager; import org.apache.iotdb.db.protocol.client.ConfigNodeInfo; @@ -84,6 +85,9 @@ public void run() { } WALManager.getInstance().syncDeleteOutdatedFilesInWALNodes(); + // Wait all deletions are flushed + DeletionResourceManager.exit(); + // We did this work because the RatisConsensus recovery mechanism is different from other // consensus algorithms, which will replace the underlying storage engine based on its // own @@ -93,7 +97,9 @@ public void run() { .getConfig() .getDataRegionConsensusProtocolClass() .equals(ConsensusFactory.RATIS_CONSENSUS)) { - DataRegionConsensusImpl.getInstance().getAllConsensusGroupIds().parallelStream() + DataRegionConsensusImpl.getInstance() + .getAllConsensusGroupIds() + .parallelStream() .forEach( id -> { try { From 848c82d5140f2acfe9fe62ab70d2029d755d52f4 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Mon, 2 Sep 2024 14:44:07 +0800 Subject: [PATCH 25/85] add log --- .../db/pipe/consensus/deletion/DeletionResourceManager.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java index 81b963e0eb5b..49b949d9537b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java @@ -116,9 +116,11 @@ private void initAndRecover() throws IOException { @Override public void close() { + LOGGER.info("Closing deletion resource manager for {}...", dataRegionId); this.deletionResources.clear(); this.deletionBuffer.close(); waitUntilFlushAllDeletions(); + LOGGER.info("Deletion resource manager for {} has been successfully closed!", dataRegionId); } private void waitUntilFlushAllDeletions() { From fa0780c9ba495e8ba71d7488a7ef91e928cfc174 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Mon, 2 Sep 2024 14:46:24 +0800 Subject: [PATCH 26/85] format --- .../PipeHistoricalDataRegionTsFileAndDeletionExtractor.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java index 2cfda02dfc89..909f0e7c3afb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java @@ -498,11 +498,9 @@ && mayTsFileResourceOverlappedWithPattern(resource)) @Override public synchronized void start() { - if (!StorageEngine.getInstance().isReadyForNonReadWriteFunctions()) { return; } - if (!shouldExtractInsertion) { return; } From edca176a01cbf4d6e60124fe7de8693cf987b34d Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Mon, 2 Sep 2024 16:01:04 +0800 Subject: [PATCH 27/85] spotless --- .../java/org/apache/iotdb/db/service/IoTDBShutdownHook.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/IoTDBShutdownHook.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/IoTDBShutdownHook.java index 06b194b7d409..b10463f59bc5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/IoTDBShutdownHook.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/IoTDBShutdownHook.java @@ -97,9 +97,7 @@ public void run() { .getConfig() .getDataRegionConsensusProtocolClass() .equals(ConsensusFactory.RATIS_CONSENSUS)) { - DataRegionConsensusImpl.getInstance() - .getAllConsensusGroupIds() - .parallelStream() + DataRegionConsensusImpl.getInstance().getAllConsensusGroupIds().parallelStream() .forEach( id -> { try { From 19a4471d864e855c1bea7a85c079c79f003e6477 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Mon, 2 Sep 2024 18:34:17 +0800 Subject: [PATCH 28/85] fix deserialize --- .../db/pipe/consensus/deletion/DeletionResource.java | 8 +++++--- .../common/schema/PipeSchemaRegionWritePlanEvent.java | 1 - 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java index c60a3a3a1443..87ed76dd13a8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java @@ -21,8 +21,10 @@ import org.apache.iotdb.commons.consensus.index.ProgressIndex; import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; +import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaSerializableEventType; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; +import java.io.IOException; import java.nio.ByteBuffer; import java.util.Objects; import java.util.concurrent.atomic.AtomicLong; @@ -88,9 +90,9 @@ public ByteBuffer serialize() { } public static DeletionResource deserialize( - final ByteBuffer buffer, final Consumer removeHook) { - PipeSchemaRegionWritePlanEvent event = new PipeSchemaRegionWritePlanEvent(); - event.deserializeFromByteBuffer(buffer); + final ByteBuffer buffer, final Consumer removeHook) throws IOException { + PipeSchemaRegionWritePlanEvent event = + (PipeSchemaRegionWritePlanEvent) PipeSchemaSerializableEventType.deserialize(buffer); return new DeletionResource(event, removeHook); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java index 4a9a58a807e1..f70491e5273a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java @@ -104,7 +104,6 @@ public ByteBuffer serializeToByteBuffer() { @Override public void deserializeFromByteBuffer(final ByteBuffer buffer) { - byte ignoredEventType = ReadWriteIOUtils.readByte(buffer); isGeneratedByPipe = ReadWriteIOUtils.readBool(buffer); planNode = PlanNodeType.deserialize(buffer); } From 2262724b6775b6be284da54f2ee5ed1b1c0b96c6 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Tue, 3 Sep 2024 11:32:35 +0800 Subject: [PATCH 29/85] keep config consistent with wal --- .../db/pipe/consensus/deletion/persist/DeletionBuffer.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java index f27d3e550480..634b23a2685c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java @@ -51,7 +51,7 @@ public class DeletionBuffer implements AutoCloseable { private static final Logger LOGGER = LoggerFactory.getLogger(DeletionBuffer.class); private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig(); - // TODO: make it deletion own + // Buffer config keep consistent with WAL. private static final int ONE_THIRD_WAL_BUFFER_SIZE = config.getWalBufferSize() / 3; private static final double FSYNC_BUFFER_RATIO = 0.95; private static final int QUEUE_CAPACITY = config.getWalBufferQueueCapacity(); @@ -210,9 +210,9 @@ private void serialize() { while (totalSize < ONE_THIRD_WAL_BUFFER_SIZE * FSYNC_BUFFER_RATIO) { DeletionResource deletionResource = null; try { - // TODO: add deletion timeout to config + // Timeout config keep consistent with WAL async mode. deletionResource = - deletionResources.poll(config.getWalSyncModeFsyncDelayInMs(), TimeUnit.MILLISECONDS); + deletionResources.poll(config.getWalAsyncModeFsyncDelayInMs(), TimeUnit.MILLISECONDS); } catch (InterruptedException e) { LOGGER.warn( "Interrupted when waiting for taking WALEntry from blocking queue to serialize."); From e60bea62814453f85c3a8b30aace503bea62c9ad Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Thu, 5 Sep 2024 14:15:15 +0800 Subject: [PATCH 30/85] fix review --- .../consensus/deletion/DeletionResourceManager.java | 2 +- .../consensus/deletion/recover/DeletionReader.java | 12 +++++++++--- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java index 49b949d9537b..f9404b55fd1b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java @@ -67,7 +67,7 @@ public class DeletionResourceManager implements AutoCloseable { private final List deletionResources = new CopyOnWriteArrayList<>(); private final Lock recoverLock = new ReentrantLock(); private final Condition recoveryReadyCondition = recoverLock.newCondition(); - private boolean hasCompletedRecovery = false; + private volatile boolean hasCompletedRecovery = false; private DeletionResourceManager(String dataRegionId) throws IOException { this.dataRegionId = dataRegionId; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java index e3d7ffd187bd..630b51b739a6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java @@ -58,14 +58,18 @@ public List readAllDeletions() throws IOException { fileChannel.read(magicStringBuffer); magicStringBuffer.flip(); String magicVersion = new String(magicStringBuffer.array(), StandardCharsets.UTF_8); - LOGGER.debug("Read deletion file-{} magic version: {}", logFile, magicVersion); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("Read deletion file-{} magic version: {}", logFile, magicVersion); + } // Read metaData ByteBuffer intBuffer = ByteBuffer.allocate(4); fileChannel.read(intBuffer); intBuffer.flip(); int deletionNum = intBuffer.getInt(); - LOGGER.debug("Read deletion file-{} contains {} deletions", logFile, deletionNum); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("Read deletion file-{} contains {} deletions", logFile, deletionNum); + } // Read deletions long remainingBytes = fileChannel.size() - fileChannel.position(); @@ -77,7 +81,9 @@ public List readAllDeletions() throws IOException { for (int i = 0; i < deletionNum; i++) { DeletionResource deletionResource = DeletionResource.deserialize(byteBuffer, removeHook); deletions.add(deletionResource); - LOGGER.debug("Read deletion: {} from file {}", deletionResource, logFile); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("Read deletion: {} from file {}", deletionResource, logFile); + } } return deletions; } From 360637a345f4b62a27f10eef2d2dda001ad2322c Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Tue, 10 Sep 2024 16:25:49 +0800 Subject: [PATCH 31/85] introduce page cache version DAL and fix review --- .../consensus/deletion/DeletionResource.java | 45 +++ .../deletion/DeletionResourceManager.java | 6 +- .../deletion/persist/DeletionBuffer.java | 366 +---------------- .../persist/PageCacheDeletionBuffer.java | 317 +++++++++++++++ .../persist/TwoStageDeletionBuffer.java | 375 ++++++++++++++++++ .../realtime/PipeRealtimeEventFactory.java | 2 - .../PipeInsertionDataNodeListener.java | 17 +- .../storageengine/dataregion/DataRegion.java | 24 +- 8 files changed, 775 insertions(+), 377 deletions(-) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/TwoStageDeletionBuffer.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java index 87ed76dd13a8..abce9c99e627 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java @@ -24,6 +24,9 @@ import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaSerializableEventType; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.IOException; import java.nio.ByteBuffer; import java.util.Objects; @@ -36,14 +39,20 @@ * framework, PipeConsensus will use {@link PipeSchemaRegionWritePlanEvent} */ public class DeletionResource { + private static final Logger LOGGER = LoggerFactory.getLogger(DeletionResource.class); private final Consumer removeHook; private final AtomicLong latestUpdateTime; private PipeSchemaRegionWritePlanEvent deletionEvent; + private volatile Status currentStatus; + // it's safe to use volatile here to make this reference thread-safe. + @SuppressWarnings("squid:S3077") + private volatile Exception cause; public DeletionResource( PipeSchemaRegionWritePlanEvent deletionEvent, Consumer removeHook) { this.deletionEvent = deletionEvent; this.removeHook = removeHook; + this.currentStatus = Status.RUNNING; latestUpdateTime = new AtomicLong(System.currentTimeMillis()); } @@ -77,6 +86,32 @@ public long getLatestUpdateTime() { return latestUpdateTime.get(); } + public void onPersistFailed(Exception e) { + cause = e; + currentStatus = Status.FAILURE; + this.notifyAll(); + } + + public void onPersistSucceed() { + currentStatus = Status.SUCCESS; + this.notifyAll(); + } + + /** @return true if this object has been successfully persisted, false if persist failed. */ + public synchronized Status waitForResult() { + while (currentStatus == Status.RUNNING) { + try { + this.wait(); + } catch (InterruptedException e) { + LOGGER.warn("Interrupted when waiting for result.", e); + Thread.currentThread().interrupt(); + currentStatus = Status.FAILURE; + break; + } + } + return currentStatus; + } + public ProgressIndex getProgressIndex() { return ((DeleteDataNode) deletionEvent.getPlanNode()).getProgressIndex(); } @@ -124,4 +159,14 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(deletionEvent, latestUpdateTime); } + + public Exception getCause() { + return cause; + } + + public enum Status { + SUCCESS, + FAILURE, + RUNNING, + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java index f9404b55fd1b..6b6a7d30c5a3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java @@ -28,6 +28,7 @@ import org.apache.iotdb.db.consensus.DataRegionConsensusImpl; import org.apache.iotdb.db.pipe.consensus.ProgressIndexDataNodeManager; import org.apache.iotdb.db.pipe.consensus.deletion.persist.DeletionBuffer; +import org.apache.iotdb.db.pipe.consensus.deletion.persist.PageCacheDeletionBuffer; import org.apache.iotdb.db.pipe.consensus.deletion.recover.DeletionReader; import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; @@ -76,7 +77,7 @@ private DeletionResourceManager(String dataRegionId) throws IOException { IoTDBDescriptor.getInstance().getConfig().getPipeConsensusDeletionFileDir() + File.separator + dataRegionId); - this.deletionBuffer = new DeletionBuffer(dataRegionId, storageDir.getAbsolutePath()); + this.deletionBuffer = new PageCacheDeletionBuffer(dataRegionId, storageDir.getAbsolutePath()); initAndRecover(); // Only after initAndRecover can we start serialize and sync new deletions. this.deletionBuffer.start(); @@ -134,11 +135,12 @@ private void waitUntilFlushAllDeletions() { } } - public void registerDeletionResource(PipeSchemaRegionWritePlanEvent event) { + public DeletionResource registerDeletionResource(PipeSchemaRegionWritePlanEvent event) { DeletionResource deletionResource = new DeletionResource(event, this::removeDeletionResource); event.setDeletionResource(deletionResource); this.deletionResources.add(deletionResource); deletionBuffer.registerDeletionResource(deletionResource); + return deletionResource; } public List getAllDeletionResources() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java index 634b23a2685c..41e44991d41e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java @@ -1,369 +1,13 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - package org.apache.iotdb.db.pipe.consensus.deletion.persist; -import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory; -import org.apache.iotdb.commons.concurrent.ThreadName; -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.SimpleProgressIndex; -import org.apache.iotdb.db.conf.IoTDBConfig; -import org.apache.iotdb.db.conf.IoTDBDescriptor; -import org.apache.iotdb.db.pipe.consensus.ProgressIndexDataNodeManager; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; -import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; -import org.apache.iotdb.db.utils.MmapUtil; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.channels.FileChannel; -import java.nio.charset.StandardCharsets; -import java.util.concurrent.ArrayBlockingQueue; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.locks.Condition; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; - -public class DeletionBuffer implements AutoCloseable { - private static final Logger LOGGER = LoggerFactory.getLogger(DeletionBuffer.class); - private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig(); - // Buffer config keep consistent with WAL. - private static final int ONE_THIRD_WAL_BUFFER_SIZE = config.getWalBufferSize() / 3; - private static final double FSYNC_BUFFER_RATIO = 0.95; - private static final int QUEUE_CAPACITY = config.getWalBufferQueueCapacity(); - - // whether close method is called - private volatile boolean isClosed = false; - // DeletionResources - private final BlockingQueue deletionResources = - new ArrayBlockingQueue<>(QUEUE_CAPACITY); - // lock to provide synchronization for double buffers mechanism, protecting buffers status - private final Lock buffersLock = new ReentrantLock(); - // condition to guarantee correctness of switching buffers - private final Condition idleBufferReadyCondition = buffersLock.newCondition(); - private final String groupId; - - // region these variables should be protected by buffersLock - /** two buffers switch between three statuses (there is always 1 buffer working). */ - // buffer in working status, only updated by serializeThread - // it's safe to use volatile here to make this reference thread-safe. - @SuppressWarnings("squid:S3077") - private volatile ByteBuffer workingBuffer; - - // buffer in idle status - // it's safe to use volatile here to make this reference thread-safe. - @SuppressWarnings("squid:S3077") - private volatile ByteBuffer idleBuffer; - - // buffer in syncing status, serializeThread makes sure no more writes to syncingBuffer - // it's safe to use volatile here to make this reference thread-safe. - @SuppressWarnings("squid:S3077") - private volatile ByteBuffer syncingBuffer; - - // single thread to serialize WALEntry to workingBuffer - private final ExecutorService serializeThread; - // single thread to sync syncingBuffer to disk - private final ExecutorService syncBufferThread; - // directory to store .deletion files - private final String baseDirectory; - - public DeletionBuffer(String groupId, String baseDirectory) { - this.groupId = groupId; - this.baseDirectory = baseDirectory; - allocateBuffers(); - serializeThread = - IoTDBThreadPoolFactory.newSingleThreadExecutor( - ThreadName.PIPE_CONSENSUS_DELETION_SERIALIZE.getName() + "(group-" + groupId + ")"); - syncBufferThread = - IoTDBThreadPoolFactory.newSingleThreadExecutor( - ThreadName.PIPE_CONSENSUS_DELETION_SYNC.getName() + "(group-" + groupId + ")"); - } - - public void start() { - // Start serialize and sync pipeline. - serializeThread.submit(new SerializeTask()); - } - - public void registerDeletionResource(DeletionResource deletionResource) { - if (isClosed) { - LOGGER.error( - "Fail to register DeletionResource into deletionBuffer-{} because this buffer is closed.", - groupId); - return; - } - deletionResources.add(deletionResource); - } - - public boolean isAllDeletionFlushed() { - buffersLock.lock(); - try { - return deletionResources.isEmpty() && workingBuffer.position() == 0 && syncingBuffer == null; - } finally { - buffersLock.unlock(); - } - } - - private void allocateBuffers() { - try { - workingBuffer = ByteBuffer.allocateDirect(ONE_THIRD_WAL_BUFFER_SIZE); - idleBuffer = ByteBuffer.allocateDirect(ONE_THIRD_WAL_BUFFER_SIZE); - } catch (OutOfMemoryError e) { - LOGGER.error( - "Fail to allocate deletionBuffer-group-{}'s buffer because out of memory.", groupId, e); - close(); - throw e; - } - } - - /** Notice: this method only called when buffer is exhausted by SerializeTask. */ - private void syncWorkingBuffer(ProgressIndex maxProgressIndexInCurrentBatch, int deletionNum) { - switchWorkingBufferToFlushing(); - try { - syncBufferThread.submit(new SyncBufferTask(maxProgressIndexInCurrentBatch, deletionNum)); - } catch (IOException e) { - LOGGER.warn( - "Failed to submit syncBufferTask, May because file open error and cause data inconsistency. Please check your file system. ", - e); - } - } - - // only called by serializeThread - private void switchWorkingBufferToFlushing() { - buffersLock.lock(); - try { - while (idleBuffer == null) { - idleBufferReadyCondition.await(); - } - syncingBuffer = workingBuffer; - workingBuffer = idleBuffer; - workingBuffer.clear(); - idleBuffer = null; - } catch (InterruptedException e) { - LOGGER.warn("Interrupted When waiting for available working buffer."); - Thread.currentThread().interrupt(); - } finally { - buffersLock.unlock(); - } - } - - private class SerializeTask implements Runnable { - // Total size of this batch. - private int totalSize = 0; - // Deletion num of this batch. - private int deletionNum = 0; - // Max progressIndex among this batch. Used by SyncTask for naming .deletion file. - private ProgressIndex maxProgressIndexInCurrentBatch = MinimumProgressIndex.INSTANCE; - - @Override - public void run() { - try { - serialize(); - } finally { - if (!isClosed) { - serializeThread.submit(new SerializeTask()); - } - } - } - - private void serialize() { - // For first deletion we use blocking take() method. - try { - DeletionResource firstDeletionResource = deletionResources.take(); - // For first serialization, we don't need to judge whether working buffer is exhausted. - // Because a single DeleteDataNode can't exceed size of working buffer. - serializeToWorkingBuffer(firstDeletionResource); - maxProgressIndexInCurrentBatch = - maxProgressIndexInCurrentBatch.updateToMinimumEqualOrIsAfterProgressIndex( - firstDeletionResource.getProgressIndex()); - } catch (InterruptedException e) { - LOGGER.warn( - "Interrupted when waiting for taking DeletionResource from blocking queue to serialize."); - Thread.currentThread().interrupt(); - } - - // For further deletion, we use non-blocking poll() method to persist existing deletion of - // current batch in time. - while (totalSize < ONE_THIRD_WAL_BUFFER_SIZE * FSYNC_BUFFER_RATIO) { - DeletionResource deletionResource = null; - try { - // Timeout config keep consistent with WAL async mode. - deletionResource = - deletionResources.poll(config.getWalAsyncModeFsyncDelayInMs(), TimeUnit.MILLISECONDS); - } catch (InterruptedException e) { - LOGGER.warn( - "Interrupted when waiting for taking WALEntry from blocking queue to serialize."); - Thread.currentThread().interrupt(); - } - // If timeout, flush deletions to disk. - if (deletionResource == null) { - break; - } - // Serialize deletion - while (!serializeToWorkingBuffer(deletionResource)) { - // If working buffer is exhausted, submit a syncTask to consume current batch and switch - // buffer to start a new batch. - syncWorkingBuffer(maxProgressIndexInCurrentBatch, deletionNum); - // Reset maxProgressIndex and deletionNum for new batch. - maxProgressIndexInCurrentBatch = MinimumProgressIndex.INSTANCE; - deletionNum = 0; - } - // Update max progressIndex - maxProgressIndexInCurrentBatch = - maxProgressIndexInCurrentBatch.updateToMinimumEqualOrIsAfterProgressIndex( - deletionResource.getProgressIndex()); - } - // Persist deletions; Defensive programming here, just in case. - if (totalSize > 0) { - syncWorkingBuffer(maxProgressIndexInCurrentBatch, deletionNum); - } - } - - /** - * Serialize deletionResource to working buffer. Return true if serialize successfully, false - * otherwise. - */ - private boolean serializeToWorkingBuffer(DeletionResource deletionResource) { - ByteBuffer buffer = deletionResource.serialize(); - // if working buffer doesn't have enough space - if (buffer.position() > workingBuffer.remaining()) { - return false; - } - workingBuffer.put(buffer.array()); - totalSize += buffer.position(); - deletionNum++; - return true; - } - } - - // only called by syncBufferThread - private void switchSyncingBufferToIdle() { - buffersLock.lock(); - try { - // No need to judge whether idleBuffer is null because syncingBuffer is not null - // and there is only one buffer can be null between syncingBuffer and idleBuffer - idleBuffer = syncingBuffer; - syncingBuffer = null; - idleBufferReadyCondition.signalAll(); - } finally { - buffersLock.unlock(); - } - } - - private class SyncBufferTask implements Runnable { - private final int deletionNum; - private final File logFile; - private final FileOutputStream logStream; - private final FileChannel logChannel; - - public SyncBufferTask(ProgressIndex maxProgressIndexInCurrentBatch, int deletionNum) - throws IOException { - this.deletionNum = deletionNum; - // PipeConsensus ensures that deleteDataNodes use recoverProgressIndex. - ProgressIndex curProgressIndex = - ProgressIndexDataNodeManager.extractLocalSimpleProgressIndex( - maxProgressIndexInCurrentBatch); - if (!(curProgressIndex instanceof SimpleProgressIndex)) { - throw new IOException("Invalid deletion progress index: " + curProgressIndex); - } - SimpleProgressIndex progressIndex = (SimpleProgressIndex) curProgressIndex; - // Deletion file name format: "_{rebootTimes}_{memTableFlushOrderId}.deletion" - this.logFile = - new File( - baseDirectory, - String.format( - "_%d-%d%s", - progressIndex.getRebootTimes(), - progressIndex.getMemTableFlushOrderId(), - DeletionResourceManager.DELETION_FILE_SUFFIX)); - this.logStream = new FileOutputStream(logFile, true); - this.logChannel = logStream.getChannel(); - // Create file && write magic string - if (!logFile.exists() || logFile.length() == 0) { - this.logChannel.write( - ByteBuffer.wrap( - DeletionResourceManager.MAGIC_VERSION_V1.getBytes(StandardCharsets.UTF_8))); - } - } - @Override - public void run() { - // Sync deletion to disk. - workingBuffer.flip(); - try { - // Write metaData. - ByteBuffer metaData = ByteBuffer.allocate(4); - metaData.putInt(deletionNum); - metaData.flip(); - this.logChannel.write(metaData); - // Write deletions. - syncingBuffer.flip(); - this.logChannel.write(syncingBuffer); - } catch (IOException e) { - LOGGER.warn( - "Deletion persist: Cannot write to {}, may cause data inconsistency.", logFile, e); - } finally { - switchSyncingBufferToIdle(); - } - // Close resource. - try { - this.logChannel.close(); - this.logStream.close(); - } catch (IOException e) { - LOGGER.warn("Failed to close deletion writing resource when writing to {}.", logFile, e); - } - } - } +public interface DeletionBuffer extends AutoCloseable { + void start(); - @Override - public void close() { - isClosed = true; - // Force sync existing data in memory to disk. - // first waiting serialize and sync tasks finished, then release all resources - if (serializeThread != null) { - shutdownThread(serializeThread, ThreadName.PIPE_CONSENSUS_DELETION_SERIALIZE); - } - if (syncBufferThread != null) { - shutdownThread(syncBufferThread, ThreadName.PIPE_CONSENSUS_DELETION_SYNC); - } + void close(); - MmapUtil.clean(workingBuffer); - MmapUtil.clean(workingBuffer); - MmapUtil.clean(syncingBuffer); - } + void registerDeletionResource(DeletionResource deletionResource); - private void shutdownThread(ExecutorService thread, ThreadName threadName) { - thread.shutdown(); - try { - if (!thread.awaitTermination(30, TimeUnit.SECONDS)) { - LOGGER.warn("Waiting thread {} to be terminated is timeout", threadName.getName()); - } - } catch (InterruptedException e) { - LOGGER.warn("Thread {} still doesn't exit after 30s", threadName.getName()); - Thread.currentThread().interrupt(); - } - } + boolean isAllDeletionFlushed(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java new file mode 100644 index 000000000000..477edfd7d57b --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java @@ -0,0 +1,317 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iotdb.db.pipe.consensus.deletion.persist; + +import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory; +import org.apache.iotdb.commons.concurrent.ThreadName; +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.SimpleProgressIndex; +import org.apache.iotdb.db.conf.IoTDBConfig; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.pipe.consensus.ProgressIndexDataNodeManager; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; +import org.apache.iotdb.db.utils.MmapUtil; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +/** + * The core idea of this buffer is to delete writes to the Page cache and fsync when certain + * conditions are met. This design does not decouple serialization and writing, but provides easier + * writing. + */ +public class PageCacheDeletionBuffer implements DeletionBuffer { + private static final Logger LOGGER = LoggerFactory.getLogger(TwoStageDeletionBuffer.class); + private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig(); + // Buffer config keep consistent with WAL. + private static final int ONE_THIRD_WAL_BUFFER_SIZE = config.getWalBufferSize() / 3; + private static final double FSYNC_BUFFER_RATIO = 0.95; + private static final int QUEUE_CAPACITY = config.getWalBufferQueueCapacity(); + + // DeletionResources + private final BlockingQueue deletionResources = + new ArrayBlockingQueue<>(QUEUE_CAPACITY); + // Data region id + private final String groupId; + // directory to store .deletion files + private final String baseDirectory; + // single thread to serialize WALEntry to workingBuffer + private final ExecutorService persistThread; + private final Lock buffersLock = new ReentrantLock(); + // Total size of this batch. + private final AtomicInteger totalSize = new AtomicInteger(0); + // Deletion num of this batch. + private final AtomicInteger deletionNum = new AtomicInteger(0); + // All deletions that will be written to the current file + private final List pendingDeletions = new ArrayList<>(); + + // whether close method is called + private volatile boolean isClosed = false; + // Serialize buffer in current batch + private volatile ByteBuffer serializeBuffer; + // Current Logging file. + private volatile File logFile; + private volatile FileOutputStream logStream; + private volatile FileChannel logChannel; + // Max progressIndex among last batch. Used by PersistTask for naming .deletion file. + // Since deletions are written serially, DAL is also written serially. This ensures that the + // maxProgressIndex of each batch increases in the same order as the physical time. + private volatile ProgressIndex maxProgressIndexInLastBatch = MinimumProgressIndex.INSTANCE; + + public PageCacheDeletionBuffer(String groupId, String baseDirectory) { + this.groupId = groupId; + this.baseDirectory = baseDirectory; + allocateBuffers(); + persistThread = + IoTDBThreadPoolFactory.newSingleThreadExecutor( + ThreadName.PIPE_CONSENSUS_DELETION_SERIALIZE.getName() + "(group-" + groupId + ")"); + } + + @Override + public void start() { + persistThread.submit(new PersistTask()); + } + + @Override + public boolean isAllDeletionFlushed() { + buffersLock.lock(); + try { + return deletionResources.isEmpty() && serializeBuffer.position() == 0; + } finally { + buffersLock.unlock(); + } + } + + private void allocateBuffers() { + try { + serializeBuffer = ByteBuffer.allocateDirect(ONE_THIRD_WAL_BUFFER_SIZE); + } catch (OutOfMemoryError e) { + LOGGER.error( + "Fail to allocate deletionBuffer-group-{}'s buffer because out of memory.", groupId, e); + close(); + throw e; + } + } + + public void registerDeletionResource(DeletionResource deletionResource) { + if (isClosed) { + LOGGER.error( + "Fail to register DeletionResource into deletionBuffer-{} because this buffer is closed.", + groupId); + return; + } + deletionResources.add(deletionResource); + } + + private void appendCurrentBatch() throws IOException { + serializeBuffer.flip(); + logChannel.write(serializeBuffer); + } + + private void fsyncCurrentLoggingFileAndReset(int deletionNum, ProgressIndex curMaxProgressIndex) + throws IOException { + try { + // Write metaData. + ByteBuffer metaData = ByteBuffer.allocate(4); + metaData.putInt(deletionNum); + metaData.flip(); + this.logChannel.write(metaData); + // Close old resource to fsync. + this.logStream.close(); + this.logChannel.close(); + // Mark DeletionResources to persisted + this.pendingDeletions.forEach(DeletionResource::onPersistSucceed); + } finally { + reset(curMaxProgressIndex); + } + } + + private void reset(ProgressIndex curMaxProgressIndex) { + // Reset file attributes. + this.totalSize.set(0); + this.deletionNum.set(0); + this.maxProgressIndexInLastBatch = curMaxProgressIndex; + this.pendingDeletions.clear(); + // Clear serialize buffer + buffersLock.lock(); + try { + serializeBuffer.clear(); + } finally { + buffersLock.unlock(); + } + } + + private void switchLoggingFile() throws IOException { + // PipeConsensus ensures that deleteDataNodes use recoverProgressIndex. + ProgressIndex curProgressIndex = + ProgressIndexDataNodeManager.extractLocalSimpleProgressIndex(maxProgressIndexInLastBatch); + if (!(curProgressIndex instanceof SimpleProgressIndex)) { + throw new IOException("Invalid deletion progress index: " + curProgressIndex); + } + SimpleProgressIndex progressIndex = (SimpleProgressIndex) curProgressIndex; + // Deletion file name format: "_{rebootTimes}_{memTableFlushOrderId}.deletion" + this.logFile = + new File( + baseDirectory, + String.format( + "_%d-%d%s", + progressIndex.getRebootTimes(), + progressIndex.getMemTableFlushOrderId(), + DeletionResourceManager.DELETION_FILE_SUFFIX)); + this.logStream = new FileOutputStream(logFile, true); + this.logChannel = logStream.getChannel(); + // Create file && write magic string + if (!logFile.exists() || logFile.length() == 0) { + this.logChannel.write( + ByteBuffer.wrap( + DeletionResourceManager.MAGIC_VERSION_V1.getBytes(StandardCharsets.UTF_8))); + } + } + + private class PersistTask implements Runnable { + // Max progressIndex among this batch. Used by SyncTask for naming .deletion file. + private ProgressIndex maxProgressIndexInCurrentBatch = MinimumProgressIndex.INSTANCE; + + @Override + public void run() { + try { + persistDeletion(); + } catch (IOException e) { + LOGGER.warn( + "Deletion persist: Cannot write to {}, may cause data inconsistency.", logFile, e); + pendingDeletions.forEach(deletionResource -> deletionResource.onPersistFailed(e)); + reset(maxProgressIndexInLastBatch); + } finally { + if (!isClosed) { + persistThread.submit(new PersistTask()); + } + } + } + + private boolean serializeDeletionToBatchBuffer(DeletionResource deletionResource) { + ByteBuffer buffer = deletionResource.serialize(); + // if working buffer doesn't have enough space + if (buffer.position() > serializeBuffer.remaining()) { + return false; + } + serializeBuffer.put(buffer.array()); + totalSize.addAndGet(buffer.position()); + deletionNum.incrementAndGet(); + return true; + } + + private void persistDeletion() throws IOException { + // For first deletion we use blocking take() method. + try { + DeletionResource firstDeletionResource = deletionResources.take(); + pendingDeletions.add(firstDeletionResource); + serializeDeletionToBatchBuffer(firstDeletionResource); + maxProgressIndexInCurrentBatch = + maxProgressIndexInCurrentBatch.updateToMinimumEqualOrIsAfterProgressIndex( + firstDeletionResource.getProgressIndex()); + } catch (InterruptedException e) { + LOGGER.warn( + "Interrupted when waiting for taking DeletionResource from blocking queue to serialize."); + Thread.currentThread().interrupt(); + } + + // For further deletion, we use non-blocking poll() method to persist existing deletion of + // current batch in time. + while (totalSize.get() < ONE_THIRD_WAL_BUFFER_SIZE * FSYNC_BUFFER_RATIO) { + DeletionResource deletionResource = null; + try { + // Timeout config keep consistent with WAL async mode. + deletionResource = + deletionResources.poll(config.getWalAsyncModeFsyncDelayInMs(), TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + LOGGER.warn( + "Interrupted when waiting for taking WALEntry from blocking queue to serialize."); + Thread.currentThread().interrupt(); + } + // If timeout, flush deletions to disk. + if (deletionResource == null) { + // append to current file and not switch file + appendCurrentBatch(); + return; + } + // Serialize deletion + if (!serializeDeletionToBatchBuffer(deletionResource)) { + // If working buffer is exhausted, fsync immediately and roll to a new file. + appendCurrentBatch(); + fsyncCurrentLoggingFileAndReset(deletionNum.get(), maxProgressIndexInCurrentBatch); + switchLoggingFile(); + return; + } + // Update max progressIndex + maxProgressIndexInCurrentBatch = + maxProgressIndexInCurrentBatch.updateToMinimumEqualOrIsAfterProgressIndex( + deletionResource.getProgressIndex()); + pendingDeletions.add(deletionResource); + } + // Persist deletions; Defensive programming here, just in case. + if (totalSize.get() > 0) { + appendCurrentBatch(); + fsyncCurrentLoggingFileAndReset(deletionNum.get(), maxProgressIndexInCurrentBatch); + switchLoggingFile(); + } + } + } + + @Override + public void close() { + isClosed = true; + // Force sync existing data in memory to disk. + // first waiting serialize and sync tasks finished, then release all resources + if (persistThread != null) { + shutdownThread(persistThread, ThreadName.PIPE_CONSENSUS_DELETION_SERIALIZE); + } + // clean buffer + MmapUtil.clean(serializeBuffer); + } + + private void shutdownThread(ExecutorService thread, ThreadName threadName) { + thread.shutdown(); + try { + if (!thread.awaitTermination(30, TimeUnit.SECONDS)) { + LOGGER.warn("Waiting thread {} to be terminated is timeout", threadName.getName()); + } + } catch (InterruptedException e) { + LOGGER.warn("Thread {} still doesn't exit after 30s", threadName.getName()); + Thread.currentThread().interrupt(); + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/TwoStageDeletionBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/TwoStageDeletionBuffer.java new file mode 100644 index 000000000000..a12b089d9a35 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/TwoStageDeletionBuffer.java @@ -0,0 +1,375 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.consensus.deletion.persist; + +import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory; +import org.apache.iotdb.commons.concurrent.ThreadName; +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.SimpleProgressIndex; +import org.apache.iotdb.db.conf.IoTDBConfig; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.pipe.consensus.ProgressIndexDataNodeManager; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; +import org.apache.iotdb.db.utils.MmapUtil; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.charset.StandardCharsets; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +/** + * The core idea of this buffer is to decouple the serialization and writing tasks. Similar to + * WALBuffer, it can perform subsequent operations such as compression in the application state. + */ +public class TwoStageDeletionBuffer implements DeletionBuffer { + private static final Logger LOGGER = LoggerFactory.getLogger(TwoStageDeletionBuffer.class); + private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig(); + // Buffer config keep consistent with WAL. + private static final int ONE_THIRD_WAL_BUFFER_SIZE = config.getWalBufferSize() / 3; + private static final double FSYNC_BUFFER_RATIO = 0.95; + private static final int QUEUE_CAPACITY = config.getWalBufferQueueCapacity(); + + // whether close method is called + private volatile boolean isClosed = false; + // DeletionResources + private final BlockingQueue deletionResources = + new ArrayBlockingQueue<>(QUEUE_CAPACITY); + // lock to provide synchronization for double buffers mechanism, protecting buffers status + private final Lock buffersLock = new ReentrantLock(); + // condition to guarantee correctness of switching buffers + private final Condition idleBufferReadyCondition = buffersLock.newCondition(); + private final String groupId; + + // region these variables should be protected by buffersLock + /** two buffers switch between three statuses (there is always 1 buffer working). */ + // buffer in working status, only updated by serializeThread + // it's safe to use volatile here to make this reference thread-safe. + @SuppressWarnings("squid:S3077") + private volatile ByteBuffer workingBuffer; + + // buffer in idle status + // it's safe to use volatile here to make this reference thread-safe. + @SuppressWarnings("squid:S3077") + private volatile ByteBuffer idleBuffer; + + // buffer in syncing status, serializeThread makes sure no more writes to syncingBuffer + // it's safe to use volatile here to make this reference thread-safe. + @SuppressWarnings("squid:S3077") + private volatile ByteBuffer syncingBuffer; + + // single thread to serialize WALEntry to workingBuffer + private final ExecutorService serializeThread; + // single thread to sync syncingBuffer to disk + private final ExecutorService syncBufferThread; + // directory to store .deletion files + private final String baseDirectory; + + public TwoStageDeletionBuffer(String groupId, String baseDirectory) { + this.groupId = groupId; + this.baseDirectory = baseDirectory; + allocateBuffers(); + serializeThread = + IoTDBThreadPoolFactory.newSingleThreadExecutor( + ThreadName.PIPE_CONSENSUS_DELETION_SERIALIZE.getName() + "(group-" + groupId + ")"); + syncBufferThread = + IoTDBThreadPoolFactory.newSingleThreadExecutor( + ThreadName.PIPE_CONSENSUS_DELETION_SYNC.getName() + "(group-" + groupId + ")"); + } + + @Override + public void start() { + // Start serialize and sync pipeline. + serializeThread.submit(new SerializeTask()); + } + + @Override + public boolean isAllDeletionFlushed() { + buffersLock.lock(); + try { + return deletionResources.isEmpty() && workingBuffer.position() == 0 && syncingBuffer == null; + } finally { + buffersLock.unlock(); + } + } + + public void registerDeletionResource(DeletionResource deletionResource) { + if (isClosed) { + LOGGER.error( + "Fail to register DeletionResource into deletionBuffer-{} because this buffer is closed.", + groupId); + return; + } + deletionResources.add(deletionResource); + } + + private void allocateBuffers() { + try { + workingBuffer = ByteBuffer.allocateDirect(ONE_THIRD_WAL_BUFFER_SIZE); + idleBuffer = ByteBuffer.allocateDirect(ONE_THIRD_WAL_BUFFER_SIZE); + } catch (OutOfMemoryError e) { + LOGGER.error( + "Fail to allocate deletionBuffer-group-{}'s buffer because out of memory.", groupId, e); + close(); + throw e; + } + } + + /** Notice: this method only called when buffer is exhausted by SerializeTask. */ + private void syncWorkingBuffer(ProgressIndex maxProgressIndexInCurrentBatch, int deletionNum) { + switchWorkingBufferToFlushing(); + try { + syncBufferThread.submit(new SyncBufferTask(maxProgressIndexInCurrentBatch, deletionNum)); + } catch (IOException e) { + LOGGER.warn( + "Failed to submit syncBufferTask, May because file open error and cause data inconsistency. Please check your file system. ", + e); + } + } + + // only called by serializeThread + private void switchWorkingBufferToFlushing() { + buffersLock.lock(); + try { + while (idleBuffer == null) { + idleBufferReadyCondition.await(); + } + syncingBuffer = workingBuffer; + workingBuffer = idleBuffer; + workingBuffer.clear(); + idleBuffer = null; + } catch (InterruptedException e) { + LOGGER.warn("Interrupted When waiting for available working buffer."); + Thread.currentThread().interrupt(); + } finally { + buffersLock.unlock(); + } + } + + private class SerializeTask implements Runnable { + // Total size of this batch. + private int totalSize = 0; + // Deletion num of this batch. + private int deletionNum = 0; + // Max progressIndex among this batch. Used by SyncTask for naming .deletion file. + private ProgressIndex maxProgressIndexInCurrentBatch = MinimumProgressIndex.INSTANCE; + + @Override + public void run() { + try { + serialize(); + } finally { + if (!isClosed) { + serializeThread.submit(new SerializeTask()); + } + } + } + + private void serialize() { + // For first deletion we use blocking take() method. + try { + DeletionResource firstDeletionResource = deletionResources.take(); + // For first serialization, we don't need to judge whether working buffer is exhausted. + // Because a single DeleteDataNode can't exceed size of working buffer. + serializeToWorkingBuffer(firstDeletionResource); + maxProgressIndexInCurrentBatch = + maxProgressIndexInCurrentBatch.updateToMinimumEqualOrIsAfterProgressIndex( + firstDeletionResource.getProgressIndex()); + } catch (InterruptedException e) { + LOGGER.warn( + "Interrupted when waiting for taking DeletionResource from blocking queue to serialize."); + Thread.currentThread().interrupt(); + } + + // For further deletion, we use non-blocking poll() method to persist existing deletion of + // current batch in time. + while (totalSize < ONE_THIRD_WAL_BUFFER_SIZE * FSYNC_BUFFER_RATIO) { + DeletionResource deletionResource = null; + try { + // Timeout config keep consistent with WAL async mode. + deletionResource = + deletionResources.poll(config.getWalAsyncModeFsyncDelayInMs(), TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + LOGGER.warn( + "Interrupted when waiting for taking WALEntry from blocking queue to serialize."); + Thread.currentThread().interrupt(); + } + // If timeout, flush deletions to disk. + if (deletionResource == null) { + break; + } + // Serialize deletion + while (!serializeToWorkingBuffer(deletionResource)) { + // If working buffer is exhausted, submit a syncTask to consume current batch and switch + // buffer to start a new batch. + syncWorkingBuffer(maxProgressIndexInCurrentBatch, deletionNum); + // Reset maxProgressIndex and deletionNum for new batch. + maxProgressIndexInCurrentBatch = MinimumProgressIndex.INSTANCE; + deletionNum = 0; + } + // Update max progressIndex + maxProgressIndexInCurrentBatch = + maxProgressIndexInCurrentBatch.updateToMinimumEqualOrIsAfterProgressIndex( + deletionResource.getProgressIndex()); + } + // Persist deletions; Defensive programming here, just in case. + if (totalSize > 0) { + syncWorkingBuffer(maxProgressIndexInCurrentBatch, deletionNum); + } + } + + /** + * Serialize deletionResource to working buffer. Return true if serialize successfully, false + * otherwise. + */ + private boolean serializeToWorkingBuffer(DeletionResource deletionResource) { + ByteBuffer buffer = deletionResource.serialize(); + // if working buffer doesn't have enough space + if (buffer.position() > workingBuffer.remaining()) { + return false; + } + workingBuffer.put(buffer.array()); + totalSize += buffer.position(); + deletionNum++; + return true; + } + } + + // only called by syncBufferThread + private void switchSyncingBufferToIdle() { + buffersLock.lock(); + try { + // No need to judge whether idleBuffer is null because syncingBuffer is not null + // and there is only one buffer can be null between syncingBuffer and idleBuffer + idleBuffer = syncingBuffer; + syncingBuffer = null; + idleBufferReadyCondition.signalAll(); + } finally { + buffersLock.unlock(); + } + } + + private class SyncBufferTask implements Runnable { + private final int deletionNum; + private final File logFile; + private final FileOutputStream logStream; + private final FileChannel logChannel; + + public SyncBufferTask(ProgressIndex maxProgressIndexInCurrentBatch, int deletionNum) + throws IOException { + this.deletionNum = deletionNum; + // PipeConsensus ensures that deleteDataNodes use recoverProgressIndex. + ProgressIndex curProgressIndex = + ProgressIndexDataNodeManager.extractLocalSimpleProgressIndex( + maxProgressIndexInCurrentBatch); + if (!(curProgressIndex instanceof SimpleProgressIndex)) { + throw new IOException("Invalid deletion progress index: " + curProgressIndex); + } + SimpleProgressIndex progressIndex = (SimpleProgressIndex) curProgressIndex; + // Deletion file name format: "_{rebootTimes}_{memTableFlushOrderId}.deletion" + this.logFile = + new File( + baseDirectory, + String.format( + "_%d-%d%s", + progressIndex.getRebootTimes(), + progressIndex.getMemTableFlushOrderId(), + DeletionResourceManager.DELETION_FILE_SUFFIX)); + this.logStream = new FileOutputStream(logFile, true); + this.logChannel = logStream.getChannel(); + // Create file && write magic string + if (!logFile.exists() || logFile.length() == 0) { + this.logChannel.write( + ByteBuffer.wrap( + DeletionResourceManager.MAGIC_VERSION_V1.getBytes(StandardCharsets.UTF_8))); + } + } + + @Override + public void run() { + // Sync deletion to disk. + workingBuffer.flip(); + try { + // Write deletions. + syncingBuffer.flip(); + this.logChannel.write(syncingBuffer); + // Write metaData. + ByteBuffer metaData = ByteBuffer.allocate(4); + metaData.putInt(deletionNum); + metaData.flip(); + this.logChannel.write(metaData); + } catch (IOException e) { + LOGGER.warn( + "Deletion persist: Cannot write to {}, may cause data inconsistency.", logFile, e); + } finally { + switchSyncingBufferToIdle(); + } + // Close resource. + try { + this.logChannel.close(); + this.logStream.close(); + } catch (IOException e) { + LOGGER.warn("Failed to close deletion writing resource when writing to {}.", logFile, e); + } + } + } + + @Override + public void close() { + isClosed = true; + // Force sync existing data in memory to disk. + // first waiting serialize and sync tasks finished, then release all resources + if (serializeThread != null) { + shutdownThread(serializeThread, ThreadName.PIPE_CONSENSUS_DELETION_SERIALIZE); + } + if (syncBufferThread != null) { + shutdownThread(syncBufferThread, ThreadName.PIPE_CONSENSUS_DELETION_SYNC); + } + + MmapUtil.clean(workingBuffer); + MmapUtil.clean(workingBuffer); + MmapUtil.clean(syncingBuffer); + } + + private void shutdownThread(ExecutorService thread, ThreadName threadName) { + thread.shutdown(); + try { + if (!thread.awaitTermination(30, TimeUnit.SECONDS)) { + LOGGER.warn("Waiting thread {} to be terminated is timeout", threadName.getName()); + } + } catch (InterruptedException e) { + LOGGER.warn("Thread {} still doesn't exit after 30s", threadName.getName()); + Thread.currentThread().interrupt(); + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java index 3e6430b8f969..2bcca7959cdb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java @@ -68,8 +68,6 @@ public static PipeRealtimeEvent createRealtimeEvent( final DeleteDataNode node, final String regionId) { PipeSchemaRegionWritePlanEvent deletionEvent = new PipeSchemaRegionWritePlanEvent(node, node.isGeneratedByPipe()); - Optional.ofNullable(DeletionResourceManager.getInstance(regionId)) - .ifPresent(mgr -> mgr.registerDeletionResource(deletionEvent)); return new PipeRealtimeEvent(deletionEvent, null, null, null); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java index 371cc379eae3..52bffce79617 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java @@ -21,6 +21,10 @@ import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; +import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; +import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEventFactory; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionExtractor; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.assigner.PipeDataRegionAssigner; @@ -141,10 +145,15 @@ public void listenToHeartbeat(boolean shouldPrintMessage) { PipeRealtimeEventFactory.createRealtimeEvent(key, shouldPrintMessage))); } - public void listenToDeleteData(DeleteDataNode node, String regionId) { - dataRegionId2Assigner.forEach( - (key, value) -> - value.publishToAssign(PipeRealtimeEventFactory.createRealtimeEvent(node, regionId))); + public DeletionResource listenToDeleteData(DeleteDataNode node, String regionId) { + PipeRealtimeEvent realtimeEvent = PipeRealtimeEventFactory.createRealtimeEvent(node, regionId); + dataRegionId2Assigner.forEach((key, value) -> value.publishToAssign(realtimeEvent)); + // log deletion to DAL + DeletionResourceManager mgr = DeletionResourceManager.getInstance(regionId); + if (mgr == null) { + return null; + } + return mgr.registerDeletionResource((PipeSchemaRegionWritePlanEvent) realtimeEvent.getEvent()); } /////////////////////////////// singleton /////////////////////////////// diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java index de6fe214270f..3610a85b5ea7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java @@ -48,6 +48,8 @@ import org.apache.iotdb.db.exception.query.OutOfTTLException; import org.apache.iotdb.db.exception.query.QueryProcessException; import org.apache.iotdb.db.exception.quota.ExceedQuotaException; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource.Status; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.listener.PipeInsertionDataNodeListener; import org.apache.iotdb.db.queryengine.common.DeviceContext; import org.apache.iotdb.db.queryengine.execution.fragment.QueryContext; @@ -2268,9 +2270,7 @@ private void getTwoKindsOfTsFiles( .forEach(unsealedResource::add); } - /** - * @param pattern Must be a pattern start with a precise device path - */ + /** @param pattern Must be a pattern start with a precise device path */ public void deleteByDevice(MeasurementPath pattern, DeleteDataNode node) throws IOException { if (SettleService.getINSTANCE().getFilesToBeSettledCount().get() != 0) { throw new IOException( @@ -2309,7 +2309,12 @@ public void deleteByDevice(MeasurementPath pattern, DeleteDataNode node) throws // deviceMatchInfo contains the DeviceId means this device matched the pattern Set deviceMatchInfo = new HashSet<>(); deleteDataInFiles(unsealedTsFileResource, deletion, devicePaths, deviceMatchInfo); - PipeInsertionDataNodeListener.getInstance().listenToDeleteData(node, dataRegionId); + // capture deleteDataNode and wait it to be persisted to DAL. + DeletionResource deletionResource = + PipeInsertionDataNodeListener.getInstance().listenToDeleteData(node, dataRegionId); + if (deletionResource != null && deletionResource.waitForResult() == Status.FAILURE) { + throw deletionResource.getCause(); + } writeUnlock(); hasReleasedLock = true; @@ -2353,7 +2358,12 @@ public void deleteDataDirectly(MeasurementPath pathToDelete, DeleteDataNode node List unsealedTsFileResource = new ArrayList<>(); getTwoKindsOfTsFiles(sealedTsFileResource, unsealedTsFileResource, startTime, endTime); deleteDataDirectlyInFile(unsealedTsFileResource, pathToDelete, startTime, endTime); - PipeInsertionDataNodeListener.getInstance().listenToDeleteData(node, dataRegionId); + // capture deleteDataNode and wait it to be persisted to DAL. + DeletionResource deletionResource = + PipeInsertionDataNodeListener.getInstance().listenToDeleteData(node, dataRegionId); + if (deletionResource != null && deletionResource.waitForResult() == Status.FAILURE) { + throw deletionResource.getCause(); + } writeUnlock(); releasedLock = true; deleteDataDirectlyInFile(sealedTsFileResource, pathToDelete, startTime, endTime); @@ -3707,9 +3717,7 @@ public void insertTablets(InsertMultiTabletsNode insertMultiTabletsNode) } } - /** - * @return the disk space occupied by this data region, unit is MB - */ + /** @return the disk space occupied by this data region, unit is MB */ public long countRegionDiskSize() { AtomicLong diskSize = new AtomicLong(0); TierManager.getInstance() From d8743fcefa06aca4cf986a87fd4cfcb5e360dcdc Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Tue, 10 Sep 2024 20:23:05 +0800 Subject: [PATCH 32/85] remove and recover for new version of DAL --- .../deletion/DeletionResourceManager.java | 37 +++++++++++++++++-- .../deletion/recover/DeletionReader.java | 7 ++-- 2 files changed, 37 insertions(+), 7 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java index 6b6a7d30c5a3..ac9d8b41526e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java @@ -181,11 +181,17 @@ private synchronized void removeDeletionResource(DeletionResource deletionResour .filter(path -> path.getFileName().toString().matches(DELETION_FILE_NAME_PATTERN)) .filter( path -> - isCurrentFileCanBeDeleted( + isFileProgressBehindGivenProgress( path.getFileName().toString(), currentProgressIndex)) + .sorted(this::compareFileProgressIndex) .toArray(Path[]::new); - for (Path path : deletionPaths) { - FileUtils.deleteFileOrDirectory(path.toFile()); + // File name represents the max progressIndex in its previous file. If currentProgressIndex is + // larger than a fileName's progressIndex, it means that the file before this file has been + // fully synchronized and can be deleted. + // So here we cannot guarantee that the last file can be deleted, we can only guarantee that + // the first n-1 files can be deleted (if the length of deletionPaths is n) + for (int i = 0; i < deletionPaths.length - 1; i++) { + FileUtils.deleteFileOrDirectory(deletionPaths[i].toFile()); } } catch (IOException e) { LOGGER.warn( @@ -195,7 +201,30 @@ private synchronized void removeDeletionResource(DeletionResource deletionResour } } - private boolean isCurrentFileCanBeDeleted(String fileName, ProgressIndex currentProgressIndex) { + private int compareFileProgressIndex(Path file1, Path file2) { + Pattern pattern = Pattern.compile(DELETION_FILE_NAME_PATTERN); + String fileName1 = file1.getFileName().toString(); + String fileName2 = file2.getFileName().toString(); + Matcher matcher1 = pattern.matcher(fileName1); + Matcher matcher2 = pattern.matcher(fileName2); + // Definitely match. Because upper caller has filtered fileNames. + if (matcher1.matches() && matcher2.matches()) { + int fileRebootTimes1 = Integer.parseInt(matcher1.group(REBOOT_TIME)); + long fileMemTableFlushOrderId1 = Long.parseLong(matcher1.group(MEM_TABLE_FLUSH_ORDER)); + + int fileRebootTimes2 = Integer.parseInt(matcher2.group(REBOOT_TIME)); + long fileMemTableFlushOrderId2 = Long.parseLong(matcher2.group(MEM_TABLE_FLUSH_ORDER)); + + int rebootCompareRes = Integer.compare(fileRebootTimes1, fileRebootTimes2); + return rebootCompareRes == 0 + ? Long.compare(fileMemTableFlushOrderId1, fileMemTableFlushOrderId2) + : rebootCompareRes; + } + return 0; + } + + private boolean isFileProgressBehindGivenProgress( + String fileName, ProgressIndex currentProgressIndex) { if (currentProgressIndex instanceof SimpleProgressIndex) { SimpleProgressIndex simpleProgressIndex = (SimpleProgressIndex) currentProgressIndex; int curRebootTimes = simpleProgressIndex.getRebootTimes(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java index 630b51b739a6..c0729acb4b61 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java @@ -63,8 +63,9 @@ public List readAllDeletions() throws IOException { } // Read metaData - ByteBuffer intBuffer = ByteBuffer.allocate(4); - fileChannel.read(intBuffer); + ByteBuffer intBuffer = ByteBuffer.allocate(Integer.BYTES); + long position = fileChannel.size() - Integer.BYTES; + fileChannel.read(intBuffer, position); intBuffer.flip(); int deletionNum = intBuffer.getInt(); if (LOGGER.isDebugEnabled()) { @@ -72,7 +73,7 @@ public List readAllDeletions() throws IOException { } // Read deletions - long remainingBytes = fileChannel.size() - fileChannel.position(); + long remainingBytes = fileChannel.size() - fileChannel.position() - Integer.BYTES; ByteBuffer byteBuffer = ByteBuffer.allocate((int) remainingBytes); fileChannel.read(byteBuffer); byteBuffer.flip(); From 19216ca0c2e53f741f0d01876d100d7dbe10285f Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Tue, 10 Sep 2024 20:56:45 +0800 Subject: [PATCH 33/85] historical data process --- .../consensus/deletion/DeletionResource.java | 9 +- .../realtime/PipeRealtimeEventFactory.java | 3 - ...lDataRegionTsFileAndDeletionExtractor.java | 140 +++++++++--------- .../storageengine/dataregion/DataRegion.java | 8 +- .../dataregion/tsfile/TsFileResource.java | 8 +- .../datastructure/PersistentResource.java | 26 ++++ 6 files changed, 112 insertions(+), 82 deletions(-) create mode 100644 iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/PersistentResource.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java index abce9c99e627..24fed10f38f5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.pipe.consensus.deletion; import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.commons.pipe.datastructure.PersistentResource; import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaSerializableEventType; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; @@ -38,12 +39,13 @@ * including realtime deletion and historical deletion. In order to be compatible with user pipe * framework, PipeConsensus will use {@link PipeSchemaRegionWritePlanEvent} */ -public class DeletionResource { +public class DeletionResource implements PersistentResource { private static final Logger LOGGER = LoggerFactory.getLogger(DeletionResource.class); private final Consumer removeHook; private final AtomicLong latestUpdateTime; private PipeSchemaRegionWritePlanEvent deletionEvent; private volatile Status currentStatus; + // it's safe to use volatile here to make this reference thread-safe. @SuppressWarnings("squid:S3077") private volatile Exception cause; @@ -97,7 +99,9 @@ public void onPersistSucceed() { this.notifyAll(); } - /** @return true if this object has been successfully persisted, false if persist failed. */ + /** + * @return true if this object has been successfully persisted, false if persist failed. + */ public synchronized Status waitForResult() { while (currentStatus == Status.RUNNING) { try { @@ -112,6 +116,7 @@ public synchronized Status waitForResult() { return currentStatus; } + @Override public ProgressIndex getProgressIndex() { return ((DeleteDataNode) deletionEvent.getPlanNode()).getProgressIndex(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java index 2bcca7959cdb..c32b310b6034 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java @@ -20,7 +20,6 @@ package org.apache.iotdb.db.pipe.event.realtime; import org.apache.iotdb.commons.pipe.event.ProgressReportEvent; -import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; @@ -31,8 +30,6 @@ import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALEntryHandler; -import java.util.Optional; - public class PipeRealtimeEventFactory { private static final TsFileEpochManager TS_FILE_EPOCH_MANAGER = new TsFileEpochManager(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java index 909f0e7c3afb..fa7207a169bf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java @@ -27,6 +27,7 @@ import org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant; import org.apache.iotdb.commons.pipe.config.constant.SystemConstant; import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskExtractorRuntimeEnvironment; +import org.apache.iotdb.commons.pipe.datastructure.PersistentResource; import org.apache.iotdb.commons.pipe.pattern.PipePattern; import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; @@ -63,6 +64,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.Queue; import java.util.Set; import java.util.stream.Collectors; @@ -123,10 +125,7 @@ public class PipeHistoricalDataRegionTsFileAndDeletionExtractor private boolean shouldTerminatePipeOnAllHistoricalEventsConsumed; private boolean isTerminateSignalSent = false; - private boolean isTsFileEventAllConsumed = false; - - private Queue pendingTsFileQueue; - private Queue pendingDeletionQueue; + private Queue pendingQueue; @Override public void validate(final PipeParameterValidator validator) { @@ -373,16 +372,17 @@ private void flushDataRegionAllTsFiles() { } } - private void extractDeletions(final DeletionResourceManager deletionResourceManager) { + private void extractDeletions( + final DeletionResourceManager deletionResourceManager, + final List resourceList) { LOGGER.info("Pipe {}@{}: start to extract deletions", pipeName, dataRegionId); - List resourceList = deletionResourceManager.getAllDeletionResources(); - final int originalDeletionCount = resourceList.size(); - resourceList = - resourceList.stream() + List allDeletionResources = deletionResourceManager.getAllDeletionResources(); + final int originalDeletionCount = allDeletionResources.size(); + allDeletionResources = + allDeletionResources.stream() .filter(this::mayDeletionUnprocessed) - .sorted((o1, o2) -> o1.getProgressIndex().isAfter(o2.getProgressIndex()) ? 1 : -1) .collect(Collectors.toList()); - pendingDeletionQueue = new ArrayDeque<>(resourceList); + resourceList.addAll(allDeletionResources); LOGGER.info( "Pipe {}@{}: finish to extract deletions, extract deletions count {}/{}", pipeName, @@ -415,14 +415,14 @@ private void flushTsFilesForExtraction( } private void extractTsFiles( - final DataRegion dataRegion, final long startHistoricalExtractionTime) { + final DataRegion dataRegion, + final long startHistoricalExtractionTime, + final List resourceList) { final TsFileManager tsFileManager = dataRegion.getTsFileManager(); tsFileManager.readLock(); try { final int originalSequenceTsFileCount = tsFileManager.size(true); final int originalUnsequenceTsFileCount = tsFileManager.size(false); - final List resourceList = - new ArrayList<>(originalSequenceTsFileCount + originalUnsequenceTsFileCount); LOGGER.info( "Pipe {}@{}: start to extract historical TsFile, original sequence file count {}, " + "original unsequence file count {}, start progress index {}", @@ -466,19 +466,14 @@ && mayTsFileResourceOverlappedWithPattern(resource)) // Will unpin it after the PipeTsFileInsertionEvent is created and pinned. try { PipeDataNodeResourceManager.tsfile() - .pinTsFileResource(resource, shouldTransferModFile); + .pinTsFileResource((TsFileResource) resource, shouldTransferModFile); } catch (final IOException e) { - LOGGER.warn("Pipe: failed to pin TsFileResource {}", resource.getTsFilePath()); + LOGGER.warn( + "Pipe: failed to pin TsFileResource {}", + ((TsFileResource) resource).getTsFilePath()); } }); - resourceList.sort( - (o1, o2) -> - startIndex instanceof TimeWindowStateProgressIndex - ? Long.compare(o1.getFileStartTime(), o2.getFileStartTime()) - : o1.getMaxProgressIndex().topologicalCompareTo(o2.getMaxProgressIndex())); - pendingTsFileQueue = new ArrayDeque<>(resourceList); - LOGGER.info( "Pipe {}@{}: finish to extract historical TsFile, extracted sequence file count {}/{}, " + "extracted unsequence file count {}/{}, extracted file count {}/{}, took {} ms", @@ -508,28 +503,28 @@ public synchronized void start() { final DataRegion dataRegion = StorageEngine.getInstance().getDataRegion(new DataRegionId(dataRegionId)); if (Objects.isNull(dataRegion)) { - pendingTsFileQueue = new ArrayDeque<>(); - pendingDeletionQueue = new ArrayDeque<>(); + pendingQueue = new ArrayDeque<>(); return; } dataRegion.writeLock( "Pipe: start to extract historical TsFile and Deletion(if uses pipeConsensus)"); - // Extract deletions - DeletionResourceManager deletionResourceManager = - DeletionResourceManager.getInstance(String.valueOf(dataRegionId)); - if (deletionResourceManager == null) { - // If not uses pipeConsensus - pendingDeletionQueue = new ArrayDeque<>(); - } else { - extractDeletions(deletionResourceManager); - } - // Flush TsFiles - final long startHistoricalExtractionTime = System.currentTimeMillis(); try { + List resourceList = new ArrayList<>(); + // Extract deletions + Optional.ofNullable(DeletionResourceManager.getInstance(String.valueOf(dataRegionId))) + .ifPresent(mgr -> extractDeletions(mgr, resourceList)); + + // Flush TsFiles + final long startHistoricalExtractionTime = System.currentTimeMillis(); flushTsFilesForExtraction(dataRegion, startHistoricalExtractionTime); // Extract TsFiles - extractTsFiles(dataRegion, startHistoricalExtractionTime); + extractTsFiles(dataRegion, startHistoricalExtractionTime, resourceList); + + // Sort tsFileResource and deletionResource + resourceList.sort( + (o1, o2) -> o1.getProgressIndex().topologicalCompareTo(o2.getProgressIndex())); + pendingQueue.addAll(resourceList); } finally { dataRegion.writeUnlock(); } @@ -665,45 +660,41 @@ private Event supplyTsFileEvent(TsFileResource resource) { } } - private Event supplyDeletionEvent() { - final DeletionResource deletionResource = pendingDeletionQueue.poll(); - if (deletionResource == null) { - final PipeTerminateEvent terminateEvent = - new PipeTerminateEvent(pipeName, creationTime, pipeTaskMeta, dataRegionId); - if (!terminateEvent.increaseReferenceCount( - PipeHistoricalDataRegionTsFileAndDeletionExtractor.class.getName())) { - LOGGER.warn( - "Pipe {}@{}: failed to increase reference count for terminate event, will resend it", - pipeName, - dataRegionId); - return null; - } - isTerminateSignalSent = true; - return terminateEvent; - } + private Event supplyDeletionEvent(final DeletionResource deletionResource) { PipeSchemaRegionWritePlanEvent event = deletionResource.getDeletionEvent(); event.increaseReferenceCount( PipeHistoricalDataRegionTsFileAndDeletionExtractor.class.getName()); return event; } + private Event supplyTerminateEvent() { + final PipeTerminateEvent terminateEvent = + new PipeTerminateEvent(pipeName, creationTime, pipeTaskMeta, dataRegionId); + if (!terminateEvent.increaseReferenceCount( + PipeHistoricalDataRegionTsFileAndDeletionExtractor.class.getName())) { + LOGGER.warn( + "Pipe {}@{}: failed to increase reference count for terminate event, will resend it", + pipeName, + dataRegionId); + return null; + } + isTerminateSignalSent = true; + return terminateEvent; + } + @Override public synchronized Event supply() { - if (Objects.isNull(pendingTsFileQueue) && Objects.isNull(pendingDeletionQueue)) { + if (Objects.isNull(pendingQueue)) { return null; } - // Consume tsFile first - if (!isTsFileEventAllConsumed) { - final TsFileResource resource = pendingTsFileQueue.poll(); - if (resource == null) { - isTsFileEventAllConsumed = true; - return supplyDeletionEvent(); - } - return supplyTsFileEvent(resource); + final PersistentResource resource = pendingQueue.poll(); + if (resource == null) { + return supplyTerminateEvent(); + } else if (resource instanceof TsFileResource) { + return supplyTsFileEvent((TsFileResource) resource); } else { - // Consume deletions - return supplyDeletionEvent(); + return supplyDeletionEvent((DeletionResource) resource); } } @@ -711,34 +702,35 @@ public synchronized Event supply() { public synchronized boolean hasConsumedAll() { // If the pendingQueues are null when the function is called, it implies that the extractor only // extracts deletion thus the historical event has nothing to consume. - return (Objects.isNull(pendingTsFileQueue) && Objects.isNull(pendingDeletionQueue)) - || pendingTsFileQueue.isEmpty() - && pendingDeletionQueue.isEmpty() + return (Objects.isNull(pendingQueue)) + || pendingQueue.isEmpty() && (!shouldTerminatePipeOnAllHistoricalEventsConsumed || isTerminateSignalSent); } @Override public int getPendingQueueSize() { - return Objects.nonNull(pendingTsFileQueue) ? pendingTsFileQueue.size() : 0; + return Objects.nonNull(pendingQueue) ? pendingQueue.size() : 0; } @Override public synchronized void close() { - if (Objects.nonNull(pendingTsFileQueue)) { - pendingTsFileQueue.forEach( + if (Objects.nonNull(pendingQueue)) { + pendingQueue.forEach( resource -> { try { - PipeDataNodeResourceManager.tsfile().unpinTsFileResource(resource); + if (resource instanceof TsFileResource) { + PipeDataNodeResourceManager.tsfile().unpinTsFileResource((TsFileResource) resource); + } } catch (final IOException e) { LOGGER.warn( "Pipe {}@{}: failed to unpin TsFileResource after dropping pipe, original path: {}", pipeName, dataRegionId, - resource.getTsFilePath()); + ((TsFileResource) resource).getTsFilePath()); } }); - pendingTsFileQueue.clear(); - pendingTsFileQueue = null; + pendingQueue.clear(); + pendingQueue = null; } } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java index 3610a85b5ea7..93a9673bb0b4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java @@ -2270,7 +2270,9 @@ private void getTwoKindsOfTsFiles( .forEach(unsealedResource::add); } - /** @param pattern Must be a pattern start with a precise device path */ + /** + * @param pattern Must be a pattern start with a precise device path + */ public void deleteByDevice(MeasurementPath pattern, DeleteDataNode node) throws IOException { if (SettleService.getINSTANCE().getFilesToBeSettledCount().get() != 0) { throw new IOException( @@ -3717,7 +3719,9 @@ public void insertTablets(InsertMultiTabletsNode insertMultiTabletsNode) } } - /** @return the disk space occupied by this data region, unit is MB */ + /** + * @return the disk space occupied by this data region, unit is MB + */ public long countRegionDiskSize() { AtomicLong diskSize = new AtomicLong(0); TierManager.getInstance() diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java index 97dcada99849..927ad955e57a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java @@ -24,6 +24,7 @@ import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; import org.apache.iotdb.commons.path.IFullPath; import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.pipe.datastructure.PersistentResource; import org.apache.iotdb.commons.utils.CommonDateTimeUtils; import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.db.conf.IoTDBConfig; @@ -76,7 +77,7 @@ import static org.apache.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX; @SuppressWarnings("java:S1135") // ignore todos -public class TsFileResource { +public class TsFileResource implements PersistentResource { private static final long INSTANCE_SIZE = RamUsageEstimator.shallowSizeOfInstance(TsFileResource.class) @@ -1143,6 +1144,11 @@ public void setProgressIndex(ProgressIndex progressIndex) { maxProgressIndex = progressIndex; } + @Override + public ProgressIndex getProgressIndex() { + return getMaxProgressIndex(); + } + public ProgressIndex getMaxProgressIndexAfterClose() throws IllegalStateException { if (getStatus().equals(TsFileResourceStatus.UNCLOSED)) { throw new IllegalStateException( diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/PersistentResource.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/PersistentResource.java new file mode 100644 index 000000000000..188b4a0a9251 --- /dev/null +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/PersistentResource.java @@ -0,0 +1,26 @@ +/* + * 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.datastructure; + +import org.apache.iotdb.commons.consensus.index.ProgressIndex; + +public interface PersistentResource { + ProgressIndex getProgressIndex(); +} From 7835d8b51789229285c25e766b0a1d21c9b2025a Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Tue, 10 Sep 2024 20:59:43 +0800 Subject: [PATCH 34/85] add ut --- .../pipe/consensus/DeletionRecoverTest.java | 85 ++++++++++ .../pipe/consensus/DeletionResourceTest.java | 151 ++++++++++++++++++ 2 files changed, 236 insertions(+) create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java new file mode 100644 index 000000000000..294df9029616 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.consensus; + +import org.apache.iotdb.commons.consensus.index.impl.RecoverProgressIndex; +import org.apache.iotdb.commons.consensus.index.impl.SimpleProgressIndex; +import org.apache.iotdb.commons.path.MeasurementPath; +import org.apache.iotdb.commons.utils.FileUtils; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; +import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.util.Collections; + +public class DeletionRecoverTest { + private static final String FAKE_DATE_REGION_ID = "1"; + private static final int THIS_DATANODE_ID = + IoTDBDescriptor.getInstance().getConfig().getDataNodeId(); + private static final String DELETION_BASE_DIR = + IoTDBDescriptor.getInstance().getConfig().getPipeConsensusDeletionFileDir(); + private static final String BASE_PATH = DELETION_BASE_DIR + File.separator + FAKE_DATE_REGION_ID; + private final int deletionCount = 10; + private DeletionResourceManager deletionResourceManager; + + @Before + public void setUp() throws Exception { + File baseDir = new File(BASE_PATH); + if (baseDir.exists()) { + FileUtils.deleteFileOrDirectory(baseDir); + } + DeletionResourceManager.buildForTest(); + deletionResourceManager = DeletionResourceManager.getInstance(FAKE_DATE_REGION_ID); + // Create some deletion files + int rebootTimes = 0; + MeasurementPath path = new MeasurementPath("root.vehicle.d2.s0"); + for (int i = 0; i < deletionCount; i++) { + DeleteDataNode deleteDataNode = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 150); + deleteDataNode.setProgressIndex( + new RecoverProgressIndex(THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, i))); + PipeSchemaRegionWritePlanEvent deletionEvent = + new PipeSchemaRegionWritePlanEvent(deleteDataNode, true); + deletionResourceManager.registerDeletionResource(deletionEvent); + } + // Manually close for further test + deletionResourceManager.close(); + } + + @After + public void tearDown() throws Exception { + deletionResourceManager.close(); + } + + @Test + public void testDeletionRecover() throws Exception { + Assert.assertEquals(0, deletionResourceManager.getAllDeletionResources().size()); + deletionResourceManager.recoverForTest(); + Assert.assertEquals(deletionCount, deletionResourceManager.getAllDeletionResources().size()); + } +} diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java new file mode 100644 index 000000000000..3e7aaef792ba --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.consensus; + +import org.apache.iotdb.commons.consensus.index.impl.RecoverProgressIndex; +import org.apache.iotdb.commons.consensus.index.impl.SimpleProgressIndex; +import org.apache.iotdb.commons.exception.IllegalPathException; +import org.apache.iotdb.commons.path.MeasurementPath; +import org.apache.iotdb.commons.utils.FileUtils; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; +import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Collections; +import java.util.stream.Stream; + +public class DeletionResourceTest { + private static final String[] FAKE_DATE_REGION_IDS = {"2", "3", "4", "5"}; + private static final String DELETION_BASE_DIR = + IoTDBDescriptor.getInstance().getConfig().getPipeConsensusDeletionFileDir(); + private static final int THIS_DATANODE_ID = + IoTDBDescriptor.getInstance().getConfig().getDataNodeId(); + private DeletionResourceManager deletionResourceManager; + + @Before + public void setUp() throws Exception { + DeletionResourceManager.buildForTest(); + } + + @After + public void tearDown() throws Exception { + for (String FAKE_DATE_REGION_ID : FAKE_DATE_REGION_IDS) { + File baseDir = new File(DELETION_BASE_DIR + File.separator + FAKE_DATE_REGION_ID); + if (baseDir.exists()) { + FileUtils.deleteFileOrDirectory(baseDir); + } + } + } + + @Test + public void testCreateBaseDir() { + deletionResourceManager = DeletionResourceManager.getInstance(FAKE_DATE_REGION_IDS[0]); + File baseDir = new File(DELETION_BASE_DIR); + File dataRegionDir = new File(baseDir + File.separator + FAKE_DATE_REGION_IDS[0]); + Assert.assertTrue(baseDir.exists()); + Assert.assertTrue(dataRegionDir.exists()); + } + + @Test + public void testAddBatchDeletionResource() + throws IllegalPathException, InterruptedException, IOException { + deletionResourceManager = DeletionResourceManager.getInstance(FAKE_DATE_REGION_IDS[1]); + int deletionCount = 10; + int rebootTimes = 0; + MeasurementPath path = new MeasurementPath("root.vehicle.d2.s0"); + for (int i = 0; i < deletionCount; i++) { + DeleteDataNode deleteDataNode = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 150); + deleteDataNode.setProgressIndex( + new RecoverProgressIndex(THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, i))); + PipeSchemaRegionWritePlanEvent deletionEvent = + new PipeSchemaRegionWritePlanEvent(deleteDataNode, true); + deletionResourceManager.registerDeletionResource(deletionEvent); + } + // Sleep to wait deletion being persisted + Thread.sleep(1000); + Stream paths = + Files.list(Paths.get(DELETION_BASE_DIR + File.separator + FAKE_DATE_REGION_IDS[1])); + ; + Assert.assertTrue(paths.anyMatch(Files::isRegularFile)); + } + + @Test + public void testAddDeletionResourceTimeout() + throws IllegalPathException, InterruptedException, IOException { + deletionResourceManager = DeletionResourceManager.getInstance(FAKE_DATE_REGION_IDS[2]); + int rebootTimes = 0; + MeasurementPath path = new MeasurementPath("root.vehicle.d2.s0"); + DeleteDataNode deleteDataNode = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 150); + deleteDataNode.setProgressIndex( + new RecoverProgressIndex(THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, 1))); + PipeSchemaRegionWritePlanEvent deletionEvent = + new PipeSchemaRegionWritePlanEvent(deleteDataNode, true); + // Only register one deletionResource + deletionResourceManager.registerDeletionResource(deletionEvent); + // Sleep to wait deletion being persisted + Thread.sleep(5000); + Stream paths = + Files.list(Paths.get(DELETION_BASE_DIR + File.separator + FAKE_DATE_REGION_IDS[2])); + Assert.assertTrue(paths.anyMatch(Files::isRegularFile)); + } + + @Test + public void testDeletionRemove() throws IllegalPathException, InterruptedException, IOException { + deletionResourceManager = DeletionResourceManager.getInstance(FAKE_DATE_REGION_IDS[3]); + // new a deletion + int rebootTimes = 0; + MeasurementPath path = new MeasurementPath("root.vehicle.d2.s0"); + DeleteDataNode deleteDataNode = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 150); + deleteDataNode.setProgressIndex( + new RecoverProgressIndex(THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, 1))); + PipeSchemaRegionWritePlanEvent deletionEvent = + new PipeSchemaRegionWritePlanEvent(deleteDataNode, true); + // Only register one deletionResource + deletionResourceManager.registerDeletionResource(deletionEvent); + deletionEvent.increaseReferenceCount("test"); + // Sleep to wait deletion being persisted + Thread.sleep(1000); + Stream paths = + Files.list(Paths.get(DELETION_BASE_DIR + File.separator + FAKE_DATE_REGION_IDS[3])); + Assert.assertTrue(paths.anyMatch(Files::isRegularFile)); + // Remove deletion + deletionEvent.decreaseReferenceCount("test", false); + // Sleep to wait deletion being removed + Thread.sleep(1000); + Stream newPaths = + Files.list(Paths.get(DELETION_BASE_DIR + File.separator + FAKE_DATE_REGION_IDS[3])); + Assert.assertFalse(newPaths.anyMatch(Files::isRegularFile)); + } +} From cadbaab3d77036e6f592c60a8534ca8044a6fba0 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Tue, 10 Sep 2024 21:05:49 +0800 Subject: [PATCH 35/85] merge master --- .../org/apache/iotdb/db/service/DataNodeShutdownHook.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java index 10a0afa94688..a4672582598d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java @@ -76,6 +76,9 @@ public void run() { // Wait all wal are flushed WALManager.getInstance().waitAllWALFlushed(); + // Wait all deletions are flushed + DeletionResourceManager.exit(); + // Flush data to Tsfile and remove WAL log files if (!IoTDBDescriptor.getInstance() .getConfig() @@ -97,7 +100,9 @@ public void run() { .getConfig() .getDataRegionConsensusProtocolClass() .equals(ConsensusFactory.RATIS_CONSENSUS)) { - DataRegionConsensusImpl.getInstance().getAllConsensusGroupIds().parallelStream() + DataRegionConsensusImpl.getInstance() + .getAllConsensusGroupIds() + .parallelStream() .forEach( id -> { try { From baa71f84a5408c09001e41feee8780494ae2d044 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Wed, 11 Sep 2024 20:12:56 +0800 Subject: [PATCH 36/85] fix bug and ut --- .../consensus/deletion/DeletionResource.java | 8 +-- .../persist/PageCacheDeletionBuffer.java | 56 +++++++++++++------ .../deletion/recover/DeletionReader.java | 56 +++++++++---------- ...ontinuousSameSearchIndexSeparatorNode.java | 11 ++++ .../db/service/DataNodeShutdownHook.java | 4 +- .../pipe/consensus/DeletionResourceTest.java | 49 ++++++++++------ 6 files changed, 110 insertions(+), 74 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java index 24fed10f38f5..80b8861b859b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java @@ -88,20 +88,18 @@ public long getLatestUpdateTime() { return latestUpdateTime.get(); } - public void onPersistFailed(Exception e) { + public synchronized void onPersistFailed(Exception e) { cause = e; currentStatus = Status.FAILURE; this.notifyAll(); } - public void onPersistSucceed() { + public synchronized void onPersistSucceed() { currentStatus = Status.SUCCESS; this.notifyAll(); } - /** - * @return true if this object has been successfully persisted, false if persist failed. - */ + /** @return true if this object has been successfully persisted, false if persist failed. */ public synchronized Status waitForResult() { while (currentStatus == Status.RUNNING) { try { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java index 477edfd7d57b..1bb2e5d3b64f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java @@ -74,8 +74,6 @@ public class PageCacheDeletionBuffer implements DeletionBuffer { private final Lock buffersLock = new ReentrantLock(); // Total size of this batch. private final AtomicInteger totalSize = new AtomicInteger(0); - // Deletion num of this batch. - private final AtomicInteger deletionNum = new AtomicInteger(0); // All deletions that will be written to the current file private final List pendingDeletions = new ArrayList<>(); @@ -104,6 +102,27 @@ public PageCacheDeletionBuffer(String groupId, String baseDirectory) { @Override public void start() { persistThread.submit(new PersistTask()); + try { + // initial file is the minimumProgressIndex + this.logFile = + new File( + baseDirectory, + String.format("_%d-%d%s", 0, 0, DeletionResourceManager.DELETION_FILE_SUFFIX)); + this.logStream = new FileOutputStream(logFile, true); + this.logChannel = logStream.getChannel(); + // Create file && write magic string + if (!logFile.exists() || logFile.length() == 0) { + this.logChannel.write( + ByteBuffer.wrap( + DeletionResourceManager.MAGIC_VERSION_V1.getBytes(StandardCharsets.UTF_8))); + } + } catch (IOException e) { + LOGGER.warn( + "Deletion persist: Cannot create file {}, please check your file system manually.", + logFile, + e); + throw new RuntimeException(e); + } } @Override @@ -140,32 +159,34 @@ public void registerDeletionResource(DeletionResource deletionResource) { private void appendCurrentBatch() throws IOException { serializeBuffer.flip(); logChannel.write(serializeBuffer); + // Mark DeletionResources to persisted once deletion has been written to page cache + pendingDeletions.forEach(DeletionResource::onPersistSucceed); + resetTaskAttribute(); } - private void fsyncCurrentLoggingFileAndReset(int deletionNum, ProgressIndex curMaxProgressIndex) + private void fsyncCurrentLoggingFileAndReset(ProgressIndex curMaxProgressIndex) throws IOException { try { - // Write metaData. - ByteBuffer metaData = ByteBuffer.allocate(4); - metaData.putInt(deletionNum); - metaData.flip(); - this.logChannel.write(metaData); // Close old resource to fsync. this.logStream.close(); this.logChannel.close(); - // Mark DeletionResources to persisted - this.pendingDeletions.forEach(DeletionResource::onPersistSucceed); } finally { - reset(curMaxProgressIndex); + resetFileAttribute(curMaxProgressIndex); } } - private void reset(ProgressIndex curMaxProgressIndex) { + private void resetTaskAttribute() { + this.pendingDeletions.clear(); + clearBuffer(); + } + + private void resetFileAttribute(ProgressIndex curMaxProgressIndex) { // Reset file attributes. this.totalSize.set(0); - this.deletionNum.set(0); this.maxProgressIndexInLastBatch = curMaxProgressIndex; - this.pendingDeletions.clear(); + } + + private void clearBuffer() { // Clear serialize buffer buffersLock.lock(); try { @@ -214,7 +235,7 @@ public void run() { LOGGER.warn( "Deletion persist: Cannot write to {}, may cause data inconsistency.", logFile, e); pendingDeletions.forEach(deletionResource -> deletionResource.onPersistFailed(e)); - reset(maxProgressIndexInLastBatch); + resetFileAttribute(maxProgressIndexInLastBatch); } finally { if (!isClosed) { persistThread.submit(new PersistTask()); @@ -230,7 +251,6 @@ private boolean serializeDeletionToBatchBuffer(DeletionResource deletionResource } serializeBuffer.put(buffer.array()); totalSize.addAndGet(buffer.position()); - deletionNum.incrementAndGet(); return true; } @@ -272,7 +292,7 @@ private void persistDeletion() throws IOException { if (!serializeDeletionToBatchBuffer(deletionResource)) { // If working buffer is exhausted, fsync immediately and roll to a new file. appendCurrentBatch(); - fsyncCurrentLoggingFileAndReset(deletionNum.get(), maxProgressIndexInCurrentBatch); + fsyncCurrentLoggingFileAndReset(maxProgressIndexInCurrentBatch); switchLoggingFile(); return; } @@ -285,7 +305,7 @@ private void persistDeletion() throws IOException { // Persist deletions; Defensive programming here, just in case. if (totalSize.get() > 0) { appendCurrentBatch(); - fsyncCurrentLoggingFileAndReset(deletionNum.get(), maxProgressIndexInCurrentBatch); + fsyncCurrentLoggingFileAndReset(maxProgressIndexInCurrentBatch); switchLoggingFile(); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java index c0729acb4b61..56d3938c8e2e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java @@ -52,41 +52,37 @@ public DeletionReader(File logFile, Consumer removeHook) throw this.removeHook = removeHook; } - public List readAllDeletions() throws IOException { - // Read magic string - ByteBuffer magicStringBuffer = ByteBuffer.allocate(MAGIC_STRING_BYTES_SIZE); - fileChannel.read(magicStringBuffer); - magicStringBuffer.flip(); - String magicVersion = new String(magicStringBuffer.array(), StandardCharsets.UTF_8); - if (LOGGER.isDebugEnabled()) { - LOGGER.debug("Read deletion file-{} magic version: {}", logFile, magicVersion); - } + public List readAllDeletions() { + try { + // Read magic string + ByteBuffer magicStringBuffer = ByteBuffer.allocate(MAGIC_STRING_BYTES_SIZE); + fileChannel.read(magicStringBuffer); + magicStringBuffer.flip(); + String magicVersion = new String(magicStringBuffer.array(), StandardCharsets.UTF_8); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("Read deletion file-{} magic version: {}", logFile, magicVersion); + } - // Read metaData - ByteBuffer intBuffer = ByteBuffer.allocate(Integer.BYTES); - long position = fileChannel.size() - Integer.BYTES; - fileChannel.read(intBuffer, position); - intBuffer.flip(); - int deletionNum = intBuffer.getInt(); - if (LOGGER.isDebugEnabled()) { - LOGGER.debug("Read deletion file-{} contains {} deletions", logFile, deletionNum); - } + // Read deletions + long remainingBytes = fileChannel.size() - fileChannel.position(); + ByteBuffer byteBuffer = ByteBuffer.allocate((int) remainingBytes); + fileChannel.read(byteBuffer); + byteBuffer.flip(); - // Read deletions - long remainingBytes = fileChannel.size() - fileChannel.position() - Integer.BYTES; - ByteBuffer byteBuffer = ByteBuffer.allocate((int) remainingBytes); - fileChannel.read(byteBuffer); - byteBuffer.flip(); + List deletions = new ArrayList<>(); - List deletions = new ArrayList<>(); - for (int i = 0; i < deletionNum; i++) { - DeletionResource deletionResource = DeletionResource.deserialize(byteBuffer, removeHook); - deletions.add(deletionResource); - if (LOGGER.isDebugEnabled()) { - LOGGER.debug("Read deletion: {} from file {}", deletionResource, logFile); + while (byteBuffer.hasRemaining()) { + DeletionResource deletionResource = DeletionResource.deserialize(byteBuffer, removeHook); + deletions.add(deletionResource); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("Read deletion: {} from file {}", deletionResource, logFile); + } } + return deletions; + } catch (IOException e) { + // TODO: 如果文件写坏了 } - return deletions; + return new ArrayList<>(); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/ContinuousSameSearchIndexSeparatorNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/ContinuousSameSearchIndexSeparatorNode.java index 521f720be789..0cd6c58ac2bf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/ContinuousSameSearchIndexSeparatorNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/ContinuousSameSearchIndexSeparatorNode.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.queryengine.plan.planner.plan.node.write; import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet; +import org.apache.iotdb.commons.consensus.index.ProgressIndex; import org.apache.iotdb.db.queryengine.plan.analyze.IAnalysis; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; @@ -121,4 +122,14 @@ protected void serializeAttributes(DataOutputStream stream) throws IOException { public List splitByPartition(IAnalysis analysis) { throw new UnsupportedOperationException(UNSUPPORTED_MESSAGE); } + + @Override + public void setProgressIndex(ProgressIndex progressIndex) { + throw new UnsupportedOperationException(UNSUPPORTED_MESSAGE); + } + + @Override + public ProgressIndex getProgressIndex() { + return null; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java index a4672582598d..d48a30c868da 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java @@ -100,9 +100,7 @@ public void run() { .getConfig() .getDataRegionConsensusProtocolClass() .equals(ConsensusFactory.RATIS_CONSENSUS)) { - DataRegionConsensusImpl.getInstance() - .getAllConsensusGroupIds() - .parallelStream() + DataRegionConsensusImpl.getInstance().getAllConsensusGroupIds().parallelStream() .forEach( id -> { try { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java index 3e7aaef792ba..0f1632881951 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java @@ -40,7 +40,10 @@ import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; +import java.util.ArrayList; import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; import java.util.stream.Stream; public class DeletionResourceTest { @@ -91,8 +94,7 @@ public void testAddBatchDeletionResource() new PipeSchemaRegionWritePlanEvent(deleteDataNode, true); deletionResourceManager.registerDeletionResource(deletionEvent); } - // Sleep to wait deletion being persisted - Thread.sleep(1000); + Stream paths = Files.list(Paths.get(DELETION_BASE_DIR + File.separator + FAKE_DATE_REGION_IDS[1])); ; @@ -125,27 +127,38 @@ public void testDeletionRemove() throws IllegalPathException, InterruptedExcepti deletionResourceManager = DeletionResourceManager.getInstance(FAKE_DATE_REGION_IDS[3]); // new a deletion int rebootTimes = 0; + int deletionCount = 100; MeasurementPath path = new MeasurementPath("root.vehicle.d2.s0"); - DeleteDataNode deleteDataNode = - new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 150); - deleteDataNode.setProgressIndex( - new RecoverProgressIndex(THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, 1))); - PipeSchemaRegionWritePlanEvent deletionEvent = - new PipeSchemaRegionWritePlanEvent(deleteDataNode, true); - // Only register one deletionResource - deletionResourceManager.registerDeletionResource(deletionEvent); - deletionEvent.increaseReferenceCount("test"); + List deletionEvents = new ArrayList<>(); + for (int i = 0; i < deletionCount; i++) { + DeleteDataNode deleteDataNode = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 150); + deleteDataNode.setProgressIndex( + new RecoverProgressIndex(THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, i))); + PipeSchemaRegionWritePlanEvent deletionEvent = + new PipeSchemaRegionWritePlanEvent(deleteDataNode, true); + deletionEvents.add(deletionEvent); + deletionResourceManager.registerDeletionResource(deletionEvent); + } + deletionEvents.forEach(deletionEvent -> deletionEvent.increaseReferenceCount("test")); // Sleep to wait deletion being persisted Thread.sleep(1000); - Stream paths = - Files.list(Paths.get(DELETION_BASE_DIR + File.separator + FAKE_DATE_REGION_IDS[3])); - Assert.assertTrue(paths.anyMatch(Files::isRegularFile)); + List paths = + Files.list(Paths.get(DELETION_BASE_DIR + File.separator + FAKE_DATE_REGION_IDS[3])) + .collect(Collectors.toList()); + Assert.assertTrue(paths.stream().anyMatch(Files::isRegularFile)); + int beforeFileCount = paths.size(); + if (beforeFileCount < 2) { + return; + } // Remove deletion - deletionEvent.decreaseReferenceCount("test", false); + deletionEvents.forEach(deletionEvent -> deletionEvent.decreaseReferenceCount("test", false)); // Sleep to wait deletion being removed Thread.sleep(1000); - Stream newPaths = - Files.list(Paths.get(DELETION_BASE_DIR + File.separator + FAKE_DATE_REGION_IDS[3])); - Assert.assertFalse(newPaths.anyMatch(Files::isRegularFile)); + List newPaths = + Files.list(Paths.get(DELETION_BASE_DIR + File.separator + FAKE_DATE_REGION_IDS[3])) + .collect(Collectors.toList()); + int afterCount = newPaths.size(); + Assert.assertTrue(afterCount < beforeFileCount); } } From e251c43302916c9ebdcef64101b5175adbb72910 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Wed, 11 Sep 2024 20:14:25 +0800 Subject: [PATCH 37/85] remove metadata(deletionNum) --- .../persist/TwoStageDeletionBuffer.java | 22 +++++-------------- 1 file changed, 5 insertions(+), 17 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/TwoStageDeletionBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/TwoStageDeletionBuffer.java index a12b089d9a35..cbe89d43e4d9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/TwoStageDeletionBuffer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/TwoStageDeletionBuffer.java @@ -146,10 +146,10 @@ private void allocateBuffers() { } /** Notice: this method only called when buffer is exhausted by SerializeTask. */ - private void syncWorkingBuffer(ProgressIndex maxProgressIndexInCurrentBatch, int deletionNum) { + private void syncWorkingBuffer(ProgressIndex maxProgressIndexInCurrentBatch) { switchWorkingBufferToFlushing(); try { - syncBufferThread.submit(new SyncBufferTask(maxProgressIndexInCurrentBatch, deletionNum)); + syncBufferThread.submit(new SyncBufferTask(maxProgressIndexInCurrentBatch)); } catch (IOException e) { LOGGER.warn( "Failed to submit syncBufferTask, May because file open error and cause data inconsistency. Please check your file system. ", @@ -179,8 +179,6 @@ private void switchWorkingBufferToFlushing() { private class SerializeTask implements Runnable { // Total size of this batch. private int totalSize = 0; - // Deletion num of this batch. - private int deletionNum = 0; // Max progressIndex among this batch. Used by SyncTask for naming .deletion file. private ProgressIndex maxProgressIndexInCurrentBatch = MinimumProgressIndex.INSTANCE; @@ -232,10 +230,9 @@ private void serialize() { while (!serializeToWorkingBuffer(deletionResource)) { // If working buffer is exhausted, submit a syncTask to consume current batch and switch // buffer to start a new batch. - syncWorkingBuffer(maxProgressIndexInCurrentBatch, deletionNum); + syncWorkingBuffer(maxProgressIndexInCurrentBatch); // Reset maxProgressIndex and deletionNum for new batch. maxProgressIndexInCurrentBatch = MinimumProgressIndex.INSTANCE; - deletionNum = 0; } // Update max progressIndex maxProgressIndexInCurrentBatch = @@ -244,7 +241,7 @@ private void serialize() { } // Persist deletions; Defensive programming here, just in case. if (totalSize > 0) { - syncWorkingBuffer(maxProgressIndexInCurrentBatch, deletionNum); + syncWorkingBuffer(maxProgressIndexInCurrentBatch); } } @@ -260,7 +257,6 @@ private boolean serializeToWorkingBuffer(DeletionResource deletionResource) { } workingBuffer.put(buffer.array()); totalSize += buffer.position(); - deletionNum++; return true; } } @@ -280,14 +276,11 @@ private void switchSyncingBufferToIdle() { } private class SyncBufferTask implements Runnable { - private final int deletionNum; private final File logFile; private final FileOutputStream logStream; private final FileChannel logChannel; - public SyncBufferTask(ProgressIndex maxProgressIndexInCurrentBatch, int deletionNum) - throws IOException { - this.deletionNum = deletionNum; + public SyncBufferTask(ProgressIndex maxProgressIndexInCurrentBatch) throws IOException { // PipeConsensus ensures that deleteDataNodes use recoverProgressIndex. ProgressIndex curProgressIndex = ProgressIndexDataNodeManager.extractLocalSimpleProgressIndex( @@ -323,11 +316,6 @@ public void run() { // Write deletions. syncingBuffer.flip(); this.logChannel.write(syncingBuffer); - // Write metaData. - ByteBuffer metaData = ByteBuffer.allocate(4); - metaData.putInt(deletionNum); - metaData.flip(); - this.logChannel.write(metaData); } catch (IOException e) { LOGGER.warn( "Deletion persist: Cannot write to {}, may cause data inconsistency.", logFile, e); From c3d3460535cafd9dd67c751f4a3b016819200f92 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Wed, 11 Sep 2024 20:23:14 +0800 Subject: [PATCH 38/85] merge --- .../iotdb/db/pipe/consensus/deletion/DeletionResource.java | 4 +++- .../consensus/deletion/persist/PageCacheDeletionBuffer.java | 6 +++--- .../org/apache/iotdb/db/service/DataNodeShutdownHook.java | 1 - 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java index 80b8861b859b..ea1093968780 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java @@ -99,7 +99,9 @@ public synchronized void onPersistSucceed() { this.notifyAll(); } - /** @return true if this object has been successfully persisted, false if persist failed. */ + /** + * @return true if this object has been successfully persisted, false if persist failed. + */ public synchronized Status waitForResult() { while (currentStatus == Status.RUNNING) { try { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java index 1bb2e5d3b64f..cf26679f2c6b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java @@ -50,9 +50,9 @@ import java.util.concurrent.locks.ReentrantLock; /** - * The core idea of this buffer is to delete writes to the Page cache and fsync when certain - * conditions are met. This design does not decouple serialization and writing, but provides easier - * writing. + * The core idea of this buffer is to write deletion to the Page cache and fsync them to disk when + * certain conditions are met. This design does not decouple serialization and writing, but provides + * an easier way to maintain and understand. */ public class PageCacheDeletionBuffer implements DeletionBuffer { private static final Logger LOGGER = LoggerFactory.getLogger(TwoStageDeletionBuffer.class); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java index 1f221118ff0c..e4de436cfd4b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNodeShutdownHook.java @@ -28,7 +28,6 @@ import org.apache.iotdb.consensus.exception.ConsensusException; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.consensus.DataRegionConsensusImpl; -import org.apache.iotdb.db.consensus.SchemaRegionConsensusImpl; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; import org.apache.iotdb.db.protocol.client.ConfigNodeClient; import org.apache.iotdb.db.protocol.client.ConfigNodeClientManager; From fb8a029be5d229c7297d72cdc4dc0e5a1d0fda9b Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Wed, 11 Sep 2024 23:46:08 +0800 Subject: [PATCH 39/85] handle corrupted file --- .../consensus/deletion/DeletionResourceManager.java | 5 +++++ .../consensus/deletion/recover/DeletionReader.java | 10 +++++++--- 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java index ac9d8b41526e..aab6f0b620bc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java @@ -105,6 +105,11 @@ private void initAndRecover() throws IOException { try (DeletionReader deletionReader = new DeletionReader(path.toFile(), this::removeDeletionResource)) { deletionResources.addAll(deletionReader.readAllDeletions()); + } catch (IOException e) { + LOGGER.warn( + "Detect file corrupted when recover DAL-{}, discard all subsequent DALs...", + path.getFileName()); + break; } } hasCompletedRecovery = true; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java index 56d3938c8e2e..51ab624e7d19 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java @@ -52,7 +52,7 @@ public DeletionReader(File logFile, Consumer removeHook) throw this.removeHook = removeHook; } - public List readAllDeletions() { + public List readAllDeletions() throws IOException { try { // Read magic string ByteBuffer magicStringBuffer = ByteBuffer.allocate(MAGIC_STRING_BYTES_SIZE); @@ -80,9 +80,13 @@ public List readAllDeletions() { } return deletions; } catch (IOException e) { - // TODO: 如果文件写坏了 + // if file is corrupted, throw an exception and skip subsequence DAL. + LOGGER.warn( + "Failed to read deletion file {}, may because this file corrupted when writing it.", + logFile, + e); + throw e; } - return new ArrayList<>(); } @Override From 30f7956a8845aa5f530d166ca738f5f5710121b1 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Wed, 11 Sep 2024 23:59:31 +0800 Subject: [PATCH 40/85] add ut --- .../PipeInsertionDataNodeListener.java | 7 +++---- .../pipe/consensus/DeletionResourceTest.java | 20 ++++++++++++++++++- 2 files changed, 22 insertions(+), 5 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java index 52bffce79617..46b8bac7ff2f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java @@ -150,10 +150,9 @@ public DeletionResource listenToDeleteData(DeleteDataNode node, String regionId) dataRegionId2Assigner.forEach((key, value) -> value.publishToAssign(realtimeEvent)); // log deletion to DAL DeletionResourceManager mgr = DeletionResourceManager.getInstance(regionId); - if (mgr == null) { - return null; - } - return mgr.registerDeletionResource((PipeSchemaRegionWritePlanEvent) realtimeEvent.getEvent()); + return mgr == null + ? null + : mgr.registerDeletionResource((PipeSchemaRegionWritePlanEvent) realtimeEvent.getEvent()); } /////////////////////////////// singleton /////////////////////////////// diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java index 0f1632881951..281a323bbf0d 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java @@ -25,8 +25,11 @@ import org.apache.iotdb.commons.path.MeasurementPath; import org.apache.iotdb.commons.utils.FileUtils; import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource.Status; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; +import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.listener.PipeInsertionDataNodeListener; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; @@ -47,7 +50,7 @@ import java.util.stream.Stream; public class DeletionResourceTest { - private static final String[] FAKE_DATE_REGION_IDS = {"2", "3", "4", "5"}; + private static final String[] FAKE_DATE_REGION_IDS = {"2", "3", "4", "5", "6"}; private static final String DELETION_BASE_DIR = IoTDBDescriptor.getInstance().getConfig().getPipeConsensusDeletionFileDir(); private static final int THIS_DATANODE_ID = @@ -161,4 +164,19 @@ public void testDeletionRemove() throws IllegalPathException, InterruptedExcepti int afterCount = newPaths.size(); Assert.assertTrue(afterCount < beforeFileCount); } + + @Test + public void testWaitForResult() throws Exception { + deletionResourceManager = DeletionResourceManager.getInstance(FAKE_DATE_REGION_IDS[4]); + int rebootTimes = 0; + MeasurementPath path = new MeasurementPath("root.vehicle.d2.s0"); + DeleteDataNode deleteDataNode = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 150); + deleteDataNode.setProgressIndex( + new RecoverProgressIndex(THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, 1))); + DeletionResource deletionResource = + PipeInsertionDataNodeListener.getInstance() + .listenToDeleteData(deleteDataNode, FAKE_DATE_REGION_IDS[4]); + Assert.assertSame(deletionResource.waitForResult(), Status.SUCCESS); + } } From b514c97d417847c1a79df21bcc29225b23fe3a24 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Thu, 12 Sep 2024 00:03:38 +0800 Subject: [PATCH 41/85] add license --- .../deletion/persist/DeletionBuffer.java | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java index 41e44991d41e..4b773ebcf6fa 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/DeletionBuffer.java @@ -1,3 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.iotdb.db.pipe.consensus.deletion.persist; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; From 56ff4bc8937f128c259dee08c990487ac175521f Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Thu, 12 Sep 2024 14:10:01 +0800 Subject: [PATCH 42/85] retrigger ci --- .../iotdb/db/pipe/consensus/deletion/DeletionResource.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java index ea1093968780..f5f344c97a55 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java @@ -35,9 +35,9 @@ import java.util.function.Consumer; /** - * DeletionResource is designed for PipeConsensus to manage the lifecycle of all deletion operations - * including realtime deletion and historical deletion. In order to be compatible with user pipe - * framework, PipeConsensus will use {@link PipeSchemaRegionWritePlanEvent} + * DeletionResource is designed for IoTConsensusV2 to manage the lifecycle of all deletion + * operations including realtime deletion and historical deletion. In order to be compatible with + * user pipe framework, PipeConsensus will use {@link PipeSchemaRegionWritePlanEvent} */ public class DeletionResource implements PersistentResource { private static final Logger LOGGER = LoggerFactory.getLogger(DeletionResource.class); From 3c7d48db44157d307660255d9c0d44ec3784216b Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Sat, 14 Sep 2024 15:27:28 +0800 Subject: [PATCH 43/85] fix review --- .../db/pipe/consensus/deletion/DeletionResource.java | 10 ++++++++++ ...HistoricalDataRegionTsFileAndDeletionExtractor.java | 7 +++++-- .../dataregion/tsfile/TsFileResource.java | 2 ++ .../resources/conf/iotdb-system.properties.template | 1 - .../commons/pipe/datastructure/PersistentResource.java | 4 ++++ 5 files changed, 21 insertions(+), 3 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java index f5f344c97a55..2ad0d6f6f93b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java @@ -121,6 +121,16 @@ public ProgressIndex getProgressIndex() { return ((DeleteDataNode) deletionEvent.getPlanNode()).getProgressIndex(); } + @Override + public long getFileStartTime() { + return 0; + } + + @Override + public long getFileEndTime() { + return 0; + } + public PipeSchemaRegionWritePlanEvent getDeletionEvent() { return deletionEvent; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java index 11a8fb86e7f9..47bdf815fe29 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java @@ -523,8 +523,11 @@ public synchronized void start() { // Sort tsFileResource and deletionResource resourceList.sort( - (o1, o2) -> o1.getProgressIndex().topologicalCompareTo(o2.getProgressIndex())); - pendingQueue.addAll(resourceList); + (o1, o2) -> + startIndex instanceof TimeWindowStateProgressIndex + ? Long.compare(o1.getFileStartTime(), o2.getFileStartTime()) + : o1.getProgressIndex().topologicalCompareTo(o2.getProgressIndex())); + pendingQueue = new ArrayDeque<>(resourceList); } finally { dataRegion.writeUnlock(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java index 7da19d849986..5bae8daf587c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java @@ -447,11 +447,13 @@ public long getOrderTime(IDeviceID deviceId, boolean ascending) { return ascending ? getStartTime(deviceId) : getEndTime(deviceId); } + @Override public long getFileStartTime() { return timeIndex.getMinStartTime(); } /** Open file's end time is Long.MIN_VALUE */ + @Override public long getFileEndTime() { return timeIndex.getMaxEndTime(); } diff --git a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template index 49b21e6a357f..18a7c05856b5 100644 --- a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template +++ b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template @@ -336,7 +336,6 @@ pipe_consensus_receiver_file_dirs=data/datanode/system/pipe/consensus/receiver # If this property is unset, system will save the data in the default relative path directory under the IoTDB folder(i.e., %IOTDB_HOME%/${dn_system_dir}/pipe/consensus/deletion). # If it is absolute, system will save the data in the exact location it points to. # If it is relative, system will save the data in the relative path directory it indicates under the IoTDB folder. -# If there are more than one directory, please separate them by commas ",". # Note: If pipe_consensus_deletion_file_dir is assigned an empty string(i.e.,zero-size), it will be handled as a relative path. # effectiveMode: restart # For windows platform diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/PersistentResource.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/PersistentResource.java index 188b4a0a9251..246f6b262790 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/PersistentResource.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/PersistentResource.java @@ -23,4 +23,8 @@ public interface PersistentResource { ProgressIndex getProgressIndex(); + + long getFileStartTime(); + + long getFileEndTime(); } From b4de6d83b56c5b6f122b828918eeeadd11fde408 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Wed, 18 Sep 2024 14:50:17 +0800 Subject: [PATCH 44/85] fix review --- .../pipeconsensus/PipeConsensusSyncConnector.java | 3 ++- .../db/pipe/event/realtime/PipeRealtimeEventFactory.java | 8 +++----- .../realtime/listener/PipeInsertionDataNodeListener.java | 2 +- 3 files changed, 6 insertions(+), 7 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java index 65c4261a7658..765d2cfc8f91 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java @@ -73,6 +73,7 @@ public class PipeConsensusSyncConnector extends IoTDBConnector { private static final String TABLET_INSERTION_NODE_SCENARIO = "transfer insertionNode tablet"; private static final String TSFILE_SCENARIO = "transfer tsfile"; private static final String TABLET_BATCH_SCENARIO = "transfer tablet batch"; + private static final String DELETION_SCENARIO = "transfer deletion"; private final IClientManager syncRetryClientManager; private final List peers; private final int thisDataNodeId; @@ -263,7 +264,7 @@ private void doTransfer(final PipeSchemaRegionWritePlanEvent pipeSchemaRegionWri PIPE_CONSENSUS_SYNC_CONNECTION_FAILED_FORMAT, getFollowerUrl().getIp(), getFollowerUrl().getPort(), - TABLET_INSERTION_NODE_SCENARIO, + DELETION_SCENARIO, e.getMessage()), e); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java index 15c7d4620b13..3c3023c58ee3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java @@ -61,11 +61,9 @@ public static PipeRealtimeEvent createRealtimeEvent( new PipeHeartbeatEvent(dataRegionId, shouldPrintMessage), null, null, null); } - public static PipeRealtimeEvent createRealtimeEvent( - final DeleteDataNode node, final String regionId) { - PipeSchemaRegionWritePlanEvent deletionEvent = - new PipeSchemaRegionWritePlanEvent(node, node.isGeneratedByPipe()); - return new PipeRealtimeEvent(deletionEvent, null, null, null); + public static PipeRealtimeEvent createRealtimeEvent(final DeleteDataNode node) { + return new PipeRealtimeEvent( + new PipeSchemaRegionWritePlanEvent(node, node.isGeneratedByPipe()), null, null, null); } public static PipeRealtimeEvent createRealtimeEvent(final ProgressReportEvent event) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java index 46b8bac7ff2f..dd0c8cbcb53a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java @@ -146,7 +146,7 @@ public void listenToHeartbeat(boolean shouldPrintMessage) { } public DeletionResource listenToDeleteData(DeleteDataNode node, String regionId) { - PipeRealtimeEvent realtimeEvent = PipeRealtimeEventFactory.createRealtimeEvent(node, regionId); + PipeRealtimeEvent realtimeEvent = PipeRealtimeEventFactory.createRealtimeEvent(node); dataRegionId2Assigner.forEach((key, value) -> value.publishToAssign(realtimeEvent)); // log deletion to DAL DeletionResourceManager mgr = DeletionResourceManager.getInstance(regionId); From b7e9ca3f22c23c7caefddddd5e11f916ae3f98b1 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Thu, 19 Sep 2024 15:05:05 +0800 Subject: [PATCH 45/85] introduce PipeDeleteDataNodeEvent & fix review --- .../airgap/IoTDBDataNodeAirGapConnector.java | 44 ----- .../IoTDBDataRegionAirGapConnector.java | 46 ++++- .../IoTDBSchemaRegionAirGapConnector.java | 40 +++++ .../PipeConsensusAsyncConnector.java | 4 +- .../PipeConsensusSyncConnector.java | 33 ++-- ...q.java => PipeConsensusDeleteNodeReq.java} | 53 +++--- .../sync/IoTDBDataNodeSyncConnector.java | 67 ------- .../sync/IoTDBDataRegionSyncConnector.java | 64 ++++++- .../sync/IoTDBSchemaRegionConnector.java | 59 +++++++ .../consensus/deletion/DeletionResource.java | 17 +- .../deletion/DeletionResourceManager.java | 4 +- .../deletion/PipeDeleteDataNodeEvent.java | 166 ++++++++++++++++++ .../realtime/PipeRealtimeEventFactory.java | 4 +- .../PipeInsertionDataNodeListener.java | 4 +- .../pipeconsensus/PipeConsensusReceiver.java | 8 +- 15 files changed, 429 insertions(+), 184 deletions(-) rename iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/{PipeConsensusPlanNodeReq.java => PipeConsensusDeleteNodeReq.java} (68%) create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataNodeAirGapConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataNodeAirGapConnector.java index 33be8e002f4f..cda47a860554 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataNodeAirGapConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataNodeAirGapConnector.java @@ -20,7 +20,6 @@ package org.apache.iotdb.db.pipe.connector.protocol.airgap; import org.apache.iotdb.common.rpc.thrift.TEndPoint; -import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.conf.CommonDescriptor; import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.connector.payload.thrift.common.PipeTransferHandshakeConstant; @@ -29,11 +28,7 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferDataNodeHandshakeV1Req; import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferDataNodeHandshakeV2Req; -import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferPlanNodeReq; -import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; -import org.apache.iotdb.pipe.api.exception.PipeException; -import org.apache.iotdb.rpc.TSStatusCode; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -108,43 +103,4 @@ protected byte[] generateHandShakeV2Payload() throws IOException { return PipeTransferDataNodeHandshakeV2Req.toTPipeTransferBytes(params); } - - protected void doTransferWrapper( - final AirGapSocket socket, - final PipeSchemaRegionWritePlanEvent pipeSchemaRegionWritePlanEvent) - throws PipeException, IOException { - // We increase the reference count for this event to determine if the event may be released. - if (!pipeSchemaRegionWritePlanEvent.increaseReferenceCount( - IoTDBDataNodeAirGapConnector.class.getName())) { - return; - } - try { - doTransfer(socket, pipeSchemaRegionWritePlanEvent); - } finally { - pipeSchemaRegionWritePlanEvent.decreaseReferenceCount( - IoTDBDataNodeAirGapConnector.class.getName(), false); - } - } - - private void doTransfer( - final AirGapSocket socket, - final PipeSchemaRegionWritePlanEvent pipeSchemaRegionWritePlanEvent) - throws PipeException, IOException { - if (!send( - pipeSchemaRegionWritePlanEvent.getPipeName(), - pipeSchemaRegionWritePlanEvent.getCreationTime(), - socket, - PipeTransferPlanNodeReq.toTPipeTransferBytes( - pipeSchemaRegionWritePlanEvent.getPlanNode()))) { - final String errorMessage = - String.format( - "Transfer data node write plan %s error. Socket: %s.", - pipeSchemaRegionWritePlanEvent.getPlanNode().getType(), socket); - receiverStatusHandler.handle( - new TSStatus(TSStatusCode.PIPE_RECEIVER_USER_CONFLICT_EXCEPTION.getStatusCode()) - .setMessage(errorMessage), - errorMessage, - pipeSchemaRegionWritePlanEvent.toString()); - } - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataRegionAirGapConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataRegionAirGapConnector.java index 7b6c4376d0a9..4178983b6e46 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataRegionAirGapConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataRegionAirGapConnector.java @@ -21,6 +21,7 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; +import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferPlanNodeReq; import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferTabletBinaryReq; import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferTabletInsertNodeReq; import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferTabletRawReq; @@ -28,8 +29,8 @@ import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferTsFilePieceWithModReq; import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferTsFileSealReq; import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferTsFileSealWithModReq; +import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; -import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.terminate.PipeTerminateEvent; @@ -128,8 +129,8 @@ public void transfer(final Event event) throws Exception { final AirGapSocket socket = sockets.get(socketIndex); try { - if (event instanceof PipeSchemaRegionWritePlanEvent) { - doTransferWrapper(socket, (PipeSchemaRegionWritePlanEvent) event); + if (event instanceof PipeDeleteDataNodeEvent) { + doTransferWrapper(socket, (PipeDeleteDataNodeEvent) event); } else if (!(event instanceof PipeHeartbeatEvent || event instanceof PipeTerminateEvent)) { LOGGER.warn( "IoTDBDataRegionAirGapConnector does not support transferring generic event: {}.", @@ -141,11 +142,48 @@ public void transfer(final Event event) throws Exception { throw new PipeConnectionException( String.format( "Network error when transfer tsfile event %s, because %s.", - ((PipeSchemaRegionWritePlanEvent) event).coreReportMessage(), e.getMessage()), + ((PipeDeleteDataNodeEvent) event).coreReportMessage(), e.getMessage()), e); } } + private void doTransferWrapper( + final AirGapSocket socket, final PipeDeleteDataNodeEvent pipeDeleteDataNodeEvent) + throws PipeException, IOException { + // We increase the reference count for this event to determine if the event may be released. + if (!pipeDeleteDataNodeEvent.increaseReferenceCount( + IoTDBDataNodeAirGapConnector.class.getName())) { + return; + } + try { + doTransfer(socket, pipeDeleteDataNodeEvent); + } finally { + pipeDeleteDataNodeEvent.decreaseReferenceCount( + IoTDBDataNodeAirGapConnector.class.getName(), false); + } + } + + private void doTransfer( + final AirGapSocket socket, final PipeDeleteDataNodeEvent pipeDeleteDataNodeEvent) + throws PipeException, IOException { + if (!send( + pipeDeleteDataNodeEvent.getPipeName(), + pipeDeleteDataNodeEvent.getCreationTime(), + socket, + PipeTransferPlanNodeReq.toTPipeTransferBytes( + pipeDeleteDataNodeEvent.getDeleteDataNode()))) { + final String errorMessage = + String.format( + "Transfer deletion %s error. Socket: %s.", + pipeDeleteDataNodeEvent.getDeleteDataNode().getType(), socket); + receiverStatusHandler.handle( + new TSStatus(TSStatusCode.PIPE_RECEIVER_USER_CONFLICT_EXCEPTION.getStatusCode()) + .setMessage(errorMessage), + errorMessage, + pipeDeleteDataNodeEvent.toString()); + } + } + private void doTransferWrapper( final AirGapSocket socket, final PipeInsertNodeTabletInsertionEvent pipeInsertNodeTabletInsertionEvent) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBSchemaRegionAirGapConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBSchemaRegionAirGapConnector.java index 342807a5dfbc..ead26390dc6c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBSchemaRegionAirGapConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBSchemaRegionAirGapConnector.java @@ -21,6 +21,7 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; +import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferPlanNodeReq; import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferSchemaSnapshotPieceReq; import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferSchemaSnapshotSealReq; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; @@ -83,6 +84,45 @@ public void transfer(final Event event) throws Exception { } } + private void doTransferWrapper( + final AirGapSocket socket, + final PipeSchemaRegionWritePlanEvent pipeSchemaRegionWritePlanEvent) + throws PipeException, IOException { + // We increase the reference count for this event to determine if the event may be released. + if (!pipeSchemaRegionWritePlanEvent.increaseReferenceCount( + IoTDBDataNodeAirGapConnector.class.getName())) { + return; + } + try { + doTransfer(socket, pipeSchemaRegionWritePlanEvent); + } finally { + pipeSchemaRegionWritePlanEvent.decreaseReferenceCount( + IoTDBDataNodeAirGapConnector.class.getName(), false); + } + } + + private void doTransfer( + final AirGapSocket socket, + final PipeSchemaRegionWritePlanEvent pipeSchemaRegionWritePlanEvent) + throws PipeException, IOException { + if (!send( + pipeSchemaRegionWritePlanEvent.getPipeName(), + pipeSchemaRegionWritePlanEvent.getCreationTime(), + socket, + PipeTransferPlanNodeReq.toTPipeTransferBytes( + pipeSchemaRegionWritePlanEvent.getPlanNode()))) { + final String errorMessage = + String.format( + "Transfer data node write plan %s error. Socket: %s.", + pipeSchemaRegionWritePlanEvent.getPlanNode().getType(), socket); + receiverStatusHandler.handle( + new TSStatus(TSStatusCode.PIPE_RECEIVER_USER_CONFLICT_EXCEPTION.getStatusCode()) + .setMessage(errorMessage), + errorMessage, + pipeSchemaRegionWritePlanEvent.toString()); + } + } + private void doTransferWrapper( final AirGapSocket socket, final PipeSchemaRegionSnapshotEvent pipeSchemaRegionSnapshotEvent) throws PipeException, IOException { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java index 53480f0dcb93..c4e51a82bc2f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusAsyncConnector.java @@ -46,8 +46,8 @@ import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletBinaryReq; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletInsertNodeReq; import org.apache.iotdb.db.pipe.consensus.metric.PipeConsensusConnectorMetrics; +import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; -import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; @@ -402,7 +402,7 @@ public void transfer(Event event) throws Exception { transferBatchedEventsIfNecessary(); // Transfer deletion - if (event instanceof PipeSchemaRegionWritePlanEvent) { + if (event instanceof PipeDeleteDataNodeEvent) { retryConnector.transfer(event); return; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java index 765d2cfc8f91..9589ce376e96 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java @@ -34,7 +34,7 @@ import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusBatchTransferResp; import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferResp; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.builder.PipeConsensusSyncBatchReqBuilder; -import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusPlanNodeReq; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusDeleteNodeReq; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletBinaryReq; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletInsertNodeReq; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFilePieceReq; @@ -42,7 +42,7 @@ import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFileSealReq; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFileSealWithModReq; import org.apache.iotdb.db.pipe.consensus.metric.PipeConsensusConnectorMetrics; -import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; +import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode; @@ -180,7 +180,7 @@ public void transfer(Event event) throws Exception { } // Only deletion event will be passed here. - doTransferWrapper((PipeSchemaRegionWritePlanEvent) event); + doTransferWrapper((PipeDeleteDataNodeEvent) event); } private void doTransfer() { @@ -221,39 +221,38 @@ private void doTransfer() { } } - private void doTransferWrapper( - final PipeSchemaRegionWritePlanEvent pipeSchemaRegionWritePlanEvent) throws PipeException { + private void doTransferWrapper(final PipeDeleteDataNodeEvent pipeDeleteDataNodeEvent) + throws PipeException { try { // We increase the reference count for this event to determine if the event may be released. - if (!pipeSchemaRegionWritePlanEvent.increaseReferenceCount( + if (!pipeDeleteDataNodeEvent.increaseReferenceCount( PipeConsensusSyncConnector.class.getName())) { return; } - doTransfer(pipeSchemaRegionWritePlanEvent); + doTransfer(pipeDeleteDataNodeEvent); } finally { - pipeSchemaRegionWritePlanEvent.decreaseReferenceCount( + pipeDeleteDataNodeEvent.decreaseReferenceCount( PipeConsensusSyncConnector.class.getName(), false); } } - private void doTransfer(final PipeSchemaRegionWritePlanEvent pipeSchemaRegionWritePlanEvent) + private void doTransfer(final PipeDeleteDataNodeEvent pipeDeleteDataNodeEvent) throws PipeException { final ProgressIndex progressIndex; final TPipeConsensusTransferResp resp; TCommitId tCommitId = new TCommitId( - pipeSchemaRegionWritePlanEvent.getCommitId(), - pipeSchemaRegionWritePlanEvent.getRebootTimes()); + pipeDeleteDataNodeEvent.getCommitId(), pipeDeleteDataNodeEvent.getRebootTimes()); TConsensusGroupId tConsensusGroupId = new TConsensusGroupId(TConsensusGroupType.DataRegion, consensusGroupId); try (final SyncPipeConsensusServiceClient syncPipeConsensusServiceClient = syncRetryClientManager.borrowClient(getFollowerUrl())) { - progressIndex = pipeSchemaRegionWritePlanEvent.getProgressIndex(); + progressIndex = pipeDeleteDataNodeEvent.getProgressIndex(); resp = syncPipeConsensusServiceClient.pipeConsensusTransfer( - PipeConsensusPlanNodeReq.toTPipeConsensusTransferReq( - pipeSchemaRegionWritePlanEvent.getPlanNode(), + PipeConsensusDeleteNodeReq.toTPipeConsensusTransferReq( + pipeDeleteDataNodeEvent.getDeleteDataNode(), tCommitId, tConsensusGroupId, progressIndex, @@ -277,14 +276,14 @@ private void doTransfer(final PipeSchemaRegionWritePlanEvent pipeSchemaRegionWri status, String.format( "PipeConsensus transfer DeletionEvent %s error, result status %s.", - pipeSchemaRegionWritePlanEvent.getDeletionResource(), status), - pipeSchemaRegionWritePlanEvent.getPlanNode().toString()); + pipeDeleteDataNodeEvent.getDeletionResource(), status), + pipeDeleteDataNodeEvent.getDeleteDataNode().toString()); } if (LOGGER.isDebugEnabled()) { LOGGER.debug( "Successfully transferred deletion event {}.", - pipeSchemaRegionWritePlanEvent.getDeletionResource()); + pipeDeleteDataNodeEvent.getDeletionResource()); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusPlanNodeReq.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusDeleteNodeReq.java similarity index 68% rename from iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusPlanNodeReq.java rename to iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusDeleteNodeReq.java index 55c7923c3bdb..1bf9916b0b82 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusPlanNodeReq.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusDeleteNodeReq.java @@ -25,8 +25,7 @@ import org.apache.iotdb.commons.pipe.connector.payload.pipeconsensus.request.PipeConsensusRequestVersion; import org.apache.iotdb.consensus.pipe.thrift.TCommitId; import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq; -import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; -import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeType; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; import org.apache.tsfile.utils.PublicBAOS; import org.slf4j.Logger; @@ -37,36 +36,36 @@ import java.nio.ByteBuffer; import java.util.Objects; -public class PipeConsensusPlanNodeReq extends TPipeConsensusTransferReq { - private static final Logger LOGGER = LoggerFactory.getLogger(PipeConsensusPlanNodeReq.class); - private transient PlanNode planNode; +public class PipeConsensusDeleteNodeReq extends TPipeConsensusTransferReq { + private static final Logger LOGGER = LoggerFactory.getLogger(PipeConsensusDeleteNodeReq.class); + private transient DeleteDataNode deleteDataNode; - private PipeConsensusPlanNodeReq() { + private PipeConsensusDeleteNodeReq() { // Do nothing } - public PlanNode getPlanNode() { - return planNode; + public DeleteDataNode getDeleteDataNode() { + return deleteDataNode; } /////////////////////////////// Thrift /////////////////////////////// - public static PipeConsensusPlanNodeReq toTPipeConsensusTransferReq( - PlanNode planNode, + public static PipeConsensusDeleteNodeReq toTPipeConsensusTransferReq( + DeleteDataNode deleteDataNode, TCommitId commitId, TConsensusGroupId consensusGroupId, ProgressIndex progressIndex, int thisDataNodeId) { - final PipeConsensusPlanNodeReq req = new PipeConsensusPlanNodeReq(); + final PipeConsensusDeleteNodeReq req = new PipeConsensusDeleteNodeReq(); - req.planNode = planNode; + req.deleteDataNode = deleteDataNode; req.commitId = commitId; req.consensusGroupId = consensusGroupId; req.dataNodeId = thisDataNodeId; req.version = PipeConsensusRequestVersion.VERSION_1.getVersion(); req.type = PipeConsensusRequestType.TRANSFER_DELETION.getType(); - req.body = planNode.serializeToByteBuffer(); + req.body = deleteDataNode.serializeToByteBuffer(); try (final PublicBAOS byteArrayOutputStream = new PublicBAOS(); final DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { @@ -80,21 +79,21 @@ public static PipeConsensusPlanNodeReq toTPipeConsensusTransferReq( return req; } - public static PipeConsensusPlanNodeReq fromTPipeConsensusTransferReq( + public static PipeConsensusDeleteNodeReq fromTPipeConsensusTransferReq( TPipeConsensusTransferReq transferReq) { - final PipeConsensusPlanNodeReq planNodeReq = new PipeConsensusPlanNodeReq(); + final PipeConsensusDeleteNodeReq deleteNodeReq = new PipeConsensusDeleteNodeReq(); - planNodeReq.planNode = PlanNodeType.deserialize(transferReq.body); + deleteNodeReq.deleteDataNode = DeleteDataNode.deserialize(transferReq.body); - planNodeReq.version = transferReq.version; - planNodeReq.type = transferReq.type; - planNodeReq.body = transferReq.body; - planNodeReq.commitId = transferReq.commitId; - planNodeReq.dataNodeId = transferReq.dataNodeId; - planNodeReq.consensusGroupId = transferReq.consensusGroupId; - planNodeReq.progressIndex = transferReq.progressIndex; + deleteNodeReq.version = transferReq.version; + deleteNodeReq.type = transferReq.type; + deleteNodeReq.body = transferReq.body; + deleteNodeReq.commitId = transferReq.commitId; + deleteNodeReq.dataNodeId = transferReq.dataNodeId; + deleteNodeReq.consensusGroupId = transferReq.consensusGroupId; + deleteNodeReq.progressIndex = transferReq.progressIndex; - return planNodeReq; + return deleteNodeReq; } /////////////////////////////// Object /////////////////////////////// @@ -107,8 +106,8 @@ public boolean equals(Object obj) { if (obj == null || getClass() != obj.getClass()) { return false; } - PipeConsensusPlanNodeReq that = (PipeConsensusPlanNodeReq) obj; - return planNode.equals(that.planNode) + PipeConsensusDeleteNodeReq that = (PipeConsensusDeleteNodeReq) obj; + return deleteDataNode.equals(that.deleteDataNode) && version == that.version && type == that.type && Objects.equals(body, that.body) @@ -121,6 +120,6 @@ public boolean equals(Object obj) { @Override public int hashCode() { return Objects.hash( - planNode, version, type, body, commitId, consensusGroupId, dataNodeId, progressIndex); + deleteDataNode, version, type, body, commitId, consensusGroupId, dataNodeId, progressIndex); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataNodeSyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataNodeSyncConnector.java index 294e69a769ab..b8b778afbfb5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataNodeSyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataNodeSyncConnector.java @@ -20,24 +20,14 @@ package org.apache.iotdb.db.pipe.connector.protocol.thrift.sync; import org.apache.iotdb.common.rpc.thrift.TEndPoint; -import org.apache.iotdb.common.rpc.thrift.TSStatus; -import org.apache.iotdb.commons.pipe.connector.client.IoTDBSyncClient; import org.apache.iotdb.commons.pipe.connector.client.IoTDBSyncClientManager; import org.apache.iotdb.commons.pipe.connector.protocol.IoTDBSslSyncConnector; import org.apache.iotdb.commons.utils.NodeUrlUtils; import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.pipe.connector.client.IoTDBDataNodeSyncClientManager; -import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferPlanNodeReq; -import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameterValidator; -import org.apache.iotdb.pipe.api.exception.PipeConnectionException; -import org.apache.iotdb.pipe.api.exception.PipeException; -import org.apache.iotdb.rpc.TSStatusCode; -import org.apache.iotdb.service.rpc.thrift.TPipeTransferReq; -import org.apache.iotdb.service.rpc.thrift.TPipeTransferResp; -import org.apache.tsfile.utils.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -101,61 +91,4 @@ protected IoTDBSyncClientManager constructClient( loadTsFileStrategy); return clientManager; } - - protected void doTransferWrapper( - final PipeSchemaRegionWritePlanEvent pipeSchemaRegionWritePlanEvent) throws PipeException { - // We increase the reference count for this event to determine if the event may be released. - if (!pipeSchemaRegionWritePlanEvent.increaseReferenceCount( - IoTDBDataNodeSyncConnector.class.getName())) { - return; - } - try { - doTransfer(pipeSchemaRegionWritePlanEvent); - } finally { - pipeSchemaRegionWritePlanEvent.decreaseReferenceCount( - IoTDBDataNodeSyncConnector.class.getName(), false); - } - } - - protected void doTransfer(final PipeSchemaRegionWritePlanEvent pipeSchemaRegionWritePlanEvent) - throws PipeException { - final Pair clientAndStatus = clientManager.getClient(); - - final TPipeTransferResp resp; - try { - final TPipeTransferReq req = - compressIfNeeded( - PipeTransferPlanNodeReq.toTPipeTransferReq( - pipeSchemaRegionWritePlanEvent.getPlanNode())); - rateLimitIfNeeded( - pipeSchemaRegionWritePlanEvent.getPipeName(), - pipeSchemaRegionWritePlanEvent.getCreationTime(), - clientAndStatus.getLeft().getEndPoint(), - req.getBody().length); - resp = clientAndStatus.getLeft().pipeTransfer(req); - } catch (final Exception e) { - clientAndStatus.setRight(false); - throw new PipeConnectionException( - String.format( - "Network error when transfer schema region write plan %s, because %s.", - pipeSchemaRegionWritePlanEvent.getPlanNode().getType(), e.getMessage()), - e); - } - - final TSStatus status = resp.getStatus(); - // Only handle the failed statuses to avoid string format performance overhead - if (resp.getStatus().getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode() - && resp.getStatus().getCode() != TSStatusCode.REDIRECTION_RECOMMEND.getStatusCode()) { - receiverStatusHandler.handle( - status, - String.format( - "Transfer data node write plan %s error, result status %s.", - pipeSchemaRegionWritePlanEvent.getPlanNode().getType(), status), - pipeSchemaRegionWritePlanEvent.getPlanNode().toString()); - } - - if (LOGGER.isDebugEnabled()) { - LOGGER.debug("Successfully transferred schema event {}.", pipeSchemaRegionWritePlanEvent); - } - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java index 772a55b0e17a..c80619c67096 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java @@ -28,6 +28,7 @@ import org.apache.iotdb.db.pipe.connector.payload.evolvable.batch.PipeTabletEventPlainBatch; import org.apache.iotdb.db.pipe.connector.payload.evolvable.batch.PipeTabletEventTsFileBatch; import org.apache.iotdb.db.pipe.connector.payload.evolvable.batch.PipeTransferBatchReqBuilder; +import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferPlanNodeReq; import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferTabletBinaryReq; import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferTabletInsertNodeReq; import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferTabletRawReq; @@ -36,8 +37,8 @@ import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferTsFileSealReq; import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferTsFileSealWithModReq; import org.apache.iotdb.db.pipe.connector.util.LeaderCacheUtils; +import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; -import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.terminate.PipeTerminateEvent; @@ -163,8 +164,8 @@ public void transfer(final TsFileInsertionEvent tsFileInsertionEvent) throws Exc @Override public void transfer(final Event event) throws Exception { - if (event instanceof PipeSchemaRegionWritePlanEvent) { - doTransferWrapper((PipeSchemaRegionWritePlanEvent) event); + if (event instanceof PipeDeleteDataNodeEvent) { + doTransferWrapper((PipeDeleteDataNodeEvent) event); return; } @@ -179,6 +180,63 @@ public void transfer(final Event event) throws Exception { } } + private void doTransferWrapper(final PipeDeleteDataNodeEvent pipeDeleteDataNodeEvent) + throws PipeException { + // We increase the reference count for this event to determine if the event may be released. + if (!pipeDeleteDataNodeEvent.increaseReferenceCount( + IoTDBDataNodeSyncConnector.class.getName())) { + return; + } + try { + doTransfer(pipeDeleteDataNodeEvent); + } finally { + pipeDeleteDataNodeEvent.decreaseReferenceCount( + IoTDBDataNodeSyncConnector.class.getName(), false); + } + } + + private void doTransfer(final PipeDeleteDataNodeEvent pipeDeleteDataNodeEvent) + throws PipeException { + final Pair clientAndStatus = clientManager.getClient(); + + final TPipeTransferResp resp; + try { + final TPipeTransferReq req = + compressIfNeeded( + PipeTransferPlanNodeReq.toTPipeTransferReq( + pipeDeleteDataNodeEvent.getDeleteDataNode())); + rateLimitIfNeeded( + pipeDeleteDataNodeEvent.getPipeName(), + pipeDeleteDataNodeEvent.getCreationTime(), + clientAndStatus.getLeft().getEndPoint(), + req.getBody().length); + resp = clientAndStatus.getLeft().pipeTransfer(req); + } catch (final Exception e) { + clientAndStatus.setRight(false); + throw new PipeConnectionException( + String.format( + "Network error when transfer deletion %s, because %s.", + pipeDeleteDataNodeEvent.getDeleteDataNode().getType(), e.getMessage()), + e); + } + + final TSStatus status = resp.getStatus(); + // Only handle the failed statuses to avoid string format performance overhead + if (resp.getStatus().getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode() + && resp.getStatus().getCode() != TSStatusCode.REDIRECTION_RECOMMEND.getStatusCode()) { + receiverStatusHandler.handle( + status, + String.format( + "Transfer deletion %s error, result status %s.", + pipeDeleteDataNodeEvent.getDeleteDataNode().getType(), status), + pipeDeleteDataNodeEvent.getDeletionResource().toString()); + } + + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("Successfully transferred deletion event {}.", pipeDeleteDataNodeEvent); + } + } + private void doTransferWrapper() throws IOException, WriteProcessException { for (final Pair nonEmptyBatch : tabletBatchBuilder.getAllNonEmptyBatches()) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBSchemaRegionConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBSchemaRegionConnector.java index f70e18c06511..c71679017ba1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBSchemaRegionConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBSchemaRegionConnector.java @@ -19,8 +19,10 @@ package org.apache.iotdb.db.pipe.connector.protocol.thrift.sync; +import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.pipe.connector.client.IoTDBSyncClient; import org.apache.iotdb.commons.pipe.connector.payload.thrift.request.PipeTransferFilePieceReq; +import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferPlanNodeReq; import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferSchemaSnapshotPieceReq; import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferSchemaSnapshotSealReq; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; @@ -72,6 +74,63 @@ public void transfer(final Event event) throws Exception { } } + private void doTransferWrapper( + final PipeSchemaRegionWritePlanEvent pipeSchemaRegionWritePlanEvent) throws PipeException { + // We increase the reference count for this event to determine if the event may be released. + if (!pipeSchemaRegionWritePlanEvent.increaseReferenceCount( + IoTDBDataNodeSyncConnector.class.getName())) { + return; + } + try { + doTransfer(pipeSchemaRegionWritePlanEvent); + } finally { + pipeSchemaRegionWritePlanEvent.decreaseReferenceCount( + IoTDBDataNodeSyncConnector.class.getName(), false); + } + } + + private void doTransfer(final PipeSchemaRegionWritePlanEvent pipeSchemaRegionWritePlanEvent) + throws PipeException { + final Pair clientAndStatus = clientManager.getClient(); + + final TPipeTransferResp resp; + try { + final TPipeTransferReq req = + compressIfNeeded( + PipeTransferPlanNodeReq.toTPipeTransferReq( + pipeSchemaRegionWritePlanEvent.getPlanNode())); + rateLimitIfNeeded( + pipeSchemaRegionWritePlanEvent.getPipeName(), + pipeSchemaRegionWritePlanEvent.getCreationTime(), + clientAndStatus.getLeft().getEndPoint(), + req.getBody().length); + resp = clientAndStatus.getLeft().pipeTransfer(req); + } catch (final Exception e) { + clientAndStatus.setRight(false); + throw new PipeConnectionException( + String.format( + "Network error when transfer schema region write plan %s, because %s.", + pipeSchemaRegionWritePlanEvent.getPlanNode().getType(), e.getMessage()), + e); + } + + final TSStatus status = resp.getStatus(); + // Only handle the failed statuses to avoid string format performance overhead + if (resp.getStatus().getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode() + && resp.getStatus().getCode() != TSStatusCode.REDIRECTION_RECOMMEND.getStatusCode()) { + receiverStatusHandler.handle( + status, + String.format( + "Transfer data node write plan %s error, result status %s.", + pipeSchemaRegionWritePlanEvent.getPlanNode().getType(), status), + pipeSchemaRegionWritePlanEvent.getPlanNode().toString()); + } + + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("Successfully transferred schema event {}.", pipeSchemaRegionWritePlanEvent); + } + } + private void doTransferWrapper(final PipeSchemaRegionSnapshotEvent pipeSchemaRegionSnapshotEvent) throws PipeException, IOException { // We increase the reference count for this event to determine if the event may be released. diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java index 2ad0d6f6f93b..07808a1e42d5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java @@ -21,9 +21,7 @@ import org.apache.iotdb.commons.consensus.index.ProgressIndex; import org.apache.iotdb.commons.pipe.datastructure.PersistentResource; -import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; -import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaSerializableEventType; -import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; +import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,13 +35,13 @@ /** * DeletionResource is designed for IoTConsensusV2 to manage the lifecycle of all deletion * operations including realtime deletion and historical deletion. In order to be compatible with - * user pipe framework, PipeConsensus will use {@link PipeSchemaRegionWritePlanEvent} + * user pipe framework, PipeConsensus will use {@link PipeDeleteDataNodeEvent} */ public class DeletionResource implements PersistentResource { private static final Logger LOGGER = LoggerFactory.getLogger(DeletionResource.class); private final Consumer removeHook; private final AtomicLong latestUpdateTime; - private PipeSchemaRegionWritePlanEvent deletionEvent; + private PipeDeleteDataNodeEvent deletionEvent; private volatile Status currentStatus; // it's safe to use volatile here to make this reference thread-safe. @@ -51,7 +49,7 @@ public class DeletionResource implements PersistentResource { private volatile Exception cause; public DeletionResource( - PipeSchemaRegionWritePlanEvent deletionEvent, Consumer removeHook) { + PipeDeleteDataNodeEvent deletionEvent, Consumer removeHook) { this.deletionEvent = deletionEvent; this.removeHook = removeHook; this.currentStatus = Status.RUNNING; @@ -118,7 +116,7 @@ public synchronized Status waitForResult() { @Override public ProgressIndex getProgressIndex() { - return ((DeleteDataNode) deletionEvent.getPlanNode()).getProgressIndex(); + return deletionEvent.getDeleteDataNode().getProgressIndex(); } @Override @@ -131,7 +129,7 @@ public long getFileEndTime() { return 0; } - public PipeSchemaRegionWritePlanEvent getDeletionEvent() { + public PipeDeleteDataNodeEvent getDeletionEvent() { return deletionEvent; } @@ -141,8 +139,7 @@ public ByteBuffer serialize() { public static DeletionResource deserialize( final ByteBuffer buffer, final Consumer removeHook) throws IOException { - PipeSchemaRegionWritePlanEvent event = - (PipeSchemaRegionWritePlanEvent) PipeSchemaSerializableEventType.deserialize(buffer); + PipeDeleteDataNodeEvent event = PipeDeleteDataNodeEvent.deserialize(buffer); return new DeletionResource(event, removeHook); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java index aab6f0b620bc..0c02c02af10f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java @@ -30,7 +30,7 @@ import org.apache.iotdb.db.pipe.consensus.deletion.persist.DeletionBuffer; import org.apache.iotdb.db.pipe.consensus.deletion.persist.PageCacheDeletionBuffer; import org.apache.iotdb.db.pipe.consensus.deletion.recover.DeletionReader; -import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; +import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import com.google.common.collect.ImmutableList; import org.slf4j.Logger; @@ -140,7 +140,7 @@ private void waitUntilFlushAllDeletions() { } } - public DeletionResource registerDeletionResource(PipeSchemaRegionWritePlanEvent event) { + public DeletionResource registerDeletionResource(PipeDeleteDataNodeEvent event) { DeletionResource deletionResource = new DeletionResource(event, this::removeDeletionResource); event.setDeletionResource(deletionResource); this.deletionResources.add(deletionResource); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java new file mode 100644 index 000000000000..45a7a5edc65a --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java @@ -0,0 +1,166 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.pipe.event.common.deletion; + +import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; +import org.apache.iotdb.commons.pipe.event.EnrichedEvent; +import org.apache.iotdb.commons.pipe.event.SerializableEvent; +import org.apache.iotdb.commons.pipe.pattern.PipePattern; +import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeType; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; + +import org.apache.tsfile.utils.ReadWriteIOUtils; + +import java.nio.ByteBuffer; + +public class PipeDeleteDataNodeEvent extends EnrichedEvent implements SerializableEvent { + private DeleteDataNode deleteDataNode; + private DeletionResource deletionResource; + private boolean isGeneratedByPipe; + private final ProgressIndex progressIndex; + + public PipeDeleteDataNodeEvent() { + // Used for deserialization + this(null, false); + } + + public PipeDeleteDataNodeEvent( + final DeleteDataNode deleteDataNode, final boolean isGeneratedByPipe) { + this(deleteDataNode, null, 0, null, null, isGeneratedByPipe); + } + + public PipeDeleteDataNodeEvent( + final DeleteDataNode deleteDataNode, + final String pipeName, + final long creationTime, + final PipeTaskMeta pipeTaskMeta, + final PipePattern pattern, + final boolean isGeneratedByPipe) { + super(pipeName, creationTime, pipeTaskMeta, pattern, Long.MIN_VALUE, Long.MAX_VALUE); + this.isGeneratedByPipe = isGeneratedByPipe; + this.deleteDataNode = deleteDataNode; + this.progressIndex = deleteDataNode.getProgressIndex(); + } + + public DeleteDataNode getDeleteDataNode() { + return deleteDataNode; + } + + public DeletionResource getDeletionResource() { + return deletionResource; + } + + public void setDeletionResource(DeletionResource deletionResource) { + this.deletionResource = deletionResource; + } + + @Override + public boolean internallyIncreaseResourceReferenceCount(String holderMessage) { + return true; + } + + @Override + public boolean internallyDecreaseResourceReferenceCount(String holderMessage) { + if (deletionResource != null) { + // Trigger hook function. + deletionResource.removeSelf(); + // Resolve circular reference to let GC reclaim them all. + deletionResource.releaseSelf(); + deletionResource = null; + } + return true; + } + + @Override + public ProgressIndex getProgressIndex() { + return progressIndex == null ? MinimumProgressIndex.INSTANCE : progressIndex; + } + + @Override + public EnrichedEvent shallowCopySelfAndBindPipeTaskMetaForProgressReport( + String pipeName, + long creationTime, + PipeTaskMeta pipeTaskMeta, + PipePattern pattern, + long startTime, + long endTime) { + return new PipeDeleteDataNodeEvent( + deleteDataNode, pipeName, creationTime, pipeTaskMeta, pattern, isGeneratedByPipe); + } + + @Override + public boolean isGeneratedByPipe() { + return isGeneratedByPipe; + } + + @Override + public boolean mayEventTimeOverlappedWithTimeRange() { + return true; + } + + @Override + public boolean mayEventPathsOverlappedWithPattern() { + return true; + } + + @Override + public ByteBuffer serializeToByteBuffer() { + final ByteBuffer planBuffer = deleteDataNode.serializeToByteBuffer(); + final ByteBuffer result = ByteBuffer.allocate(Byte.BYTES + planBuffer.limit()); + ReadWriteIOUtils.write(isGeneratedByPipe, result); + result.put(planBuffer); + return result; + } + + @Override + public void deserializeFromByteBuffer(ByteBuffer buffer) { + isGeneratedByPipe = ReadWriteIOUtils.readBool(buffer); + deleteDataNode = (DeleteDataNode) PlanNodeType.deserialize(buffer); + } + + public static PipeDeleteDataNodeEvent deserialize(ByteBuffer buffer) { + final PipeDeleteDataNodeEvent event = new PipeDeleteDataNodeEvent(); + event.deserializeFromByteBuffer(buffer); + return event; + } + + /////////////////////////// Object /////////////////////////// + + @Override + public String toString() { + return String.format( + "PipDeleteDataNodeEvent{progressIndex=%s, isGeneratedByPipe=%s}", + progressIndex, isGeneratedByPipe) + + " - " + + super.toString(); + } + + @Override + public String coreReportMessage() { + return String.format( + "PipeDeleteDataNodeEvent{progressIndex=%s, isGeneratedByPipe=%s}", + progressIndex, isGeneratedByPipe) + + " - " + + super.coreReportMessage(); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java index 3c3023c58ee3..7c6193708f48 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java @@ -20,8 +20,8 @@ package org.apache.iotdb.db.pipe.event.realtime; import org.apache.iotdb.commons.pipe.event.ProgressReportEvent; +import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; -import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.epoch.TsFileEpochManager; @@ -63,7 +63,7 @@ public static PipeRealtimeEvent createRealtimeEvent( public static PipeRealtimeEvent createRealtimeEvent(final DeleteDataNode node) { return new PipeRealtimeEvent( - new PipeSchemaRegionWritePlanEvent(node, node.isGeneratedByPipe()), null, null, null); + new PipeDeleteDataNodeEvent(node, node.isGeneratedByPipe()), null, null, null); } public static PipeRealtimeEvent createRealtimeEvent(final ProgressReportEvent event) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java index dd0c8cbcb53a..b272228d3286 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java @@ -23,7 +23,7 @@ import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; -import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; +import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEventFactory; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionExtractor; @@ -152,7 +152,7 @@ public DeletionResource listenToDeleteData(DeleteDataNode node, String regionId) DeletionResourceManager mgr = DeletionResourceManager.getInstance(regionId); return mgr == null ? null - : mgr.registerDeletionResource((PipeSchemaRegionWritePlanEvent) realtimeEvent.getEvent()); + : mgr.registerDeletionResource((PipeDeleteDataNodeEvent) realtimeEvent.getEvent()); } /////////////////////////////// singleton /////////////////////////////// diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java index 1f5505f5c8a8..47d1bb35cf7e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java @@ -42,7 +42,7 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.exception.DiskSpaceInsufficientException; import org.apache.iotdb.db.exception.LoadFileException; -import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusPlanNodeReq; +import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusDeleteNodeReq; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletBinaryReq; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTabletInsertNodeReq; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFilePieceReq; @@ -256,7 +256,7 @@ private TPipeConsensusTransferResp loadEvent(final TPipeConsensusTransferReq req PipeConsensusTabletBinaryReq.fromTPipeConsensusTransferReq(req)); case TRANSFER_DELETION: return handleTransferDeletion( - PipeConsensusPlanNodeReq.fromTPipeConsensusTransferReq(req)); + PipeConsensusDeleteNodeReq.fromTPipeConsensusTransferReq(req)); case TRANSFER_TS_FILE_PIECE: return handleTransferFilePiece( PipeConsensusTsFilePieceReq.fromTPipeConsensusTransferReq(req), true); @@ -317,12 +317,12 @@ private TPipeConsensusTransferResp handleTransferTabletBinary( return new TPipeConsensusTransferResp(impl.writeOnFollowerReplica(insertNode)); } - private TPipeConsensusTransferResp handleTransferDeletion(final PipeConsensusPlanNodeReq req) + private TPipeConsensusTransferResp handleTransferDeletion(final PipeConsensusDeleteNodeReq req) throws ConsensusGroupNotExistException { PipeConsensusServerImpl impl = Optional.ofNullable(pipeConsensus.getImpl(consensusGroupId)) .orElseThrow(() -> new ConsensusGroupNotExistException(consensusGroupId)); - final DeleteDataNode planNode = (DeleteDataNode) req.getPlanNode(); + final DeleteDataNode planNode = req.getDeleteDataNode(); planNode.markAsGeneratedByRemoteConsensusLeader(); planNode.setProgressIndex( ProgressIndexType.deserializeFrom(ByteBuffer.wrap(req.getProgressIndex()))); From 434f10377eeaf48036d8d4bd22c7c715bd1fff81 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Thu, 19 Sep 2024 22:23:18 +0800 Subject: [PATCH 46/85] fix review: bind pipe task's corresponding event for deletionResource --- .../consensus/deletion/DeletionResource.java | 58 ++++++++----------- .../deletion/DeletionResourceManager.java | 31 ++++++++-- .../PipeSchemaRegionWritePlanEvent.java | 22 ------- ...lDataRegionTsFileAndDeletionExtractor.java | 4 +- .../PipeRealtimeDataRegionExtractor.java | 4 ++ .../assigner/PipeDataRegionAssigner.java | 14 +++++ .../pipe/consensus/DeletionRecoverTest.java | 6 +- .../pipe/consensus/DeletionResourceTest.java | 43 +++++++------- 8 files changed, 94 insertions(+), 88 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java index 07808a1e42d5..4ddd67a15937 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java @@ -29,7 +29,6 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Objects; -import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; /** @@ -40,20 +39,28 @@ public class DeletionResource implements PersistentResource { private static final Logger LOGGER = LoggerFactory.getLogger(DeletionResource.class); private final Consumer removeHook; - private final AtomicLong latestUpdateTime; - private PipeDeleteDataNodeEvent deletionEvent; + private PipeDeleteDataNodeEvent correspondingPipeTaskEvent; private volatile Status currentStatus; // it's safe to use volatile here to make this reference thread-safe. @SuppressWarnings("squid:S3077") private volatile Exception cause; - public DeletionResource( - PipeDeleteDataNodeEvent deletionEvent, Consumer removeHook) { - this.deletionEvent = deletionEvent; + // For first register in DataRegion + public DeletionResource(Consumer removeHook) { this.removeHook = removeHook; this.currentStatus = Status.RUNNING; - latestUpdateTime = new AtomicLong(System.currentTimeMillis()); + } + + // For deserialize + public DeletionResource(PipeDeleteDataNodeEvent event, Consumer removeHook) { + this.correspondingPipeTaskEvent = event; + this.removeHook = removeHook; + this.currentStatus = Status.RUNNING; + } + + public void setCorrespondingPipeTaskEvent(PipeDeleteDataNodeEvent correspondingPipeTaskEvent) { + this.correspondingPipeTaskEvent = correspondingPipeTaskEvent; } /** @@ -62,28 +69,15 @@ public DeletionResource( * deletionResource and deletionEvent so that GC can reclaim them. */ public void releaseSelf() { - deletionEvent = null; + correspondingPipeTaskEvent = null; } public void removeSelf() { removeHook.accept(this); } - public void increaseReferenceCount() { - deletionEvent.increaseReferenceCount(DeletionResource.class.getSimpleName()); - updateLatestUpdateTime(); - } - - public void decreaseReferenceCount() { - deletionEvent.decreaseReferenceCount(DeletionResource.class.getSimpleName(), false); - } - public long getReferenceCount() { - return deletionEvent.getReferenceCount(); - } - - public long getLatestUpdateTime() { - return latestUpdateTime.get(); + return correspondingPipeTaskEvent.getReferenceCount(); } public synchronized void onPersistFailed(Exception e) { @@ -116,7 +110,7 @@ public synchronized Status waitForResult() { @Override public ProgressIndex getProgressIndex() { - return deletionEvent.getDeleteDataNode().getProgressIndex(); + return correspondingPipeTaskEvent.getDeleteDataNode().getProgressIndex(); } @Override @@ -129,12 +123,12 @@ public long getFileEndTime() { return 0; } - public PipeDeleteDataNodeEvent getDeletionEvent() { - return deletionEvent; + public PipeDeleteDataNodeEvent getCorrespondingPipeTaskEvent() { + return correspondingPipeTaskEvent; } public ByteBuffer serialize() { - return deletionEvent.serializeToByteBuffer(); + return correspondingPipeTaskEvent.serializeToByteBuffer(); } public static DeletionResource deserialize( @@ -143,15 +137,10 @@ public static DeletionResource deserialize( return new DeletionResource(event, removeHook); } - private void updateLatestUpdateTime() { - latestUpdateTime.set(System.currentTimeMillis()); - } - @Override public String toString() { return String.format( - "DeletionResource[%s]{referenceCount=%s, latestUpdateTime=%s}", - deletionEvent, getReferenceCount(), getLatestUpdateTime()); + "DeletionResource[%s]{referenceCount=%s}", correspondingPipeTaskEvent, getReferenceCount()); } @Override @@ -163,13 +152,12 @@ public boolean equals(Object o) { return false; } final DeletionResource otherEvent = (DeletionResource) o; - return Objects.equals(deletionEvent, otherEvent.deletionEvent) - && latestUpdateTime.get() == otherEvent.latestUpdateTime.get(); + return Objects.equals(correspondingPipeTaskEvent, otherEvent.correspondingPipeTaskEvent); } @Override public int hashCode() { - return Objects.hash(deletionEvent, latestUpdateTime); + return Objects.hash(correspondingPipeTaskEvent); } public Exception getCause() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java index 0c02c02af10f..4157e61e6a35 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java @@ -43,6 +43,7 @@ import java.nio.file.Paths; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.locks.Condition; @@ -65,6 +66,8 @@ public class DeletionResourceManager implements AutoCloseable { private final String dataRegionId; private final DeletionBuffer deletionBuffer; private final File storageDir; + private final Map eventHash2DeletionResources = + new ConcurrentHashMap<>(); private final List deletionResources = new CopyOnWriteArrayList<>(); private final Lock recoverLock = new ReentrantLock(); private final Condition recoveryReadyCondition = recoverLock.newCondition(); @@ -140,14 +143,33 @@ private void waitUntilFlushAllDeletions() { } } - public DeletionResource registerDeletionResource(PipeDeleteDataNodeEvent event) { - DeletionResource deletionResource = new DeletionResource(event, this::removeDeletionResource); - event.setDeletionResource(deletionResource); + /** + * In this method, we only new an instance and return it to DataRegion and not persist + * deletionResource. Because currently deletionResource can not bind corresponding pipe task's + * deletionEvent. + */ + public DeletionResource registerDeletionResource(PipeDeleteDataNodeEvent originEvent) { + DeletionResource deletionResource = + eventHash2DeletionResources.computeIfAbsent( + Objects.hash(originEvent, dataRegionId), + key -> new DeletionResource(this::removeDeletionResource)); this.deletionResources.add(deletionResource); - deletionBuffer.registerDeletionResource(deletionResource); return deletionResource; } + /** This method will bind event for deletionResource and persist it. */ + public void enrichDeletionResourceAndPersist( + PipeDeleteDataNodeEvent originEvent, PipeDeleteDataNodeEvent copiedEvent) { + int key = Objects.hash(originEvent, dataRegionId); + DeletionResource deletionResource = eventHash2DeletionResources.get(key); + // Bind real deletion event + deletionResource.setCorrespondingPipeTaskEvent(copiedEvent); + // Register a persist task for current deletionResource + deletionBuffer.registerDeletionResource(deletionResource); + // Now, we can safely remove this entry from map. Since this entry will not be used anymore. + eventHash2DeletionResources.remove(key); + } + public List getAllDeletionResources() { recoverLock.lock(); try { @@ -174,6 +196,7 @@ public List getAllDeletionResources() { private synchronized void removeDeletionResource(DeletionResource deletionResource) { // Clean memory deletionResources.remove(deletionResource); + eventHash2DeletionResources.remove(deletionResource.getCorrespondingPipeTaskEvent()); // Clean disk ProgressIndex currentProgressIndex = ProgressIndexDataNodeManager.extractLocalSimpleProgressIndex( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java index f70491e5273a..d6eefe086e55 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/schema/PipeSchemaRegionWritePlanEvent.java @@ -23,7 +23,6 @@ import org.apache.iotdb.commons.pipe.event.PipeWritePlanEvent; import org.apache.iotdb.commons.pipe.pattern.PipePattern; import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta; -import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeType; @@ -34,7 +33,6 @@ public class PipeSchemaRegionWritePlanEvent extends PipeWritePlanEvent { private PlanNode planNode; - private DeletionResource deletionResource; public PipeSchemaRegionWritePlanEvent() { // Used for deserialization @@ -60,26 +58,6 @@ public PlanNode getPlanNode() { return planNode; } - public DeletionResource getDeletionResource() { - return deletionResource; - } - - public void setDeletionResource(DeletionResource deletionResource) { - this.deletionResource = deletionResource; - } - - @Override - public boolean internallyDecreaseResourceReferenceCount(String holderMessage) { - if (deletionResource != null) { - // Trigger hook function. - deletionResource.removeSelf(); - // Resolve circular reference to let GC reclaim them all. - deletionResource.releaseSelf(); - deletionResource = null; - } - return true; - } - @Override public EnrichedEvent shallowCopySelfAndBindPipeTaskMetaForProgressReport( final String pipeName, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java index 47bdf815fe29..924b54546878 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java @@ -32,7 +32,7 @@ import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; -import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; +import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.apache.iotdb.db.pipe.event.common.terminate.PipeTerminateEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.pipe.extractor.dataregion.DataRegionListeningFilter; @@ -664,7 +664,7 @@ private Event supplyTsFileEvent(TsFileResource resource) { } private Event supplyDeletionEvent(final DeletionResource deletionResource) { - PipeSchemaRegionWritePlanEvent event = deletionResource.getDeletionEvent(); + PipeDeleteDataNodeEvent event = deletionResource.getCorrespondingPipeTaskEvent(); event.increaseReferenceCount( PipeHistoricalDataRegionTsFileAndDeletionExtractor.class.getName()); return event; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java index 55faf2db59ca..d247e74450ce 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java @@ -458,6 +458,10 @@ public final PipePattern getPipePattern() { return pipePattern; } + public final String getDataRegionId() { + return dataRegionId; + } + public final long getRealtimeDataExtractionStartTime() { return realtimeDataExtractionStartTime; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java index c36a8877dff3..d3b1fbba0836 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java @@ -22,6 +22,8 @@ import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.event.ProgressReportEvent; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; +import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; @@ -36,6 +38,7 @@ import org.slf4j.LoggerFactory; import java.io.Closeable; +import java.util.Optional; public class PipeDataRegionAssigner implements Closeable { @@ -126,6 +129,17 @@ public void assignToExtractor( extractor.getPipePattern(), extractor.getRealtimeDataExtractionStartTime(), extractor.getRealtimeDataExtractionEndTime()); + // Log deletion event to DAL + if (copiedEvent.getEvent() instanceof PipeDeleteDataNodeEvent) { + Optional.ofNullable( + DeletionResourceManager.getInstance(extractor.getDataRegionId())) + .ifPresent( + mgr -> + mgr.enrichDeletionResourceAndPersist( + (PipeDeleteDataNodeEvent) event.getEvent(), + (PipeDeleteDataNodeEvent) copiedEvent.getEvent())); + } + final EnrichedEvent innerEvent = copiedEvent.getEvent(); if (innerEvent instanceof PipeTsFileInsertionEvent) { ((PipeTsFileInsertionEvent) innerEvent) diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java index 294df9029616..d069301221f1 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java @@ -25,7 +25,7 @@ import org.apache.iotdb.commons.utils.FileUtils; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; -import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; +import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; @@ -63,9 +63,9 @@ public void setUp() throws Exception { new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 150); deleteDataNode.setProgressIndex( new RecoverProgressIndex(THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, i))); - PipeSchemaRegionWritePlanEvent deletionEvent = - new PipeSchemaRegionWritePlanEvent(deleteDataNode, true); + PipeDeleteDataNodeEvent deletionEvent = new PipeDeleteDataNodeEvent(deleteDataNode, true); deletionResourceManager.registerDeletionResource(deletionEvent); + deletionResourceManager.enrichDeletionResourceAndPersist(deletionEvent, deletionEvent); } // Manually close for further test deletionResourceManager.close(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java index 281a323bbf0d..773cd67ba01d 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java @@ -28,7 +28,7 @@ import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource.Status; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; -import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; +import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.listener.PipeInsertionDataNodeListener; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; @@ -50,7 +50,7 @@ import java.util.stream.Stream; public class DeletionResourceTest { - private static final String[] FAKE_DATE_REGION_IDS = {"2", "3", "4", "5", "6"}; + private static final String[] FAKE_DATA_REGION_IDS = {"2", "3", "4", "5", "6"}; private static final String DELETION_BASE_DIR = IoTDBDescriptor.getInstance().getConfig().getPipeConsensusDeletionFileDir(); private static final int THIS_DATANODE_ID = @@ -64,7 +64,7 @@ public void setUp() throws Exception { @After public void tearDown() throws Exception { - for (String FAKE_DATE_REGION_ID : FAKE_DATE_REGION_IDS) { + for (String FAKE_DATE_REGION_ID : FAKE_DATA_REGION_IDS) { File baseDir = new File(DELETION_BASE_DIR + File.separator + FAKE_DATE_REGION_ID); if (baseDir.exists()) { FileUtils.deleteFileOrDirectory(baseDir); @@ -74,9 +74,9 @@ public void tearDown() throws Exception { @Test public void testCreateBaseDir() { - deletionResourceManager = DeletionResourceManager.getInstance(FAKE_DATE_REGION_IDS[0]); + deletionResourceManager = DeletionResourceManager.getInstance(FAKE_DATA_REGION_IDS[0]); File baseDir = new File(DELETION_BASE_DIR); - File dataRegionDir = new File(baseDir + File.separator + FAKE_DATE_REGION_IDS[0]); + File dataRegionDir = new File(baseDir + File.separator + FAKE_DATA_REGION_IDS[0]); Assert.assertTrue(baseDir.exists()); Assert.assertTrue(dataRegionDir.exists()); } @@ -84,7 +84,7 @@ public void testCreateBaseDir() { @Test public void testAddBatchDeletionResource() throws IllegalPathException, InterruptedException, IOException { - deletionResourceManager = DeletionResourceManager.getInstance(FAKE_DATE_REGION_IDS[1]); + deletionResourceManager = DeletionResourceManager.getInstance(FAKE_DATA_REGION_IDS[1]); int deletionCount = 10; int rebootTimes = 0; MeasurementPath path = new MeasurementPath("root.vehicle.d2.s0"); @@ -93,61 +93,60 @@ public void testAddBatchDeletionResource() new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 150); deleteDataNode.setProgressIndex( new RecoverProgressIndex(THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, i))); - PipeSchemaRegionWritePlanEvent deletionEvent = - new PipeSchemaRegionWritePlanEvent(deleteDataNode, true); + PipeDeleteDataNodeEvent deletionEvent = new PipeDeleteDataNodeEvent(deleteDataNode, true); deletionResourceManager.registerDeletionResource(deletionEvent); + deletionResourceManager.enrichDeletionResourceAndPersist(deletionEvent, deletionEvent); } Stream paths = - Files.list(Paths.get(DELETION_BASE_DIR + File.separator + FAKE_DATE_REGION_IDS[1])); - ; + Files.list(Paths.get(DELETION_BASE_DIR + File.separator + FAKE_DATA_REGION_IDS[1])); Assert.assertTrue(paths.anyMatch(Files::isRegularFile)); } @Test public void testAddDeletionResourceTimeout() throws IllegalPathException, InterruptedException, IOException { - deletionResourceManager = DeletionResourceManager.getInstance(FAKE_DATE_REGION_IDS[2]); + deletionResourceManager = DeletionResourceManager.getInstance(FAKE_DATA_REGION_IDS[2]); int rebootTimes = 0; MeasurementPath path = new MeasurementPath("root.vehicle.d2.s0"); DeleteDataNode deleteDataNode = new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 150); deleteDataNode.setProgressIndex( new RecoverProgressIndex(THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, 1))); - PipeSchemaRegionWritePlanEvent deletionEvent = - new PipeSchemaRegionWritePlanEvent(deleteDataNode, true); + PipeDeleteDataNodeEvent deletionEvent = new PipeDeleteDataNodeEvent(deleteDataNode, true); // Only register one deletionResource deletionResourceManager.registerDeletionResource(deletionEvent); + deletionResourceManager.enrichDeletionResourceAndPersist(deletionEvent, deletionEvent); // Sleep to wait deletion being persisted Thread.sleep(5000); Stream paths = - Files.list(Paths.get(DELETION_BASE_DIR + File.separator + FAKE_DATE_REGION_IDS[2])); + Files.list(Paths.get(DELETION_BASE_DIR + File.separator + FAKE_DATA_REGION_IDS[2])); Assert.assertTrue(paths.anyMatch(Files::isRegularFile)); } @Test public void testDeletionRemove() throws IllegalPathException, InterruptedException, IOException { - deletionResourceManager = DeletionResourceManager.getInstance(FAKE_DATE_REGION_IDS[3]); + deletionResourceManager = DeletionResourceManager.getInstance(FAKE_DATA_REGION_IDS[3]); // new a deletion int rebootTimes = 0; int deletionCount = 100; MeasurementPath path = new MeasurementPath("root.vehicle.d2.s0"); - List deletionEvents = new ArrayList<>(); + List deletionEvents = new ArrayList<>(); for (int i = 0; i < deletionCount; i++) { DeleteDataNode deleteDataNode = new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 150); deleteDataNode.setProgressIndex( new RecoverProgressIndex(THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, i))); - PipeSchemaRegionWritePlanEvent deletionEvent = - new PipeSchemaRegionWritePlanEvent(deleteDataNode, true); + PipeDeleteDataNodeEvent deletionEvent = new PipeDeleteDataNodeEvent(deleteDataNode, true); deletionEvents.add(deletionEvent); deletionResourceManager.registerDeletionResource(deletionEvent); + deletionResourceManager.enrichDeletionResourceAndPersist(deletionEvent, deletionEvent); } deletionEvents.forEach(deletionEvent -> deletionEvent.increaseReferenceCount("test")); // Sleep to wait deletion being persisted Thread.sleep(1000); List paths = - Files.list(Paths.get(DELETION_BASE_DIR + File.separator + FAKE_DATE_REGION_IDS[3])) + Files.list(Paths.get(DELETION_BASE_DIR + File.separator + FAKE_DATA_REGION_IDS[3])) .collect(Collectors.toList()); Assert.assertTrue(paths.stream().anyMatch(Files::isRegularFile)); int beforeFileCount = paths.size(); @@ -159,7 +158,7 @@ public void testDeletionRemove() throws IllegalPathException, InterruptedExcepti // Sleep to wait deletion being removed Thread.sleep(1000); List newPaths = - Files.list(Paths.get(DELETION_BASE_DIR + File.separator + FAKE_DATE_REGION_IDS[3])) + Files.list(Paths.get(DELETION_BASE_DIR + File.separator + FAKE_DATA_REGION_IDS[3])) .collect(Collectors.toList()); int afterCount = newPaths.size(); Assert.assertTrue(afterCount < beforeFileCount); @@ -167,7 +166,7 @@ public void testDeletionRemove() throws IllegalPathException, InterruptedExcepti @Test public void testWaitForResult() throws Exception { - deletionResourceManager = DeletionResourceManager.getInstance(FAKE_DATE_REGION_IDS[4]); + deletionResourceManager = DeletionResourceManager.getInstance(FAKE_DATA_REGION_IDS[4]); int rebootTimes = 0; MeasurementPath path = new MeasurementPath("root.vehicle.d2.s0"); DeleteDataNode deleteDataNode = @@ -176,7 +175,7 @@ public void testWaitForResult() throws Exception { new RecoverProgressIndex(THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, 1))); DeletionResource deletionResource = PipeInsertionDataNodeListener.getInstance() - .listenToDeleteData(deleteDataNode, FAKE_DATE_REGION_IDS[4]); + .listenToDeleteData(deleteDataNode, FAKE_DATA_REGION_IDS[4]); Assert.assertSame(deletionResource.waitForResult(), Status.SUCCESS); } } From 7e9414119fe5e1ed74a4fc9089dbf5406351bb48 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Thu, 19 Sep 2024 23:52:38 +0800 Subject: [PATCH 47/85] fix review: deletion will only be assigned to corresponding dataRegion's extractor & deletion will not be extracted when persist failed. --- .../deletion/DeletionResourceManager.java | 3 ++- .../assigner/PipeDataRegionAssigner.java | 24 ++++++++++++------- .../PipeInsertionDataNodeListener.java | 20 ++++++++++++---- 3 files changed, 33 insertions(+), 14 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java index 4157e61e6a35..af94d89b9f1a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java @@ -158,7 +158,7 @@ public DeletionResource registerDeletionResource(PipeDeleteDataNodeEvent originE } /** This method will bind event for deletionResource and persist it. */ - public void enrichDeletionResourceAndPersist( + public DeletionResource enrichDeletionResourceAndPersist( PipeDeleteDataNodeEvent originEvent, PipeDeleteDataNodeEvent copiedEvent) { int key = Objects.hash(originEvent, dataRegionId); DeletionResource deletionResource = eventHash2DeletionResources.get(key); @@ -168,6 +168,7 @@ public void enrichDeletionResourceAndPersist( deletionBuffer.registerDeletionResource(deletionResource); // Now, we can safely remove this entry from map. Since this entry will not be used anymore. eventHash2DeletionResources.remove(key); + return deletionResource; } public List getAllDeletionResources() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java index d3b1fbba0836..e3ffa36ed34c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java @@ -22,6 +22,8 @@ import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.event.ProgressReportEvent; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource.Status; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; @@ -38,7 +40,7 @@ import org.slf4j.LoggerFactory; import java.io.Closeable; -import java.util.Optional; +import java.util.Objects; public class PipeDataRegionAssigner implements Closeable { @@ -131,13 +133,19 @@ public void assignToExtractor( extractor.getRealtimeDataExtractionEndTime()); // Log deletion event to DAL if (copiedEvent.getEvent() instanceof PipeDeleteDataNodeEvent) { - Optional.ofNullable( - DeletionResourceManager.getInstance(extractor.getDataRegionId())) - .ifPresent( - mgr -> - mgr.enrichDeletionResourceAndPersist( - (PipeDeleteDataNodeEvent) event.getEvent(), - (PipeDeleteDataNodeEvent) copiedEvent.getEvent())); + DeletionResourceManager mgr = + DeletionResourceManager.getInstance(extractor.getDataRegionId()); + if (Objects.nonNull(mgr)) { + DeletionResource deletionResource = + mgr.enrichDeletionResourceAndPersist( + (PipeDeleteDataNodeEvent) event.getEvent(), + (PipeDeleteDataNodeEvent) copiedEvent.getEvent()); + // if persist failed, skip sending this event to keep consistency with the + // behavior of storage engine. + if (deletionResource.waitForResult() == Status.FAILURE) { + return; + } + } } final EnrichedEvent innerEvent = copiedEvent.getEvent(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java index b272228d3286..28db982ce554 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java @@ -33,6 +33,7 @@ import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource; import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALEntryHandler; +import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicInteger; @@ -147,12 +148,21 @@ public void listenToHeartbeat(boolean shouldPrintMessage) { public DeletionResource listenToDeleteData(DeleteDataNode node, String regionId) { PipeRealtimeEvent realtimeEvent = PipeRealtimeEventFactory.createRealtimeEvent(node); - dataRegionId2Assigner.forEach((key, value) -> value.publishToAssign(realtimeEvent)); - // log deletion to DAL + final PipeDataRegionAssigner assigner = dataRegionId2Assigner.get(regionId); + // only events from registered data region will be extracted + if (assigner == null) { + return null; + } + // register a deletionResource and return it to DataRegion DeletionResourceManager mgr = DeletionResourceManager.getInstance(regionId); - return mgr == null - ? null - : mgr.registerDeletionResource((PipeDeleteDataNodeEvent) realtimeEvent.getEvent()); + DeletionResource deletionResource = null; + if (Objects.nonNull(mgr)) { + deletionResource = + mgr.registerDeletionResource((PipeDeleteDataNodeEvent) realtimeEvent.getEvent()); + } + // register first, then publish. + assigner.publishToAssign(realtimeEvent); + return deletionResource; } /////////////////////////////// singleton /////////////////////////////// From e13b3e73e123714aba31fe6fc4590ed2ac5cfe2f Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Fri, 20 Sep 2024 11:28:21 +0800 Subject: [PATCH 48/85] fix review --- .../deletion/DeletionResourceManager.java | 16 ++-------------- .../persist/PageCacheDeletionBuffer.java | 15 +++++++++++++++ 2 files changed, 17 insertions(+), 14 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java index af94d89b9f1a..49287e37c2ff 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java @@ -128,21 +128,9 @@ public void close() { LOGGER.info("Closing deletion resource manager for {}...", dataRegionId); this.deletionResources.clear(); this.deletionBuffer.close(); - waitUntilFlushAllDeletions(); LOGGER.info("Deletion resource manager for {} has been successfully closed!", dataRegionId); } - private void waitUntilFlushAllDeletions() { - while (!deletionBuffer.isAllDeletionFlushed()) { - try { - Thread.sleep(50); - } catch (InterruptedException e) { - LOGGER.error("Interrupted when waiting for all deletions flushed."); - Thread.currentThread().interrupt(); - } - } - } - /** * In this method, we only new an instance and return it to DataRegion and not persist * deletionResource. Because currently deletionResource can not bind corresponding pipe task's @@ -210,7 +198,7 @@ private synchronized void removeDeletionResource(DeletionResource deletionResour .filter(path -> path.getFileName().toString().matches(DELETION_FILE_NAME_PATTERN)) .filter( path -> - isFileProgressBehindGivenProgress( + isFileProgressCoveredByGivenProgress( path.getFileName().toString(), currentProgressIndex)) .sorted(this::compareFileProgressIndex) .toArray(Path[]::new); @@ -252,7 +240,7 @@ private int compareFileProgressIndex(Path file1, Path file2) { return 0; } - private boolean isFileProgressBehindGivenProgress( + private boolean isFileProgressCoveredByGivenProgress( String fileName, ProgressIndex currentProgressIndex) { if (currentProgressIndex instanceof SimpleProgressIndex) { SimpleProgressIndex simpleProgressIndex = (SimpleProgressIndex) currentProgressIndex; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java index cf26679f2c6b..c459be775cdf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java @@ -61,6 +61,7 @@ public class PageCacheDeletionBuffer implements DeletionBuffer { private static final int ONE_THIRD_WAL_BUFFER_SIZE = config.getWalBufferSize() / 3; private static final double FSYNC_BUFFER_RATIO = 0.95; private static final int QUEUE_CAPACITY = config.getWalBufferQueueCapacity(); + private static final long MAX_WAIT_CLOSE_TIME_IN_MS = 10000; // DeletionResources private final BlockingQueue deletionResources = @@ -316,6 +317,7 @@ public void close() { isClosed = true; // Force sync existing data in memory to disk. // first waiting serialize and sync tasks finished, then release all resources + waitUntilFlushAllDeletionsOrTimeOut(); if (persistThread != null) { shutdownThread(persistThread, ThreadName.PIPE_CONSENSUS_DELETION_SERIALIZE); } @@ -323,6 +325,19 @@ public void close() { MmapUtil.clean(serializeBuffer); } + private void waitUntilFlushAllDeletionsOrTimeOut() { + long currentTime = System.currentTimeMillis(); + while (!isAllDeletionFlushed() + && System.currentTimeMillis() - currentTime < MAX_WAIT_CLOSE_TIME_IN_MS) { + try { + Thread.sleep(50); + } catch (InterruptedException e) { + LOGGER.error("Interrupted when waiting for all deletions flushed."); + Thread.currentThread().interrupt(); + } + } + } + private void shutdownThread(ExecutorService thread, ThreadName threadName) { thread.shutdown(); try { From 535150efb37f514ed3a25d1721940b11910bac4c Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Fri, 20 Sep 2024 16:01:33 +0800 Subject: [PATCH 49/85] fix --- .../db/pipe/consensus/deletion/DeletionResourceManager.java | 1 - 1 file changed, 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java index 49287e37c2ff..d57c1554da5d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java @@ -185,7 +185,6 @@ public List getAllDeletionResources() { private synchronized void removeDeletionResource(DeletionResource deletionResource) { // Clean memory deletionResources.remove(deletionResource); - eventHash2DeletionResources.remove(deletionResource.getCorrespondingPipeTaskEvent()); // Clean disk ProgressIndex currentProgressIndex = ProgressIndexDataNodeManager.extractLocalSimpleProgressIndex( From 0b39c8b788ce0f74111ecdbd7f8f664b294d6413 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Sat, 21 Sep 2024 17:06:42 +0800 Subject: [PATCH 50/85] feat: timout sync --- .../persist/PageCacheDeletionBuffer.java | 148 ++++++++++-------- 1 file changed, 86 insertions(+), 62 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java index c459be775cdf..373e45faaae5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java @@ -41,9 +41,9 @@ import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; -import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ExecutorService; +import java.util.concurrent.PriorityBlockingQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.Lock; @@ -63,9 +63,12 @@ public class PageCacheDeletionBuffer implements DeletionBuffer { private static final int QUEUE_CAPACITY = config.getWalBufferQueueCapacity(); private static final long MAX_WAIT_CLOSE_TIME_IN_MS = 10000; - // DeletionResources + // DeletionResources received from storage engine, which is waiting to be persisted. private final BlockingQueue deletionResources = - new ArrayBlockingQueue<>(QUEUE_CAPACITY); + new PriorityBlockingQueue<>( + QUEUE_CAPACITY, + // any two progressIndex can't be equal + (o1, o2) -> o1.getProgressIndex().isAfter(o2.getProgressIndex()) ? 1 : -1); // Data region id private final String groupId; // directory to store .deletion files @@ -75,21 +78,23 @@ public class PageCacheDeletionBuffer implements DeletionBuffer { private final Lock buffersLock = new ReentrantLock(); // Total size of this batch. private final AtomicInteger totalSize = new AtomicInteger(0); - // All deletions that will be written to the current file - private final List pendingDeletions = new ArrayList<>(); + // All deletions that will be handled in a single persist task + private final List pendingDeletionsInOneTask = new ArrayList<>(); // whether close method is called private volatile boolean isClosed = false; - // Serialize buffer in current batch + // Serialize buffer in current persist task private volatile ByteBuffer serializeBuffer; // Current Logging file. private volatile File logFile; private volatile FileOutputStream logStream; private volatile FileChannel logChannel; - // Max progressIndex among last batch. Used by PersistTask for naming .deletion file. + // Max progressIndex among current .deletion file. Used by PersistTask for naming .deletion file. + private ProgressIndex maxProgressIndexInCurrentFile = MinimumProgressIndex.INSTANCE; + // Max progressIndex among last .deletion file. Used by PersistTask for naming .deletion file. // Since deletions are written serially, DAL is also written serially. This ensures that the // maxProgressIndex of each batch increases in the same order as the physical time. - private volatile ProgressIndex maxProgressIndexInLastBatch = MinimumProgressIndex.INSTANCE; + private volatile ProgressIndex maxProgressIndexInLastFile = MinimumProgressIndex.INSTANCE; public PageCacheDeletionBuffer(String groupId, String baseDirectory) { this.groupId = groupId; @@ -160,31 +165,34 @@ public void registerDeletionResource(DeletionResource deletionResource) { private void appendCurrentBatch() throws IOException { serializeBuffer.flip(); logChannel.write(serializeBuffer); - // Mark DeletionResources to persisted once deletion has been written to page cache - pendingDeletions.forEach(DeletionResource::onPersistSucceed); - resetTaskAttribute(); } - private void fsyncCurrentLoggingFileAndReset(ProgressIndex curMaxProgressIndex) - throws IOException { - try { - // Close old resource to fsync. - this.logStream.close(); - this.logChannel.close(); - } finally { - resetFileAttribute(curMaxProgressIndex); - } + private void fsyncCurrentLoggingFile() throws IOException { + this.logChannel.force(false); + pendingDeletionsInOneTask.forEach(DeletionResource::onPersistSucceed); + } + + private void closeCurrentLoggingFile() throws IOException { + // Close old resource to fsync. + this.logStream.close(); + this.logChannel.close(); + pendingDeletionsInOneTask.forEach(DeletionResource::onPersistSucceed); } private void resetTaskAttribute() { - this.pendingDeletions.clear(); + this.pendingDeletionsInOneTask.clear(); clearBuffer(); } - private void resetFileAttribute(ProgressIndex curMaxProgressIndex) { + private void resetFileAttribute() { // Reset file attributes. this.totalSize.set(0); - this.maxProgressIndexInLastBatch = curMaxProgressIndex; + this.maxProgressIndexInLastFile = this.maxProgressIndexInCurrentFile; + this.maxProgressIndexInCurrentFile = MinimumProgressIndex.INSTANCE; + } + + private void rollbackFileAttribute(int currentBatchSize) { + this.totalSize.addAndGet(-currentBatchSize); } private void clearBuffer() { @@ -198,35 +206,39 @@ private void clearBuffer() { } private void switchLoggingFile() throws IOException { - // PipeConsensus ensures that deleteDataNodes use recoverProgressIndex. - ProgressIndex curProgressIndex = - ProgressIndexDataNodeManager.extractLocalSimpleProgressIndex(maxProgressIndexInLastBatch); - if (!(curProgressIndex instanceof SimpleProgressIndex)) { - throw new IOException("Invalid deletion progress index: " + curProgressIndex); - } - SimpleProgressIndex progressIndex = (SimpleProgressIndex) curProgressIndex; - // Deletion file name format: "_{rebootTimes}_{memTableFlushOrderId}.deletion" - this.logFile = - new File( - baseDirectory, - String.format( - "_%d-%d%s", - progressIndex.getRebootTimes(), - progressIndex.getMemTableFlushOrderId(), - DeletionResourceManager.DELETION_FILE_SUFFIX)); - this.logStream = new FileOutputStream(logFile, true); - this.logChannel = logStream.getChannel(); - // Create file && write magic string - if (!logFile.exists() || logFile.length() == 0) { - this.logChannel.write( - ByteBuffer.wrap( - DeletionResourceManager.MAGIC_VERSION_V1.getBytes(StandardCharsets.UTF_8))); + try { + // PipeConsensus ensures that deleteDataNodes use recoverProgressIndex. + ProgressIndex curProgressIndex = + ProgressIndexDataNodeManager.extractLocalSimpleProgressIndex(maxProgressIndexInLastFile); + if (!(curProgressIndex instanceof SimpleProgressIndex)) { + throw new IOException("Invalid deletion progress index: " + curProgressIndex); + } + SimpleProgressIndex progressIndex = (SimpleProgressIndex) curProgressIndex; + // Deletion file name format: "_{rebootTimes}_{memTableFlushOrderId}.deletion" + this.logFile = + new File( + baseDirectory, + String.format( + "_%d-%d%s", + progressIndex.getRebootTimes(), + progressIndex.getMemTableFlushOrderId(), + DeletionResourceManager.DELETION_FILE_SUFFIX)); + this.logStream = new FileOutputStream(logFile, true); + this.logChannel = logStream.getChannel(); + // Create file && write magic string + if (!logFile.exists() || logFile.length() == 0) { + this.logChannel.write( + ByteBuffer.wrap( + DeletionResourceManager.MAGIC_VERSION_V1.getBytes(StandardCharsets.UTF_8))); + } + } finally { + resetFileAttribute(); } } private class PersistTask implements Runnable { - // Max progressIndex among this batch. Used by SyncTask for naming .deletion file. - private ProgressIndex maxProgressIndexInCurrentBatch = MinimumProgressIndex.INSTANCE; + // Batch size in current task, used to roll back. + private final AtomicInteger currentTaskBatchSize = new AtomicInteger(0); @Override public void run() { @@ -235,8 +247,9 @@ public void run() { } catch (IOException e) { LOGGER.warn( "Deletion persist: Cannot write to {}, may cause data inconsistency.", logFile, e); - pendingDeletions.forEach(deletionResource -> deletionResource.onPersistFailed(e)); - resetFileAttribute(maxProgressIndexInLastBatch); + // if any exception occurred, this batch will not be written to disk and lost. + pendingDeletionsInOneTask.forEach(deletionResource -> deletionResource.onPersistFailed(e)); + rollbackFileAttribute(currentTaskBatchSize.get()); } finally { if (!isClosed) { persistThread.submit(new PersistTask()); @@ -252,6 +265,7 @@ private boolean serializeDeletionToBatchBuffer(DeletionResource deletionResource } serializeBuffer.put(buffer.array()); totalSize.addAndGet(buffer.position()); + currentTaskBatchSize.addAndGet(buffer.position()); return true; } @@ -259,10 +273,12 @@ private void persistDeletion() throws IOException { // For first deletion we use blocking take() method. try { DeletionResource firstDeletionResource = deletionResources.take(); - pendingDeletions.add(firstDeletionResource); + // Serialize deletion. The first serialization cannot fail because a deletion cannot exceed + // size of serializeBuffer. serializeDeletionToBatchBuffer(firstDeletionResource); - maxProgressIndexInCurrentBatch = - maxProgressIndexInCurrentBatch.updateToMinimumEqualOrIsAfterProgressIndex( + pendingDeletionsInOneTask.add(firstDeletionResource); + maxProgressIndexInCurrentFile = + maxProgressIndexInCurrentFile.updateToMinimumEqualOrIsAfterProgressIndex( firstDeletionResource.getProgressIndex()); } catch (InterruptedException e) { LOGGER.warn( @@ -287,26 +303,33 @@ private void persistDeletion() throws IOException { if (deletionResource == null) { // append to current file and not switch file appendCurrentBatch(); + fsyncCurrentLoggingFile(); + resetTaskAttribute(); return; } // Serialize deletion if (!serializeDeletionToBatchBuffer(deletionResource)) { - // If working buffer is exhausted, fsync immediately and roll to a new file. + // if working buffer is exhausted, which means serialization failed. + // 1. roll back + deletionResources.add(deletionResource); + // 2. fsync immediately and roll to a new file. appendCurrentBatch(); - fsyncCurrentLoggingFileAndReset(maxProgressIndexInCurrentBatch); + closeCurrentLoggingFile(); + resetTaskAttribute(); switchLoggingFile(); return; } - // Update max progressIndex - maxProgressIndexInCurrentBatch = - maxProgressIndexInCurrentBatch.updateToMinimumEqualOrIsAfterProgressIndex( + pendingDeletionsInOneTask.add(deletionResource); + // Update max progressIndex in current file if serialized successfully. + maxProgressIndexInCurrentFile = + maxProgressIndexInCurrentFile.updateToMinimumEqualOrIsAfterProgressIndex( deletionResource.getProgressIndex()); - pendingDeletions.add(deletionResource); } // Persist deletions; Defensive programming here, just in case. if (totalSize.get() > 0) { appendCurrentBatch(); - fsyncCurrentLoggingFileAndReset(maxProgressIndexInCurrentBatch); + closeCurrentLoggingFile(); + resetTaskAttribute(); switchLoggingFile(); } } @@ -319,7 +342,7 @@ public void close() { // first waiting serialize and sync tasks finished, then release all resources waitUntilFlushAllDeletionsOrTimeOut(); if (persistThread != null) { - shutdownThread(persistThread, ThreadName.PIPE_CONSENSUS_DELETION_SERIALIZE); + shutdownThread(persistThread); } // clean buffer MmapUtil.clean(serializeBuffer); @@ -338,7 +361,8 @@ private void waitUntilFlushAllDeletionsOrTimeOut() { } } - private void shutdownThread(ExecutorService thread, ThreadName threadName) { + private void shutdownThread(ExecutorService thread) { + ThreadName threadName = ThreadName.PIPE_CONSENSUS_DELETION_SERIALIZE; thread.shutdown(); try { if (!thread.awaitTermination(30, TimeUnit.SECONDS)) { From 4045de73a7a6bf1dbd7b5590e7f4ded7cfda9d13 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Mon, 23 Sep 2024 13:22:46 +0800 Subject: [PATCH 51/85] todo: priority queue sort --- .../db/pipe/consensus/deletion/DeletionResourceManager.java | 2 +- .../consensus/deletion/persist/PageCacheDeletionBuffer.java | 5 ++++- .../apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java | 3 ++- 3 files changed, 7 insertions(+), 3 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java index d57c1554da5d..edb4bb38e6c2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java @@ -83,7 +83,7 @@ private DeletionResourceManager(String dataRegionId) throws IOException { this.deletionBuffer = new PageCacheDeletionBuffer(dataRegionId, storageDir.getAbsolutePath()); initAndRecover(); // Only after initAndRecover can we start serialize and sync new deletions. - this.deletionBuffer.start(); + // this.deletionBuffer.start(); } private void initAndRecover() throws IOException { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java index 373e45faaae5..8940295051f5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java @@ -68,7 +68,10 @@ public class PageCacheDeletionBuffer implements DeletionBuffer { new PriorityBlockingQueue<>( QUEUE_CAPACITY, // any two progressIndex can't be equal - (o1, o2) -> o1.getProgressIndex().isAfter(o2.getProgressIndex()) ? 1 : -1); + (o1, o2) -> + o1.getProgressIndex().equals(o2.getProgressIndex()) + ? 0 + : (o1.getProgressIndex().isAfter(o2.getProgressIndex()) ? 1 : -1)); // Data region id private final String groupId; // directory to store .deletion files diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java index d069301221f1..45f7b8340083 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java @@ -62,7 +62,8 @@ public void setUp() throws Exception { DeleteDataNode deleteDataNode = new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 150); deleteDataNode.setProgressIndex( - new RecoverProgressIndex(THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, i))); + new RecoverProgressIndex( + THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, deletionCount - i))); PipeDeleteDataNodeEvent deletionEvent = new PipeDeleteDataNodeEvent(deleteDataNode, true); deletionResourceManager.registerDeletionResource(deletionEvent); deletionResourceManager.enrichDeletionResourceAndPersist(deletionEvent, deletionEvent); From 22a3ca6847b8ab36ae2efd9031cf47b817be8824 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Mon, 23 Sep 2024 22:44:37 +0800 Subject: [PATCH 52/85] fix ut --- .../deletion/DeletionResourceManager.java | 2 +- .../pipe/consensus/DeletionRecoverTest.java | 10 ++------ .../pipe/consensus/DeletionResourceTest.java | 23 +++++++++++++++++++ 3 files changed, 26 insertions(+), 9 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java index edb4bb38e6c2..d57c1554da5d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java @@ -83,7 +83,7 @@ private DeletionResourceManager(String dataRegionId) throws IOException { this.deletionBuffer = new PageCacheDeletionBuffer(dataRegionId, storageDir.getAbsolutePath()); initAndRecover(); // Only after initAndRecover can we start serialize and sync new deletions. - // this.deletionBuffer.start(); + this.deletionBuffer.start(); } private void initAndRecover() throws IOException { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java index 45f7b8340083..d79da421515f 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java @@ -62,18 +62,12 @@ public void setUp() throws Exception { DeleteDataNode deleteDataNode = new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 150); deleteDataNode.setProgressIndex( - new RecoverProgressIndex( - THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, deletionCount - i))); + new RecoverProgressIndex(THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, i))); PipeDeleteDataNodeEvent deletionEvent = new PipeDeleteDataNodeEvent(deleteDataNode, true); deletionResourceManager.registerDeletionResource(deletionEvent); deletionResourceManager.enrichDeletionResourceAndPersist(deletionEvent, deletionEvent); } - // Manually close for further test - deletionResourceManager.close(); - } - - @After - public void tearDown() throws Exception { + // Manually close to ensure all deletions are persisted deletionResourceManager.close(); } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java index 773cd67ba01d..567a9bdad95c 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java @@ -23,15 +23,21 @@ import org.apache.iotdb.commons.consensus.index.impl.SimpleProgressIndex; import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.path.MeasurementPath; +import org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant; +import org.apache.iotdb.commons.pipe.config.plugin.configuraion.PipeTaskRuntimeConfiguration; +import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskExtractorRuntimeEnvironment; import org.apache.iotdb.commons.utils.FileUtils; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource.Status; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; +import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionExtractor; +import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionHybridExtractor; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.listener.PipeInsertionDataNodeListener; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; +import org.apache.iotdb.pipe.api.customizer.parameter.PipeParameters; import org.junit.After; import org.junit.Assert; @@ -45,6 +51,7 @@ import java.nio.file.Paths; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -166,6 +173,22 @@ public void testDeletionRemove() throws IllegalPathException, InterruptedExcepti @Test public void testWaitForResult() throws Exception { + // prepare pipe component + PipeRealtimeDataRegionExtractor extractor = new PipeRealtimeDataRegionHybridExtractor(); + PipeParameters parameters = + new PipeParameters( + new HashMap() { + { + put(PipeExtractorConstant.EXTRACTOR_INCLUSION_KEY, "data"); + } + }); + PipeTaskRuntimeConfiguration configuration = + new PipeTaskRuntimeConfiguration( + new PipeTaskExtractorRuntimeEnvironment("1", 1, Integer.parseInt(FAKE_DATA_REGION_IDS[4]), null)); + extractor.customize(parameters, configuration); + Assert.assertTrue(extractor.shouldExtractDeletion()); + + PipeInsertionDataNodeListener.getInstance().startListenAndAssign(FAKE_DATA_REGION_IDS[4], extractor); deletionResourceManager = DeletionResourceManager.getInstance(FAKE_DATA_REGION_IDS[4]); int rebootTimes = 0; MeasurementPath path = new MeasurementPath("root.vehicle.d2.s0"); From 8bb6cc9387562a6269571c7e71421f16a021be3b Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Mon, 23 Sep 2024 22:44:47 +0800 Subject: [PATCH 53/85] support deletion for consensus pipe --- .../consensus/pipe/consensuspipe/ConsensusPipeManager.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeManager.java b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeManager.java index eab94de768f0..d29068c94a7b 100644 --- a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeManager.java +++ b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeManager.java @@ -19,6 +19,7 @@ package org.apache.iotdb.consensus.pipe.consensuspipe; +import org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant; import org.apache.iotdb.commons.pipe.task.meta.PipeStatus; import org.apache.iotdb.consensus.common.Peer; import org.apache.iotdb.consensus.config.PipeConsensusConfig; @@ -38,11 +39,14 @@ import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.EXTRACTOR_CONSENSUS_GROUP_ID_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.EXTRACTOR_CONSENSUS_RECEIVER_DATANODE_ID_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.EXTRACTOR_CONSENSUS_SENDER_DATANODE_ID_KEY; +import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.EXTRACTOR_INCLUSION_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.EXTRACTOR_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant.EXTRACTOR_REALTIME_MODE_KEY; import static org.apache.iotdb.commons.pipe.config.constant.PipeProcessorConstant.PROCESSOR_KEY; public class ConsensusPipeManager { + // Extract data.insert and data.delete to support deletion. + private static final String CONSENSUS_EXTRACTOR_INCLUSION_VALUE = "data"; private final PipeConsensusConfig.Pipe config; private final ReplicateMode replicateMode; private final ConsensusPipeDispatcher dispatcher; @@ -61,6 +65,7 @@ public void createConsensusPipe(Peer senderPeer, Peer receiverPeer) throws Excep consensusPipeName.toString(), ImmutableMap.builder() .put(EXTRACTOR_KEY, config.getExtractorPluginName()) + .put(EXTRACTOR_INCLUSION_KEY, CONSENSUS_EXTRACTOR_INCLUSION_VALUE) .put( EXTRACTOR_CONSENSUS_GROUP_ID_KEY, consensusPipeName.getConsensusGroupId().toString()) From 1508215c1b6a444a64f002ae51e64604df8f8c0b Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Mon, 23 Sep 2024 22:48:09 +0800 Subject: [PATCH 54/85] support new deletion event and replace schemaEvent --- .../consensuspipe/ConsensusPipeManager.java | 3 +- .../deletion/PipeDeleteDataNodeEvent.java | 7 ++-- .../PipeRealtimeDataRegionExtractor.java | 6 ++++ ...PipeRealtimeDataRegionHybridExtractor.java | 6 ++-- .../PipeRealtimeDataRegionLogExtractor.java | 6 ++-- ...PipeRealtimeDataRegionTsFileExtractor.java | 6 ++-- .../pattern/CachedSchemaPatternMatcher.java | 4 +-- .../task/connection/PipeEventCollector.java | 34 ++----------------- .../pipe/consensus/DeletionRecoverTest.java | 1 - .../pipe/consensus/DeletionResourceTest.java | 20 ++++++----- 10 files changed, 37 insertions(+), 56 deletions(-) diff --git a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeManager.java b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeManager.java index d29068c94a7b..9a64c7b47dbe 100644 --- a/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeManager.java +++ b/iotdb-core/consensus/src/main/java/org/apache/iotdb/consensus/pipe/consensuspipe/ConsensusPipeManager.java @@ -19,7 +19,6 @@ package org.apache.iotdb.consensus.pipe.consensuspipe; -import org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant; import org.apache.iotdb.commons.pipe.task.meta.PipeStatus; import org.apache.iotdb.consensus.common.Peer; import org.apache.iotdb.consensus.config.PipeConsensusConfig; @@ -65,7 +64,7 @@ public void createConsensusPipe(Peer senderPeer, Peer receiverPeer) throws Excep consensusPipeName.toString(), ImmutableMap.builder() .put(EXTRACTOR_KEY, config.getExtractorPluginName()) - .put(EXTRACTOR_INCLUSION_KEY, CONSENSUS_EXTRACTOR_INCLUSION_VALUE) + .put(EXTRACTOR_INCLUSION_KEY, CONSENSUS_EXTRACTOR_INCLUSION_VALUE) .put( EXTRACTOR_CONSENSUS_GROUP_ID_KEY, consensusPipeName.getConsensusGroupId().toString()) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java index 45a7a5edc65a..048dd6473b1a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java @@ -32,12 +32,13 @@ import org.apache.tsfile.utils.ReadWriteIOUtils; import java.nio.ByteBuffer; +import java.util.Optional; public class PipeDeleteDataNodeEvent extends EnrichedEvent implements SerializableEvent { private DeleteDataNode deleteDataNode; private DeletionResource deletionResource; private boolean isGeneratedByPipe; - private final ProgressIndex progressIndex; + private ProgressIndex progressIndex; public PipeDeleteDataNodeEvent() { // Used for deserialization @@ -59,7 +60,8 @@ public PipeDeleteDataNodeEvent( super(pipeName, creationTime, pipeTaskMeta, pattern, Long.MIN_VALUE, Long.MAX_VALUE); this.isGeneratedByPipe = isGeneratedByPipe; this.deleteDataNode = deleteDataNode; - this.progressIndex = deleteDataNode.getProgressIndex(); + Optional.ofNullable(deleteDataNode) + .ifPresent(node -> this.progressIndex = deleteDataNode.getProgressIndex()); } public DeleteDataNode getDeleteDataNode() { @@ -136,6 +138,7 @@ public ByteBuffer serializeToByteBuffer() { public void deserializeFromByteBuffer(ByteBuffer buffer) { isGeneratedByPipe = ReadWriteIOUtils.readBool(buffer); deleteDataNode = (DeleteDataNode) PlanNodeType.deserialize(buffer); + progressIndex = deleteDataNode.getProgressIndex(); } public static PipeDeleteDataNodeEvent deserialize(ByteBuffer buffer) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java index d247e74450ce..4c2231b96911 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java @@ -28,6 +28,7 @@ import org.apache.iotdb.commons.pipe.pattern.PipePattern; import org.apache.iotdb.commons.pipe.task.connection.UnboundedBlockingPendingQueue; import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta; +import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.commons.utils.TimePartitionUtils; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; @@ -510,6 +511,11 @@ public String toString() { + '}'; } + @TestOnly + public void setShouldExtractDeletion(boolean shouldExtractDeletion) { + this.shouldExtractDeletion = shouldExtractDeletion; + } + //////////////////////////// APIs provided for metric framework //////////////////////////// public int getTabletInsertionEventCount() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java index ced920f378c6..bf2a5b6a966e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionHybridExtractor.java @@ -24,8 +24,8 @@ import org.apache.iotdb.commons.pipe.event.ProgressReportEvent; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; +import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; -import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; import org.apache.iotdb.db.pipe.extractor.dataregion.IoTDBDataRegionExtractor; @@ -57,7 +57,7 @@ protected void doExtract(final PipeRealtimeEvent event) { extractTsFileInsertion(event); } else if (eventToExtract instanceof PipeHeartbeatEvent) { extractHeartbeat(event); - } else if (eventToExtract instanceof PipeSchemaRegionWritePlanEvent) { + } else if (eventToExtract instanceof PipeDeleteDataNodeEvent) { extractDirectly(event); } else { throw new UnsupportedOperationException( @@ -260,7 +260,7 @@ public Event supply() { suppliedEvent = supplyTsFileInsertion(realtimeEvent); } else if (eventToSupply instanceof PipeHeartbeatEvent) { suppliedEvent = supplyHeartbeat(realtimeEvent); - } else if (eventToSupply instanceof PipeSchemaRegionWritePlanEvent + } else if (eventToSupply instanceof PipeDeleteDataNodeEvent || eventToSupply instanceof ProgressReportEvent) { suppliedEvent = supplyDirectly(realtimeEvent); } else { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionLogExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionLogExtractor.java index 4b300355c80e..1348bfdbaa9a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionLogExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionLogExtractor.java @@ -22,8 +22,8 @@ import org.apache.iotdb.commons.exception.pipe.PipeRuntimeNonCriticalException; import org.apache.iotdb.commons.pipe.event.ProgressReportEvent; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; +import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; -import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.epoch.TsFileEpoch; @@ -49,7 +49,7 @@ protected void doExtract(PipeRealtimeEvent event) { extractTsFileInsertion(event); } else if (eventToExtract instanceof PipeHeartbeatEvent) { extractHeartbeat(event); - } else if (eventToExtract instanceof PipeSchemaRegionWritePlanEvent) { + } else if (eventToExtract instanceof PipeDeleteDataNodeEvent) { extractDirectly(event); } else { throw new UnsupportedOperationException( @@ -131,7 +131,7 @@ public Event supply() { if (realtimeEvent.getEvent() instanceof PipeHeartbeatEvent) { suppliedEvent = supplyHeartbeat(realtimeEvent); - } else if (realtimeEvent.getEvent() instanceof PipeSchemaRegionWritePlanEvent + } else if (realtimeEvent.getEvent() instanceof PipeDeleteDataNodeEvent || realtimeEvent.getEvent() instanceof ProgressReportEvent) { suppliedEvent = supplyDirectly(realtimeEvent); } else if (realtimeEvent.increaseReferenceCount( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionTsFileExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionTsFileExtractor.java index 8072499b3daf..27198e349cb7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionTsFileExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionTsFileExtractor.java @@ -22,8 +22,8 @@ import org.apache.iotdb.commons.exception.pipe.PipeRuntimeNonCriticalException; import org.apache.iotdb.commons.pipe.event.ProgressReportEvent; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; +import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; -import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.epoch.TsFileEpoch; import org.apache.iotdb.pipe.api.event.Event; @@ -44,7 +44,7 @@ protected void doExtract(PipeRealtimeEvent event) { return; } - if (event.getEvent() instanceof PipeSchemaRegionWritePlanEvent) { + if (event.getEvent() instanceof PipeDeleteDataNodeEvent) { extractDirectly(event); return; } @@ -92,7 +92,7 @@ public Event supply() { if (realtimeEvent.getEvent() instanceof PipeHeartbeatEvent) { suppliedEvent = supplyHeartbeat(realtimeEvent); - } else if (realtimeEvent.getEvent() instanceof PipeSchemaRegionWritePlanEvent + } else if (realtimeEvent.getEvent() instanceof PipeDeleteDataNodeEvent || realtimeEvent.getEvent() instanceof ProgressReportEvent) { suppliedEvent = supplyDirectly(realtimeEvent); } else if (realtimeEvent.increaseReferenceCount( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/pattern/CachedSchemaPatternMatcher.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/pattern/CachedSchemaPatternMatcher.java index 546d8185c3a7..e6243d481d29 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/pattern/CachedSchemaPatternMatcher.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/pattern/CachedSchemaPatternMatcher.java @@ -21,8 +21,8 @@ import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.pattern.PipePattern; +import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; -import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionExtractor; @@ -109,7 +109,7 @@ public Set match(final PipeRealtimeEvent event) } // Deletion event will be assigned to extractors listened to it - if (event.getEvent() instanceof PipeSchemaRegionWritePlanEvent) { + if (event.getEvent() instanceof PipeDeleteDataNodeEvent) { return extractors.stream() .filter(PipeRealtimeDataRegionExtractor::shouldExtractDeletion) .collect(Collectors.toSet()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/connection/PipeEventCollector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/connection/PipeEventCollector.java index de0ede996231..dfd72a2b8227 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/connection/PipeEventCollector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/task/connection/PipeEventCollector.java @@ -21,17 +21,14 @@ import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.event.ProgressReportEvent; -import org.apache.iotdb.commons.pipe.pattern.IoTDBPipePattern; import org.apache.iotdb.commons.pipe.progress.PipeEventCommitManager; import org.apache.iotdb.commons.pipe.task.connection.UnboundedBlockingPendingQueue; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; +import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; -import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; -import org.apache.iotdb.db.pipe.extractor.schemaregion.IoTDBSchemaRegionExtractor; -import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeType; import org.apache.iotdb.pipe.api.collector.EventCollector; import org.apache.iotdb.pipe.api.event.Event; import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent; @@ -78,12 +75,8 @@ public void collect(final Event event) { parseAndCollectEvent((PipeRawTabletInsertionEvent) event); } else if (event instanceof PipeTsFileInsertionEvent) { parseAndCollectEvent((PipeTsFileInsertionEvent) event); - } else if (event instanceof PipeSchemaRegionWritePlanEvent - && ((PipeSchemaRegionWritePlanEvent) event).getPlanNode().getType() - == PlanNodeType.DELETE_DATA) { - // This is only for delete data node in data region since plan nodes in schema regions are - // already parsed in schema region extractor - parseAndCollectEvent((PipeSchemaRegionWritePlanEvent) event); + } else if (event instanceof PipeDeleteDataNodeEvent) { + collectEvent(event); } else if (!(event instanceof ProgressReportEvent)) { collectEvent(event); } @@ -141,27 +134,6 @@ private void collectParsedRawTableEvent(final PipeRawTabletInsertionEvent parsed } } - private void parseAndCollectEvent(final PipeSchemaRegionWritePlanEvent deleteDataEvent) { - // Only used by events containing delete data node, no need to bind progress index here since - // delete data event does not have progress index currently - IoTDBSchemaRegionExtractor.PATTERN_PARSE_VISITOR - .process(deleteDataEvent.getPlanNode(), (IoTDBPipePattern) deleteDataEvent.getPipePattern()) - .map( - planNode -> - new PipeSchemaRegionWritePlanEvent( - planNode, - deleteDataEvent.getPipeName(), - deleteDataEvent.getCreationTime(), - deleteDataEvent.getPipeTaskMeta(), - deleteDataEvent.getPipePattern(), - deleteDataEvent.isGeneratedByPipe())) - .ifPresent( - event -> { - hasNoGeneratedEvent = false; - collectEvent(event); - }); - } - private void collectEvent(final Event event) { if (event instanceof EnrichedEvent) { if (!((EnrichedEvent) event).increaseReferenceCount(PipeEventCollector.class.getName())) { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java index d79da421515f..df2253a12d99 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java @@ -29,7 +29,6 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; -import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java index 567a9bdad95c..f7ef7a21896e 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java @@ -176,19 +176,21 @@ public void testWaitForResult() throws Exception { // prepare pipe component PipeRealtimeDataRegionExtractor extractor = new PipeRealtimeDataRegionHybridExtractor(); PipeParameters parameters = - new PipeParameters( - new HashMap() { - { - put(PipeExtractorConstant.EXTRACTOR_INCLUSION_KEY, "data"); - } - }); + new PipeParameters( + new HashMap() { + { + put(PipeExtractorConstant.EXTRACTOR_INCLUSION_KEY, "data"); + } + }); PipeTaskRuntimeConfiguration configuration = - new PipeTaskRuntimeConfiguration( - new PipeTaskExtractorRuntimeEnvironment("1", 1, Integer.parseInt(FAKE_DATA_REGION_IDS[4]), null)); + new PipeTaskRuntimeConfiguration( + new PipeTaskExtractorRuntimeEnvironment( + "1", 1, Integer.parseInt(FAKE_DATA_REGION_IDS[4]), null)); extractor.customize(parameters, configuration); Assert.assertTrue(extractor.shouldExtractDeletion()); - PipeInsertionDataNodeListener.getInstance().startListenAndAssign(FAKE_DATA_REGION_IDS[4], extractor); + PipeInsertionDataNodeListener.getInstance() + .startListenAndAssign(FAKE_DATA_REGION_IDS[4], extractor); deletionResourceManager = DeletionResourceManager.getInstance(FAKE_DATA_REGION_IDS[4]); int rebootTimes = 0; MeasurementPath path = new MeasurementPath("root.vehicle.d2.s0"); From 872f13509add4b7f723f12b5dc40773b5bf00c82 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Mon, 23 Sep 2024 23:10:07 +0800 Subject: [PATCH 55/85] spotless --- .../task/connection/PipeEventCollector.java | 3 -- .../deletion/PipeDeleteDataNodeEvent.java | 4 +-- ...lDataRegionTsFileAndDeletionExtractor.java | 35 +++++++++---------- .../PipeRealtimeDataRegionExtractor.java | 4 --- .../assigner/PipeDataRegionAssigner.java | 2 +- .../matcher/CachedSchemaPatternMatcher.java | 3 +- .../index/impl/SimpleProgressIndex.java | 2 +- 7 files changed, 21 insertions(+), 32 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollector.java index a58def76b5a6..b546e8d4ab9b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/task/connection/PipeEventCollector.java @@ -21,11 +21,8 @@ import org.apache.iotdb.commons.pipe.agent.task.connection.UnboundedBlockingPendingQueue; import org.apache.iotdb.commons.pipe.agent.task.progress.PipeEventCommitManager; -import org.apache.iotdb.commons.pipe.datastructure.pattern.IoTDBPipePattern; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.event.ProgressReportEvent; -import org.apache.iotdb.commons.pipe.progress.PipeEventCommitManager; -import org.apache.iotdb.commons.pipe.task.connection.UnboundedBlockingPendingQueue; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java index 048dd6473b1a..49a59871cd35 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java @@ -21,10 +21,10 @@ import org.apache.iotdb.commons.consensus.index.ProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; +import org.apache.iotdb.commons.pipe.agent.task.meta.PipeTaskMeta; +import org.apache.iotdb.commons.pipe.datastructure.pattern.PipePattern; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.event.SerializableEvent; -import org.apache.iotdb.commons.pipe.pattern.PipePattern; -import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeType; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java index 336b30071f69..6ab16a5648b2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java @@ -29,12 +29,10 @@ import org.apache.iotdb.commons.pipe.config.constant.SystemConstant; import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskExtractorRuntimeEnvironment; import org.apache.iotdb.commons.pipe.datastructure.PersistentResource; -import org.apache.iotdb.commons.pipe.pattern.PipePattern; -import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta; +import org.apache.iotdb.commons.pipe.datastructure.pattern.PipePattern; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; -import org.apache.iotdb.commons.pipe.datastructure.pattern.PipePattern; import org.apache.iotdb.db.pipe.event.common.terminate.PipeTerminateEvent; import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent; import org.apache.iotdb.db.pipe.extractor.dataregion.DataRegionListeningFilter; @@ -496,18 +494,18 @@ && mayTsFileResourceOverlappedWithPattern(resource)) @Override public synchronized void start() { - if (!shouldExtractInsertion) { - hasBeenStarted = true; - return; - } - if (!StorageEngine.getInstance().isReadyForNonReadWriteFunctions()) { - LOGGER.info( - "Pipe {}@{}: failed to start to extract historical TsFile, storage engine is not ready. Will retry later.", - pipeName, - dataRegionId); - return; - } + if (!shouldExtractInsertion) { hasBeenStarted = true; + return; + } + if (!StorageEngine.getInstance().isReadyForNonReadWriteFunctions()) { + LOGGER.info( + "Pipe {}@{}: failed to start to extract historical TsFile, storage engine is not ready. Will retry later.", + pipeName, + dataRegionId); + return; + } + hasBeenStarted = true; final DataRegion dataRegion = StorageEngine.getInstance().getDataRegion(new DataRegionId(dataRegionId)); @@ -694,9 +692,9 @@ private Event supplyTerminateEvent() { @Override public synchronized Event supply() { - if (!hasBeenStarted && StorageEngine.getInstance().isReadyForNonReadWriteFunctions()) { - start(); - } + if (!hasBeenStarted && StorageEngine.getInstance().isReadyForNonReadWriteFunctions()) { + start(); + } if (Objects.isNull(pendingQueue)) { return null; @@ -716,8 +714,7 @@ public synchronized Event supply() { public synchronized boolean hasConsumedAll() { // If the pendingQueues are null when the function is called, it implies that the extractor only // extracts deletion thus the historical event has nothing to consume. - return hasBeenStarted - && (Objects.isNull(pendingQueue)) + return hasBeenStarted && (Objects.isNull(pendingQueue)) || pendingQueue.isEmpty() && (!shouldTerminatePipeOnAllHistoricalEventsConsumed || isTerminateSignalSent); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java index f4af5f47fa9b..cb1df33b0145 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/PipeRealtimeDataRegionExtractor.java @@ -28,10 +28,6 @@ import org.apache.iotdb.commons.pipe.datastructure.pattern.PipePattern; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.event.ProgressReportEvent; -import org.apache.iotdb.commons.pipe.pattern.PipePattern; -import org.apache.iotdb.commons.pipe.task.connection.UnboundedBlockingPendingQueue; -import org.apache.iotdb.commons.pipe.task.meta.PipeTaskMeta; -import org.apache.iotdb.commons.utils.TestOnly; import org.apache.iotdb.commons.utils.TimePartitionUtils; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java index 15c6b43512e2..e0118d482917 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java @@ -42,8 +42,8 @@ import org.slf4j.LoggerFactory; import java.io.Closeable; -import java.util.concurrent.atomic.AtomicReference; import java.util.Objects; +import java.util.concurrent.atomic.AtomicReference; public class PipeDataRegionAssigner implements Closeable { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/matcher/CachedSchemaPatternMatcher.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/matcher/CachedSchemaPatternMatcher.java index 5d59c729ef4c..c876cb58487b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/matcher/CachedSchemaPatternMatcher.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/matcher/CachedSchemaPatternMatcher.java @@ -20,9 +20,8 @@ package org.apache.iotdb.db.pipe.extractor.dataregion.realtime.matcher; import org.apache.iotdb.commons.pipe.config.PipeConfig; -import org.apache.iotdb.commons.pipe.pattern.PipePattern; -import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.apache.iotdb.commons.pipe.datastructure.pattern.PipePattern; +import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionExtractor; 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 95e1a380b186..d852f0cc0014 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 @@ -163,7 +163,7 @@ public ProgressIndex updateToMinimumEqualOrIsAfterProgressIndex(ProgressIndex pr } if (thisSimpleProgressIndex.memTableFlushOrderId < thatSimpleProgressIndex.memTableFlushOrderId) { - return progressIndex.deepCopy(); + return progressIndex; } // thisSimpleProgressIndex.memtableFlushOrderId == // thatSimpleProgressIndex.memtableFlushOrderId From 587c7df2a15c715d0e216d44fc33450163269a6c Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Mon, 23 Sep 2024 23:16:17 +0800 Subject: [PATCH 56/85] rename pipeConsensus to iotV2 --- .../org/apache/iotdb/db/conf/IoTDBConfig.java | 27 +++++++++---------- .../apache/iotdb/db/conf/IoTDBDescriptor.java | 4 +-- .../deletion/DeletionResourceManager.java | 2 +- .../pipe/consensus/DeletionRecoverTest.java | 2 +- .../pipe/consensus/DeletionResourceTest.java | 2 +- .../conf/iotdb-system.properties.template | 8 +++--- 6 files changed, 22 insertions(+), 23 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java index b09887ca9042..ce19b9e242e7 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java @@ -1140,6 +1140,14 @@ public class IoTDBConfig { private int iotConsensusV2PipelineSize = 5; private String iotConsensusV2Mode = ConsensusFactory.IOT_CONSENSUS_V2_BATCH_MODE; private String[] iotConsensusV2ReceiverFileDirs = new String[0]; + private String iotConsensusV2DeletionFileDir = + systemDir + + File.separator + + PIPE_FOLDER_NAME + + File.separator + + CONSENSUS_FOLDER_NAME + + File.separator + + DELETION_FOLDER_NAME; /** Load related */ private double maxAllocateMemoryRatioForLoad = 0.8; @@ -1190,15 +1198,6 @@ public class IoTDBConfig { /** initialized as empty, updated based on the latest `systemDir` during querying */ private String[] pipeReceiverFileDirs = new String[0]; - private String pipeConsensusDeletionFileDir = - systemDir - + File.separator - + PIPE_FOLDER_NAME - + File.separator - + CONSENSUS_FOLDER_NAME - + File.separator - + DELETION_FOLDER_NAME; - /** Resource control */ private boolean quotaEnable = false; @@ -1353,7 +1352,7 @@ private void formulateFolders() { systemDir = addDataHomeDir(systemDir); schemaDir = addDataHomeDir(schemaDir); consensusDir = addDataHomeDir(consensusDir); - pipeConsensusDeletionFileDir = addDataHomeDir(pipeConsensusDeletionFileDir); + iotConsensusV2DeletionFileDir = addDataHomeDir(iotConsensusV2DeletionFileDir); dataRegionConsensusDir = addDataHomeDir(dataRegionConsensusDir); ratisDataRegionSnapshotDir = addDataHomeDir(ratisDataRegionSnapshotDir); schemaRegionConsensusDir = addDataHomeDir(schemaRegionConsensusDir); @@ -1553,12 +1552,12 @@ public void setSystemDir(String systemDir) { this.systemDir = systemDir; } - public String getPipeConsensusDeletionFileDir() { - return pipeConsensusDeletionFileDir; + public String getIotConsensusV2DeletionFileDir() { + return iotConsensusV2DeletionFileDir; } - public void setPipeConsensusDeletionFileDir(String pipeConsensusDeletionFileDir) { - this.pipeConsensusDeletionFileDir = pipeConsensusDeletionFileDir; + public void setIotConsensusV2DeletionFileDir(String iotConsensusV2DeletionFileDir) { + this.iotConsensusV2DeletionFileDir = iotConsensusV2DeletionFileDir; } public String[] getLoadTsFileDirs() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java index 49565bb443f5..9d86d148900e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java @@ -2486,9 +2486,9 @@ private void loadPipeProps(Properties properties) { .filter(dir -> !dir.isEmpty()) .toArray(String[]::new)); - conf.setPipeConsensusDeletionFileDir( + conf.setIotConsensusV2DeletionFileDir( properties.getProperty( - "pipe_consensus_deletion_file_dir", conf.getPipeConsensusDeletionFileDir())); + "iot_consensus_v2_deletion_file_dir", conf.getIotConsensusV2DeletionFileDir())); } private void loadCQProps(Properties properties) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java index d57c1554da5d..3addac799c6b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java @@ -77,7 +77,7 @@ private DeletionResourceManager(String dataRegionId) throws IOException { this.dataRegionId = dataRegionId; this.storageDir = new File( - IoTDBDescriptor.getInstance().getConfig().getPipeConsensusDeletionFileDir() + IoTDBDescriptor.getInstance().getConfig().getIotConsensusV2DeletionFileDir() + File.separator + dataRegionId); this.deletionBuffer = new PageCacheDeletionBuffer(dataRegionId, storageDir.getAbsolutePath()); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java index df2253a12d99..4e7805761ef7 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java @@ -41,7 +41,7 @@ public class DeletionRecoverTest { private static final int THIS_DATANODE_ID = IoTDBDescriptor.getInstance().getConfig().getDataNodeId(); private static final String DELETION_BASE_DIR = - IoTDBDescriptor.getInstance().getConfig().getPipeConsensusDeletionFileDir(); + IoTDBDescriptor.getInstance().getConfig().getIotConsensusV2DeletionFileDir(); private static final String BASE_PATH = DELETION_BASE_DIR + File.separator + FAKE_DATE_REGION_ID; private final int deletionCount = 10; private DeletionResourceManager deletionResourceManager; diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java index f7ef7a21896e..6e6df7a2d0ab 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java @@ -59,7 +59,7 @@ public class DeletionResourceTest { private static final String[] FAKE_DATA_REGION_IDS = {"2", "3", "4", "5", "6"}; private static final String DELETION_BASE_DIR = - IoTDBDescriptor.getInstance().getConfig().getPipeConsensusDeletionFileDir(); + IoTDBDescriptor.getInstance().getConfig().getIotConsensusV2DeletionFileDir(); private static final int THIS_DATANODE_ID = IoTDBDescriptor.getInstance().getConfig().getDataNodeId(); private DeletionResourceManager deletionResourceManager; diff --git a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template index e97b35347809..56b753c7a59f 100644 --- a/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template +++ b/iotdb-core/node-commons/src/assembly/resources/conf/iotdb-system.properties.template @@ -333,18 +333,18 @@ dn_pipe_receiver_file_dirs=data/datanode/system/pipe/receiver # If its prefix is "/", then the path is absolute. Otherwise, it is relative. iot_consensus_v2_receiver_file_dirs=data/datanode/system/pipe/consensus/receiver -# pipe_consensus_deletion_file_dir +# iot_consensus_v2_deletion_file_dir # If this property is unset, system will save the data in the default relative path directory under the IoTDB folder(i.e., %IOTDB_HOME%/${dn_system_dir}/pipe/consensus/deletion). # If it is absolute, system will save the data in the exact location it points to. # If it is relative, system will save the data in the relative path directory it indicates under the IoTDB folder. -# Note: If pipe_consensus_deletion_file_dir is assigned an empty string(i.e.,zero-size), it will be handled as a relative path. +# Note: If iot_consensus_v2_deletion_file_dir is assigned an empty string(i.e.,zero-size), it will be handled as a relative path. # effectiveMode: restart # For windows platform # If its prefix is a drive specifier followed by "\\", or if its prefix is "\\\\", then the path is absolute. Otherwise, it is relative. -# pipe_consensus_deletion_file_dir=data\\datanode\\system\\pipe\\consensus\\deletion +# iot_consensus_v2_deletion_file_dir=data\\datanode\\system\\pipe\\consensus\\deletion # For Linux platform # If its prefix is "/", then the path is absolute. Otherwise, it is relative. -pipe_consensus_deletion_file_dir=data/datanode/system/pipe/consensus/deletion +iot_consensus_v2_deletion_file_dir=data/datanode/system/pipe/consensus/deletion #################### ### Metric Configuration From 85fef64f27a0709acd4ff2bca6d88d03c088eaad Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Wed, 25 Sep 2024 11:37:24 +0800 Subject: [PATCH 57/85] fix: reference management for deletionResource --- .../consensus/deletion/DeletionResource.java | 55 ++++++++--------- .../deletion/DeletionResourceManager.java | 60 +++++++++---------- .../deletion/PipeDeleteDataNodeEvent.java | 6 +- .../dataregion/DataRegionListeningFilter.java | 1 - ...lDataRegionTsFileAndDeletionExtractor.java | 41 +++++++++---- .../assigner/PipeDataRegionAssigner.java | 17 ++---- .../PipeInsertionDataNodeListener.java | 11 ++-- .../plan/node/write/DeleteDataNode.java | 20 ++++++- .../storageengine/dataregion/DataRegion.java | 2 + 9 files changed, 119 insertions(+), 94 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java index 4ddd67a15937..98c7f8b762b6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java @@ -22,6 +22,8 @@ import org.apache.iotdb.commons.consensus.index.ProgressIndex; import org.apache.iotdb.commons.pipe.datastructure.PersistentResource; import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeType; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -29,6 +31,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Objects; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; /** @@ -39,37 +42,29 @@ public class DeletionResource implements PersistentResource { private static final Logger LOGGER = LoggerFactory.getLogger(DeletionResource.class); private final Consumer removeHook; - private PipeDeleteDataNodeEvent correspondingPipeTaskEvent; + private final AtomicInteger pipeTaskReferenceCount = new AtomicInteger(0); + private final DeleteDataNode deleteDataNode; private volatile Status currentStatus; // it's safe to use volatile here to make this reference thread-safe. @SuppressWarnings("squid:S3077") private volatile Exception cause; - // For first register in DataRegion - public DeletionResource(Consumer removeHook) { + public DeletionResource(DeleteDataNode deleteDataNode, Consumer removeHook) { + this.deleteDataNode = deleteDataNode; this.removeHook = removeHook; this.currentStatus = Status.RUNNING; } - // For deserialize - public DeletionResource(PipeDeleteDataNodeEvent event, Consumer removeHook) { - this.correspondingPipeTaskEvent = event; - this.removeHook = removeHook; - this.currentStatus = Status.RUNNING; - } - - public void setCorrespondingPipeTaskEvent(PipeDeleteDataNodeEvent correspondingPipeTaskEvent) { - this.correspondingPipeTaskEvent = correspondingPipeTaskEvent; + public void increaseReference() { + pipeTaskReferenceCount.incrementAndGet(); } - /** - * This method is invoked when DeletionResource is deleted by DeleteResourceManager. In this - * method, we release the reference of deletionEvent to resolve circular references between - * deletionResource and deletionEvent so that GC can reclaim them. - */ - public void releaseSelf() { - correspondingPipeTaskEvent = null; + public synchronized void decreaseReference() { + if (pipeTaskReferenceCount.get() == 1) { + removeSelf(); + } + pipeTaskReferenceCount.decrementAndGet(); } public void removeSelf() { @@ -77,7 +72,7 @@ public void removeSelf() { } public long getReferenceCount() { - return correspondingPipeTaskEvent.getReferenceCount(); + return pipeTaskReferenceCount.get(); } public synchronized void onPersistFailed(Exception e) { @@ -92,7 +87,7 @@ public synchronized void onPersistSucceed() { } /** - * @return true if this object has been successfully persisted, false if persist failed. + * @return true, if this object has been successfully persisted, false if persist failed. */ public synchronized Status waitForResult() { while (currentStatus == Status.RUNNING) { @@ -110,7 +105,7 @@ public synchronized Status waitForResult() { @Override public ProgressIndex getProgressIndex() { - return correspondingPipeTaskEvent.getDeleteDataNode().getProgressIndex(); + return deleteDataNode.getProgressIndex(); } @Override @@ -123,24 +118,24 @@ public long getFileEndTime() { return 0; } - public PipeDeleteDataNodeEvent getCorrespondingPipeTaskEvent() { - return correspondingPipeTaskEvent; + public DeleteDataNode getDeleteDataNode() { + return deleteDataNode; } public ByteBuffer serialize() { - return correspondingPipeTaskEvent.serializeToByteBuffer(); + return deleteDataNode.serializeToByteBuffer(); } public static DeletionResource deserialize( final ByteBuffer buffer, final Consumer removeHook) throws IOException { - PipeDeleteDataNodeEvent event = PipeDeleteDataNodeEvent.deserialize(buffer); - return new DeletionResource(event, removeHook); + DeleteDataNode node = (DeleteDataNode) PlanNodeType.deserialize(buffer); + return new DeletionResource(node, removeHook); } @Override public String toString() { return String.format( - "DeletionResource[%s]{referenceCount=%s}", correspondingPipeTaskEvent, getReferenceCount()); + "DeletionResource[%s]{referenceCount=%s}", deleteDataNode, getReferenceCount()); } @Override @@ -152,12 +147,12 @@ public boolean equals(Object o) { return false; } final DeletionResource otherEvent = (DeletionResource) o; - return Objects.equals(correspondingPipeTaskEvent, otherEvent.correspondingPipeTaskEvent); + return Objects.equals(deleteDataNode, otherEvent.deleteDataNode); } @Override public int hashCode() { - return Objects.hash(correspondingPipeTaskEvent); + return Objects.hash(deleteDataNode); } public Exception getCause() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java index 3addac799c6b..b9a34f69e374 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java @@ -30,7 +30,7 @@ import org.apache.iotdb.db.pipe.consensus.deletion.persist.DeletionBuffer; import org.apache.iotdb.db.pipe.consensus.deletion.persist.PageCacheDeletionBuffer; import org.apache.iotdb.db.pipe.consensus.deletion.recover.DeletionReader; -import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; import com.google.common.collect.ImmutableList; import org.slf4j.Logger; @@ -43,9 +43,8 @@ import java.nio.file.Paths; import java.util.List; import java.util.Map; -import java.util.Objects; +import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; @@ -66,9 +65,8 @@ public class DeletionResourceManager implements AutoCloseable { private final String dataRegionId; private final DeletionBuffer deletionBuffer; private final File storageDir; - private final Map eventHash2DeletionResources = + private final Map deleteNode2ResourcesMap = new ConcurrentHashMap<>(); - private final List deletionResources = new CopyOnWriteArrayList<>(); private final Lock recoverLock = new ReentrantLock(); private final Condition recoveryReadyCondition = recoverLock.newCondition(); private volatile boolean hasCompletedRecovery = false; @@ -107,7 +105,12 @@ private void initAndRecover() throws IOException { for (Path path : deletionPaths) { try (DeletionReader deletionReader = new DeletionReader(path.toFile(), this::removeDeletionResource)) { - deletionResources.addAll(deletionReader.readAllDeletions()); + deletionReader + .readAllDeletions() + .forEach( + deletion -> + deleteNode2ResourcesMap.computeIfAbsent( + deletion.getDeleteDataNode(), key -> deletion)); } catch (IOException e) { LOGGER.warn( "Detect file corrupted when recover DAL-{}, discard all subsequent DALs...", @@ -126,36 +129,24 @@ private void initAndRecover() throws IOException { @Override public void close() { LOGGER.info("Closing deletion resource manager for {}...", dataRegionId); - this.deletionResources.clear(); + this.deleteNode2ResourcesMap.clear(); this.deletionBuffer.close(); LOGGER.info("Deletion resource manager for {} has been successfully closed!", dataRegionId); } - /** - * In this method, we only new an instance and return it to DataRegion and not persist - * deletionResource. Because currently deletionResource can not bind corresponding pipe task's - * deletionEvent. - */ - public DeletionResource registerDeletionResource(PipeDeleteDataNodeEvent originEvent) { + public DeletionResource registerDeletionResource(DeleteDataNode deleteDataNode) { DeletionResource deletionResource = - eventHash2DeletionResources.computeIfAbsent( - Objects.hash(originEvent, dataRegionId), - key -> new DeletionResource(this::removeDeletionResource)); - this.deletionResources.add(deletionResource); + deleteNode2ResourcesMap.computeIfAbsent( + deleteDataNode, + key -> new DeletionResource(deleteDataNode, this::removeDeletionResource)); + // register a persist task for current deletionResource + deletionBuffer.registerDeletionResource(deletionResource); return deletionResource; } - /** This method will bind event for deletionResource and persist it. */ - public DeletionResource enrichDeletionResourceAndPersist( - PipeDeleteDataNodeEvent originEvent, PipeDeleteDataNodeEvent copiedEvent) { - int key = Objects.hash(originEvent, dataRegionId); - DeletionResource deletionResource = eventHash2DeletionResources.get(key); - // Bind real deletion event - deletionResource.setCorrespondingPipeTaskEvent(copiedEvent); - // Register a persist task for current deletionResource - deletionBuffer.registerDeletionResource(deletionResource); - // Now, we can safely remove this entry from map. Since this entry will not be used anymore. - eventHash2DeletionResources.remove(key); + public DeletionResource increaseResourceReferenceAndGet(DeleteDataNode deleteDataNode) { + DeletionResource deletionResource = deleteNode2ResourcesMap.get(deleteDataNode); + Optional.ofNullable(deletionResource).ifPresent(DeletionResource::increaseReference); return deletionResource; } @@ -165,14 +156,14 @@ public List getAllDeletionResources() { if (!hasCompletedRecovery) { recoveryReadyCondition.await(); } - return deletionResources.stream().collect(ImmutableList.toImmutableList()); + return deleteNode2ResourcesMap.values().stream().collect(ImmutableList.toImmutableList()); } catch (InterruptedException e) { Thread.currentThread().interrupt(); LOGGER.warn( "DeletionManager-{}: current waiting is interrupted. May because current application is down. ", dataRegionId, e); - return deletionResources.stream().collect(ImmutableList.toImmutableList()); + return deleteNode2ResourcesMap.values().stream().collect(ImmutableList.toImmutableList()); } finally { recoverLock.unlock(); } @@ -184,7 +175,7 @@ public List getAllDeletionResources() { */ private synchronized void removeDeletionResource(DeletionResource deletionResource) { // Clean memory - deletionResources.remove(deletionResource); + deleteNode2ResourcesMap.remove(deletionResource.getDeleteDataNode()); // Clean disk ProgressIndex currentProgressIndex = ProgressIndexDataNodeManager.extractLocalSimpleProgressIndex( @@ -324,7 +315,12 @@ public void recoverForTest() { for (Path path : deletionPaths) { try (DeletionReader deletionReader = new DeletionReader(path.toFile(), this::removeDeletionResource)) { - deletionResources.addAll(deletionReader.readAllDeletions()); + deletionReader + .readAllDeletions() + .forEach( + deletion -> + deleteNode2ResourcesMap.computeIfAbsent( + deletion.getDeleteDataNode(), key -> deletion)); } } } catch (IOException e) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java index 49a59871cd35..6110d511a569 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java @@ -84,11 +84,7 @@ public boolean internallyIncreaseResourceReferenceCount(String holderMessage) { @Override public boolean internallyDecreaseResourceReferenceCount(String holderMessage) { if (deletionResource != null) { - // Trigger hook function. - deletionResource.removeSelf(); - // Resolve circular reference to let GC reclaim them all. - deletionResource.releaseSelf(); - deletionResource = null; + deletionResource.decreaseReference(); } return true; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/DataRegionListeningFilter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/DataRegionListeningFilter.java index a54f51560484..7f43701530f6 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/DataRegionListeningFilter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/DataRegionListeningFilter.java @@ -68,7 +68,6 @@ public static boolean shouldDataRegionBeListened(PipeParameters parameters) public static Pair parseInsertionDeletionListeningOptionPair( PipeParameters parameters) throws IllegalPathException, IllegalArgumentException { final Set listeningOptions = new HashSet<>(); - final Set inclusionOptions = parseOptions( parameters.getStringOrDefault( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java index 6ab16a5648b2..a279b1b7f00e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java @@ -518,16 +518,16 @@ public synchronized void start() { "Pipe: start to extract historical TsFile and Deletion(if uses pipeConsensus)"); try { List resourceList = new ArrayList<>(); - // Extract deletions - Optional.ofNullable(DeletionResourceManager.getInstance(String.valueOf(dataRegionId))) - .ifPresent(mgr -> extractDeletions(mgr, resourceList)); - // Flush TsFiles final long startHistoricalExtractionTime = System.currentTimeMillis(); flushTsFilesForExtraction(dataRegion, startHistoricalExtractionTime); // Extract TsFiles extractTsFiles(dataRegion, startHistoricalExtractionTime, resourceList); + // Extract deletions + Optional.ofNullable(DeletionResourceManager.getInstance(String.valueOf(dataRegionId))) + .ifPresent(mgr -> extractDeletions(mgr, resourceList)); + // Sort tsFileResource and deletionResource resourceList.sort( (o1, o2) -> @@ -648,7 +648,7 @@ private Event supplyTsFileEvent(TsFileResource resource) { PipeHistoricalDataRegionTsFileAndDeletionExtractor.class.getName()); if (!isReferenceCountIncreased) { LOGGER.warn( - "Pipe {}@{}: failed to increase reference count for historical event {}, will discard it", + "Pipe {}@{}: failed to increase reference count for historical tsfile event {}, will discard it", pipeName, dataRegionId, event); @@ -668,10 +668,32 @@ private Event supplyTsFileEvent(TsFileResource resource) { } private Event supplyDeletionEvent(final DeletionResource deletionResource) { - PipeDeleteDataNodeEvent event = deletionResource.getCorrespondingPipeTaskEvent(); - event.increaseReferenceCount( - PipeHistoricalDataRegionTsFileAndDeletionExtractor.class.getName()); - return event; + final PipeDeleteDataNodeEvent event = + new PipeDeleteDataNodeEvent( + deletionResource.getDeleteDataNode(), + pipeName, + creationTime, + pipeTaskMeta, + pipePattern, + false); + if (sloppyPattern || isDbNameCoveredByPattern) { + event.skipParsingPattern(); + } + if (sloppyTimeRange) { + event.skipParsingTime(); + } + + final boolean isReferenceCountIncreased = + event.increaseReferenceCount( + PipeHistoricalDataRegionTsFileAndDeletionExtractor.class.getName()); + if (!isReferenceCountIncreased) { + LOGGER.warn( + "Pipe {}@{}: failed to increase reference count for historical deletion event {}, will discard it", + pipeName, + dataRegionId, + event); + } + return isReferenceCountIncreased ? event : null; } private Event supplyTerminateEvent() { @@ -691,7 +713,6 @@ private Event supplyTerminateEvent() { @Override public synchronized Event supply() { - if (!hasBeenStarted && StorageEngine.getInstance().isReadyForNonReadWriteFunctions()) { start(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java index e0118d482917..25dc449c2816 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java @@ -24,8 +24,6 @@ import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.event.ProgressReportEvent; -import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; -import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource.Status; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; @@ -141,16 +139,13 @@ public void assignToExtractor( if (copiedEvent.getEvent() instanceof PipeDeleteDataNodeEvent) { DeletionResourceManager mgr = DeletionResourceManager.getInstance(extractor.getDataRegionId()); + // increase deletion resource's reference if (Objects.nonNull(mgr)) { - DeletionResource deletionResource = - mgr.enrichDeletionResourceAndPersist( - (PipeDeleteDataNodeEvent) event.getEvent(), - (PipeDeleteDataNodeEvent) copiedEvent.getEvent()); - // if persist failed, skip sending this event to keep consistency with the - // behavior of storage engine. - if (deletionResource.waitForResult() == Status.FAILURE) { - return; - } + PipeDeleteDataNodeEvent deleteDataNodeEvent = + (PipeDeleteDataNodeEvent) copiedEvent.getEvent(); + deleteDataNodeEvent.setDeletionResource( + mgr.increaseResourceReferenceAndGet( + ((PipeDeleteDataNodeEvent) event.getEvent()).getDeleteDataNode())); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java index 28db982ce554..2bbeaa9a57c3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java @@ -22,8 +22,8 @@ import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource.Status; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; -import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEventFactory; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionExtractor; @@ -157,10 +157,13 @@ public DeletionResource listenToDeleteData(DeleteDataNode node, String regionId) DeletionResourceManager mgr = DeletionResourceManager.getInstance(regionId); DeletionResource deletionResource = null; if (Objects.nonNull(mgr)) { - deletionResource = - mgr.registerDeletionResource((PipeDeleteDataNodeEvent) realtimeEvent.getEvent()); + deletionResource = mgr.registerDeletionResource(node); + // if persist failed, skip sending/publishing this event to keep consistency with the + // behavior of storage engine. + if (deletionResource.waitForResult() == Status.FAILURE) { + return deletionResource; + } } - // register first, then publish. assigner.publishToAssign(realtimeEvent); return deletionResource; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java index 3ae48ab95caa..9955f837b2e8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java @@ -21,6 +21,7 @@ import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet; import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.commons.consensus.index.ProgressIndexType; import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.partition.DataPartition; import org.apache.iotdb.commons.path.MeasurementPath; @@ -74,6 +75,19 @@ public DeleteDataNode( this.deleteEndTime = deleteEndTime; } + public DeleteDataNode( + PlanNodeId id, + List pathList, + long deleteStartTime, + long deleteEndTime, + ProgressIndex progressIndex) { + super(id); + this.pathList = pathList; + this.deleteStartTime = deleteStartTime; + this.deleteEndTime = deleteEndTime; + this.progressIndex = progressIndex; + } + public DeleteDataNode( PlanNodeId id, List pathList, @@ -207,6 +221,7 @@ protected void serializeAttributes(ByteBuffer byteBuffer) { } ReadWriteIOUtils.write(deleteStartTime, byteBuffer); ReadWriteIOUtils.write(deleteEndTime, byteBuffer); + progressIndex.serialize(byteBuffer); } @Override @@ -218,6 +233,7 @@ protected void serializeAttributes(DataOutputStream stream) throws IOException { } ReadWriteIOUtils.write(deleteStartTime, stream); ReadWriteIOUtils.write(deleteEndTime, stream); + progressIndex.serialize(stream); } public static DeleteDataNode deserialize(ByteBuffer byteBuffer) { @@ -228,12 +244,14 @@ public static DeleteDataNode deserialize(ByteBuffer byteBuffer) { } long deleteStartTime = ReadWriteIOUtils.readLong(byteBuffer); long deleteEndTime = ReadWriteIOUtils.readLong(byteBuffer); + ProgressIndex deserializedIndex = ProgressIndexType.deserializeFrom(byteBuffer); PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer); // DeleteDataNode has no child int ignoredChildrenSize = ReadWriteIOUtils.readInt(byteBuffer); - return new DeleteDataNode(planNodeId, pathList, deleteStartTime, deleteEndTime); + return new DeleteDataNode( + planNodeId, pathList, deleteStartTime, deleteEndTime, deserializedIndex); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java index 46c0983e158f..f91d285d6989 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java @@ -2325,6 +2325,7 @@ public void deleteByDevice(MeasurementPath pattern, DeleteDataNode node) throws // capture deleteDataNode and wait it to be persisted to DAL. DeletionResource deletionResource = PipeInsertionDataNodeListener.getInstance().listenToDeleteData(node, dataRegionId); + // just get result. We have already waited for result in `listenToDeleteData` if (deletionResource != null && deletionResource.waitForResult() == Status.FAILURE) { throw deletionResource.getCause(); } @@ -2374,6 +2375,7 @@ public void deleteDataDirectly(MeasurementPath pathToDelete, DeleteDataNode node // capture deleteDataNode and wait it to be persisted to DAL. DeletionResource deletionResource = PipeInsertionDataNodeListener.getInstance().listenToDeleteData(node, dataRegionId); + // just get result. We have already waited for result in `listenToDeleteData` if (deletionResource != null && deletionResource.waitForResult() == Status.FAILURE) { throw deletionResource.getCause(); } From 7062c3724e0a5eeabe98a3ca022d1c9a88c5364d Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Wed, 25 Sep 2024 20:45:01 +0800 Subject: [PATCH 58/85] fix: reference management for deletionResource --- .../org/apache/iotdb/db/conf/IoTDBConfig.java | 1 + .../request/PipeConsensusDeleteNodeReq.java | 3 ++- .../consensus/deletion/DeletionResource.java | 17 ++++++++++++++++- .../assigner/PipeDataRegionAssigner.java | 9 ++++++--- .../listener/PipeInsertionDataNodeListener.java | 4 ++-- .../planner/plan/node/write/DeleteDataNode.java | 10 ++++++---- 6 files changed, 33 insertions(+), 11 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java index ce19b9e242e7..04f5399fe530 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java @@ -1375,6 +1375,7 @@ private void formulateFolders() { for (int i = 0; i < iotConsensusV2ReceiverFileDirs.length; i++) { iotConsensusV2ReceiverFileDirs[i] = addDataHomeDir(iotConsensusV2ReceiverFileDirs[i]); } + iotConsensusV2DeletionFileDir = addDataHomeDir(iotConsensusV2DeletionFileDir); mqttDir = addDataHomeDir(mqttDir); extPipeDir = addDataHomeDir(extPipeDir); queryDir = addDataHomeDir(queryDir); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusDeleteNodeReq.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusDeleteNodeReq.java index 1bf9916b0b82..dab135f92821 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusDeleteNodeReq.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/payload/request/PipeConsensusDeleteNodeReq.java @@ -25,6 +25,7 @@ import org.apache.iotdb.commons.pipe.connector.payload.pipeconsensus.request.PipeConsensusRequestVersion; import org.apache.iotdb.consensus.pipe.thrift.TCommitId; import org.apache.iotdb.consensus.pipe.thrift.TPipeConsensusTransferReq; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeType; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; import org.apache.tsfile.utils.PublicBAOS; @@ -83,7 +84,7 @@ public static PipeConsensusDeleteNodeReq fromTPipeConsensusTransferReq( TPipeConsensusTransferReq transferReq) { final PipeConsensusDeleteNodeReq deleteNodeReq = new PipeConsensusDeleteNodeReq(); - deleteNodeReq.deleteDataNode = DeleteDataNode.deserialize(transferReq.body); + deleteNodeReq.deleteDataNode = (DeleteDataNode) PlanNodeType.deserialize(transferReq.body); deleteNodeReq.version = transferReq.version; deleteNodeReq.type = transferReq.type; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java index 98c7f8b762b6..ab1d95d756ee 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java @@ -20,7 +20,9 @@ package org.apache.iotdb.db.pipe.consensus.deletion; import org.apache.iotdb.commons.consensus.index.ProgressIndex; +import org.apache.iotdb.commons.consensus.index.impl.RecoverProgressIndex; import org.apache.iotdb.commons.pipe.datastructure.PersistentResource; +import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeType; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; @@ -123,7 +125,10 @@ public DeleteDataNode getDeleteDataNode() { } public ByteBuffer serialize() { - return deleteDataNode.serializeToByteBuffer(); + ByteBuffer deletion = deleteDataNode.serializeToByteBuffer(); + final ByteBuffer result = ByteBuffer.allocate(deletion.limit()); + result.put(deletion); + return result; } public static DeletionResource deserialize( @@ -132,6 +137,16 @@ public static DeletionResource deserialize( return new DeletionResource(node, removeHook); } + public static boolean isDeleteNodeGeneratedInLocalByIoTV2(DeleteDataNode node) { + if (node.getProgressIndex() instanceof RecoverProgressIndex) { + RecoverProgressIndex recoverProgressIndex = (RecoverProgressIndex) node.getProgressIndex(); + return recoverProgressIndex + .getDataNodeId2LocalIndex() + .containsKey(IoTDBDescriptor.getInstance().getConfig().getDataNodeId()); + } + return false; + } + @Override public String toString() { return String.format( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java index 25dc449c2816..a7853eb14ab2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java @@ -24,6 +24,7 @@ import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.event.ProgressReportEvent; +import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; @@ -139,10 +140,12 @@ public void assignToExtractor( if (copiedEvent.getEvent() instanceof PipeDeleteDataNodeEvent) { DeletionResourceManager mgr = DeletionResourceManager.getInstance(extractor.getDataRegionId()); + PipeDeleteDataNodeEvent deleteDataNodeEvent = + (PipeDeleteDataNodeEvent) copiedEvent.getEvent(); // increase deletion resource's reference - if (Objects.nonNull(mgr)) { - PipeDeleteDataNodeEvent deleteDataNodeEvent = - (PipeDeleteDataNodeEvent) copiedEvent.getEvent(); + if (Objects.nonNull(mgr) + && DeletionResource.isDeleteNodeGeneratedInLocalByIoTV2( + deleteDataNodeEvent.getDeleteDataNode())) { deleteDataNodeEvent.setDeletionResource( mgr.increaseResourceReferenceAndGet( ((PipeDeleteDataNodeEvent) event.getEvent()).getDeleteDataNode())); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java index 2bbeaa9a57c3..15f206ab7779 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java @@ -50,7 +50,6 @@ * will filter events and assign them to different PipeRealtimeEventDataRegionExtractors. */ public class PipeInsertionDataNodeListener { - private final ConcurrentMap dataRegionId2Assigner = new ConcurrentHashMap<>(); @@ -156,7 +155,8 @@ public DeletionResource listenToDeleteData(DeleteDataNode node, String regionId) // register a deletionResource and return it to DataRegion DeletionResourceManager mgr = DeletionResourceManager.getInstance(regionId); DeletionResource deletionResource = null; - if (Objects.nonNull(mgr)) { + // deleteNode generated by remote consensus leader shouldn't be persisted to DAL. + if (Objects.nonNull(mgr) && DeletionResource.isDeleteNodeGeneratedInLocalByIoTV2(node)) { deletionResource = mgr.registerDeletionResource(node); // if persist failed, skip sending/publishing this event to keep consistency with the // behavior of storage engine. diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java index 9955f837b2e8..6ac5064e4b4b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java @@ -280,20 +280,22 @@ public boolean equals(final Object obj) { return this.getPlanNodeId().equals(that.getPlanNodeId()) && Objects.equals(this.pathList, that.pathList) && Objects.equals(this.deleteStartTime, that.deleteStartTime) - && Objects.equals(this.deleteEndTime, that.deleteEndTime); + && Objects.equals(this.deleteEndTime, that.deleteEndTime) + && Objects.equals(this.progressIndex, that.progressIndex); } @Override public int hashCode() { - return Objects.hash(getPlanNodeId(), pathList, deleteStartTime, deleteEndTime); + return Objects.hash(getPlanNodeId(), pathList, deleteStartTime, deleteEndTime, progressIndex); } public String toString() { return String.format( - "DeleteDataNode-%s[ Paths: %s, Region: %s ]", + "DeleteDataNode-%s[ Paths: %s, Region: %s, ProgressIndex: %s]", getPlanNodeId(), pathList, - regionReplicaSet == null ? "Not Assigned" : regionReplicaSet.getRegionId()); + regionReplicaSet == null ? "Not Assigned" : regionReplicaSet.getRegionId(), + progressIndex == null ? "Not Assigned" : progressIndex); } @Override From 127e02be5ff62771e92de5f0e8d182f6c1f80136 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Wed, 25 Sep 2024 20:45:07 +0800 Subject: [PATCH 59/85] fix: ut --- .../db/pipe/consensus/DeletionRecoverTest.java | 11 ++++------- .../db/pipe/consensus/DeletionResourceTest.java | 13 +++++-------- 2 files changed, 9 insertions(+), 15 deletions(-) diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java index 4e7805761ef7..3cb2209e506a 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java @@ -25,7 +25,6 @@ import org.apache.iotdb.commons.utils.FileUtils; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; -import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; @@ -37,12 +36,12 @@ import java.util.Collections; public class DeletionRecoverTest { - private static final String FAKE_DATE_REGION_ID = "1"; + private static final String FAKE_DATA_REGION_ID = "1"; private static final int THIS_DATANODE_ID = IoTDBDescriptor.getInstance().getConfig().getDataNodeId(); private static final String DELETION_BASE_DIR = IoTDBDescriptor.getInstance().getConfig().getIotConsensusV2DeletionFileDir(); - private static final String BASE_PATH = DELETION_BASE_DIR + File.separator + FAKE_DATE_REGION_ID; + private static final String BASE_PATH = DELETION_BASE_DIR + File.separator + FAKE_DATA_REGION_ID; private final int deletionCount = 10; private DeletionResourceManager deletionResourceManager; @@ -53,7 +52,7 @@ public void setUp() throws Exception { FileUtils.deleteFileOrDirectory(baseDir); } DeletionResourceManager.buildForTest(); - deletionResourceManager = DeletionResourceManager.getInstance(FAKE_DATE_REGION_ID); + deletionResourceManager = DeletionResourceManager.getInstance(FAKE_DATA_REGION_ID); // Create some deletion files int rebootTimes = 0; MeasurementPath path = new MeasurementPath("root.vehicle.d2.s0"); @@ -62,9 +61,7 @@ public void setUp() throws Exception { new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 150); deleteDataNode.setProgressIndex( new RecoverProgressIndex(THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, i))); - PipeDeleteDataNodeEvent deletionEvent = new PipeDeleteDataNodeEvent(deleteDataNode, true); - deletionResourceManager.registerDeletionResource(deletionEvent); - deletionResourceManager.enrichDeletionResourceAndPersist(deletionEvent, deletionEvent); + deletionResourceManager.registerDeletionResource(deleteDataNode); } // Manually close to ensure all deletions are persisted deletionResourceManager.close(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java index 6e6df7a2d0ab..028f08057135 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java @@ -100,9 +100,7 @@ public void testAddBatchDeletionResource() new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 150); deleteDataNode.setProgressIndex( new RecoverProgressIndex(THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, i))); - PipeDeleteDataNodeEvent deletionEvent = new PipeDeleteDataNodeEvent(deleteDataNode, true); - deletionResourceManager.registerDeletionResource(deletionEvent); - deletionResourceManager.enrichDeletionResourceAndPersist(deletionEvent, deletionEvent); + deletionResourceManager.registerDeletionResource(deleteDataNode); } Stream paths = @@ -120,10 +118,8 @@ public void testAddDeletionResourceTimeout() new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 150); deleteDataNode.setProgressIndex( new RecoverProgressIndex(THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, 1))); - PipeDeleteDataNodeEvent deletionEvent = new PipeDeleteDataNodeEvent(deleteDataNode, true); // Only register one deletionResource - deletionResourceManager.registerDeletionResource(deletionEvent); - deletionResourceManager.enrichDeletionResourceAndPersist(deletionEvent, deletionEvent); + deletionResourceManager.registerDeletionResource(deleteDataNode); // Sleep to wait deletion being persisted Thread.sleep(5000); Stream paths = @@ -146,8 +142,9 @@ public void testDeletionRemove() throws IllegalPathException, InterruptedExcepti new RecoverProgressIndex(THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, i))); PipeDeleteDataNodeEvent deletionEvent = new PipeDeleteDataNodeEvent(deleteDataNode, true); deletionEvents.add(deletionEvent); - deletionResourceManager.registerDeletionResource(deletionEvent); - deletionResourceManager.enrichDeletionResourceAndPersist(deletionEvent, deletionEvent); + deletionResourceManager.registerDeletionResource(deleteDataNode); + deletionEvent.setDeletionResource( + deletionResourceManager.increaseResourceReferenceAndGet(deleteDataNode)); } deletionEvents.forEach(deletionEvent -> deletionEvent.increaseReferenceCount("test")); // Sleep to wait deletion being persisted From 486ec81f2b02dbc49638ed62ab6de97feee09258 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Wed, 25 Sep 2024 21:19:29 +0800 Subject: [PATCH 60/85] add log --- .../consensus/deletion/DeletionResource.java | 1 + .../deletion/DeletionResourceManager.java | 5 +++- .../persist/PageCacheDeletionBuffer.java | 27 ++++++++++++++----- 3 files changed, 26 insertions(+), 7 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java index ab1d95d756ee..3d80911075d3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java @@ -70,6 +70,7 @@ public synchronized void decreaseReference() { } public void removeSelf() { + LOGGER.info("DeletionResource {} has been released, trigger a remove of DAL...", this); removeHook.accept(this); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java index b9a34f69e374..f9d18f0ea9f2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java @@ -198,7 +198,10 @@ private synchronized void removeDeletionResource(DeletionResource deletionResour // So here we cannot guarantee that the last file can be deleted, we can only guarantee that // the first n-1 files can be deleted (if the length of deletionPaths is n) for (int i = 0; i < deletionPaths.length - 1; i++) { - FileUtils.deleteFileOrDirectory(deletionPaths[i].toFile()); + File fileToDelete = deletionPaths[i].toFile(); + FileUtils.deleteFileOrDirectory(fileToDelete); + LOGGER.info( + "DeletionManager-{} delete deletion file in {} dir...", dataRegionId, fileToDelete); } } catch (IOException e) { LOGGER.warn( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java index 8940295051f5..376e4455937a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java @@ -73,7 +73,7 @@ public class PageCacheDeletionBuffer implements DeletionBuffer { ? 0 : (o1.getProgressIndex().isAfter(o2.getProgressIndex()) ? 1 : -1)); // Data region id - private final String groupId; + private final String dataRegionId; // directory to store .deletion files private final String baseDirectory; // single thread to serialize WALEntry to workingBuffer @@ -99,13 +99,16 @@ public class PageCacheDeletionBuffer implements DeletionBuffer { // maxProgressIndex of each batch increases in the same order as the physical time. private volatile ProgressIndex maxProgressIndexInLastFile = MinimumProgressIndex.INSTANCE; - public PageCacheDeletionBuffer(String groupId, String baseDirectory) { - this.groupId = groupId; + public PageCacheDeletionBuffer(String dataRegionId, String baseDirectory) { + this.dataRegionId = dataRegionId; this.baseDirectory = baseDirectory; allocateBuffers(); persistThread = IoTDBThreadPoolFactory.newSingleThreadExecutor( - ThreadName.PIPE_CONSENSUS_DELETION_SERIALIZE.getName() + "(group-" + groupId + ")"); + ThreadName.PIPE_CONSENSUS_DELETION_SERIALIZE.getName() + + "(group-" + + dataRegionId + + ")"); } @Override @@ -125,6 +128,8 @@ public void start() { ByteBuffer.wrap( DeletionResourceManager.MAGIC_VERSION_V1.getBytes(StandardCharsets.UTF_8))); } + LOGGER.info( + "Deletion persist-{}: starting to persist, current writing: {}", dataRegionId, logFile); } catch (IOException e) { LOGGER.warn( "Deletion persist: Cannot create file {}, please check your file system manually.", @@ -149,7 +154,9 @@ private void allocateBuffers() { serializeBuffer = ByteBuffer.allocateDirect(ONE_THIRD_WAL_BUFFER_SIZE); } catch (OutOfMemoryError e) { LOGGER.error( - "Fail to allocate deletionBuffer-group-{}'s buffer because out of memory.", groupId, e); + "Fail to allocate deletionBuffer-group-{}'s buffer because out of memory.", + dataRegionId, + e); close(); throw e; } @@ -159,7 +166,7 @@ public void registerDeletionResource(DeletionResource deletionResource) { if (isClosed) { LOGGER.error( "Fail to register DeletionResource into deletionBuffer-{} because this buffer is closed.", - groupId); + dataRegionId); return; } deletionResources.add(deletionResource); @@ -171,11 +178,13 @@ private void appendCurrentBatch() throws IOException { } private void fsyncCurrentLoggingFile() throws IOException { + LOGGER.info("Deletion persist-{}: current batch fsync due to timeout", dataRegionId); this.logChannel.force(false); pendingDeletionsInOneTask.forEach(DeletionResource::onPersistSucceed); } private void closeCurrentLoggingFile() throws IOException { + LOGGER.info("Deletion persist-{}: current file has been closed", dataRegionId); // Close old resource to fsync. this.logStream.close(); this.logChannel.close(); @@ -234,6 +243,10 @@ private void switchLoggingFile() throws IOException { ByteBuffer.wrap( DeletionResourceManager.MAGIC_VERSION_V1.getBytes(StandardCharsets.UTF_8))); } + LOGGER.info( + "Deletion persist-{}: switching to a new file, current writing: {}", + dataRegionId, + logFile); } finally { resetFileAttribute(); } @@ -261,6 +274,8 @@ public void run() { } private boolean serializeDeletionToBatchBuffer(DeletionResource deletionResource) { + LOGGER.info( + "Deletion persist-{}: serialize deletion resource {}", dataRegionId, deletionResource); ByteBuffer buffer = deletionResource.serialize(); // if working buffer doesn't have enough space if (buffer.position() > serializeBuffer.remaining()) { From 845c196962e9af9ae94697c18fbb7a3722f7b251 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Wed, 25 Sep 2024 21:53:24 +0800 Subject: [PATCH 61/85] fix ut --- .../consensus/deletion/persist/PageCacheDeletionBuffer.java | 1 - .../node/pipe/PipeEnrichedDeleteDataNodeSerdeTest.java | 4 +++- .../plan/planner/node/write/DeleteDataNodeSerdeTest.java | 4 +++- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java index 376e4455937a..3045c1f0b7c2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java @@ -67,7 +67,6 @@ public class PageCacheDeletionBuffer implements DeletionBuffer { private final BlockingQueue deletionResources = new PriorityBlockingQueue<>( QUEUE_CAPACITY, - // any two progressIndex can't be equal (o1, o2) -> o1.getProgressIndex().equals(o2.getProgressIndex()) ? 0 diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/pipe/PipeEnrichedDeleteDataNodeSerdeTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/pipe/PipeEnrichedDeleteDataNodeSerdeTest.java index 91bd488f409c..f6add9efe89b 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/pipe/PipeEnrichedDeleteDataNodeSerdeTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/pipe/PipeEnrichedDeleteDataNodeSerdeTest.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.queryengine.plan.planner.node.pipe; +import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.path.MeasurementPath; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; @@ -43,7 +44,8 @@ public void testSerializeAndDeserialize() throws IllegalPathException { List pathList = new ArrayList<>(); pathList.add(new MeasurementPath("root.sg.d1.s1")); pathList.add(new MeasurementPath("root.sg.d2.*")); - DeleteDataNode deleteDataNode = new DeleteDataNode(planNodeId, pathList, startTime, endTime); + DeleteDataNode deleteDataNode = + new DeleteDataNode(planNodeId, pathList, startTime, endTime, MinimumProgressIndex.INSTANCE); PipeEnrichedDeleteDataNode pipeEnrichedDeleteDataNode = new PipeEnrichedDeleteDataNode(deleteDataNode); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/write/DeleteDataNodeSerdeTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/write/DeleteDataNodeSerdeTest.java index 38478acccb26..78d0b4349dee 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/write/DeleteDataNodeSerdeTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/write/DeleteDataNodeSerdeTest.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.queryengine.plan.planner.node.write; +import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.path.MeasurementPath; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; @@ -43,7 +44,8 @@ public void testSerializeAndDeserialize() throws IllegalPathException { List pathList = new ArrayList<>(); pathList.add(new MeasurementPath("root.sg.d1.s1")); pathList.add(new MeasurementPath("root.sg.d2.*")); - DeleteDataNode deleteDataNode = new DeleteDataNode(planNodeId, pathList, startTime, endTime); + DeleteDataNode deleteDataNode = + new DeleteDataNode(planNodeId, pathList, startTime, endTime, MinimumProgressIndex.INSTANCE); ByteBuffer byteBuffer = ByteBuffer.allocate(1024); deleteDataNode.serialize(byteBuffer); From ec11267a1f653d7765a45f64b3845fe1a935d550 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Fri, 27 Sep 2024 22:28:10 +0800 Subject: [PATCH 62/85] fix ut --- .../thrift/async/IoTDBDataRegionAsyncConnector.java | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java index 5aa4324a2f39..f7bcb87f26ae 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/async/IoTDBDataRegionAsyncConnector.java @@ -37,8 +37,8 @@ import org.apache.iotdb.db.pipe.connector.protocol.thrift.async.handler.PipeTransferTabletRawEventHandler; import org.apache.iotdb.db.pipe.connector.protocol.thrift.async.handler.PipeTransferTsFileHandler; import org.apache.iotdb.db.pipe.connector.protocol.thrift.sync.IoTDBDataRegionSyncConnector; +import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; -import org.apache.iotdb.db.pipe.event.common.schema.PipeSchemaRegionWritePlanEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeInsertNodeTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent; import org.apache.iotdb.db.pipe.event.common.terminate.PipeTerminateEvent; @@ -86,15 +86,11 @@ public class IoTDBDataRegionAsyncConnector extends IoTDBConnector { "Failed to borrow client from client pool when sending to receiver."; private static final String THRIFT_ERROR_FORMATTER_WITH_ENDPOINT = "Exception occurred while sending to receiver %s:%s."; - - private IoTDBDataNodeAsyncClientManager clientManager; - private final IoTDBDataRegionSyncConnector retryConnector = new IoTDBDataRegionSyncConnector(); private final BlockingQueue retryEventQueue = new LinkedBlockingQueue<>(); - - private PipeTransferBatchReqBuilder tabletBatchBuilder; - private final AtomicBoolean isClosed = new AtomicBoolean(false); + private IoTDBDataNodeAsyncClientManager clientManager; + private PipeTransferBatchReqBuilder tabletBatchBuilder; @Override public void validate(final PipeParameterValidator validator) throws Exception { @@ -366,7 +362,7 @@ public void transfer(final Event event) throws Exception { transferBatchedEventsIfNecessary(); if (!(event instanceof PipeHeartbeatEvent - || event instanceof PipeSchemaRegionWritePlanEvent + || event instanceof PipeDeleteDataNodeEvent || event instanceof PipeTerminateEvent)) { LOGGER.warn( "IoTDBThriftAsyncConnector does not support transferring generic event: {}.", event); From d6daaabef25b87eb411337900f2b8390aa8e3993 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Mon, 30 Sep 2024 00:05:59 +0800 Subject: [PATCH 63/85] fix review --- .../sync/IoTDBDataRegionSyncConnector.java | 4 +- .../ProgressIndexDataNodeManager.java | 4 +- .../consensus/deletion/DeletionResource.java | 5 +- .../persist/PageCacheDeletionBuffer.java | 84 +++++----- .../plan/node/write/DeleteDataNode.java | 155 +++++++++++------- 5 files changed, 142 insertions(+), 110 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java index e0da5ebd9fb1..f7dae53737bf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java @@ -184,14 +184,14 @@ private void doTransferWrapper(final PipeDeleteDataNodeEvent pipeDeleteDataNodeE throws PipeException { // We increase the reference count for this event to determine if the event may be released. if (!pipeDeleteDataNodeEvent.increaseReferenceCount( - IoTDBDataNodeSyncConnector.class.getName())) { + IoTDBDataRegionSyncConnector.class.getName())) { return; } try { doTransfer(pipeDeleteDataNodeEvent); } finally { pipeDeleteDataNodeEvent.decreaseReferenceCount( - IoTDBDataNodeSyncConnector.class.getName(), false); + IoTDBDataRegionSyncConnector.class.getName(), false); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ProgressIndexDataNodeManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ProgressIndexDataNodeManager.java index c492bef3bd73..e9ae4ff4454d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ProgressIndexDataNodeManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ProgressIndexDataNodeManager.java @@ -41,8 +41,8 @@ import java.util.stream.Collectors; public class ProgressIndexDataNodeManager implements ProgressIndexManager { - private final Map groupId2MaxProgressIndex; private static final int DATA_NODE_ID = IoTDBDescriptor.getInstance().getConfig().getDataNodeId(); + private final Map groupId2MaxProgressIndex; public ProgressIndexDataNodeManager() { this.groupId2MaxProgressIndex = new ConcurrentHashMap<>(); @@ -106,8 +106,6 @@ private void recoverMaxProgressIndexFromDataRegion() { (value == null ? MinimumProgressIndex.INSTANCE : value) .updateToMinimumEqualOrIsAfterProgressIndex(finalMaxProgressIndex)); }); - - // TODO: update deletion progress index } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java index 3d80911075d3..bad71da73b35 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java @@ -24,7 +24,6 @@ import org.apache.iotdb.commons.pipe.datastructure.PersistentResource; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; -import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeType; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; import org.slf4j.Logger; @@ -126,7 +125,7 @@ public DeleteDataNode getDeleteDataNode() { } public ByteBuffer serialize() { - ByteBuffer deletion = deleteDataNode.serializeToByteBuffer(); + ByteBuffer deletion = deleteDataNode.serializeToDAL(); final ByteBuffer result = ByteBuffer.allocate(deletion.limit()); result.put(deletion); return result; @@ -134,7 +133,7 @@ public ByteBuffer serialize() { public static DeletionResource deserialize( final ByteBuffer buffer, final Consumer removeHook) throws IOException { - DeleteDataNode node = (DeleteDataNode) PlanNodeType.deserialize(buffer); + DeleteDataNode node = DeleteDataNode.deserializeFromDAL(buffer); return new DeletionResource(node, removeHook); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java index 3045c1f0b7c2..4905f4d26413 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java @@ -55,7 +55,7 @@ * an easier way to maintain and understand. */ public class PageCacheDeletionBuffer implements DeletionBuffer { - private static final Logger LOGGER = LoggerFactory.getLogger(TwoStageDeletionBuffer.class); + private static final Logger LOGGER = LoggerFactory.getLogger(PageCacheDeletionBuffer.class); private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig(); // Buffer config keep consistent with WAL. private static final int ONE_THIRD_WAL_BUFFER_SIZE = config.getWalBufferSize() / 3; @@ -91,7 +91,7 @@ public class PageCacheDeletionBuffer implements DeletionBuffer { private volatile File logFile; private volatile FileOutputStream logStream; private volatile FileChannel logChannel; - // Max progressIndex among current .deletion file. Used by PersistTask for naming .deletion file. + // Max progressIndex among current .deletion file. private ProgressIndex maxProgressIndexInCurrentFile = MinimumProgressIndex.INSTANCE; // Max progressIndex among last .deletion file. Used by PersistTask for naming .deletion file. // Since deletions are written serially, DAL is also written serially. This ensures that the @@ -251,6 +251,45 @@ private void switchLoggingFile() throws IOException { } } + @Override + public void close() { + isClosed = true; + // Force sync existing data in memory to disk. + // first waiting serialize and sync tasks finished, then release all resources + waitUntilFlushAllDeletionsOrTimeOut(); + if (persistThread != null) { + shutdownThread(persistThread); + } + // clean buffer + MmapUtil.clean(serializeBuffer); + } + + private void waitUntilFlushAllDeletionsOrTimeOut() { + long currentTime = System.currentTimeMillis(); + while (!isAllDeletionFlushed() + && System.currentTimeMillis() - currentTime < MAX_WAIT_CLOSE_TIME_IN_MS) { + try { + Thread.sleep(50); + } catch (InterruptedException e) { + LOGGER.error("Interrupted when waiting for all deletions flushed."); + Thread.currentThread().interrupt(); + } + } + } + + private void shutdownThread(ExecutorService thread) { + ThreadName threadName = ThreadName.PIPE_CONSENSUS_DELETION_SERIALIZE; + thread.shutdown(); + try { + if (!thread.awaitTermination(30, TimeUnit.SECONDS)) { + LOGGER.warn("Waiting thread {} to be terminated is timeout", threadName.getName()); + } + } catch (InterruptedException e) { + LOGGER.warn("Thread {} still doesn't exit after 30s", threadName.getName()); + Thread.currentThread().interrupt(); + } + } + private class PersistTask implements Runnable { // Batch size in current task, used to roll back. private final AtomicInteger currentTaskBatchSize = new AtomicInteger(0); @@ -273,7 +312,7 @@ public void run() { } private boolean serializeDeletionToBatchBuffer(DeletionResource deletionResource) { - LOGGER.info( + LOGGER.debug( "Deletion persist-{}: serialize deletion resource {}", dataRegionId, deletionResource); ByteBuffer buffer = deletionResource.serialize(); // if working buffer doesn't have enough space @@ -351,43 +390,4 @@ private void persistDeletion() throws IOException { } } } - - @Override - public void close() { - isClosed = true; - // Force sync existing data in memory to disk. - // first waiting serialize and sync tasks finished, then release all resources - waitUntilFlushAllDeletionsOrTimeOut(); - if (persistThread != null) { - shutdownThread(persistThread); - } - // clean buffer - MmapUtil.clean(serializeBuffer); - } - - private void waitUntilFlushAllDeletionsOrTimeOut() { - long currentTime = System.currentTimeMillis(); - while (!isAllDeletionFlushed() - && System.currentTimeMillis() - currentTime < MAX_WAIT_CLOSE_TIME_IN_MS) { - try { - Thread.sleep(50); - } catch (InterruptedException e) { - LOGGER.error("Interrupted when waiting for all deletions flushed."); - Thread.currentThread().interrupt(); - } - } - } - - private void shutdownThread(ExecutorService thread) { - ThreadName threadName = ThreadName.PIPE_CONSENSUS_DELETION_SERIALIZE; - thread.shutdown(); - try { - if (!thread.awaitTermination(30, TimeUnit.SECONDS)) { - LOGGER.warn("Waiting thread {} to be terminated is timeout", threadName.getName()); - } - } catch (InterruptedException e) { - LOGGER.warn("Thread {} still doesn't exit after 30s", threadName.getName()); - Thread.currentThread().interrupt(); - } - } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java index 6ac5064e4b4b..e5299fabe670 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java @@ -23,6 +23,7 @@ import org.apache.iotdb.commons.consensus.index.ProgressIndex; import org.apache.iotdb.commons.consensus.index.ProgressIndexType; import org.apache.iotdb.commons.exception.IllegalPathException; +import org.apache.iotdb.commons.exception.runtime.SerializationRunTimeException; import org.apache.iotdb.commons.partition.DataPartition; import org.apache.iotdb.commons.path.MeasurementPath; import org.apache.iotdb.commons.path.PartialPath; @@ -41,7 +42,10 @@ import org.apache.iotdb.db.storageengine.dataregion.wal.utils.WALWriteUtils; import org.apache.tsfile.read.filter.factory.TimeFilterApi; +import org.apache.tsfile.utils.PublicBAOS; import org.apache.tsfile.utils.ReadWriteIOUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.DataInputStream; import java.io.DataOutputStream; @@ -57,6 +61,8 @@ import static org.apache.iotdb.commons.conf.IoTDBConstant.MULTI_LEVEL_PATH_WILDCARD; public class DeleteDataNode extends SearchNode implements WALEntryValue { + private static final Logger LOGGER = LoggerFactory.getLogger(DeleteDataNode.class); + /** byte: type, integer: pathList.size(), long: deleteStartTime, deleteEndTime, searchIndex */ private static final int FIXED_SERIALIZED_SIZE = Short.BYTES + Integer.BYTES + Long.BYTES * 3; @@ -101,6 +107,95 @@ public DeleteDataNode( this.regionReplicaSet = regionReplicaSet; } + public static DeleteDataNode deserializeFromWAL(DataInputStream stream) throws IOException { + long searchIndex = stream.readLong(); + int size = stream.readInt(); + List pathList = new ArrayList<>(size); + for (int i = 0; i < size; i++) { + try { + pathList.add(new MeasurementPath(ReadWriteIOUtils.readString(stream))); + } catch (IllegalPathException e) { + throw new IllegalArgumentException("Cannot deserialize InsertRowNode", e); + } + } + long deleteStartTime = stream.readLong(); + long deleteEndTime = stream.readLong(); + + DeleteDataNode deleteDataNode = + new DeleteDataNode(new PlanNodeId(""), pathList, deleteStartTime, deleteEndTime); + deleteDataNode.setSearchIndex(searchIndex); + return deleteDataNode; + } + + public static DeleteDataNode deserializeFromWAL(ByteBuffer buffer) { + long searchIndex = buffer.getLong(); + int size = buffer.getInt(); + List pathList = new ArrayList<>(size); + for (int i = 0; i < size; i++) { + try { + pathList.add(new MeasurementPath(ReadWriteIOUtils.readString(buffer))); + } catch (IllegalPathException e) { + throw new IllegalArgumentException("Cannot deserialize InsertRowNode", e); + } + } + long deleteStartTime = buffer.getLong(); + long deleteEndTime = buffer.getLong(); + + DeleteDataNode deleteDataNode = + new DeleteDataNode(new PlanNodeId(""), pathList, deleteStartTime, deleteEndTime); + deleteDataNode.setSearchIndex(searchIndex); + return deleteDataNode; + } + + public static DeleteDataNode deserialize(ByteBuffer byteBuffer) { + int size = ReadWriteIOUtils.readInt(byteBuffer); + List pathList = new ArrayList<>(size); + for (int i = 0; i < size; i++) { + pathList.add((MeasurementPath) PathDeserializeUtil.deserialize(byteBuffer)); + } + long deleteStartTime = ReadWriteIOUtils.readLong(byteBuffer); + long deleteEndTime = ReadWriteIOUtils.readLong(byteBuffer); + + PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer); + + // DeleteDataNode has no child + int ignoredChildrenSize = ReadWriteIOUtils.readInt(byteBuffer); + return new DeleteDataNode(planNodeId, pathList, deleteStartTime, deleteEndTime); + } + + public static DeleteDataNode deserializeFromDAL(ByteBuffer byteBuffer) { + short nodeType = byteBuffer.getShort(); + int size = ReadWriteIOUtils.readInt(byteBuffer); + List pathList = new ArrayList<>(size); + for (int i = 0; i < size; i++) { + pathList.add((MeasurementPath) PathDeserializeUtil.deserialize(byteBuffer)); + } + long deleteStartTime = ReadWriteIOUtils.readLong(byteBuffer); + long deleteEndTime = ReadWriteIOUtils.readLong(byteBuffer); + ProgressIndex deserializedIndex = ProgressIndexType.deserializeFrom(byteBuffer); + + PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer); + + // DeleteDataNode has no child + int ignoredChildrenSize = ReadWriteIOUtils.readInt(byteBuffer); + return new DeleteDataNode(planNodeId, pathList, deleteStartTime, deleteEndTime); + } + + public ByteBuffer serializeToDAL() { + try (PublicBAOS byteArrayOutputStream = new PublicBAOS(); + DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream)) { + serializeAttributes(outputStream); + progressIndex.serialize(outputStream); + id.serialize(outputStream); + // write children nodes size + ReadWriteIOUtils.write(0, outputStream); + return ByteBuffer.wrap(byteArrayOutputStream.getBuf(), 0, byteArrayOutputStream.size()); + } catch (IOException e) { + LOGGER.error("Unexpected error occurs when serializing deleteDataNode.", e); + throw new SerializationRunTimeException(e); + } + } + public List getPathList() { return pathList; } @@ -172,46 +267,6 @@ public void serializeToWAL(IWALByteBufferView buffer) { buffer.putLong(deleteEndTime); } - public static DeleteDataNode deserializeFromWAL(DataInputStream stream) throws IOException { - long searchIndex = stream.readLong(); - int size = stream.readInt(); - List pathList = new ArrayList<>(size); - for (int i = 0; i < size; i++) { - try { - pathList.add(new MeasurementPath(ReadWriteIOUtils.readString(stream))); - } catch (IllegalPathException e) { - throw new IllegalArgumentException("Cannot deserialize InsertRowNode", e); - } - } - long deleteStartTime = stream.readLong(); - long deleteEndTime = stream.readLong(); - - DeleteDataNode deleteDataNode = - new DeleteDataNode(new PlanNodeId(""), pathList, deleteStartTime, deleteEndTime); - deleteDataNode.setSearchIndex(searchIndex); - return deleteDataNode; - } - - public static DeleteDataNode deserializeFromWAL(ByteBuffer buffer) { - long searchIndex = buffer.getLong(); - int size = buffer.getInt(); - List pathList = new ArrayList<>(size); - for (int i = 0; i < size; i++) { - try { - pathList.add(new MeasurementPath(ReadWriteIOUtils.readString(buffer))); - } catch (IllegalPathException e) { - throw new IllegalArgumentException("Cannot deserialize InsertRowNode", e); - } - } - long deleteStartTime = buffer.getLong(); - long deleteEndTime = buffer.getLong(); - - DeleteDataNode deleteDataNode = - new DeleteDataNode(new PlanNodeId(""), pathList, deleteStartTime, deleteEndTime); - deleteDataNode.setSearchIndex(searchIndex); - return deleteDataNode; - } - @Override protected void serializeAttributes(ByteBuffer byteBuffer) { PlanNodeType.DELETE_DATA.serialize(byteBuffer); @@ -221,7 +276,6 @@ protected void serializeAttributes(ByteBuffer byteBuffer) { } ReadWriteIOUtils.write(deleteStartTime, byteBuffer); ReadWriteIOUtils.write(deleteEndTime, byteBuffer); - progressIndex.serialize(byteBuffer); } @Override @@ -233,25 +287,6 @@ protected void serializeAttributes(DataOutputStream stream) throws IOException { } ReadWriteIOUtils.write(deleteStartTime, stream); ReadWriteIOUtils.write(deleteEndTime, stream); - progressIndex.serialize(stream); - } - - public static DeleteDataNode deserialize(ByteBuffer byteBuffer) { - int size = ReadWriteIOUtils.readInt(byteBuffer); - List pathList = new ArrayList<>(size); - for (int i = 0; i < size; i++) { - pathList.add((MeasurementPath) PathDeserializeUtil.deserialize(byteBuffer)); - } - long deleteStartTime = ReadWriteIOUtils.readLong(byteBuffer); - long deleteEndTime = ReadWriteIOUtils.readLong(byteBuffer); - ProgressIndex deserializedIndex = ProgressIndexType.deserializeFrom(byteBuffer); - - PlanNodeId planNodeId = PlanNodeId.deserialize(byteBuffer); - - // DeleteDataNode has no child - int ignoredChildrenSize = ReadWriteIOUtils.readInt(byteBuffer); - return new DeleteDataNode( - planNodeId, pathList, deleteStartTime, deleteEndTime, deserializedIndex); } @Override From e7b64d250089bda10ab8db84a381456c3b4d1f37 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Mon, 30 Sep 2024 00:06:42 +0800 Subject: [PATCH 64/85] delete TwoStageDeletionBuffer.java --- .../persist/TwoStageDeletionBuffer.java | 363 ------------------ 1 file changed, 363 deletions(-) delete mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/TwoStageDeletionBuffer.java diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/TwoStageDeletionBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/TwoStageDeletionBuffer.java deleted file mode 100644 index cbe89d43e4d9..000000000000 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/TwoStageDeletionBuffer.java +++ /dev/null @@ -1,363 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.iotdb.db.pipe.consensus.deletion.persist; - -import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory; -import org.apache.iotdb.commons.concurrent.ThreadName; -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.SimpleProgressIndex; -import org.apache.iotdb.db.conf.IoTDBConfig; -import org.apache.iotdb.db.conf.IoTDBDescriptor; -import org.apache.iotdb.db.pipe.consensus.ProgressIndexDataNodeManager; -import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; -import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; -import org.apache.iotdb.db.utils.MmapUtil; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.channels.FileChannel; -import java.nio.charset.StandardCharsets; -import java.util.concurrent.ArrayBlockingQueue; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.locks.Condition; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; - -/** - * The core idea of this buffer is to decouple the serialization and writing tasks. Similar to - * WALBuffer, it can perform subsequent operations such as compression in the application state. - */ -public class TwoStageDeletionBuffer implements DeletionBuffer { - private static final Logger LOGGER = LoggerFactory.getLogger(TwoStageDeletionBuffer.class); - private static final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig(); - // Buffer config keep consistent with WAL. - private static final int ONE_THIRD_WAL_BUFFER_SIZE = config.getWalBufferSize() / 3; - private static final double FSYNC_BUFFER_RATIO = 0.95; - private static final int QUEUE_CAPACITY = config.getWalBufferQueueCapacity(); - - // whether close method is called - private volatile boolean isClosed = false; - // DeletionResources - private final BlockingQueue deletionResources = - new ArrayBlockingQueue<>(QUEUE_CAPACITY); - // lock to provide synchronization for double buffers mechanism, protecting buffers status - private final Lock buffersLock = new ReentrantLock(); - // condition to guarantee correctness of switching buffers - private final Condition idleBufferReadyCondition = buffersLock.newCondition(); - private final String groupId; - - // region these variables should be protected by buffersLock - /** two buffers switch between three statuses (there is always 1 buffer working). */ - // buffer in working status, only updated by serializeThread - // it's safe to use volatile here to make this reference thread-safe. - @SuppressWarnings("squid:S3077") - private volatile ByteBuffer workingBuffer; - - // buffer in idle status - // it's safe to use volatile here to make this reference thread-safe. - @SuppressWarnings("squid:S3077") - private volatile ByteBuffer idleBuffer; - - // buffer in syncing status, serializeThread makes sure no more writes to syncingBuffer - // it's safe to use volatile here to make this reference thread-safe. - @SuppressWarnings("squid:S3077") - private volatile ByteBuffer syncingBuffer; - - // single thread to serialize WALEntry to workingBuffer - private final ExecutorService serializeThread; - // single thread to sync syncingBuffer to disk - private final ExecutorService syncBufferThread; - // directory to store .deletion files - private final String baseDirectory; - - public TwoStageDeletionBuffer(String groupId, String baseDirectory) { - this.groupId = groupId; - this.baseDirectory = baseDirectory; - allocateBuffers(); - serializeThread = - IoTDBThreadPoolFactory.newSingleThreadExecutor( - ThreadName.PIPE_CONSENSUS_DELETION_SERIALIZE.getName() + "(group-" + groupId + ")"); - syncBufferThread = - IoTDBThreadPoolFactory.newSingleThreadExecutor( - ThreadName.PIPE_CONSENSUS_DELETION_SYNC.getName() + "(group-" + groupId + ")"); - } - - @Override - public void start() { - // Start serialize and sync pipeline. - serializeThread.submit(new SerializeTask()); - } - - @Override - public boolean isAllDeletionFlushed() { - buffersLock.lock(); - try { - return deletionResources.isEmpty() && workingBuffer.position() == 0 && syncingBuffer == null; - } finally { - buffersLock.unlock(); - } - } - - public void registerDeletionResource(DeletionResource deletionResource) { - if (isClosed) { - LOGGER.error( - "Fail to register DeletionResource into deletionBuffer-{} because this buffer is closed.", - groupId); - return; - } - deletionResources.add(deletionResource); - } - - private void allocateBuffers() { - try { - workingBuffer = ByteBuffer.allocateDirect(ONE_THIRD_WAL_BUFFER_SIZE); - idleBuffer = ByteBuffer.allocateDirect(ONE_THIRD_WAL_BUFFER_SIZE); - } catch (OutOfMemoryError e) { - LOGGER.error( - "Fail to allocate deletionBuffer-group-{}'s buffer because out of memory.", groupId, e); - close(); - throw e; - } - } - - /** Notice: this method only called when buffer is exhausted by SerializeTask. */ - private void syncWorkingBuffer(ProgressIndex maxProgressIndexInCurrentBatch) { - switchWorkingBufferToFlushing(); - try { - syncBufferThread.submit(new SyncBufferTask(maxProgressIndexInCurrentBatch)); - } catch (IOException e) { - LOGGER.warn( - "Failed to submit syncBufferTask, May because file open error and cause data inconsistency. Please check your file system. ", - e); - } - } - - // only called by serializeThread - private void switchWorkingBufferToFlushing() { - buffersLock.lock(); - try { - while (idleBuffer == null) { - idleBufferReadyCondition.await(); - } - syncingBuffer = workingBuffer; - workingBuffer = idleBuffer; - workingBuffer.clear(); - idleBuffer = null; - } catch (InterruptedException e) { - LOGGER.warn("Interrupted When waiting for available working buffer."); - Thread.currentThread().interrupt(); - } finally { - buffersLock.unlock(); - } - } - - private class SerializeTask implements Runnable { - // Total size of this batch. - private int totalSize = 0; - // Max progressIndex among this batch. Used by SyncTask for naming .deletion file. - private ProgressIndex maxProgressIndexInCurrentBatch = MinimumProgressIndex.INSTANCE; - - @Override - public void run() { - try { - serialize(); - } finally { - if (!isClosed) { - serializeThread.submit(new SerializeTask()); - } - } - } - - private void serialize() { - // For first deletion we use blocking take() method. - try { - DeletionResource firstDeletionResource = deletionResources.take(); - // For first serialization, we don't need to judge whether working buffer is exhausted. - // Because a single DeleteDataNode can't exceed size of working buffer. - serializeToWorkingBuffer(firstDeletionResource); - maxProgressIndexInCurrentBatch = - maxProgressIndexInCurrentBatch.updateToMinimumEqualOrIsAfterProgressIndex( - firstDeletionResource.getProgressIndex()); - } catch (InterruptedException e) { - LOGGER.warn( - "Interrupted when waiting for taking DeletionResource from blocking queue to serialize."); - Thread.currentThread().interrupt(); - } - - // For further deletion, we use non-blocking poll() method to persist existing deletion of - // current batch in time. - while (totalSize < ONE_THIRD_WAL_BUFFER_SIZE * FSYNC_BUFFER_RATIO) { - DeletionResource deletionResource = null; - try { - // Timeout config keep consistent with WAL async mode. - deletionResource = - deletionResources.poll(config.getWalAsyncModeFsyncDelayInMs(), TimeUnit.MILLISECONDS); - } catch (InterruptedException e) { - LOGGER.warn( - "Interrupted when waiting for taking WALEntry from blocking queue to serialize."); - Thread.currentThread().interrupt(); - } - // If timeout, flush deletions to disk. - if (deletionResource == null) { - break; - } - // Serialize deletion - while (!serializeToWorkingBuffer(deletionResource)) { - // If working buffer is exhausted, submit a syncTask to consume current batch and switch - // buffer to start a new batch. - syncWorkingBuffer(maxProgressIndexInCurrentBatch); - // Reset maxProgressIndex and deletionNum for new batch. - maxProgressIndexInCurrentBatch = MinimumProgressIndex.INSTANCE; - } - // Update max progressIndex - maxProgressIndexInCurrentBatch = - maxProgressIndexInCurrentBatch.updateToMinimumEqualOrIsAfterProgressIndex( - deletionResource.getProgressIndex()); - } - // Persist deletions; Defensive programming here, just in case. - if (totalSize > 0) { - syncWorkingBuffer(maxProgressIndexInCurrentBatch); - } - } - - /** - * Serialize deletionResource to working buffer. Return true if serialize successfully, false - * otherwise. - */ - private boolean serializeToWorkingBuffer(DeletionResource deletionResource) { - ByteBuffer buffer = deletionResource.serialize(); - // if working buffer doesn't have enough space - if (buffer.position() > workingBuffer.remaining()) { - return false; - } - workingBuffer.put(buffer.array()); - totalSize += buffer.position(); - return true; - } - } - - // only called by syncBufferThread - private void switchSyncingBufferToIdle() { - buffersLock.lock(); - try { - // No need to judge whether idleBuffer is null because syncingBuffer is not null - // and there is only one buffer can be null between syncingBuffer and idleBuffer - idleBuffer = syncingBuffer; - syncingBuffer = null; - idleBufferReadyCondition.signalAll(); - } finally { - buffersLock.unlock(); - } - } - - private class SyncBufferTask implements Runnable { - private final File logFile; - private final FileOutputStream logStream; - private final FileChannel logChannel; - - public SyncBufferTask(ProgressIndex maxProgressIndexInCurrentBatch) throws IOException { - // PipeConsensus ensures that deleteDataNodes use recoverProgressIndex. - ProgressIndex curProgressIndex = - ProgressIndexDataNodeManager.extractLocalSimpleProgressIndex( - maxProgressIndexInCurrentBatch); - if (!(curProgressIndex instanceof SimpleProgressIndex)) { - throw new IOException("Invalid deletion progress index: " + curProgressIndex); - } - SimpleProgressIndex progressIndex = (SimpleProgressIndex) curProgressIndex; - // Deletion file name format: "_{rebootTimes}_{memTableFlushOrderId}.deletion" - this.logFile = - new File( - baseDirectory, - String.format( - "_%d-%d%s", - progressIndex.getRebootTimes(), - progressIndex.getMemTableFlushOrderId(), - DeletionResourceManager.DELETION_FILE_SUFFIX)); - this.logStream = new FileOutputStream(logFile, true); - this.logChannel = logStream.getChannel(); - // Create file && write magic string - if (!logFile.exists() || logFile.length() == 0) { - this.logChannel.write( - ByteBuffer.wrap( - DeletionResourceManager.MAGIC_VERSION_V1.getBytes(StandardCharsets.UTF_8))); - } - } - - @Override - public void run() { - // Sync deletion to disk. - workingBuffer.flip(); - try { - // Write deletions. - syncingBuffer.flip(); - this.logChannel.write(syncingBuffer); - } catch (IOException e) { - LOGGER.warn( - "Deletion persist: Cannot write to {}, may cause data inconsistency.", logFile, e); - } finally { - switchSyncingBufferToIdle(); - } - // Close resource. - try { - this.logChannel.close(); - this.logStream.close(); - } catch (IOException e) { - LOGGER.warn("Failed to close deletion writing resource when writing to {}.", logFile, e); - } - } - } - - @Override - public void close() { - isClosed = true; - // Force sync existing data in memory to disk. - // first waiting serialize and sync tasks finished, then release all resources - if (serializeThread != null) { - shutdownThread(serializeThread, ThreadName.PIPE_CONSENSUS_DELETION_SERIALIZE); - } - if (syncBufferThread != null) { - shutdownThread(syncBufferThread, ThreadName.PIPE_CONSENSUS_DELETION_SYNC); - } - - MmapUtil.clean(workingBuffer); - MmapUtil.clean(workingBuffer); - MmapUtil.clean(syncingBuffer); - } - - private void shutdownThread(ExecutorService thread, ThreadName threadName) { - thread.shutdown(); - try { - if (!thread.awaitTermination(30, TimeUnit.SECONDS)) { - LOGGER.warn("Waiting thread {} to be terminated is timeout", threadName.getName()); - } - } catch (InterruptedException e) { - LOGGER.warn("Thread {} still doesn't exit after 30s", threadName.getName()); - Thread.currentThread().interrupt(); - } - } -} From f65fcdea4b823f9dfb134a04c63a6fdd6c6c15f7 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Mon, 30 Sep 2024 00:14:30 +0800 Subject: [PATCH 65/85] fix: load tsFile's progress index datanode_id can be positive --- .../iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java index cb9484c49e70..58ea998e8de8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java @@ -167,7 +167,7 @@ public RecoverProgressIndex getNextProgressIndexForTsFileLoad() { // because the load directly sends the tsfile to all replicas. Therefore, it is necessary to // ensure that the datanode id generated by the load is negative. return new RecoverProgressIndex( - -DATA_NODE_ID, simpleProgressIndexAssigner.getSimpleProgressIndex()); + DATA_NODE_ID, simpleProgressIndexAssigner.getSimpleProgressIndex()); } ////////////////////// Recover ProgressIndex Assigner ////////////////////// From 4121175c3135741f831ed5c473f7eaf7d64fb0c7 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Mon, 30 Sep 2024 12:22:00 +0800 Subject: [PATCH 66/85] fix ut --- .../plan/planner/plan/node/write/DeleteDataNode.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java index e5299fabe670..1e73c13de5a9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/DeleteDataNode.java @@ -178,7 +178,8 @@ public static DeleteDataNode deserializeFromDAL(ByteBuffer byteBuffer) { // DeleteDataNode has no child int ignoredChildrenSize = ReadWriteIOUtils.readInt(byteBuffer); - return new DeleteDataNode(planNodeId, pathList, deleteStartTime, deleteEndTime); + return new DeleteDataNode( + planNodeId, pathList, deleteStartTime, deleteEndTime, deserializedIndex); } public ByteBuffer serializeToDAL() { From 80e91914ddfdd34170784c7e5fb189564e72eea1 Mon Sep 17 00:00:00 2001 From: Steve Yurong Su Date: Mon, 30 Sep 2024 16:39:28 +0800 Subject: [PATCH 67/85] refactor --- .../PipeInsertionDataNodeListener.java | 36 ++++++++++--------- .../storageengine/dataregion/DataRegion.java | 4 +-- .../pipe/consensus/DeletionResourceTest.java | 2 +- 3 files changed, 22 insertions(+), 20 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java index 15f206ab7779..a34d601f92b1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java @@ -22,9 +22,7 @@ import org.apache.iotdb.commons.pipe.config.PipeConfig; import org.apache.iotdb.db.pipe.agent.PipeDataNodeAgent; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; -import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource.Status; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; -import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEventFactory; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionExtractor; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.assigner.PipeDataRegionAssigner; @@ -138,36 +136,40 @@ public void listenToInsertNode( PipeRealtimeEventFactory.createRealtimeEvent(walEntryHandler, insertNode, tsFileResource)); } - public void listenToHeartbeat(boolean shouldPrintMessage) { - dataRegionId2Assigner.forEach( - (key, value) -> - value.publishToAssign( - PipeRealtimeEventFactory.createRealtimeEvent(key, shouldPrintMessage))); - } - - public DeletionResource listenToDeleteData(DeleteDataNode node, String regionId) { - PipeRealtimeEvent realtimeEvent = PipeRealtimeEventFactory.createRealtimeEvent(node); + public DeletionResource listenToDeleteData(final String regionId, final DeleteDataNode node) { final PipeDataRegionAssigner assigner = dataRegionId2Assigner.get(regionId); // only events from registered data region will be extracted if (assigner == null) { return null; } + + final DeletionResource deletionResource; // register a deletionResource and return it to DataRegion - DeletionResourceManager mgr = DeletionResourceManager.getInstance(regionId); - DeletionResource deletionResource = null; + final DeletionResourceManager manager = DeletionResourceManager.getInstance(regionId); // deleteNode generated by remote consensus leader shouldn't be persisted to DAL. - if (Objects.nonNull(mgr) && DeletionResource.isDeleteNodeGeneratedInLocalByIoTV2(node)) { - deletionResource = mgr.registerDeletionResource(node); + if (Objects.nonNull(manager) && DeletionResource.isDeleteNodeGeneratedInLocalByIoTV2(node)) { + deletionResource = manager.registerDeletionResource(node); // if persist failed, skip sending/publishing this event to keep consistency with the // behavior of storage engine. - if (deletionResource.waitForResult() == Status.FAILURE) { + if (deletionResource.waitForResult() == DeletionResource.Status.FAILURE) { return deletionResource; } + } else { + deletionResource = null; } - assigner.publishToAssign(realtimeEvent); + + assigner.publishToAssign(PipeRealtimeEventFactory.createRealtimeEvent(node)); + return deletionResource; } + public void listenToHeartbeat(boolean shouldPrintMessage) { + dataRegionId2Assigner.forEach( + (key, value) -> + value.publishToAssign( + PipeRealtimeEventFactory.createRealtimeEvent(key, shouldPrintMessage))); + } + /////////////////////////////// singleton /////////////////////////////// private PipeInsertionDataNodeListener() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java index f91d285d6989..0f729d83c30f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java @@ -2324,7 +2324,7 @@ public void deleteByDevice(MeasurementPath pattern, DeleteDataNode node) throws deleteDataInFiles(unsealedTsFileResource, deletion, devicePaths, deviceMatchInfo); // capture deleteDataNode and wait it to be persisted to DAL. DeletionResource deletionResource = - PipeInsertionDataNodeListener.getInstance().listenToDeleteData(node, dataRegionId); + PipeInsertionDataNodeListener.getInstance().listenToDeleteData(dataRegionId, node); // just get result. We have already waited for result in `listenToDeleteData` if (deletionResource != null && deletionResource.waitForResult() == Status.FAILURE) { throw deletionResource.getCause(); @@ -2374,7 +2374,7 @@ public void deleteDataDirectly(MeasurementPath pathToDelete, DeleteDataNode node deleteDataDirectlyInFile(unsealedTsFileResource, pathToDelete, startTime, endTime); // capture deleteDataNode and wait it to be persisted to DAL. DeletionResource deletionResource = - PipeInsertionDataNodeListener.getInstance().listenToDeleteData(node, dataRegionId); + PipeInsertionDataNodeListener.getInstance().listenToDeleteData(dataRegionId, node); // just get result. We have already waited for result in `listenToDeleteData` if (deletionResource != null && deletionResource.waitForResult() == Status.FAILURE) { throw deletionResource.getCause(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java index 028f08057135..c629103f8b09 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java @@ -197,7 +197,7 @@ public void testWaitForResult() throws Exception { new RecoverProgressIndex(THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, 1))); DeletionResource deletionResource = PipeInsertionDataNodeListener.getInstance() - .listenToDeleteData(deleteDataNode, FAKE_DATA_REGION_IDS[4]); + .listenToDeleteData(FAKE_DATA_REGION_IDS[4], deleteDataNode); Assert.assertSame(deletionResource.waitForResult(), Status.SUCCESS); } } From 004663ed654c951ffa62e7aca20d17acda28d390 Mon Sep 17 00:00:00 2001 From: Steve Yurong Su Date: Mon, 30 Sep 2024 17:03:41 +0800 Subject: [PATCH 68/85] Update PipeDataRegionAssigner.java --- .../assigner/PipeDataRegionAssigner.java | 33 ++++++++++--------- 1 file changed, 17 insertions(+), 16 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java index a7853eb14ab2..abc81c318a31 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java @@ -136,23 +136,8 @@ public void assignToExtractor( extractor.getPipePattern(), extractor.getRealtimeDataExtractionStartTime(), extractor.getRealtimeDataExtractionEndTime()); - // Log deletion event to DAL - if (copiedEvent.getEvent() instanceof PipeDeleteDataNodeEvent) { - DeletionResourceManager mgr = - DeletionResourceManager.getInstance(extractor.getDataRegionId()); - PipeDeleteDataNodeEvent deleteDataNodeEvent = - (PipeDeleteDataNodeEvent) copiedEvent.getEvent(); - // increase deletion resource's reference - if (Objects.nonNull(mgr) - && DeletionResource.isDeleteNodeGeneratedInLocalByIoTV2( - deleteDataNodeEvent.getDeleteDataNode())) { - deleteDataNodeEvent.setDeletionResource( - mgr.increaseResourceReferenceAndGet( - ((PipeDeleteDataNodeEvent) event.getEvent()).getDeleteDataNode())); - } - } - final EnrichedEvent innerEvent = copiedEvent.getEvent(); + if (innerEvent instanceof PipeTsFileInsertionEvent) { final PipeTsFileInsertionEvent tsFileInsertionEvent = (PipeTsFileInsertionEvent) innerEvent; @@ -161,6 +146,22 @@ public void assignToExtractor( bindOrUpdateProgressIndexForTsFileInsertionEvent(tsFileInsertionEvent); } + if (innerEvent instanceof PipeDeleteDataNodeEvent) { + final PipeDeleteDataNodeEvent deleteDataNodeEvent = + (PipeDeleteDataNodeEvent) innerEvent; + final DeletionResourceManager manager = + DeletionResourceManager.getInstance(extractor.getDataRegionId()); + // increase deletion resource's reference + if (Objects.nonNull(manager) + && DeletionResource.isDeleteNodeGeneratedInLocalByIoTV2( + deleteDataNodeEvent.getDeleteDataNode())) { + // Log deletion event to DAL + deleteDataNodeEvent.setDeletionResource( + manager.increaseResourceReferenceAndGet( + ((PipeDeleteDataNodeEvent) event.getEvent()).getDeleteDataNode())); + } + } + if (!copiedEvent.increaseReferenceCount(PipeDataRegionAssigner.class.getName())) { LOGGER.warn( "The reference count of the event {} cannot be increased, skipping it.", From 1fb8853b59039b7ce31a1624d6d47a7ff05a7f3d Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Mon, 30 Sep 2024 11:15:17 +0800 Subject: [PATCH 69/85] add comment --- .../db/pipe/consensus/deletion/DeletionResource.java | 8 ++++++++ .../realtime/assigner/PipeDataRegionAssigner.java | 3 +-- .../commons/pipe/datastructure/PersistentResource.java | 2 ++ 3 files changed, 11 insertions(+), 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java index bad71da73b35..bace950a10d1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java @@ -110,11 +110,19 @@ public ProgressIndex getProgressIndex() { return deleteDataNode.getProgressIndex(); } + /** + * Only the default implementation is provided here, which will not be called in practice, but is + * only used to implement the interface + */ @Override public long getFileStartTime() { return 0; } + /** + * Only the default implementation is provided here, which will not be called in practice, but is + * only used to implement the interface + */ @Override public long getFileEndTime() { return 0; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java index a7853eb14ab2..b69233673550 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java @@ -136,13 +136,12 @@ public void assignToExtractor( extractor.getPipePattern(), extractor.getRealtimeDataExtractionStartTime(), extractor.getRealtimeDataExtractionEndTime()); - // Log deletion event to DAL if (copiedEvent.getEvent() instanceof PipeDeleteDataNodeEvent) { DeletionResourceManager mgr = DeletionResourceManager.getInstance(extractor.getDataRegionId()); PipeDeleteDataNodeEvent deleteDataNodeEvent = (PipeDeleteDataNodeEvent) copiedEvent.getEvent(); - // increase deletion resource's reference + // increase deletion resource's reference and bind real deletionEvent if (Objects.nonNull(mgr) && DeletionResource.isDeleteNodeGeneratedInLocalByIoTV2( deleteDataNodeEvent.getDeleteDataNode())) { diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/PersistentResource.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/PersistentResource.java index 246f6b262790..8d562f7552c7 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/PersistentResource.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/pipe/datastructure/PersistentResource.java @@ -24,7 +24,9 @@ public interface PersistentResource { ProgressIndex getProgressIndex(); + /** Used in PipeHistoricalDataRegionTsFileAndDeletionExtractor to sort PersistentResource */ long getFileStartTime(); + /** Used in PipeHistoricalDataRegionTsFileAndDeletionExtractor to sort PersistentResource */ long getFileEndTime(); } From 26443e845e35a60d1fe0b46706dce429c42f277e Mon Sep 17 00:00:00 2001 From: Steve Yurong Su Date: Mon, 30 Sep 2024 17:25:14 +0800 Subject: [PATCH 70/85] Update PipeHistoricalDataRegionTsFileAndDeletionExtractor.java --- ...lDataRegionTsFileAndDeletionExtractor.java | 135 +++++++++--------- 1 file changed, 66 insertions(+), 69 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java index a279b1b7f00e..e4f9ba03ad1a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java @@ -373,23 +373,52 @@ private void flushDataRegionAllTsFiles() { } } - private void extractDeletions( - final DeletionResourceManager deletionResourceManager, - final List resourceList) { - LOGGER.info("Pipe {}@{}: start to extract deletions", pipeName, dataRegionId); - List allDeletionResources = deletionResourceManager.getAllDeletionResources(); - final int originalDeletionCount = allDeletionResources.size(); - allDeletionResources = - allDeletionResources.stream() - .filter(this::mayDeletionUnprocessed) - .collect(Collectors.toList()); - resourceList.addAll(allDeletionResources); - LOGGER.info( - "Pipe {}@{}: finish to extract deletions, extract deletions count {}/{}", - pipeName, - dataRegionId, - resourceList.size(), - originalDeletionCount); + @Override + public synchronized void start() { + if (!shouldExtractInsertion) { + hasBeenStarted = true; + return; + } + if (!StorageEngine.getInstance().isReadyForNonReadWriteFunctions()) { + LOGGER.info( + "Pipe {}@{}: failed to start to extract historical TsFile, storage engine is not ready. Will retry later.", + pipeName, + dataRegionId); + return; + } + hasBeenStarted = true; + + final DataRegion dataRegion = + StorageEngine.getInstance().getDataRegion(new DataRegionId(dataRegionId)); + if (Objects.isNull(dataRegion)) { + pendingQueue = new ArrayDeque<>(); + return; + } + + dataRegion.writeLock( + "Pipe: start to extract historical TsFile and Deletion(if uses pipeConsensus)"); + try { + List resourceList = new ArrayList<>(); + // Flush TsFiles + final long startHistoricalExtractionTime = System.currentTimeMillis(); + flushTsFilesForExtraction(dataRegion, startHistoricalExtractionTime); + // Extract TsFiles + extractTsFiles(dataRegion, startHistoricalExtractionTime, resourceList); + + // Extract deletions + Optional.ofNullable(DeletionResourceManager.getInstance(String.valueOf(dataRegionId))) + .ifPresent(mgr -> extractDeletions(mgr, resourceList)); + + // Sort tsFileResource and deletionResource + resourceList.sort( + (o1, o2) -> + startIndex instanceof TimeWindowStateProgressIndex + ? Long.compare(o1.getFileStartTime(), o2.getFileStartTime()) + : o1.getProgressIndex().topologicalCompareTo(o2.getProgressIndex())); + pendingQueue = new ArrayDeque<>(resourceList); + } finally { + dataRegion.writeUnlock(); + } } private void flushTsFilesForExtraction( @@ -492,58 +521,6 @@ && mayTsFileResourceOverlappedWithPattern(resource)) } } - @Override - public synchronized void start() { - if (!shouldExtractInsertion) { - hasBeenStarted = true; - return; - } - if (!StorageEngine.getInstance().isReadyForNonReadWriteFunctions()) { - LOGGER.info( - "Pipe {}@{}: failed to start to extract historical TsFile, storage engine is not ready. Will retry later.", - pipeName, - dataRegionId); - return; - } - hasBeenStarted = true; - - final DataRegion dataRegion = - StorageEngine.getInstance().getDataRegion(new DataRegionId(dataRegionId)); - if (Objects.isNull(dataRegion)) { - pendingQueue = new ArrayDeque<>(); - return; - } - - dataRegion.writeLock( - "Pipe: start to extract historical TsFile and Deletion(if uses pipeConsensus)"); - try { - List resourceList = new ArrayList<>(); - // Flush TsFiles - final long startHistoricalExtractionTime = System.currentTimeMillis(); - flushTsFilesForExtraction(dataRegion, startHistoricalExtractionTime); - // Extract TsFiles - extractTsFiles(dataRegion, startHistoricalExtractionTime, resourceList); - - // Extract deletions - Optional.ofNullable(DeletionResourceManager.getInstance(String.valueOf(dataRegionId))) - .ifPresent(mgr -> extractDeletions(mgr, resourceList)); - - // Sort tsFileResource and deletionResource - resourceList.sort( - (o1, o2) -> - startIndex instanceof TimeWindowStateProgressIndex - ? Long.compare(o1.getFileStartTime(), o2.getFileStartTime()) - : o1.getProgressIndex().topologicalCompareTo(o2.getProgressIndex())); - pendingQueue = new ArrayDeque<>(resourceList); - } finally { - dataRegion.writeUnlock(); - } - } - - private boolean mayDeletionUnprocessed(final DeletionResource resource) { - return !startIndex.isAfter(resource.getProgressIndex()); - } - private boolean mayTsFileContainUnprocessedData(final TsFileResource resource) { if (startIndex instanceof TimeWindowStateProgressIndex) { // The resource is closed thus the TsFileResource#getFileEndTime() is safe to use @@ -620,6 +597,26 @@ private boolean isTsFileGeneratedAfterExtractionTimeLowerBound(final TsFileResou } } + private void extractDeletions( + final DeletionResourceManager deletionResourceManager, + final List resourceList) { + LOGGER.info("Pipe {}@{}: start to extract deletions", pipeName, dataRegionId); + List allDeletionResources = deletionResourceManager.getAllDeletionResources(); + final int originalDeletionCount = allDeletionResources.size(); + allDeletionResources = + allDeletionResources.stream() + // filter if deletion is unprocessed + .filter(resource -> !startIndex.isAfter(resource.getProgressIndex())) + .collect(Collectors.toList()); + resourceList.addAll(allDeletionResources); + LOGGER.info( + "Pipe {}@{}: finish to extract deletions, extract deletions count {}/{}", + pipeName, + dataRegionId, + resourceList.size(), + originalDeletionCount); + } + private Event supplyTsFileEvent(TsFileResource resource) { final PipeTsFileInsertionEvent event = new PipeTsFileInsertionEvent( From fa2a329521994791f0baabad511debfb2ec3e77a Mon Sep 17 00:00:00 2001 From: Steve Yurong Su Date: Mon, 30 Sep 2024 17:28:07 +0800 Subject: [PATCH 71/85] Update PipeHistoricalDataRegionTsFileAndDeletionExtractor.java --- ...lDataRegionTsFileAndDeletionExtractor.java | 40 +++++++++---------- 1 file changed, 20 insertions(+), 20 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java index e4f9ba03ad1a..ebc74009dd33 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java @@ -617,6 +617,26 @@ private void extractDeletions( originalDeletionCount); } + @Override + public synchronized Event supply() { + if (!hasBeenStarted && StorageEngine.getInstance().isReadyForNonReadWriteFunctions()) { + start(); + } + + if (Objects.isNull(pendingQueue)) { + return null; + } + + final PersistentResource resource = pendingQueue.poll(); + if (resource == null) { + return supplyTerminateEvent(); + } else if (resource instanceof TsFileResource) { + return supplyTsFileEvent((TsFileResource) resource); + } else { + return supplyDeletionEvent((DeletionResource) resource); + } + } + private Event supplyTsFileEvent(TsFileResource resource) { final PipeTsFileInsertionEvent event = new PipeTsFileInsertionEvent( @@ -708,26 +728,6 @@ private Event supplyTerminateEvent() { return terminateEvent; } - @Override - public synchronized Event supply() { - if (!hasBeenStarted && StorageEngine.getInstance().isReadyForNonReadWriteFunctions()) { - start(); - } - - if (Objects.isNull(pendingQueue)) { - return null; - } - - final PersistentResource resource = pendingQueue.poll(); - if (resource == null) { - return supplyTerminateEvent(); - } else if (resource instanceof TsFileResource) { - return supplyTsFileEvent((TsFileResource) resource); - } else { - return supplyDeletionEvent((DeletionResource) resource); - } - } - @Override public synchronized boolean hasConsumedAll() { // If the pendingQueues are null when the function is called, it implies that the extractor only From f194928499132fa7285cc2deb25898bf5404a046 Mon Sep 17 00:00:00 2001 From: Steve Yurong Su Date: Mon, 30 Sep 2024 17:30:11 +0800 Subject: [PATCH 72/85] Update PipeHistoricalDataRegionTsFileAndDeletionExtractor.java --- ...lDataRegionTsFileAndDeletionExtractor.java | 30 +++++++++---------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java index ebc74009dd33..73cde7e2d1d1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java @@ -637,6 +637,21 @@ public synchronized Event supply() { } } + private Event supplyTerminateEvent() { + final PipeTerminateEvent terminateEvent = + new PipeTerminateEvent(pipeName, creationTime, pipeTaskMeta, dataRegionId); + if (!terminateEvent.increaseReferenceCount( + PipeHistoricalDataRegionTsFileAndDeletionExtractor.class.getName())) { + LOGGER.warn( + "Pipe {}@{}: failed to increase reference count for terminate event, will resend it", + pipeName, + dataRegionId); + return null; + } + isTerminateSignalSent = true; + return terminateEvent; + } + private Event supplyTsFileEvent(TsFileResource resource) { final PipeTsFileInsertionEvent event = new PipeTsFileInsertionEvent( @@ -713,21 +728,6 @@ private Event supplyDeletionEvent(final DeletionResource deletionResource) { return isReferenceCountIncreased ? event : null; } - private Event supplyTerminateEvent() { - final PipeTerminateEvent terminateEvent = - new PipeTerminateEvent(pipeName, creationTime, pipeTaskMeta, dataRegionId); - if (!terminateEvent.increaseReferenceCount( - PipeHistoricalDataRegionTsFileAndDeletionExtractor.class.getName())) { - LOGGER.warn( - "Pipe {}@{}: failed to increase reference count for terminate event, will resend it", - pipeName, - dataRegionId); - return null; - } - isTerminateSignalSent = true; - return terminateEvent; - } - @Override public synchronized boolean hasConsumedAll() { // If the pendingQueues are null when the function is called, it implies that the extractor only From bae8bfe0cc0f662b01a6df48237e78c336d5b190 Mon Sep 17 00:00:00 2001 From: Steve Yurong Su Date: Mon, 30 Sep 2024 18:05:23 +0800 Subject: [PATCH 73/85] Update PipeHistoricalDataRegionTsFileAndDeletionExtractor.java --- ...lDataRegionTsFileAndDeletionExtractor.java | 23 ++++++++++--------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java index 73cde7e2d1d1..bb74bfaf24a1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java @@ -732,9 +732,10 @@ private Event supplyDeletionEvent(final DeletionResource deletionResource) { public synchronized boolean hasConsumedAll() { // If the pendingQueues are null when the function is called, it implies that the extractor only // extracts deletion thus the historical event has nothing to consume. - return hasBeenStarted && (Objects.isNull(pendingQueue)) - || pendingQueue.isEmpty() - && (!shouldTerminatePipeOnAllHistoricalEventsConsumed || isTerminateSignalSent); + return hasBeenStarted + && (Objects.isNull(pendingQueue) + || pendingQueue.isEmpty() + && (!shouldTerminatePipeOnAllHistoricalEventsConsumed || isTerminateSignalSent)); } @Override @@ -747,16 +748,16 @@ public synchronized void close() { if (Objects.nonNull(pendingQueue)) { pendingQueue.forEach( resource -> { - try { - if (resource instanceof TsFileResource) { + if (resource instanceof TsFileResource) { + try { PipeDataNodeResourceManager.tsfile().unpinTsFileResource((TsFileResource) resource); + } catch (final IOException e) { + LOGGER.warn( + "Pipe {}@{}: failed to unpin TsFileResource after dropping pipe, original path: {}", + pipeName, + dataRegionId, + ((TsFileResource) resource).getTsFilePath()); } - } catch (final IOException e) { - LOGGER.warn( - "Pipe {}@{}: failed to unpin TsFileResource after dropping pipe, original path: {}", - pipeName, - dataRegionId, - ((TsFileResource) resource).getTsFilePath()); } }); pendingQueue.clear(); From a9d13fe8ec955a54d1f7a6d46f28621785b604fc Mon Sep 17 00:00:00 2001 From: Steve Yurong Su Date: Mon, 30 Sep 2024 18:07:23 +0800 Subject: [PATCH 74/85] Update PipeHistoricalDataRegionTsFileAndDeletionExtractor.java --- .../PipeHistoricalDataRegionTsFileAndDeletionExtractor.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java index bb74bfaf24a1..74684ad62cec 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java @@ -125,9 +125,10 @@ public class PipeHistoricalDataRegionTsFileAndDeletionExtractor private boolean shouldTerminatePipeOnAllHistoricalEventsConsumed; private boolean isTerminateSignalSent = false; - private Queue pendingQueue; private volatile boolean hasBeenStarted = false; + private Queue pendingQueue; + @Override public void validate(final PipeParameterValidator validator) { final PipeParameters parameters = validator.getParameters(); @@ -730,7 +731,7 @@ private Event supplyDeletionEvent(final DeletionResource deletionResource) { @Override public synchronized boolean hasConsumedAll() { - // If the pendingQueues are null when the function is called, it implies that the extractor only + // If the pendingQueue is null when the function is called, it implies that the extractor only // extracts deletion thus the historical event has nothing to consume. return hasBeenStarted && (Objects.isNull(pendingQueue) From fe255f9d20310523df9981685c31838fcedc805a Mon Sep 17 00:00:00 2001 From: Steve Yurong Su Date: Mon, 30 Sep 2024 18:12:12 +0800 Subject: [PATCH 75/85] Update PipeHistoricalDataRegionTsFileAndDeletionExtractor.java --- .../PipeHistoricalDataRegionTsFileAndDeletionExtractor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java index 74684ad62cec..2e2eb04eb243 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java @@ -396,12 +396,12 @@ public synchronized void start() { return; } + final long startHistoricalExtractionTime = System.currentTimeMillis(); dataRegion.writeLock( "Pipe: start to extract historical TsFile and Deletion(if uses pipeConsensus)"); try { List resourceList = new ArrayList<>(); // Flush TsFiles - final long startHistoricalExtractionTime = System.currentTimeMillis(); flushTsFilesForExtraction(dataRegion, startHistoricalExtractionTime); // Extract TsFiles extractTsFiles(dataRegion, startHistoricalExtractionTime, resourceList); From d93ec4ce1bf0a54c6b7a71fe0a4d31df68c93e8f Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Tue, 8 Oct 2024 09:41:05 +0800 Subject: [PATCH 76/85] fix review --- ...lDataRegionTsFileAndDeletionExtractor.java | 42 ++++++++++++------- 1 file changed, 28 insertions(+), 14 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java index a279b1b7f00e..a86b556c24fb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java @@ -120,6 +120,7 @@ public class PipeHistoricalDataRegionTsFileAndDeletionExtractor private Pair listeningOptionPair; private boolean shouldExtractInsertion; + private boolean shouldExtractDeletion; private boolean shouldTransferModFile; // Whether to transfer mods private boolean shouldTerminatePipeOnAllHistoricalEventsConsumed; @@ -256,8 +257,9 @@ public void customize( final PipeParameters parameters, final PipeExtractorRuntimeConfiguration configuration) throws IllegalPathException { shouldExtractInsertion = listeningOptionPair.getLeft(); - // Do nothing if only extract deletion - if (!shouldExtractInsertion) { + shouldExtractDeletion = listeningOptionPair.getRight(); + // Do nothing if extract nothing + if (!shouldExtractInsertion && !shouldExtractDeletion) { return; } @@ -377,6 +379,7 @@ private void extractDeletions( final DeletionResourceManager deletionResourceManager, final List resourceList) { LOGGER.info("Pipe {}@{}: start to extract deletions", pipeName, dataRegionId); + long startTime = System.currentTimeMillis(); List allDeletionResources = deletionResourceManager.getAllDeletionResources(); final int originalDeletionCount = allDeletionResources.size(); allDeletionResources = @@ -385,11 +388,12 @@ private void extractDeletions( .collect(Collectors.toList()); resourceList.addAll(allDeletionResources); LOGGER.info( - "Pipe {}@{}: finish to extract deletions, extract deletions count {}/{}", + "Pipe {}@{}: finish to extract deletions, extract deletions count {}/{}, took {} ms", pipeName, dataRegionId, resourceList.size(), - originalDeletionCount); + originalDeletionCount, + System.currentTimeMillis() - startTime); } private void flushTsFilesForExtraction( @@ -494,7 +498,7 @@ && mayTsFileResourceOverlappedWithPattern(resource)) @Override public synchronized void start() { - if (!shouldExtractInsertion) { + if (!shouldExtractInsertion && !shouldExtractDeletion) { hasBeenStarted = true; return; } @@ -518,22 +522,32 @@ public synchronized void start() { "Pipe: start to extract historical TsFile and Deletion(if uses pipeConsensus)"); try { List resourceList = new ArrayList<>(); - // Flush TsFiles - final long startHistoricalExtractionTime = System.currentTimeMillis(); - flushTsFilesForExtraction(dataRegion, startHistoricalExtractionTime); - // Extract TsFiles - extractTsFiles(dataRegion, startHistoricalExtractionTime, resourceList); - - // Extract deletions - Optional.ofNullable(DeletionResourceManager.getInstance(String.valueOf(dataRegionId))) - .ifPresent(mgr -> extractDeletions(mgr, resourceList)); + if (shouldExtractInsertion) { + // Flush TsFiles + final long startHistoricalExtractionTime = System.currentTimeMillis(); + flushTsFilesForExtraction(dataRegion, startHistoricalExtractionTime); + // Extract TsFiles + extractTsFiles(dataRegion, startHistoricalExtractionTime, resourceList); + } + if (shouldExtractDeletion) { + // Extract deletions + Optional.ofNullable(DeletionResourceManager.getInstance(String.valueOf(dataRegionId))) + .ifPresent(mgr -> extractDeletions(mgr, resourceList)); + } // Sort tsFileResource and deletionResource + long startTime = System.currentTimeMillis(); + LOGGER.info("Pipe {}@{}: start to sort all extracted resources", pipeName, dataRegionId); resourceList.sort( (o1, o2) -> startIndex instanceof TimeWindowStateProgressIndex ? Long.compare(o1.getFileStartTime(), o2.getFileStartTime()) : o1.getProgressIndex().topologicalCompareTo(o2.getProgressIndex())); + LOGGER.info( + "Pipe {}@{}: finish to sort all extracted resources, took {} ms", + pipeName, + dataRegionId, + System.currentTimeMillis() - startTime); pendingQueue = new ArrayDeque<>(resourceList); } finally { dataRegion.writeUnlock(); From a8d51d273463622d28c2413701944b8c44f7df32 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Tue, 8 Oct 2024 09:48:31 +0800 Subject: [PATCH 77/85] merge --- ...lDataRegionTsFileAndDeletionExtractor.java | 97 +++++++++---------- 1 file changed, 48 insertions(+), 49 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java index 0fc43e397503..57bc9aa9c6db 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java @@ -259,8 +259,8 @@ public void customize( throws IllegalPathException { shouldExtractInsertion = listeningOptionPair.getLeft(); shouldExtractDeletion = listeningOptionPair.getRight(); - // Do nothing if extract nothing - if (!shouldExtractInsertion && !shouldExtractDeletion) { + // Do nothing if extract deletion + if (!shouldExtractInsertion) { return; } @@ -378,7 +378,7 @@ private void flushDataRegionAllTsFiles() { @Override public synchronized void start() { - if (!shouldExtractInsertion && !shouldExtractDeletion) { + if (!shouldExtractInsertion) { hasBeenStarted = true; return; } @@ -403,34 +403,33 @@ public synchronized void start() { "Pipe: start to extract historical TsFile and Deletion(if uses pipeConsensus)"); try { List resourceList = new ArrayList<>(); - if (shouldExtractInsertion) { - // Flush TsFiles - final long startHistoricalExtractionTime = System.currentTimeMillis(); - flushTsFilesForExtraction(dataRegion, startHistoricalExtractionTime); - // Extract TsFiles - extractTsFiles(dataRegion, startHistoricalExtractionTime, resourceList); - } + if (shouldExtractInsertion) { + // Flush TsFiles + flushTsFilesForExtraction(dataRegion, startHistoricalExtractionTime); + // Extract TsFiles + extractTsFiles(dataRegion, startHistoricalExtractionTime, resourceList); + } - if (shouldExtractDeletion) { - // Extract deletions - Optional.ofNullable(DeletionResourceManager.getInstance(String.valueOf(dataRegionId))) - .ifPresent(mgr -> extractDeletions(mgr, resourceList)); - } - // Sort tsFileResource and deletionResource - long startTime = System.currentTimeMillis(); - LOGGER.info("Pipe {}@{}: start to sort all extracted resources", pipeName, dataRegionId); - resourceList.sort( - (o1, o2) -> - startIndex instanceof TimeWindowStateProgressIndex - ? Long.compare(o1.getFileStartTime(), o2.getFileStartTime()) - : o1.getProgressIndex().topologicalCompareTo(o2.getProgressIndex())); - LOGGER.info( - "Pipe {}@{}: finish to sort all extracted resources, took {} ms", - pipeName, - dataRegionId, - System.currentTimeMillis() - startTime); + if (shouldExtractDeletion) { + // Extract deletions + Optional.ofNullable(DeletionResourceManager.getInstance(String.valueOf(dataRegionId))) + .ifPresent(mgr -> extractDeletions(mgr, resourceList)); + } + // Sort tsFileResource and deletionResource + long startTime = System.currentTimeMillis(); + LOGGER.info("Pipe {}@{}: start to sort all extracted resources", pipeName, dataRegionId); + resourceList.sort( + (o1, o2) -> + startIndex instanceof TimeWindowStateProgressIndex + ? Long.compare(o1.getFileStartTime(), o2.getFileStartTime()) + : o1.getProgressIndex().topologicalCompareTo(o2.getProgressIndex())); + LOGGER.info( + "Pipe {}@{}: finish to sort all extracted resources, took {} ms", + pipeName, + dataRegionId, + System.currentTimeMillis() - startTime); - pendingQueue = new ArrayDeque<>(resourceList); + pendingQueue = new ArrayDeque<>(resourceList); } finally { dataRegion.writeUnlock(); } @@ -612,26 +611,26 @@ private boolean isTsFileGeneratedAfterExtractionTimeLowerBound(final TsFileResou } } - private void extractDeletions( - final DeletionResourceManager deletionResourceManager, - final List resourceList) { - LOGGER.info("Pipe {}@{}: start to extract deletions", pipeName, dataRegionId); - long startTime = System.currentTimeMillis(); - List allDeletionResources = deletionResourceManager.getAllDeletionResources(); - final int originalDeletionCount = allDeletionResources.size(); - allDeletionResources = - allDeletionResources.stream() - .filter(resource -> !startIndex.isAfter(resource.getProgressIndex())) - .collect(Collectors.toList()); - resourceList.addAll(allDeletionResources); - LOGGER.info( - "Pipe {}@{}: finish to extract deletions, extract deletions count {}/{}, took {} ms", - pipeName, - dataRegionId, - resourceList.size(), - originalDeletionCount, - System.currentTimeMillis() - startTime); - } + private void extractDeletions( + final DeletionResourceManager deletionResourceManager, + final List resourceList) { + LOGGER.info("Pipe {}@{}: start to extract deletions", pipeName, dataRegionId); + long startTime = System.currentTimeMillis(); + List allDeletionResources = deletionResourceManager.getAllDeletionResources(); + final int originalDeletionCount = allDeletionResources.size(); + allDeletionResources = + allDeletionResources.stream() + .filter(resource -> !startIndex.isAfter(resource.getProgressIndex())) + .collect(Collectors.toList()); + resourceList.addAll(allDeletionResources); + LOGGER.info( + "Pipe {}@{}: finish to extract deletions, extract deletions count {}/{}, took {} ms", + pipeName, + dataRegionId, + resourceList.size(), + originalDeletionCount, + System.currentTimeMillis() - startTime); + } @Override public synchronized Event supply() { From bf7982e7555513fe4ced77ca68e05fa58d9615e6 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Fri, 11 Oct 2024 16:56:18 +0800 Subject: [PATCH 78/85] fix review: manage deletion resource's reference count when restart/historical extraction --- ...ipeHistoricalDataRegionTsFileAndDeletionExtractor.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java index 57bc9aa9c6db..28cecacb922f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java @@ -413,7 +413,7 @@ public synchronized void start() { if (shouldExtractDeletion) { // Extract deletions Optional.ofNullable(DeletionResourceManager.getInstance(String.valueOf(dataRegionId))) - .ifPresent(mgr -> extractDeletions(mgr, resourceList)); + .ifPresent(manager -> extractDeletions(manager, resourceList)); } // Sort tsFileResource and deletionResource long startTime = System.currentTimeMillis(); @@ -739,6 +739,12 @@ private Event supplyDeletionEvent(final DeletionResource deletionResource) { pipeName, dataRegionId, event); + } else { + Optional.ofNullable(DeletionResourceManager.getInstance(String.valueOf(dataRegionId))) + .ifPresent( + manager -> + event.setDeletionResource( + manager.increaseResourceReferenceAndGet(event.getDeleteDataNode()))); } return isReferenceCountIncreased ? event : null; } From 01d3cb908de8196bbdc833d44bb0660fb1647f2f Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Fri, 11 Oct 2024 17:19:45 +0800 Subject: [PATCH 79/85] adjust table model for PipeDeleteDataNodeEvent & spotless --- .../deletion/PipeDeleteDataNodeEvent.java | 28 +++++++++++++++---- .../realtime/PipeRealtimeEventFactory.java | 2 +- ...lDataRegionTsFileAndDeletionExtractor.java | 6 ++-- .../PipeInsertionDataNodeListener.java | 2 +- .../matcher/CachedSchemaPatternMatcher.java | 3 +- 5 files changed, 28 insertions(+), 13 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java index 6110d511a569..26f2d04a4465 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java @@ -22,7 +22,8 @@ import org.apache.iotdb.commons.consensus.index.ProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.MinimumProgressIndex; import org.apache.iotdb.commons.pipe.agent.task.meta.PipeTaskMeta; -import org.apache.iotdb.commons.pipe.datastructure.pattern.PipePattern; +import org.apache.iotdb.commons.pipe.datastructure.pattern.TablePattern; +import org.apache.iotdb.commons.pipe.datastructure.pattern.TreePattern; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; import org.apache.iotdb.commons.pipe.event.SerializableEvent; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; @@ -47,7 +48,7 @@ public PipeDeleteDataNodeEvent() { public PipeDeleteDataNodeEvent( final DeleteDataNode deleteDataNode, final boolean isGeneratedByPipe) { - this(deleteDataNode, null, 0, null, null, isGeneratedByPipe); + this(deleteDataNode, null, 0, null, null, null, isGeneratedByPipe); } public PipeDeleteDataNodeEvent( @@ -55,9 +56,17 @@ public PipeDeleteDataNodeEvent( final String pipeName, final long creationTime, final PipeTaskMeta pipeTaskMeta, - final PipePattern pattern, + final TreePattern treePattern, + final TablePattern tablePattern, final boolean isGeneratedByPipe) { - super(pipeName, creationTime, pipeTaskMeta, pattern, Long.MIN_VALUE, Long.MAX_VALUE); + super( + pipeName, + creationTime, + pipeTaskMeta, + treePattern, + tablePattern, + Long.MIN_VALUE, + Long.MAX_VALUE); this.isGeneratedByPipe = isGeneratedByPipe; this.deleteDataNode = deleteDataNode; Optional.ofNullable(deleteDataNode) @@ -99,11 +108,18 @@ public EnrichedEvent shallowCopySelfAndBindPipeTaskMetaForProgressReport( String pipeName, long creationTime, PipeTaskMeta pipeTaskMeta, - PipePattern pattern, + TreePattern treePattern, + TablePattern tablePattern, long startTime, long endTime) { return new PipeDeleteDataNodeEvent( - deleteDataNode, pipeName, creationTime, pipeTaskMeta, pattern, isGeneratedByPipe); + deleteDataNode, + pipeName, + creationTime, + pipeTaskMeta, + treePattern, + tablePattern, + isGeneratedByPipe); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java index 0514544edb16..1e3716b63041 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/realtime/PipeRealtimeEventFactory.java @@ -69,7 +69,7 @@ public static PipeRealtimeEvent createRealtimeEvent( public static PipeRealtimeEvent createRealtimeEvent(final DeleteDataNode node) { return new PipeRealtimeEvent( - new PipeDeleteDataNodeEvent(node, node.isGeneratedByPipe()), null, null, null); + new PipeDeleteDataNodeEvent(node, node.isGeneratedByPipe()), null, null, null, null); } public static PipeRealtimeEvent createRealtimeEvent(final ProgressReportEvent event) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java index d0ec416dbc20..c584043db9fc 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java @@ -28,10 +28,9 @@ import org.apache.iotdb.commons.pipe.config.constant.PipeExtractorConstant; import org.apache.iotdb.commons.pipe.config.constant.SystemConstant; import org.apache.iotdb.commons.pipe.config.plugin.env.PipeTaskExtractorRuntimeEnvironment; +import org.apache.iotdb.commons.pipe.datastructure.PersistentResource; import org.apache.iotdb.commons.pipe.datastructure.pattern.TablePattern; import org.apache.iotdb.commons.pipe.datastructure.pattern.TreePattern; -import org.apache.iotdb.commons.pipe.datastructure.PersistentResource; -import org.apache.iotdb.commons.pipe.datastructure.pattern.PipePattern; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResource; import org.apache.iotdb.db.pipe.consensus.deletion.DeletionResourceManager; import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; @@ -729,7 +728,8 @@ private Event supplyDeletionEvent(final DeletionResource deletionResource) { pipeName, creationTime, pipeTaskMeta, - pipePattern, + treePattern, + tablePattern, false); if (sloppyPattern || isDbNameCoveredByPattern) { event.skipParsingPattern(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java index 48a189a765ee..b0736820bb3d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/listener/PipeInsertionDataNodeListener.java @@ -140,7 +140,7 @@ public void listenToInsertNode( databaseName, walEntryHandler, insertNode, tsFileResource)); } - // TODO: record database name in enriched events? + // TODO: record database name in enriched events? public DeletionResource listenToDeleteData(final String regionId, final DeleteDataNode node) { final PipeDataRegionAssigner assigner = dataRegionId2Assigner.get(regionId); // only events from registered data region will be extracted diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/matcher/CachedSchemaPatternMatcher.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/matcher/CachedSchemaPatternMatcher.java index 017b83303544..59e261f0d782 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/matcher/CachedSchemaPatternMatcher.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/matcher/CachedSchemaPatternMatcher.java @@ -20,11 +20,10 @@ package org.apache.iotdb.db.pipe.extractor.dataregion.realtime.matcher; import org.apache.iotdb.commons.pipe.config.PipeConfig; -import org.apache.iotdb.commons.pipe.datastructure.pattern.PipePattern; -import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.apache.iotdb.commons.pipe.datastructure.pattern.TablePattern; import org.apache.iotdb.commons.pipe.datastructure.pattern.TreePattern; import org.apache.iotdb.commons.pipe.event.PipeInsertionEvent; +import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.apache.iotdb.db.pipe.event.common.heartbeat.PipeHeartbeatEvent; import org.apache.iotdb.db.pipe.event.realtime.PipeRealtimeEvent; import org.apache.iotdb.db.pipe.extractor.dataregion.realtime.PipeRealtimeDataRegionExtractor; From a3368c51c6b661466a176f441bda6f89bf01aaff Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Sat, 12 Oct 2024 15:42:15 +0800 Subject: [PATCH 80/85] fix review: improve ut and reference count --- .../consensus/deletion/DeletionResource.java | 24 ++++++++++------ .../deletion/DeletionResourceManager.java | 14 ++++------ .../persist/PageCacheDeletionBuffer.java | 15 +--------- .../deletion/recover/DeletionReader.java | 8 ++++-- ...lDataRegionTsFileAndDeletionExtractor.java | 2 +- .../assigner/PipeDataRegionAssigner.java | 2 +- .../pipe/consensus/DeletionRecoverTest.java | 9 ++++++ .../pipe/consensus/DeletionResourceTest.java | 28 ++++++++++++------- 8 files changed, 58 insertions(+), 44 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java index bace950a10d1..1c7820a24d04 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java @@ -19,10 +19,13 @@ package org.apache.iotdb.db.pipe.consensus.deletion; +import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType; +import org.apache.iotdb.commons.consensus.ConsensusGroupId; import org.apache.iotdb.commons.consensus.index.ProgressIndex; import org.apache.iotdb.commons.consensus.index.impl.RecoverProgressIndex; import org.apache.iotdb.commons.pipe.datastructure.PersistentResource; import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.consensus.DataRegionConsensusImpl; import org.apache.iotdb.db.pipe.event.common.deletion.PipeDeleteDataNodeEvent; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; @@ -43,22 +46,26 @@ public class DeletionResource implements PersistentResource { private static final Logger LOGGER = LoggerFactory.getLogger(DeletionResource.class); private final Consumer removeHook; - private final AtomicInteger pipeTaskReferenceCount = new AtomicInteger(0); + private final AtomicInteger pipeTaskReferenceCount; private final DeleteDataNode deleteDataNode; + private final ConsensusGroupId consensusGroupId; private volatile Status currentStatus; // it's safe to use volatile here to make this reference thread-safe. @SuppressWarnings("squid:S3077") private volatile Exception cause; - public DeletionResource(DeleteDataNode deleteDataNode, Consumer removeHook) { + public DeletionResource( + DeleteDataNode deleteDataNode, Consumer removeHook, String regionId) { this.deleteDataNode = deleteDataNode; this.removeHook = removeHook; this.currentStatus = Status.RUNNING; - } - - public void increaseReference() { - pipeTaskReferenceCount.incrementAndGet(); + this.consensusGroupId = + ConsensusGroupId.Factory.create( + TConsensusGroupType.DataRegion.getValue(), Integer.parseInt(regionId)); + this.pipeTaskReferenceCount = + new AtomicInteger( + DataRegionConsensusImpl.getInstance().getReplicationNum(consensusGroupId)); } public synchronized void decreaseReference() { @@ -140,9 +147,10 @@ public ByteBuffer serialize() { } public static DeletionResource deserialize( - final ByteBuffer buffer, final Consumer removeHook) throws IOException { + final ByteBuffer buffer, final String regionId, final Consumer removeHook) + throws IOException { DeleteDataNode node = DeleteDataNode.deserializeFromDAL(buffer); - return new DeletionResource(node, removeHook); + return new DeletionResource(node, removeHook, regionId); } public static boolean isDeleteNodeGeneratedInLocalByIoTV2(DeleteDataNode node) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java index f9d18f0ea9f2..8459b3d3f1ee 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResourceManager.java @@ -43,7 +43,6 @@ import java.nio.file.Paths; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.Lock; @@ -104,7 +103,7 @@ private void initAndRecover() throws IOException { for (Path path : deletionPaths) { try (DeletionReader deletionReader = - new DeletionReader(path.toFile(), this::removeDeletionResource)) { + new DeletionReader(path.toFile(), dataRegionId, this::removeDeletionResource)) { deletionReader .readAllDeletions() .forEach( @@ -138,16 +137,15 @@ public DeletionResource registerDeletionResource(DeleteDataNode deleteDataNode) DeletionResource deletionResource = deleteNode2ResourcesMap.computeIfAbsent( deleteDataNode, - key -> new DeletionResource(deleteDataNode, this::removeDeletionResource)); + key -> + new DeletionResource(deleteDataNode, this::removeDeletionResource, dataRegionId)); // register a persist task for current deletionResource deletionBuffer.registerDeletionResource(deletionResource); return deletionResource; } - public DeletionResource increaseResourceReferenceAndGet(DeleteDataNode deleteDataNode) { - DeletionResource deletionResource = deleteNode2ResourcesMap.get(deleteDataNode); - Optional.ofNullable(deletionResource).ifPresent(DeletionResource::increaseReference); - return deletionResource; + public DeletionResource getDeletionResource(DeleteDataNode deleteDataNode) { + return deleteNode2ResourcesMap.get(deleteDataNode); } public List getAllDeletionResources() { @@ -317,7 +315,7 @@ public void recoverForTest() { for (Path path : deletionPaths) { try (DeletionReader deletionReader = - new DeletionReader(path.toFile(), this::removeDeletionResource)) { + new DeletionReader(path.toFile(), dataRegionId, this::removeDeletionResource)) { deletionReader .readAllDeletions() .forEach( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java index 4905f4d26413..ab8039ce150b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/persist/PageCacheDeletionBuffer.java @@ -258,7 +258,7 @@ public void close() { // first waiting serialize and sync tasks finished, then release all resources waitUntilFlushAllDeletionsOrTimeOut(); if (persistThread != null) { - shutdownThread(persistThread); + persistThread.shutdown(); } // clean buffer MmapUtil.clean(serializeBuffer); @@ -277,19 +277,6 @@ private void waitUntilFlushAllDeletionsOrTimeOut() { } } - private void shutdownThread(ExecutorService thread) { - ThreadName threadName = ThreadName.PIPE_CONSENSUS_DELETION_SERIALIZE; - thread.shutdown(); - try { - if (!thread.awaitTermination(30, TimeUnit.SECONDS)) { - LOGGER.warn("Waiting thread {} to be terminated is timeout", threadName.getName()); - } - } catch (InterruptedException e) { - LOGGER.warn("Thread {} still doesn't exit after 30s", threadName.getName()); - Thread.currentThread().interrupt(); - } - } - private class PersistTask implements Runnable { // Batch size in current task, used to roll back. private final AtomicInteger currentTaskBatchSize = new AtomicInteger(0); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java index 51ab624e7d19..22c4b7ca41d8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/recover/DeletionReader.java @@ -40,13 +40,16 @@ public class DeletionReader implements Closeable { private static final Logger LOGGER = LoggerFactory.getLogger(DeletionReader.class); private static final int MAGIC_STRING_BYTES_SIZE = DeletionResourceManager.MAGIC_VERSION_V1.getBytes(StandardCharsets.UTF_8).length; + private final String regionId; private final Consumer removeHook; private final File logFile; private final FileInputStream fileInputStream; private final FileChannel fileChannel; - public DeletionReader(File logFile, Consumer removeHook) throws IOException { + public DeletionReader(File logFile, String regionId, Consumer removeHook) + throws IOException { this.logFile = logFile; + this.regionId = regionId; this.fileInputStream = new FileInputStream(logFile); this.fileChannel = fileInputStream.getChannel(); this.removeHook = removeHook; @@ -72,7 +75,8 @@ public List readAllDeletions() throws IOException { List deletions = new ArrayList<>(); while (byteBuffer.hasRemaining()) { - DeletionResource deletionResource = DeletionResource.deserialize(byteBuffer, removeHook); + DeletionResource deletionResource = + DeletionResource.deserialize(byteBuffer, regionId, removeHook); deletions.add(deletionResource); if (LOGGER.isDebugEnabled()) { LOGGER.debug("Read deletion: {} from file {}", deletionResource, logFile); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java index c584043db9fc..e2bc2f142885 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java @@ -752,7 +752,7 @@ private Event supplyDeletionEvent(final DeletionResource deletionResource) { .ifPresent( manager -> event.setDeletionResource( - manager.increaseResourceReferenceAndGet(event.getDeleteDataNode()))); + manager.getDeletionResource(event.getDeleteDataNode()))); } return isReferenceCountIncreased ? event : null; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java index 7f293c52b792..ae2530679c47 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/realtime/assigner/PipeDataRegionAssigner.java @@ -158,7 +158,7 @@ public void assignToExtractor( && DeletionResource.isDeleteNodeGeneratedInLocalByIoTV2( deleteDataNodeEvent.getDeleteDataNode())) { deleteDataNodeEvent.setDeletionResource( - manager.increaseResourceReferenceAndGet( + manager.getDeletionResource( ((PipeDeleteDataNodeEvent) event.getEvent()).getDeleteDataNode())); } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java index 3cb2209e506a..cf23f763d201 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionRecoverTest.java @@ -28,6 +28,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; +import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -67,6 +68,14 @@ public void setUp() throws Exception { deletionResourceManager.close(); } + @After + public void tearDown() throws Exception { + File baseDir = new File(DELETION_BASE_DIR + File.separator + FAKE_DATA_REGION_ID); + if (baseDir.exists()) { + FileUtils.deleteFileOrDirectory(baseDir); + } + } + @Test public void testDeletionRecover() throws Exception { Assert.assertEquals(0, deletionResourceManager.getAllDeletionResources().size()); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java index c629103f8b09..1b62d4b63a1e 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/DeletionResourceTest.java @@ -71,12 +71,13 @@ public void setUp() throws Exception { @After public void tearDown() throws Exception { - for (String FAKE_DATE_REGION_ID : FAKE_DATA_REGION_IDS) { - File baseDir = new File(DELETION_BASE_DIR + File.separator + FAKE_DATE_REGION_ID); + for (String FAKE_DATA_REGION_ID : FAKE_DATA_REGION_IDS) { + File baseDir = new File(DELETION_BASE_DIR + File.separator + FAKE_DATA_REGION_ID); if (baseDir.exists()) { FileUtils.deleteFileOrDirectory(baseDir); } } + deletionResourceManager.close(); } @Test @@ -119,9 +120,11 @@ public void testAddDeletionResourceTimeout() deleteDataNode.setProgressIndex( new RecoverProgressIndex(THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, 1))); // Only register one deletionResource - deletionResourceManager.registerDeletionResource(deleteDataNode); - // Sleep to wait deletion being persisted - Thread.sleep(5000); + DeletionResource deletionResource = + deletionResourceManager.registerDeletionResource(deleteDataNode); + if (deletionResource.waitForResult() != Status.SUCCESS) { + Assert.fail(); + } Stream paths = Files.list(Paths.get(DELETION_BASE_DIR + File.separator + FAKE_DATA_REGION_IDS[2])); Assert.assertTrue(paths.anyMatch(Files::isRegularFile)); @@ -142,13 +145,15 @@ public void testDeletionRemove() throws IllegalPathException, InterruptedExcepti new RecoverProgressIndex(THIS_DATANODE_ID, new SimpleProgressIndex(rebootTimes, i))); PipeDeleteDataNodeEvent deletionEvent = new PipeDeleteDataNodeEvent(deleteDataNode, true); deletionEvents.add(deletionEvent); - deletionResourceManager.registerDeletionResource(deleteDataNode); + DeletionResource deletionResource = + deletionResourceManager.registerDeletionResource(deleteDataNode); deletionEvent.setDeletionResource( - deletionResourceManager.increaseResourceReferenceAndGet(deleteDataNode)); + deletionResourceManager.getDeletionResource(deleteDataNode)); + if (deletionResource.waitForResult() != Status.SUCCESS) { + Assert.fail(); + } } deletionEvents.forEach(deletionEvent -> deletionEvent.increaseReferenceCount("test")); - // Sleep to wait deletion being persisted - Thread.sleep(1000); List paths = Files.list(Paths.get(DELETION_BASE_DIR + File.separator + FAKE_DATA_REGION_IDS[3])) .collect(Collectors.toList()); @@ -198,6 +203,9 @@ public void testWaitForResult() throws Exception { DeletionResource deletionResource = PipeInsertionDataNodeListener.getInstance() .listenToDeleteData(FAKE_DATA_REGION_IDS[4], deleteDataNode); - Assert.assertSame(deletionResource.waitForResult(), Status.SUCCESS); + Assert.assertSame(Status.SUCCESS, deletionResource.waitForResult()); + // close pipe resource + PipeInsertionDataNodeListener.getInstance() + .stopListenAndAssign(FAKE_DATA_REGION_IDS[4], extractor); } } From 6c3602dbf2e7ef79ac52a4bf3b48c184c16e6de9 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Wed, 16 Oct 2024 19:51:08 +0800 Subject: [PATCH 81/85] fix review: delete DAL when time is proper --- .../iotdb/db/pipe/consensus/deletion/DeletionResource.java | 2 +- .../event/common/deletion/PipeDeleteDataNodeEvent.java | 7 ++++++- ...PipeHistoricalDataRegionTsFileAndDeletionExtractor.java | 6 ++++++ 3 files changed, 13 insertions(+), 2 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java index 1c7820a24d04..2230f6303227 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/DeletionResource.java @@ -65,7 +65,7 @@ public DeletionResource( TConsensusGroupType.DataRegion.getValue(), Integer.parseInt(regionId)); this.pipeTaskReferenceCount = new AtomicInteger( - DataRegionConsensusImpl.getInstance().getReplicationNum(consensusGroupId)); + DataRegionConsensusImpl.getInstance().getReplicationNum(consensusGroupId) - 1); } public synchronized void decreaseReference() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java index 26f2d04a4465..00a8259a9005 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/deletion/PipeDeleteDataNodeEvent.java @@ -92,10 +92,15 @@ public boolean internallyIncreaseResourceReferenceCount(String holderMessage) { @Override public boolean internallyDecreaseResourceReferenceCount(String holderMessage) { + return true; + } + + @Override + public void onCommitted() { + super.onCommitted(); if (deletionResource != null) { deletionResource.decreaseReference(); } - return true; } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java index e2bc2f142885..9f9a6796f878 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java @@ -623,6 +623,12 @@ private void extractDeletions( long startTime = System.currentTimeMillis(); List allDeletionResources = deletionResourceManager.getAllDeletionResources(); final int originalDeletionCount = allDeletionResources.size(); + // For deletions that are filtered and will not be sent, we should manually decrease its + // reference count. Because the initial value of referenceCount is `ReplicaNum - 1` + allDeletionResources.stream() + .filter(resource -> startIndex.isAfter(resource.getProgressIndex())) + .forEach(DeletionResource::decreaseReference); + // Get deletions that should be sent. allDeletionResources = allDeletionResources.stream() .filter(resource -> !startIndex.isAfter(resource.getProgressIndex())) From 97b699d0e8e18cc3930e58c810ac6c8f59946a80 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Wed, 16 Oct 2024 20:13:42 +0800 Subject: [PATCH 82/85] merge remote --- .../protocol/pipeconsensus/PipeConsensusSyncConnector.java | 4 +++- .../protocol/thrift/sync/IoTDBDataRegionSyncConnector.java | 1 + .../protocol/pipeconsensus/PipeConsensusReceiver.java | 3 +-- .../apache/iotdb/db/storageengine/dataregion/DataRegion.java | 3 ++- 4 files changed, 7 insertions(+), 4 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java index 5e03c1e7bedc..f9fbdd01cd1c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java @@ -242,7 +242,9 @@ private void doTransfer(final PipeDeleteDataNodeEvent pipeDeleteDataNodeEvent) final TPipeConsensusTransferResp resp; TCommitId tCommitId = new TCommitId( - pipeDeleteDataNodeEvent.getCommitId(), pipeDeleteDataNodeEvent.getRebootTimes()); + pipeDeleteDataNodeEvent.getCommitId(), + pipeDeleteDataNodeEvent.getCommitterKey().getRestartTimes(), + pipeDeleteDataNodeEvent.getRebootTimes()); TConsensusGroupId tConsensusGroupId = new TConsensusGroupId(TConsensusGroupType.DataRegion, consensusGroupId); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java index 21f24315eab3..14f2ff5fd070 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/thrift/sync/IoTDBDataRegionSyncConnector.java @@ -28,6 +28,7 @@ import org.apache.iotdb.db.pipe.connector.payload.evolvable.batch.PipeTabletEventPlainBatch; import org.apache.iotdb.db.pipe.connector.payload.evolvable.batch.PipeTabletEventTsFileBatch; import org.apache.iotdb.db.pipe.connector.payload.evolvable.batch.PipeTransferBatchReqBuilder; +import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferPlanNodeReq; import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferTabletBinaryReqV2; import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferTabletInsertNodeReqV2; import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferTabletRawReqV2; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java index 6fdf9c2da84a..24a00efdb782 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/receiver/protocol/pipeconsensus/PipeConsensusReceiver.java @@ -49,9 +49,8 @@ import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFileSealReq; import org.apache.iotdb.db.pipe.connector.protocol.pipeconsensus.payload.request.PipeConsensusTsFileSealWithModReq; import org.apache.iotdb.db.pipe.consensus.metric.PipeConsensusReceiverMetrics; -import org.apache.iotdb.db.pipe.event.common.tsfile.TsFileInsertionPointCounter; -import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; import org.apache.iotdb.db.pipe.event.common.tsfile.aggregator.TsFileInsertionPointCounter; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.DeleteDataNode; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.write.InsertNode; import org.apache.iotdb.db.storageengine.StorageEngine; import org.apache.iotdb.db.storageengine.dataregion.DataRegion; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java index 55fd07534503..f24a96a093e1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java @@ -2209,7 +2209,8 @@ private void getTwoKindsOfTsFiles( .forEach(unsealedResource::add); } - public void deleteByDevice(final MeasurementPath pattern, final DeleteDataNode node) throws IOException { + public void deleteByDevice(final MeasurementPath pattern, final DeleteDataNode node) + throws IOException { if (SettleService.getINSTANCE().getFilesToBeSettledCount().get() != 0) { throw new IOException( "Delete failed. " + "Please do not delete until the old files settled."); From ff3c9fcab0b4d86238e42d0215e571ef2d3e6251 Mon Sep 17 00:00:00 2001 From: Peng Junzhi <201250214@smail.nju.edu.cn> Date: Fri, 18 Oct 2024 15:47:28 +0800 Subject: [PATCH 83/85] merge remote --- .../protocol/airgap/IoTDBDataRegionAirGapConnector.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataRegionAirGapConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataRegionAirGapConnector.java index 232d6c3760ae..ec92b2e9ddd8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataRegionAirGapConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/airgap/IoTDBDataRegionAirGapConnector.java @@ -21,13 +21,10 @@ import org.apache.iotdb.common.rpc.thrift.TSStatus; import org.apache.iotdb.commons.pipe.event.EnrichedEvent; +import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferPlanNodeReq; import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferTabletBinaryReqV2; import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferTabletInsertNodeReqV2; import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferTabletRawReqV2; -import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferPlanNodeReq; -import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferTabletBinaryReq; -import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferTabletInsertNodeReq; -import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferTabletRawReq; import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferTsFilePieceReq; import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferTsFilePieceWithModReq; import org.apache.iotdb.db.pipe.connector.payload.evolvable.request.PipeTransferTsFileSealWithModReq; From 86faecab03f1774d2a2c29db2dcb4b29b74198b5 Mon Sep 17 00:00:00 2001 From: Steve Yurong Su Date: Sat, 19 Oct 2024 02:23:17 +0800 Subject: [PATCH 84/85] Update PipeHistoricalDataRegionTsFileAndDeletionExtractor.java --- ...istoricalDataRegionTsFileAndDeletionExtractor.java | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java index 0e882d3b2b55..ad36ab5285ce 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/extractor/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionExtractor.java @@ -416,18 +416,16 @@ public synchronized void start() { "Pipe: start to extract historical TsFile and Deletion(if uses pipeConsensus)"); try { List resourceList = new ArrayList<>(); + if (shouldExtractInsertion) { - // Flush TsFiles flushTsFilesForExtraction(dataRegion, startHistoricalExtractionTime); - // Extract TsFiles extractTsFiles(dataRegion, startHistoricalExtractionTime, resourceList); } - if (shouldExtractDeletion) { - // Extract deletions Optional.ofNullable(DeletionResourceManager.getInstance(String.valueOf(dataRegionId))) .ifPresent(manager -> extractDeletions(manager, resourceList)); } + // Sort tsFileResource and deletionResource long startTime = System.currentTimeMillis(); LOGGER.info("Pipe {}@{}: start to sort all extracted resources", pipeName, dataRegionId); @@ -436,13 +434,13 @@ public synchronized void start() { startIndex instanceof TimeWindowStateProgressIndex ? Long.compare(o1.getFileStartTime(), o2.getFileStartTime()) : o1.getProgressIndex().topologicalCompareTo(o2.getProgressIndex())); + pendingQueue = new ArrayDeque<>(resourceList); + LOGGER.info( "Pipe {}@{}: finish to sort all extracted resources, took {} ms", pipeName, dataRegionId, System.currentTimeMillis() - startTime); - - pendingQueue = new ArrayDeque<>(resourceList); } finally { dataRegion.writeUnlock(); } @@ -771,6 +769,7 @@ private Event supplyDeletionEvent(final DeletionResource deletionResource) { treePattern, tablePattern, false); + if (sloppyPattern || isDbNameCoveredByPattern) { event.skipParsingPattern(); } From 47813afee804b909b83c6dc948fbe98d135c7009 Mon Sep 17 00:00:00 2001 From: Steve Yurong Su Date: Sat, 19 Oct 2024 02:30:45 +0800 Subject: [PATCH 85/85] resolve conflicts --- .../pipe/agent/runtime/PipeDataNodeRuntimeAgent.java | 6 ------ .../pipeconsensus/PipeConsensusSyncConnector.java | 10 +++++----- 2 files changed, 5 insertions(+), 11 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java index 09173fbad783..7b2a6d69dccf 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/agent/runtime/PipeDataNodeRuntimeAgent.java @@ -161,12 +161,6 @@ public void assignProgressIndexForTsFileLoad(TsFileResource tsFileResource) { } public RecoverProgressIndex getNextProgressIndexForTsFileLoad() { - // The data node id must be negative. Because in the consensus layer Pipe, the processor of the - // pipe task will filter the progress index of the event. Leaving the recover progress index - // containing the datanode id of the datanode where the pipe task is located. The tsfile loaded - // in cannot be filtered by the processor of the datanode where the pipe task is located, - // because the load directly sends the tsfile to all replicas. Therefore, it is necessary to - // ensure that the datanode id generated by the load is negative. return new RecoverProgressIndex( DATA_NODE_ID, simpleProgressIndexAssigner.getSimpleProgressIndex()); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java index f9fbdd01cd1c..96602638010a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/connector/protocol/pipeconsensus/PipeConsensusSyncConnector.java @@ -223,12 +223,12 @@ private void doTransfer() { private void doTransferWrapper(final PipeDeleteDataNodeEvent pipeDeleteDataNodeEvent) throws PipeException { + // We increase the reference count for this event to determine if the event may be released. + if (!pipeDeleteDataNodeEvent.increaseReferenceCount( + PipeConsensusSyncConnector.class.getName())) { + return; + } try { - // We increase the reference count for this event to determine if the event may be released. - if (!pipeDeleteDataNodeEvent.increaseReferenceCount( - PipeConsensusSyncConnector.class.getName())) { - return; - } doTransfer(pipeDeleteDataNodeEvent); } finally { pipeDeleteDataNodeEvent.decreaseReferenceCount(