From e060d1e70892a794ec3fe700f8d025675863cf2e Mon Sep 17 00:00:00 2001
From: Peng Junzhi <201250214@smail.nju.edu.cn>
Date: Sun, 22 Mar 2026 22:22:08 +0800
Subject: [PATCH 1/9] fix: pipe tsfile miss deletion mods
---
.../tsfile/PipeTsFileInsertionEvent.java | 31 ++++++--
.../IoTConsensusV2SyncSink.java | 2 +-
...onsensusV2TsFileInsertionEventHandler.java | 8 +-
.../async/IoTDBDataRegionAsyncSink.java | 11 ++-
.../event/PipeTsFileInsertionEventTest.java | 78 +++++++++++++++++++
5 files changed, 113 insertions(+), 17 deletions(-)
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java
index 316b728a278a9..c1bc726cac22b 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java
@@ -81,6 +81,8 @@ public class PipeTsFileInsertionEvent extends PipeInsertionEvent
private File tsFile;
private long extractTime = 0;
+ // Whether this event should transfer mod files if they exist.
+ private boolean shouldTransferModFile;
// This is true iff the modFile exists and should be transferred
private boolean isWithMod;
private File modFile;
@@ -174,8 +176,8 @@ public PipeTsFileInsertionEvent(
// hard-link it to each pipe dir
this.tsFile = Objects.isNull(tsFile) ? resource.getTsFile() : tsFile;
- this.isWithMod = isWithMod && resource.anyModFileExists();
- this.modFile = this.isWithMod ? resource.getExclusiveModFile().getFile() : null;
+ this.shouldTransferModFile = isWithMod;
+ refreshModFileState();
// TODO: process the shared mod file
this.sharedModFile =
resource.getSharedModFile() != null ? resource.getSharedModFile().getFile() : null;
@@ -276,6 +278,7 @@ public File getTsFile() {
}
public File getModFile() {
+ refreshModFileState();
return modFile;
}
@@ -284,13 +287,13 @@ public File getSharedModFile() {
}
public boolean isWithMod() {
+ refreshModFileState();
return isWithMod;
}
- // If the previous "isWithMod" is false, the modFile has been set to "null", then the isWithMod
- // can't be set to true
- public void disableMod4NonTransferPipes(final boolean isWithMod) {
- this.isWithMod = isWithMod && this.isWithMod;
+ public void disableMod4NonTransferPipes(final boolean shouldTransferModFile) {
+ this.shouldTransferModFile = shouldTransferModFile && this.shouldTransferModFile;
+ refreshModFileState();
}
public boolean isLoaded() {
@@ -323,6 +326,7 @@ public long getExtractTime() {
public boolean internallyIncreaseResourceReferenceCount(final String holderMessage) {
extractTime = System.nanoTime();
try {
+ refreshModFileState();
tsFile = PipeDataNodeResourceManager.tsfile().increaseFileReference(tsFile, true, pipeName);
if (isWithMod) {
modFile =
@@ -423,7 +427,7 @@ public PipeTsFileInsertionEvent shallowCopySelfAndBindPipeTaskMetaForProgressRep
getSourceDatabaseNameFromDataRegion(),
resource,
tsFile,
- isWithMod,
+ shouldTransferModFile,
isLoaded,
isGeneratedByHistoricalExtractor,
tableNames,
@@ -754,6 +758,7 @@ public boolean isGeneratedByHistoricalExtractor() {
private TsFileInsertionEventParser initEventParser() {
try {
+ refreshModFileState();
eventParser.compareAndSet(
null,
new TsFileInsertionEventParserProvider(
@@ -848,6 +853,7 @@ public void trackResource() {
@Override
public PipeEventResource eventResourceBuilder() {
+ refreshModFileState();
return new PipeTsFileInsertionEventResource(
this.isReleased,
this.referenceCount,
@@ -859,6 +865,17 @@ public PipeEventResource eventResourceBuilder() {
this.eventParser);
}
+ private void refreshModFileState() {
+ if (!shouldTransferModFile || Objects.isNull(resource)) {
+ isWithMod = false;
+ modFile = null;
+ return;
+ }
+
+ isWithMod = resource.anyModFileExists();
+ modFile = isWithMod ? resource.getExclusiveModFile().getFile() : null;
+ }
+
private static class PipeTsFileInsertionEventResource extends PipeEventResource {
private final File tsFile;
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/iotconsensusv2/IoTConsensusV2SyncSink.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/iotconsensusv2/IoTConsensusV2SyncSink.java
index 5866a5ceeaa6c..76bebb976bbcd 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/iotconsensusv2/IoTConsensusV2SyncSink.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/iotconsensusv2/IoTConsensusV2SyncSink.java
@@ -382,7 +382,7 @@ private void doTransfer(final PipeTsFileInsertionEvent pipeTsFileInsertionEvent)
new TConsensusGroupId(TConsensusGroupType.DataRegion, consensusGroupId);
// 1. Transfer tsFile, and mod file if exists
- if (pipeTsFileInsertionEvent.isWithMod()) {
+ if (modFile != null) {
transferFilePieces(
modFile, syncIoTConsensusV2ServiceClient, true, tCommitId, tConsensusGroupId);
transferFilePieces(
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/iotconsensusv2/handler/IoTConsensusV2TsFileInsertionEventHandler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/iotconsensusv2/handler/IoTConsensusV2TsFileInsertionEventHandler.java
index 28850bc543dd9..90cdac84b896f 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/iotconsensusv2/handler/IoTConsensusV2TsFileInsertionEventHandler.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/iotconsensusv2/handler/IoTConsensusV2TsFileInsertionEventHandler.java
@@ -46,7 +46,6 @@
import java.io.IOException;
import java.io.RandomAccessFile;
import java.util.Arrays;
-import java.util.Objects;
import java.util.concurrent.atomic.AtomicBoolean;
public class IoTConsensusV2TsFileInsertionEventHandler
@@ -101,17 +100,14 @@ public IoTConsensusV2TsFileInsertionEventHandler(
tsFile = event.getTsFile();
modFile = event.getModFile();
- transferMod = event.isWithMod();
+ transferMod = modFile != null;
currentFile = transferMod ? modFile : tsFile;
readFileBufferSize = PipeConfig.getInstance().getPipeSinkReadFileBufferSize();
readBuffer = new byte[readFileBufferSize];
position = 0;
- reader =
- Objects.nonNull(modFile)
- ? new RandomAccessFile(modFile, "r")
- : new RandomAccessFile(tsFile, "r");
+ reader = transferMod ? new RandomAccessFile(modFile, "r") : new RandomAccessFile(tsFile, "r");
isSealSignalSent = new AtomicBoolean(false);
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java
index f8d0b104096f1..4e92cffc98ae5 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java
@@ -404,6 +404,12 @@ private boolean transferWithoutCheck(final TsFileInsertionEvent tsFileInsertionE
throw new FileNotFoundException(pipeTsFileInsertionEvent.getTsFile().getAbsolutePath());
}
+ final boolean supportMod = clientManager.supportModsIfIsDataNodeReceiver();
+ final File modFile =
+ (supportMod && pipeTsFileInsertionEvent.isWithMod())
+ ? pipeTsFileInsertionEvent.getModFile()
+ : null;
+
final PipeTransferTsFileHandler pipeTransferTsFileHandler =
new PipeTransferTsFileHandler(
this,
@@ -416,9 +422,8 @@ private boolean transferWithoutCheck(final TsFileInsertionEvent tsFileInsertionE
new AtomicInteger(1),
new AtomicBoolean(false),
pipeTsFileInsertionEvent.getTsFile(),
- pipeTsFileInsertionEvent.getModFile(),
- pipeTsFileInsertionEvent.isWithMod()
- && clientManager.supportModsIfIsDataNodeReceiver(),
+ modFile,
+ modFile != null,
pipeTsFileInsertionEvent.isTableModelEvent()
? pipeTsFileInsertionEvent.getTableModelDatabaseName()
: null);
diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java
index 5ba0843bf8003..6dbe3c629ace5 100644
--- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java
+++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java
@@ -24,6 +24,7 @@
import org.apache.iotdb.commons.auth.entity.PrivilegeType;
import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.commons.exception.auth.AccessDeniedException;
+import org.apache.iotdb.commons.path.MeasurementPath;
import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.commons.pipe.datastructure.pattern.IoTDBTreePattern;
import org.apache.iotdb.commons.pipe.datastructure.pattern.TablePattern;
@@ -36,6 +37,7 @@
import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.RelationalAuthorStatement;
import org.apache.iotdb.db.queryengine.plan.statement.Statement;
import org.apache.iotdb.db.storageengine.dataregion.compaction.utils.CompactionTestFileWriter;
+import org.apache.iotdb.db.storageengine.dataregion.modification.TreeDeletionEntry;
import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource;
import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResourceStatus;
import org.apache.iotdb.db.storageengine.dataregion.tsfile.generator.TsFileNameGenerator;
@@ -157,6 +159,82 @@ public void testAuthCheck() throws Exception {
}
}
+ @Test
+ public void testLateCreatedModFileCanStillBeObservedAfterShallowCopy() throws Exception {
+ final File baseDir = new File(TestConstant.BASE_OUTPUT_PATH, "late-mod");
+ final File tsFile = new File(baseDir, "late-mod.tsfile");
+ PipeTsFileInsertionEvent originalEvent = null;
+ PipeTsFileInsertionEvent copiedEvent = null;
+ try {
+ Assert.assertTrue(baseDir.mkdirs() || baseDir.exists());
+ Assert.assertTrue(tsFile.createNewFile() || tsFile.exists());
+
+ final TsFileResource resource = new TsFileResource(tsFile);
+ resource.setStatus(TsFileResourceStatus.NORMAL);
+
+ originalEvent =
+ new PipeTsFileInsertionEvent(
+ false,
+ "root.db",
+ resource,
+ null,
+ true,
+ false,
+ false,
+ null,
+ "testPipe",
+ 1L,
+ null,
+ buildUnionPattern(true, Collections.singletonList(new IoTDBTreePattern(true, null))),
+ new TablePattern(false, null, null),
+ null,
+ null,
+ null,
+ true,
+ Long.MIN_VALUE,
+ Long.MAX_VALUE);
+ copiedEvent =
+ originalEvent.shallowCopySelfAndBindPipeTaskMetaForProgressReport(
+ "testPipeCopy",
+ 2L,
+ null,
+ buildUnionPattern(true, Collections.singletonList(new IoTDBTreePattern(true, null))),
+ new TablePattern(false, null, null),
+ null,
+ null,
+ null,
+ true,
+ Long.MIN_VALUE,
+ Long.MAX_VALUE);
+
+ Assert.assertFalse(originalEvent.isWithMod());
+ Assert.assertFalse(copiedEvent.isWithMod());
+
+ resource
+ .getExclusiveModFile()
+ .write(new TreeDeletionEntry(new MeasurementPath("root.db.d1.s1"), 0, 1));
+ final File modFile = resource.getExclusiveModFile().getFile();
+ Assert.assertTrue(modFile.exists());
+
+ Assert.assertTrue(originalEvent.isWithMod());
+ Assert.assertEquals(modFile, originalEvent.getModFile());
+ Assert.assertTrue(copiedEvent.isWithMod());
+ Assert.assertEquals(modFile, copiedEvent.getModFile());
+
+ copiedEvent.disableMod4NonTransferPipes(false);
+ Assert.assertFalse(copiedEvent.isWithMod());
+ Assert.assertNull(copiedEvent.getModFile());
+ } finally {
+ if (originalEvent != null) {
+ originalEvent.close();
+ }
+ if (copiedEvent != null) {
+ copiedEvent.close();
+ }
+ FileUtils.deleteFileOrDirectory(baseDir);
+ }
+ }
+
static class TestAccessControl implements AccessControl {
@Override
From 4262a1f0e462318e1af48c23519e5935c9eb859b Mon Sep 17 00:00:00 2001
From: Peng Junzhi <201250214@smail.nju.edu.cn>
Date: Sun, 22 Mar 2026 23:17:39 +0800
Subject: [PATCH 2/9] fix: keep pinned mod path stable during tsfile transfer
---
.../tsfile/PipeTsFileInsertionEvent.java | 8 +++
.../event/PipeTsFileInsertionEventTest.java | 65 +++++++++++++++++++
2 files changed, 73 insertions(+)
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java
index c1bc726cac22b..1be0d04b080e1 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java
@@ -866,6 +866,14 @@ public PipeEventResource eventResourceBuilder() {
}
private void refreshModFileState() {
+ // Once the event is pinned (referenceCount > 0), `modFile` should already be the
+ // hardlinked/copied file managed by PipeTsFileResourceManager. Refreshing it from
+ // `resource.getExclusiveModFile()` would overwrite the pinned path, which may break reference
+ // tracking and cause the sink to transfer an unpinned file.
+ if (referenceCount.get() > 0) {
+ return;
+ }
+
if (!shouldTransferModFile || Objects.isNull(resource)) {
isWithMod = false;
modFile = null;
diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java
index 6dbe3c629ace5..95d63f040e538 100644
--- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java
+++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java
@@ -235,6 +235,71 @@ public void testLateCreatedModFileCanStillBeObservedAfterShallowCopy() throws Ex
}
}
+ @Test
+ public void testPinnedModFilePathIsStableAfterIncreaseReferenceCount() throws Exception {
+ final File tsFile =
+ new File(
+ TsFileNameGenerator.generateNewTsFilePath(
+ TestConstant.BASE_OUTPUT_PATH + IoTDBConstant.SEQUENCE_FOLDER_NAME, 1, 1, 1, 2));
+ PipeTsFileInsertionEvent event = null;
+ try {
+ Assert.assertTrue(tsFile.getParentFile().mkdirs() || tsFile.getParentFile().exists());
+ Assert.assertTrue(tsFile.createNewFile() || tsFile.exists());
+
+ final TsFileResource resource = new TsFileResource(tsFile);
+ resource.setStatus(TsFileResourceStatus.NORMAL);
+
+ event =
+ new PipeTsFileInsertionEvent(
+ false,
+ "root.db",
+ resource,
+ null,
+ true,
+ false,
+ false,
+ null,
+ "testPipe",
+ 1L,
+ null,
+ buildUnionPattern(true, Collections.singletonList(new IoTDBTreePattern(true, null))),
+ new TablePattern(false, null, null),
+ null,
+ null,
+ null,
+ true,
+ Long.MIN_VALUE,
+ Long.MAX_VALUE);
+
+ Assert.assertFalse(event.isWithMod());
+
+ // Create the mod file after event construction but before pinning.
+ resource
+ .getExclusiveModFile()
+ .write(new TreeDeletionEntry(new MeasurementPath("root.db.d1.s1"), 0, 1));
+ final File originalModFile = resource.getExclusiveModFile().getFile();
+ Assert.assertTrue(originalModFile.exists());
+ Assert.assertTrue(event.isWithMod());
+ Assert.assertEquals(originalModFile.getAbsolutePath(), event.getModFile().getAbsolutePath());
+
+ // Pin the event: mod file should be copied to pipe dir and must not be overwritten by refresh.
+ Assert.assertTrue(event.increaseReferenceCount("test"));
+ final File pinnedModFile = event.getModFile();
+ Assert.assertNotNull(pinnedModFile);
+ Assert.assertNotEquals(originalModFile.getAbsolutePath(), pinnedModFile.getAbsolutePath());
+
+ // Ensure subsequent refresh calls won't revert it to the original path.
+ Assert.assertTrue(event.isWithMod());
+ Assert.assertEquals(pinnedModFile.getAbsolutePath(), event.getModFile().getAbsolutePath());
+ } finally {
+ if (event != null) {
+ event.clearReferenceCount("test");
+ event.close();
+ }
+ FileUtils.deleteFileOrDirectory(new File(TestConstant.BASE_OUTPUT_PATH));
+ }
+ }
+
static class TestAccessControl implements AccessControl {
@Override
From 86c8fe98c403eefce6b824ee2b82859f1d8c80ab Mon Sep 17 00:00:00 2001
From: Peng Junzhi <201250214@smail.nju.edu.cn>
Date: Sun, 22 Mar 2026 23:49:49 +0800
Subject: [PATCH 3/9] fix: avoid mod TOCTOU and pin late-created mod safely
---
.../tsfile/PipeTsFileInsertionEvent.java | 32 ++++++++++++++-----
.../async/IoTDBDataRegionAsyncSink.java | 5 +--
2 files changed, 25 insertions(+), 12 deletions(-)
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java
index 1be0d04b080e1..1a3182afe3521 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java
@@ -866,20 +866,36 @@ public PipeEventResource eventResourceBuilder() {
}
private void refreshModFileState() {
- // Once the event is pinned (referenceCount > 0), `modFile` should already be the
- // hardlinked/copied file managed by PipeTsFileResourceManager. Refreshing it from
- // `resource.getExclusiveModFile()` would overwrite the pinned path, which may break reference
- // tracking and cause the sink to transfer an unpinned file.
- if (referenceCount.get() > 0) {
- return;
- }
-
if (!shouldTransferModFile || Objects.isNull(resource)) {
isWithMod = false;
modFile = null;
return;
}
+ // Once the event is pinned (referenceCount > 0), `modFile` should already be the
+ // hardlinked/copied file managed by PipeTsFileResourceManager. We must not overwrite it by
+ // reloading the original `resource.getExclusiveModFile()` path, otherwise reference tracking
+ // will be broken. However, we still need to support the case where the mod file is created
+ // after pinning: in this case we pin it lazily and then keep the pinned path stable.
+ if (referenceCount.get() > 0) {
+ if (Objects.isNull(modFile) && resource.getExclusiveModFile().exists()) {
+ try {
+ modFile =
+ PipeDataNodeResourceManager.tsfile()
+ .increaseFileReference(resource.getExclusiveModFile().getFile(), false, pipeName);
+ } catch (final Exception e) {
+ LOGGER.warn(
+ "Failed to pin late-created mod file {} for tsfile {}, will transfer without mod.",
+ resource.getExclusiveModFile().getFile(),
+ tsFile,
+ e);
+ modFile = null;
+ }
+ }
+ isWithMod = Objects.nonNull(modFile);
+ return;
+ }
+
isWithMod = resource.anyModFileExists();
modFile = isWithMod ? resource.getExclusiveModFile().getFile() : null;
}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java
index 4e92cffc98ae5..a10abf8272713 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/sink/protocol/thrift/async/IoTDBDataRegionAsyncSink.java
@@ -405,10 +405,7 @@ private boolean transferWithoutCheck(final TsFileInsertionEvent tsFileInsertionE
}
final boolean supportMod = clientManager.supportModsIfIsDataNodeReceiver();
- final File modFile =
- (supportMod && pipeTsFileInsertionEvent.isWithMod())
- ? pipeTsFileInsertionEvent.getModFile()
- : null;
+ final File modFile = supportMod ? pipeTsFileInsertionEvent.getModFile() : null;
final PipeTransferTsFileHandler pipeTransferTsFileHandler =
new PipeTransferTsFileHandler(
From b4418e4dd35b9092e728f2f2b08d2fa685071b2c Mon Sep 17 00:00:00 2001
From: Peng Junzhi <201250214@smail.nju.edu.cn>
Date: Mon, 23 Mar 2026 00:40:06 +0800
Subject: [PATCH 4/9] spotless
---
.../iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java | 3 ++-
1 file changed, 2 insertions(+), 1 deletion(-)
diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java
index 95d63f040e538..437c24cf9cf50 100644
--- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java
+++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java
@@ -282,7 +282,8 @@ public void testPinnedModFilePathIsStableAfterIncreaseReferenceCount() throws Ex
Assert.assertTrue(event.isWithMod());
Assert.assertEquals(originalModFile.getAbsolutePath(), event.getModFile().getAbsolutePath());
- // Pin the event: mod file should be copied to pipe dir and must not be overwritten by refresh.
+ // Pin the event: mod file should be copied to pipe dir and must not be overwritten by
+ // refresh.
Assert.assertTrue(event.increaseReferenceCount("test"));
final File pinnedModFile = event.getModFile();
Assert.assertNotNull(pinnedModFile);
From f2c61a04fcf8073efd3cf327e09c807a394bc3b3 Mon Sep 17 00:00:00 2001
From: Peng Junzhi <201250214@smail.nju.edu.cn>
Date: Mon, 23 Mar 2026 01:21:22 +0800
Subject: [PATCH 5/9] fix: wait delete mod materialization before pinning
tsfile for IoTV2
---
...ensusv2_delete_tsfile_mod_inconsistency.md | 361 ++++++++++++++++++
.../deletion/PipeTsFileDeletionBarrier.java | 157 ++++++++
.../tsfile/PipeTsFileInsertionEvent.java | 5 +
.../PipeInsertionDataNodeListener.java | 1 -
.../storageengine/dataregion/DataRegion.java | 108 ++++--
.../event/PipeTsFileInsertionEventTest.java | 93 +++++
6 files changed, 692 insertions(+), 33 deletions(-)
create mode 100644 iotconsensusv2_delete_tsfile_mod_inconsistency.md
create mode 100644 iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/PipeTsFileDeletionBarrier.java
diff --git a/iotconsensusv2_delete_tsfile_mod_inconsistency.md b/iotconsensusv2_delete_tsfile_mod_inconsistency.md
new file mode 100644
index 0000000000000..0cf713870eb84
--- /dev/null
+++ b/iotconsensusv2_delete_tsfile_mod_inconsistency.md
@@ -0,0 +1,361 @@
+# IoTConsensusV2 并发 Delete + TsFile 同步导致副本不一致:证据链与修复说明
+
+## 0. 结论摘要
+
+在本次 benchmark 场景里,IoTConsensusV2 的副本不一致,根因已经收敛为:
+
+1. 本地 IoTV2 `DELETE` 的传播,与 sealed TsFile 上 `.mods` 的物化,不是一个原子动作;
+2. `PipeTsFileInsertionEvent` 的第一次 `increaseReferenceCount()` 会把 tsfile/mod 的传输快照 pin 下来;
+3. 如果这个第一次 pin 发生在 delete 把对应 `.mods` 写出来之前,那么后续即使 delete 已经成功同步,tsfile 事件也可能仍然带着“无 mod”的旧快照;
+4. follower 若先执行 `delete(N)`,后 `seal/load tsfile(N+1)`,这个 tsfile 上本应删除的数据就会“回弹”。
+
+这里要特别区分两件事:
+
+- “event 被创建 / TsFileEpoch 被 extracted”
+- “event 第一次被 pin / snapshot 用于传输”
+
+真正缺失同步的是第二件事,不是单纯的“event 创建得早”。
+
+另外,之前分支上做的两类修复仍然是有价值的,但它们不是根因闭环:
+
+- `PipeTsFileInsertionEvent` 动态 refresh mod,可避免把 mod 是否存在永久冻结在构造时;
+- pinned mod path 稳定化,可避免 reference tracking 与实际传输文件不一致。
+
+但仅靠 refresh 仍然不够,因为 tsfile 可能已经更早被 pin 住;此时再 refresh,也改不了已经固定下来的传输快照。最终根修复需要把“delete 的 sealed-file materialization”与“tsfile 的第一次 pin”串起来。
+
+---
+
+## 1. 测试场景与异常现象
+
+### 1.1 场景
+
+- 共识协议:IoTConsensusV2(默认参数)
+- 负载:benchmark 并发读写,过程中 CLI 执行 delete
+- 操作:未执行任何节点故障/启停
+
+### 1.2 校验 SQL
+
+```sql
+select
+ count(s_0),count(s_1000),count(s_2000),count(s_3000),count(s_4000),
+ count(s_5000),count(s_6000),count(s_7000),count(s_8000),count(s_9999)
+from root.treedb.g_0.** align by device;
+```
+
+### 1.3 结果差异(Tree)
+
+证据文件:
+
+- `/Users/pengjunzhi/Code/iotdb-11/bug-report-and-log/v2082rc6_iotv2_query_result/q_all_online_tree.out`
+- `/Users/pengjunzhi/Code/iotdb-11/bug-report-and-log/v2082rc6_iotv2_query_result/q_stop_ip2_tree.out`
+
+关键行均为第 58 行:
+
+- all-online:`root.treedb.g_0.nonaligned_12` 的 `count(s_1000)=26960`
+- stop-ip2:`root.treedb.g_0.nonaligned_12` 的 `count(s_1000)=27350`
+
+差值为 `+390`,说明某个副本上 `s_1000` 的 delete 没有完整体现在查询结果里。
+
+### 1.4 删除语句确实覆盖问题列
+
+证据文件:
+
+- `/Users/pengjunzhi/Code/iotdb-11/bug-report-and-log/delete-sql.txt`
+
+第 4 行包含:
+
+```text
+delete from root.treedb.g_0.nonaligned_12.s_1000 ...
+```
+
+---
+
+## 2. 关键证据链
+
+本节只关注同一个问题 tsfile:
+
+- Region:`DataRegion 21`
+- tsfile:`1773977715979-129-0-0.tsfile`
+- 路径:`root.treedb.g_0.nonaligned_12.s_1000`
+
+### 2.1 Leader(dn_ip2)侧证据
+
+日志:
+
+- `/Users/pengjunzhi/Code/iotdb-11/bug-report-and-log/v2082rc6_iotv2_deletion_logs/dn_ip2/log_datanode_all.log`
+
+片段可用:
+
+```bash
+nl -ba /Users/pengjunzhi/Code/iotdb-11/bug-report-and-log/v2082rc6_iotv2_deletion_logs/dn_ip2/log_datanode_all.log | sed -n '19935,19946p'
+```
+
+关键日志:
+
+```text
+19937 2026-03-20 11:36:02,284 ... TsFileEpoch not found ... creating a new one
+19938 2026-03-20 11:36:02,285 ... All data in TsFileEpoch ... was extracted
+19939 2026-03-20 11:36:02,293 ... [Deletion] ... nonaligned_12.s_1000 ... written into 19 mod files
+19940 2026-03-20 11:36:02,494 ... DeleteNodeTransfer: no.160 event successfully processed!
+19945 2026-03-20 11:36:04,356 ... transferred file ... replicate index=161
+```
+
+这段日志能证明:
+
+1. `11:36:02.285`,leader 已经把这个 tsfile 对应的 epoch 提取出来;
+2. `11:36:02.293`,delete 随后才把 `nonaligned_12.s_1000` 写入 sealed tsfile 的 `.mods`;
+3. `11:36:02.494`,delete 事件以 `replicateIndex=160` 处理成功;
+4. `11:36:04.356`,对应 tsfile 事件以 `replicateIndex=161` 完成传输。
+
+这正是“leader 侧先出现 tsfile 提取,再出现 deletion 写 mod,再出现 delete 复制成功,最后才完成 tsfile 传输”的完整证据链。
+
+### 2.2 Follower(dn_ip4)侧证据
+
+日志:
+
+- `/Users/pengjunzhi/Code/iotdb-11/bug-report-and-log/v2082rc6_iotv2_deletion_logs/dn_ip4/log_datanode_all.log`
+
+片段可用:
+
+```bash
+nl -ba /Users/pengjunzhi/Code/iotdb-11/bug-report-and-log/v2082rc6_iotv2_deletion_logs/dn_ip4/log_datanode_all.log | sed -n '18304,18338p'
+```
+
+关键日志:
+
+```text
+18306 2026-03-20 11:36:02,488 ... start to receive ... replicateIndex:160
+18307 2026-03-20 11:36:02,493 ... [Deletion] ... nonaligned_12.s_1000 ... written into 6 mod files
+18308 2026-03-20 11:36:02,493 ... process ... replicateIndex:160 ... successfully
+
+18309 2026-03-20 11:36:02,520 ... start to receive ... replicateIndex:161
+18326 2026-03-20 11:36:03,677 ... starting to receive tsFile seal
+18328 2026-03-20 11:36:04,352 ... Load tsfile in unsequence list ...
+18331 2026-03-20 11:36:04,353 ... TsFile ... successfully loaded ...
+18338 2026-03-20 11:36:04,354 ... process ... replicateIndex:161 ... successfully
+```
+
+这段日志能证明:
+
+1. follower 先完整执行了 `delete(160)`;
+2. 然后才接收并 seal/load 了 `tsfile(161)`;
+3. 如果这个 tsfile 传过来时不带 `.mods`,那么 follower 上该 tsfile 的删除效果就不会被补上,最终查询结果会偏大。
+
+---
+
+## 3. 关于 replicateIndex 与 extracted 时序的澄清
+
+之前一个很容易混淆的点是:
+
+- `TsFileEpoch` 被 extracted 的时间
+- `replicateIndex` 被赋值的时间
+- `PipeTsFileInsertionEvent` 第一次被 pin 的时间
+
+它们不是同一件事。
+
+`replicateIndex` 的赋值发生在 Pipe 分发阶段,而不是 `TsFileEpoch` 被 extracted 的那个瞬间。代码位置:
+
+- `/Users/pengjunzhi/Code/iotdb-11/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/PipeDataRegionAssigner.java`
+
+关键逻辑:
+
+```java
+if (DataRegionConsensusImpl.getInstance() instanceof IoTConsensusV2
+ && IoTConsensusV2Processor.isShouldReplicate(innerEvent)) {
+ innerEvent.setReplicateIndexForIoTV2(
+ ReplicateProgressDataNodeManager.assignReplicateIndexForIoTV2(source.getPipeName()));
+}
+```
+
+所以,leader 日志里“tsfile 更早 extracted”只说明它更早进入了 realtime 提取链路,并不能单独推出“它的 replicateIndex 一定更小或更大”。
+
+本次最终收敛后的判断是:
+
+- follower 日志已经足够证明最终外部执行顺序是 `delete(160)` 在前、`tsfile(161)` 在后;
+- leader 日志已经足够证明 `.mods` 是在 tsfile 被 extracted 之后才物化出来;
+- 真正决定是否丢 delete 的,是 tsfile 第一次 pin/snapshot 是否早于 delete 的 sealed-file materialization。
+
+也就是说,根因并不是“必须存在 replicateIndex 逆序赋值 bug”,而是“pin 时刻缺少与 delete materialization 的同步”。
+
+---
+
+## 4. 为什么“仅在传输前 refresh mod”仍然不够
+
+分支之前的修复做了两件正确的事:
+
+1. `PipeTsFileInsertionEvent` 不再把 `isWithMod/modFile` 永久冻结在构造时,而是在使用前 refresh;
+2. 一旦 mod 已被 pin,就保持 pinned mod path 稳定,不再被后续 refresh 覆盖。
+
+这些修复解决了两类真实问题:
+
+- event 创建后、真正传输前,晚到的 `.mods` 能被看见;
+- 避免 reference count 绑定的是一个 mod 文件,而实际传输又换成另一个 mod 文件。
+
+但它仍然不能覆盖下面这个窗口:
+
+1. tsfile event 已经在上游某个环节调用了第一次 `increaseReferenceCount()`;
+2. 这次调用把 tsfile 以及当时能看到的 mod 快照 pin 下来;
+3. 随后 delete 才去 sealed tsfile 上写 `.mods`;
+4. 再晚一点即使 `refreshModFileState()` 能看到新 mod,也不一定还能安全替换已经 pinned 的传输快照。
+
+换句话说:
+
+- refresh 解决的是“看不看得见晚到 mod”
+- barrier 解决的是“能不能保证第一次 pin 不早于 delete materialization”
+
+后者才是这次副本不一致的根因闭环。
+
+---
+
+## 5. 真实根因(代码级)
+
+### 5.1 缺失同步的位置
+
+delete 路径里,本地 IoTV2 删除会经历:
+
+1. WAL / delete node 处理;
+2. 通过 `PipeInsertionDataNodeListener.listenToDeleteData(...)` 把 delete 事件发布到 Pipe / IoTV2;
+3. 再把对应 sealed tsfile 上的 `.mods` 真正写出来。
+
+而 tsfile 事件在另一条链路里,会在第一次 `increaseReferenceCount()` 时 pin 住将要传输的 tsfile/mod。
+
+原来这两条链路之间没有 per-tsfile 的同步机制,所以会出现:
+
+- delete 已经发出;
+- 但对应 sealed tsfile 的 `.mods` 还没写完;
+- tsfile 事件已经先一步完成了第一次 pin。
+
+### 5.2 这会怎样导致 follower 不一致
+
+当 follower 的外部执行顺序是:
+
+1. `delete(160)` 先执行;
+2. `tsfile(161)` 后 seal/load;
+
+如果 `tsfile(161)` 传来的仍然是 delete 物化前 pin 下来的“无 mod 快照”,那么 follower 载入这个 tsfile 后,对应删除就会缺失,表现出来就是 count 偏大。
+
+---
+
+## 6. 修复方案
+
+### 6.1 既有加固:继续保留动态 mod refresh 与 pinned path 稳定化
+
+文件:
+
+- `/Users/pengjunzhi/Code/iotdb-11/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java`
+
+这部分是分支上已有修复,仍然保留:
+
+1. `refreshModFileState()` 在使用前重新读取 `resource.anyModFileExists()`;
+2. 当 `referenceCount > 0` 时,不再用原始 `resource.getExclusiveModFile()` 覆盖已经 pinned 的 `modFile`;
+3. 如果 mod 是 pin 后才出现,则按当前逻辑 best-effort 地 lazy pin,并保持 pinned path 稳定。
+
+这一步解决的是“晚到 mod 可见性 + pinned mod path 正确性”。
+
+### 6.2 根修复:给 sealed tsfile 的 delete materialization 增加 barrier
+
+新增文件:
+
+- `/Users/pengjunzhi/Code/iotdb-11/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/PipeTsFileDeletionBarrier.java`
+
+核心思路:
+
+1. delete 路径先算出这次本地 IoTV2 delete 会影响哪些 sealed tsfile;
+2. 在真正发布 delete 并物化 `.mods` 之前,把这些 tsfile path 注册到 barrier;
+3. tsfile event 在第一次 `increaseReferenceCount()` 前,先检查该 tsfile 是否存在 pending deletion;
+4. 若存在,则等待 delete 完成 sealed-file `.mods` 物化后再继续 pin/snapshot;
+5. delete 完成后释放 barrier。
+
+落地位置:
+
+- `DataRegion.deleteByDevice(...)`
+- `DataRegion.deleteByTable(...)`
+- `DataRegion.deleteDataDirectly(...)`
+- `PipeTsFileInsertionEvent.internallyIncreaseResourceReferenceCount(...)`
+
+对应代码文件:
+
+- `/Users/pengjunzhi/Code/iotdb-11/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java`
+- `/Users/pengjunzhi/Code/iotdb-11/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java`
+
+### 6.3 为什么这个 barrier 才是根因闭环
+
+它把系统收敛到两个都安全的分支:
+
+1. tsfile event 的第一次 pin 早于 delete 注册 barrier
+
+这说明 tsfile 已经先进入复制顺序,它会保持更早的外部顺序;follower 先 load tsfile、后 apply delete,是安全的。
+
+2. delete 先注册 barrier,再发生 tsfile event 的第一次 pin
+
+这时 tsfile event 会等待,直到 delete 完成 sealed-file `.mods` 物化,再把带 mod 的快照 pin 下来;follower 即使先 apply delete、后 load tsfile,也仍然安全。
+
+真正不安全的只有第三种:
+
+- delete 已经开始影响该 tsfile,但 barrier 尚未建立;
+- 同时 tsfile 又先完成了第一次 pin。
+
+这正是本次修复消除的窗口。
+
+---
+
+## 7. 回归测试
+
+测试文件:
+
+- `/Users/pengjunzhi/Code/iotdb-11/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java`
+
+本次新增:
+
+- `testIncreaseReferenceCountWaitsForPendingDeletionBarrier`
+
+覆盖点:
+
+1. 手工为某个 tsfile path 注册 barrier;
+2. 在另一个线程里调用 `event.increaseReferenceCount(...)`;
+3. 断言它会先阻塞;
+4. 阻塞期间创建 mod 文件;
+5. 释放 barrier 后,断言事件成功完成 pin,并且能观察到 mod。
+
+分支上原有的 late-created mod 可见性测试也继续保留,因此现在覆盖了两类行为:
+
+- mod 在 event 创建后、pin 前出现
+- delete 先占住 barrier,tsfile 的第一次 pin 必须等待
+
+---
+
+## 8. 验证结果
+
+执行命令:
+
+```bash
+./mvnw -pl iotdb-core/datanode -am \
+ -Dtest=org.apache.iotdb.db.pipe.event.PipeTsFileInsertionEventTest \
+ -Dsurefire.failIfNoSpecifiedTests=false -DfailIfNoTests=false \
+ test -DskipITs
+```
+
+结果:
+
+- `BUILD SUCCESS`
+
+---
+
+## 9. 仍建议后续单独跟进的问题
+
+`historicalExtractor` 里 historical deletion 没有 assign replicateIndex,这个判断依然成立,而且值得单独修。
+
+它对应的是另一个独立风险:
+
+1. historical tsfile 路径会 assign IoTV2 replicateIndex;
+2. historical deletion 路径当前没有补齐 replicateIndex;
+3. `IoTConsensusV2Processor` 会过滤 `NO_COMMIT_ID` 事件;
+4. 因而 historical deletion 存在被 IoTV2 处理链过滤掉的风险。
+
+但它不是这次 benchmark 事故的主因,因为本次事故里的 deletion 已经明确在 leader/follower 两侧都被成功处理了。
+
+建议另开 issue 跟进:
+
+1. 在 historical deletion 路径补齐 IoTV2 replicateIndex;
+2. 增加对应回归测试;
+3. 与本次副本不一致修复分开提交,降低回归面。
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/PipeTsFileDeletionBarrier.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/PipeTsFileDeletionBarrier.java
new file mode 100644
index 0000000000000..63fd88c225205
--- /dev/null
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/PipeTsFileDeletionBarrier.java
@@ -0,0 +1,157 @@
+/*
+ * 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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * A lightweight barrier for coordinating delete materialization and TsFile transfer.
+ *
+ *
In IoTDB, a {@code DELETE} is first published to pipe/consensus and then its modifications are
+ * materialized to sealed TsFiles as mod files. Under concurrent flush and delete, it is possible
+ * that a TsFile event gets pinned/transferred before its corresponding mod file is generated,
+ * causing followers to miss deletions if they apply delete before loading that TsFile.
+ *
+ *
This barrier allows the delete path to mark a set of TsFiles as "pending deletion
+ * materialization". A TsFile event will wait (best effort) for the pending count to drop to zero
+ * before pinning/snapshotting the TsFile and its mod.
+ */
+public class PipeTsFileDeletionBarrier {
+
+ private static final Logger LOGGER = LoggerFactory.getLogger(PipeTsFileDeletionBarrier.class);
+
+ private final ConcurrentMap tsFilePath2PendingCounter =
+ new ConcurrentHashMap<>();
+
+ public void registerPendingDeletion(final Collection tsFilePaths) {
+ if (Objects.isNull(tsFilePaths) || tsFilePaths.isEmpty()) {
+ return;
+ }
+ tsFilePaths.forEach(this::registerPendingDeletion);
+ }
+
+ public void registerPendingDeletion(final String tsFilePath) {
+ if (Objects.isNull(tsFilePath)) {
+ return;
+ }
+
+ tsFilePath2PendingCounter.compute(
+ tsFilePath,
+ (k, v) -> {
+ final PendingCounter counter = v == null ? new PendingCounter() : v;
+ counter.pendingCount.incrementAndGet();
+ return counter;
+ });
+ }
+
+ public void releasePendingDeletion(final Collection tsFilePaths) {
+ if (Objects.isNull(tsFilePaths) || tsFilePaths.isEmpty()) {
+ return;
+ }
+ tsFilePaths.forEach(this::releasePendingDeletion);
+ }
+
+ public void releasePendingDeletion(final String tsFilePath) {
+ if (Objects.isNull(tsFilePath)) {
+ return;
+ }
+
+ final PendingCounter counter = tsFilePath2PendingCounter.get(tsFilePath);
+ if (counter == null) {
+ return;
+ }
+
+ final int remaining = counter.pendingCount.decrementAndGet();
+ if (remaining > 0) {
+ return;
+ }
+
+ if (remaining < 0) {
+ LOGGER.warn(
+ "PipeTsFileDeletionBarrier: pending deletion count becomes negative for {}, current {}. "
+ + "This may indicate mismatched register/release.",
+ tsFilePath,
+ remaining);
+ }
+
+ tsFilePath2PendingCounter.remove(tsFilePath, counter);
+ synchronized (counter.monitor) {
+ counter.monitor.notifyAll();
+ }
+ }
+
+ public void awaitPendingDeletionIfNecessary(final String tsFilePath) {
+ if (Objects.isNull(tsFilePath)) {
+ return;
+ }
+
+ final PendingCounter counter = tsFilePath2PendingCounter.get(tsFilePath);
+ if (counter == null) {
+ return;
+ }
+
+ synchronized (counter.monitor) {
+ while (counter.pendingCount.get() > 0) {
+ try {
+ counter.monitor.wait();
+ } catch (final InterruptedException e) {
+ Thread.currentThread().interrupt();
+ LOGGER.warn(
+ "PipeTsFileDeletionBarrier: interrupted while waiting pending deletions for {}, "
+ + "will continue without waiting.",
+ tsFilePath,
+ e);
+ return;
+ }
+ }
+ }
+ }
+
+ private static class PendingCounter {
+ private final AtomicInteger pendingCount = new AtomicInteger(0);
+ private final Object monitor = new Object();
+ }
+
+ /////////////////////////////// singleton ///////////////////////////////
+
+ private static class PipeTsFileDeletionBarrierHolder {
+ private static final PipeTsFileDeletionBarrier INSTANCE = new PipeTsFileDeletionBarrier();
+
+ private PipeTsFileDeletionBarrierHolder() {
+ // empty constructor
+ }
+ }
+
+ public static PipeTsFileDeletionBarrier getInstance() {
+ return PipeTsFileDeletionBarrierHolder.INSTANCE;
+ }
+
+ private PipeTsFileDeletionBarrier() {
+ // empty constructor
+ }
+}
+
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java
index 1a3182afe3521..5e078ccb3d0f0 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java
@@ -35,6 +35,7 @@
import org.apache.iotdb.commons.pipe.datastructure.pattern.TreePattern;
import org.apache.iotdb.commons.pipe.resource.ref.PipePhantomReferenceManager.PipeEventResource;
import org.apache.iotdb.db.auth.AuthorityChecker;
+import org.apache.iotdb.db.pipe.consensus.deletion.PipeTsFileDeletionBarrier;
import org.apache.iotdb.db.pipe.event.ReferenceTrackableEvent;
import org.apache.iotdb.db.pipe.event.common.PipeInsertionEvent;
import org.apache.iotdb.db.pipe.event.common.tablet.PipeRawTabletInsertionEvent;
@@ -326,6 +327,10 @@ public long getExtractTime() {
public boolean internallyIncreaseResourceReferenceCount(final String holderMessage) {
extractTime = System.nanoTime();
try {
+ if (Objects.nonNull(resource)) {
+ PipeTsFileDeletionBarrier.getInstance()
+ .awaitPendingDeletionIfNecessary(resource.getTsFilePath());
+ }
refreshModFileState();
tsFile = PipeDataNodeResourceManager.tsfile().increaseFileReference(tsFile, true, pipeName);
if (isWithMod) {
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/listener/PipeInsertionDataNodeListener.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/listener/PipeInsertionDataNodeListener.java
index 3cce521a51e0b..541b3432272f6 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/listener/PipeInsertionDataNodeListener.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/listener/PipeInsertionDataNodeListener.java
@@ -164,7 +164,6 @@ public DeletionResource listenToDeleteData(
}
assigner.publishToAssign(PipeRealtimeEventFactory.createRealtimeEvent(node));
-
return deletionResource;
}
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 eb37cb1d5d21b..b0e9be5eb2acd 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
@@ -67,6 +67,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.consensus.deletion.PipeTsFileDeletionBarrier;
import org.apache.iotdb.db.pipe.consensus.deletion.persist.PageCacheDeletionBuffer;
import org.apache.iotdb.db.pipe.resource.memory.PipeMemoryWeightUtil;
import org.apache.iotdb.db.pipe.source.dataregion.realtime.listener.PipeInsertionDataNodeListener;
@@ -2870,18 +2871,33 @@ public void deleteByDevice(final MeasurementPath pattern, final DeleteDataNode n
// deviceMatchInfo is used for filter the matched deviceId in TsFileResource
// deviceMatchInfo contains the DeviceId means this device matched the pattern
deleteDataInUnsealedFiles(unsealedTsFileResource, deletion, sealedTsFileResource);
- // capture deleteDataNode and wait it to be persisted to DAL.
- DeletionResource deletionResource =
- PipeInsertionDataNodeListener.getInstance()
- .listenToDeleteData(dataRegionId.getId(), node);
- // just get result. We have already waited for result in `listenToDeleteData`
- if (deletionResource != null && deletionResource.waitForResult() == Status.FAILURE) {
- throw deletionResource.getCause();
- }
- writeUnlock();
- hasReleasedLock = true;
- deleteDataInSealedFiles(sealedTsFileResource, deletion);
+ // Prevent a TsFile event from being pinned/transferred before this deletion is materialized
+ // to its corresponding mod files.
+ final Set tsFilePathsPendingDeletion =
+ DeletionResource.isDeleteNodeGeneratedInLocalByIoTV2(node)
+ ? sealedTsFileResource.stream()
+ .filter(resource -> !canSkipDelete(resource, deletion))
+ .map(TsFileResource::getTsFilePath)
+ .collect(Collectors.toSet())
+ : Collections.emptySet();
+ PipeTsFileDeletionBarrier.getInstance().registerPendingDeletion(tsFilePathsPendingDeletion);
+ try {
+ // capture deleteDataNode and wait it to be persisted to DAL.
+ final DeletionResource deletionResource =
+ PipeInsertionDataNodeListener.getInstance()
+ .listenToDeleteData(dataRegionId.getId(), node);
+ // just get result. We have already waited for result in `listenToDeleteData`
+ if (deletionResource != null && deletionResource.waitForResult() == Status.FAILURE) {
+ throw deletionResource.getCause();
+ }
+ writeUnlock();
+ hasReleasedLock = true;
+
+ deleteDataInSealedFiles(sealedTsFileResource, deletion);
+ } finally {
+ PipeTsFileDeletionBarrier.getInstance().releasePendingDeletion(tsFilePathsPendingDeletion);
+ }
} catch (Exception e) {
throw new IOException(e);
} finally {
@@ -2974,20 +2990,36 @@ public void deleteByTable(RelationalDeleteDataNode node) throws IOException {
sealedTsFileResourceLists.add(sealedTsFileResource);
}
- // capture deleteDataNode and wait it to be persisted to DAL.
- DeletionResource deletionResource =
- PipeInsertionDataNodeListener.getInstance()
- .listenToDeleteData(dataRegionId.getId(), node);
- // just get result. We have already waited for result in `listenToDeleteData`
- if (deletionResource != null && deletionResource.waitForResult() == Status.FAILURE) {
- throw deletionResource.getCause();
+ final Set tsFilePathsPendingDeletion = new HashSet<>();
+ if (DeletionResource.isDeleteNodeGeneratedInLocalByIoTV2(node)) {
+ for (int i = 0; i < modEntries.size(); i++) {
+ final TableDeletionEntry modEntry = modEntries.get(i);
+ tsFilePathsPendingDeletion.addAll(
+ sealedTsFileResourceLists.get(i).stream()
+ .filter(resource -> !canSkipDelete(resource, modEntry))
+ .map(TsFileResource::getTsFilePath)
+ .collect(Collectors.toSet()));
+ }
}
+ PipeTsFileDeletionBarrier.getInstance().registerPendingDeletion(tsFilePathsPendingDeletion);
+ try {
+ // capture deleteDataNode and wait it to be persisted to DAL.
+ final DeletionResource deletionResource =
+ PipeInsertionDataNodeListener.getInstance()
+ .listenToDeleteData(dataRegionId.getId(), node);
+ // just get result. We have already waited for result in `listenToDeleteData`
+ if (deletionResource != null && deletionResource.waitForResult() == Status.FAILURE) {
+ throw deletionResource.getCause();
+ }
- writeUnlock();
- hasReleasedLock = true;
+ writeUnlock();
+ hasReleasedLock = true;
- for (int i = 0; i < modEntries.size(); i++) {
- deleteDataInSealedFiles(sealedTsFileResourceLists.get(i), modEntries.get(i));
+ for (int i = 0; i < modEntries.size(); i++) {
+ deleteDataInSealedFiles(sealedTsFileResourceLists.get(i), modEntries.get(i));
+ }
+ } finally {
+ PipeTsFileDeletionBarrier.getInstance().releasePendingDeletion(tsFilePathsPendingDeletion);
}
} catch (Exception e) {
throw new IOException(e);
@@ -3032,17 +3064,29 @@ public void deleteDataDirectly(MeasurementPath pathToDelete, DeleteDataNode node
List unsealedTsFileResource = new ArrayList<>();
getTwoKindsOfTsFiles(sealedTsFileResource, unsealedTsFileResource, startTime, endTime);
deleteDataDirectlyInFile(unsealedTsFileResource, deletion);
- // capture deleteDataNode and wait it to be persisted to DAL.
- DeletionResource deletionResource =
- PipeInsertionDataNodeListener.getInstance()
- .listenToDeleteData(dataRegionId.getId(), node);
- // just get result. We have already waited for result in `listenToDeleteData`
- if (deletionResource != null && deletionResource.waitForResult() == Status.FAILURE) {
- throw deletionResource.getCause();
+ final Set tsFilePathsPendingDeletion =
+ DeletionResource.isDeleteNodeGeneratedInLocalByIoTV2(node)
+ ? sealedTsFileResource.stream()
+ .filter(resource -> !canSkipDelete(resource, deletion))
+ .map(TsFileResource::getTsFilePath)
+ .collect(Collectors.toSet())
+ : Collections.emptySet();
+ PipeTsFileDeletionBarrier.getInstance().registerPendingDeletion(tsFilePathsPendingDeletion);
+ try {
+ // capture deleteDataNode and wait it to be persisted to DAL.
+ final DeletionResource deletionResource =
+ PipeInsertionDataNodeListener.getInstance()
+ .listenToDeleteData(dataRegionId.getId(), node);
+ // just get result. We have already waited for result in `listenToDeleteData`
+ if (deletionResource != null && deletionResource.waitForResult() == Status.FAILURE) {
+ throw deletionResource.getCause();
+ }
+ writeUnlock();
+ releasedLock = true;
+ deleteDataDirectlyInFile(sealedTsFileResource, deletion);
+ } finally {
+ PipeTsFileDeletionBarrier.getInstance().releasePendingDeletion(tsFilePathsPendingDeletion);
}
- writeUnlock();
- releasedLock = true;
- deleteDataDirectlyInFile(sealedTsFileResource, deletion);
} catch (Exception e) {
throw new IOException(e);
} finally {
diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java
index 437c24cf9cf50..8d40122ed6b4b 100644
--- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java
+++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java
@@ -30,6 +30,7 @@
import org.apache.iotdb.commons.pipe.datastructure.pattern.TablePattern;
import org.apache.iotdb.commons.utils.FileUtils;
import org.apache.iotdb.db.auth.AuthorityChecker;
+import org.apache.iotdb.db.pipe.consensus.deletion.PipeTsFileDeletionBarrier;
import org.apache.iotdb.db.pipe.event.common.tsfile.PipeTsFileInsertionEvent;
import org.apache.iotdb.db.queryengine.plan.relational.metadata.QualifiedObjectName;
import org.apache.iotdb.db.queryengine.plan.relational.security.AccessControl;
@@ -58,6 +59,9 @@
import java.util.Collection;
import java.util.Collections;
import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
@@ -301,6 +305,95 @@ public void testPinnedModFilePathIsStableAfterIncreaseReferenceCount() throws Ex
}
}
+ @Test
+ public void testIncreaseReferenceCountWaitsForPendingDeletionBarrier() throws Exception {
+ final File tsFile =
+ new File(
+ TsFileNameGenerator.generateNewTsFilePath(
+ TestConstant.BASE_OUTPUT_PATH + IoTDBConstant.SEQUENCE_FOLDER_NAME, 1, 1, 1, 3));
+ PipeTsFileInsertionEvent event = null;
+ final String holder = "test";
+ String tsFilePath = null;
+ Thread t = null;
+ try {
+ Assert.assertTrue(tsFile.getParentFile().mkdirs() || tsFile.getParentFile().exists());
+ Assert.assertTrue(tsFile.createNewFile() || tsFile.exists());
+
+ final TsFileResource resource = new TsFileResource(tsFile);
+ resource.setStatus(TsFileResourceStatus.NORMAL);
+
+ event =
+ new PipeTsFileInsertionEvent(
+ false,
+ "root.db",
+ resource,
+ null,
+ true,
+ false,
+ false,
+ null,
+ "testPipe",
+ 1L,
+ null,
+ buildUnionPattern(true, Collections.singletonList(new IoTDBTreePattern(true, null))),
+ new TablePattern(false, null, null),
+ null,
+ null,
+ null,
+ true,
+ Long.MIN_VALUE,
+ Long.MAX_VALUE);
+ final PipeTsFileInsertionEvent finalEvent = event;
+
+ tsFilePath = resource.getTsFilePath();
+ PipeTsFileDeletionBarrier.getInstance().registerPendingDeletion(tsFilePath);
+ final CountDownLatch started = new CountDownLatch(1);
+ final CountDownLatch finished = new CountDownLatch(1);
+ final AtomicBoolean increased = new AtomicBoolean(false);
+
+ t =
+ new Thread(
+ () -> {
+ started.countDown();
+ increased.set(finalEvent.increaseReferenceCount(holder));
+ finished.countDown();
+ },
+ "test-tsfile-delete-barrier");
+ t.start();
+
+ Assert.assertTrue(started.await(5, TimeUnit.SECONDS));
+ Assert.assertFalse(finished.await(200, TimeUnit.MILLISECONDS));
+
+ resource
+ .getExclusiveModFile()
+ .write(new TreeDeletionEntry(new MeasurementPath("root.db.d1.s1"), 0, 1));
+ final File originalModFile = resource.getExclusiveModFile().getFile();
+ Assert.assertTrue(originalModFile.exists());
+
+ PipeTsFileDeletionBarrier.getInstance().releasePendingDeletion(tsFilePath);
+ tsFilePath = null;
+
+ Assert.assertTrue(finished.await(5, TimeUnit.SECONDS));
+ Assert.assertTrue(increased.get());
+ Assert.assertTrue(event.isWithMod());
+ Assert.assertNotNull(event.getModFile());
+ Assert.assertNotEquals(originalModFile.getAbsolutePath(), event.getModFile().getAbsolutePath());
+ } finally {
+ // Ensure the singleton barrier is not leaked into other tests.
+ if (tsFilePath != null) {
+ PipeTsFileDeletionBarrier.getInstance().releasePendingDeletion(tsFilePath);
+ }
+ if (t != null) {
+ t.join(TimeUnit.SECONDS.toMillis(5));
+ }
+ if (event != null) {
+ event.clearReferenceCount(holder);
+ event.close();
+ }
+ FileUtils.deleteFileOrDirectory(new File(TestConstant.BASE_OUTPUT_PATH));
+ }
+ }
+
static class TestAccessControl implements AccessControl {
@Override
From 76306a2bddd0c3776e5026c0c29095188b553909 Mon Sep 17 00:00:00 2001
From: Peng Junzhi <201250214@smail.nju.edu.cn>
Date: Mon, 23 Mar 2026 10:00:38 +0800
Subject: [PATCH 6/9] fix: freeze iotv2 tsfile mod snapshots after first pin
---
...ensusv2_delete_tsfile_mod_inconsistency.md | 68 ++++++-
.../deletion/PipeTsFileDeletionBarrier.java | 1 -
.../PipeCompactedTsFileInsertionEvent.java | 2 +
.../tsfile/PipeTsFileInsertionEvent.java | 115 ++++++++---
...icalDataRegionTsFileAndDeletionSource.java | 2 +
.../event/PipeTsFileInsertionEventTest.java | 187 +++++++++++++++++-
.../event/TsFileInsertionEventParserTest.java | 2 +
7 files changed, 334 insertions(+), 43 deletions(-)
diff --git a/iotconsensusv2_delete_tsfile_mod_inconsistency.md b/iotconsensusv2_delete_tsfile_mod_inconsistency.md
index 0cf713870eb84..9c769d2354305 100644
--- a/iotconsensusv2_delete_tsfile_mod_inconsistency.md
+++ b/iotconsensusv2_delete_tsfile_mod_inconsistency.md
@@ -19,10 +19,24 @@
另外,之前分支上做的两类修复仍然是有价值的,但它们不是根因闭环:
- `PipeTsFileInsertionEvent` 动态 refresh mod,可避免把 mod 是否存在永久冻结在构造时;
-- pinned mod path 稳定化,可避免 reference tracking 与实际传输文件不一致。
+- pinned mod path 稳定化,可避免 reference tracking 与实际传输文件不一致;
+- 更关键的是,第一次 pin 完成后必须冻结 snapshot,后续不能再从 live resource 吸收“未来才出现”的 mod。
但仅靠 refresh 仍然不够,因为 tsfile 可能已经更早被 pin 住;此时再 refresh,也改不了已经固定下来的传输快照。最终根修复需要把“delete 的 sealed-file materialization”与“tsfile 的第一次 pin”串起来。
+在把 barrier 补上之后,又进一步发现一个 residual window:
+
+1. realtime 原始 event 在 assigner 第一次 pin 后,`tsFile` 已经变成 pipe 目录里的 hardlink;
+2. 但后续 `shallowCopy` 给各个 pipe source 创建副本时,旧代码并没有继承已经 pin 好的 `modFile`;
+3. 这些副本仍然会重新读取 live `resource.getExclusiveModFile()`;
+4. 如果此时 source mod 已经被 compaction / settle / replace 删除或替换,副本仍然可能丢 mod。
+
+因此最终修复除了 delete barrier 之外,还要保证:
+
+- 首次 snapshot/pin 在资源锁保护下完成;
+- 第一次 pin 完成后,事件的 mod snapshot 立即冻结;
+- `shallowCopy` 继承已经 pin 好的 mod snapshot 状态,而不是回头读取 live resource。
+
---
## 1. 测试场景与异常现象
@@ -244,13 +258,17 @@ delete 路径里,本地 IoTV2 删除会经历:
- `/Users/pengjunzhi/Code/iotdb-11/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java`
-这部分是分支上已有修复,仍然保留:
+这部分是分支上已有修复,但最终语义已经收敛为“pin 前可刷新,pin 后冻结”:
+
+1. 在第一次 pin 之前,`refreshModFileState()` 仍然会读取 live `resource.anyModFileExists()`,因此 event 创建后、pin 前晚到的 `.mods` 仍可被观察到;
+2. 一旦第一次 pin 完成,`PipeTsFileInsertionEvent` 会立刻冻结 mod snapshot,不再回头读取 live `resource.getExclusiveModFile()`;
+3. 已经 pin 好的 `modFile` 路径会保持稳定,不会再被后续 refresh 覆盖。
-1. `refreshModFileState()` 在使用前重新读取 `resource.anyModFileExists()`;
-2. 当 `referenceCount > 0` 时,不再用原始 `resource.getExclusiveModFile()` 覆盖已经 pinned 的 `modFile`;
-3. 如果 mod 是 pin 后才出现,则按当前逻辑 best-effort 地 lazy pin,并保持 pinned path 稳定。
+这一步解决的是三件事:
-这一步解决的是“晚到 mod 可见性 + pinned mod path 正确性”。
+- pin 前的晚到 mod 可见性;
+- pinned mod path 正确性;
+- 防止旧 tsfile event 在 pin 之后误吸收未来 delete 产生的 mod。
### 6.2 根修复:给 sealed tsfile 的 delete materialization 增加 barrier
@@ -278,7 +296,35 @@ delete 路径里,本地 IoTV2 删除会经历:
- `/Users/pengjunzhi/Code/iotdb-11/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java`
- `/Users/pengjunzhi/Code/iotdb-11/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java`
-### 6.3 为什么这个 barrier 才是根因闭环
+### 6.3 补齐剩余窗口:首次 snapshot 加锁,shallow copy 继承 frozen snapshot
+
+在补上 barrier 之后,还需要再解决一个很隐蔽的 realtime-only 问题:
+
+1. 原始 realtime `PipeTsFileInsertionEvent` 在 assigner 的第一次 `increaseReferenceCount()` 时,会把 tsfile/mod snapshot 到 common pipe 目录;
+2. 随后 assigner 会对 event 做 `shallowCopy`,为每个 pipe source 生成副本;
+3. 旧代码里,`shallowCopy` 只继承了已经 pin 好的 `tsFile`,却没有继承“mod snapshot 已冻结”这个状态;
+4. 当原始 event 已经 pin 到有 mod 时,副本可能丢掉已 pin 的 `modFile`,重新读取 live `resource.getExclusiveModFile()`;
+5. 当原始 event 已经 pin 到“无 mod”时,副本也可能重新看见未来才出现的 mod;
+6. 如果此时源 `.mods` 已被 compaction / settle / replace 删除或替换,副本还可能再次丢 mod。
+
+本次补充修复做了两件事:
+
+1. `PipeTsFileInsertionEvent` 的第一次 snapshot/pin 改为在 `TsFileResource.readLock()` 下完成;
+2. 若需要 pin live exclusive mod,则在同一阶段持有对应 `ModificationFile.writeLock()` 复制 mod;
+3. 第一次 pin 完成后,event 会把“mod snapshot 是否还允许从 live resource refresh”切成 false,从而冻结 snapshot;
+4. `shallowCopy` 会直接继承这份 frozen snapshot 状态,包括:
+ - 已经 pin 好的 `modFile` 路径;
+ - 以及“已经确认当前 snapshot 没有 mod”这个空快照状态;
+5. 后续 pipe source 副本再做第一次 pin 时,会基于这份 frozen snapshot 继续复制,而不是回头读 live resource。
+
+这样可以避免:
+
+- 首次 snapshot 过程中,source tsfile 被 compaction/remove 删除;
+- 原始 event 已经拿到 pinned snapshot,但 shallow copy 仍然回头依赖 live mod;
+- 原始 event 已经 pin 到“无 mod”快照,但 shallow copy 又错误吸收未来 mod;
+- source mod 被 merge/replace 后,pipe 副本看不到已经 pin 好的 mod。
+
+### 6.4 为什么这套组合修复才是当前闭环
它把系统收敛到两个都安全的分支:
@@ -308,6 +354,8 @@ delete 路径里,本地 IoTV2 删除会经历:
本次新增:
- `testIncreaseReferenceCountWaitsForPendingDeletionBarrier`
+- `testShallowCopyKeepsPinnedModSnapshotAfterSourceModDisappears`
+- `testPinnedEventDoesNotAdoptFutureModFile`
覆盖点:
@@ -317,10 +365,12 @@ delete 路径里,本地 IoTV2 删除会经历:
4. 阻塞期间创建 mod 文件;
5. 释放 barrier 后,断言事件成功完成 pin,并且能观察到 mod。
-分支上原有的 late-created mod 可见性测试也继续保留,因此现在覆盖了两类行为:
+分支上原有的 late-created mod 可见性测试也继续保留,因此现在覆盖了三类行为:
- mod 在 event 创建后、pin 前出现
- delete 先占住 barrier,tsfile 的第一次 pin 必须等待
+- 原始 event 已经 pin 了 mod snapshot 后,即使 source mod 消失,shallow copy 仍然沿用 pinned snapshot
+- 原始 event 已经 pin 到“无 mod”快照后,即使未来真的出现 mod,旧事件及其 shallow copy 也不会再吸收它
---
@@ -330,7 +380,7 @@ delete 路径里,本地 IoTV2 删除会经历:
```bash
./mvnw -pl iotdb-core/datanode -am \
- -Dtest=org.apache.iotdb.db.pipe.event.PipeTsFileInsertionEventTest \
+ -Dtest=org.apache.iotdb.db.pipe.event.PipeTsFileInsertionEventTest,org.apache.iotdb.db.pipe.event.TsFileInsertionEventParserTest \
-Dsurefire.failIfNoSpecifiedTests=false -DfailIfNoTests=false \
test -DskipITs
```
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/PipeTsFileDeletionBarrier.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/PipeTsFileDeletionBarrier.java
index 63fd88c225205..96b81eafd83aa 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/PipeTsFileDeletionBarrier.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/PipeTsFileDeletionBarrier.java
@@ -154,4 +154,3 @@ private PipeTsFileDeletionBarrier() {
// empty constructor
}
}
-
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java
index 343c8d8932931..47eb15a27aeec 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeCompactedTsFileInsertionEvent.java
@@ -53,6 +53,8 @@ public PipeCompactedTsFileInsertionEvent(
anyOfOriginalEvents.getTreeModelDatabaseName(),
tsFileResource,
null,
+ null,
+ true,
bindIsWithMod(originalEvents),
bindIsLoaded(originalEvents),
bindIsGeneratedByHistoricalExtractor(originalEvents),
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java
index 5e078ccb3d0f0..6bd5c7859f3b2 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java
@@ -49,6 +49,7 @@
import org.apache.iotdb.db.pipe.source.dataregion.realtime.assigner.PipeTsFileEpochProgressIndexKeeper;
import org.apache.iotdb.db.queryengine.plan.relational.metadata.QualifiedObjectName;
import org.apache.iotdb.db.storageengine.dataregion.memtable.TsFileProcessor;
+import org.apache.iotdb.db.storageengine.dataregion.modification.ModificationFile;
import org.apache.iotdb.db.storageengine.dataregion.tsfile.TsFileResource;
import org.apache.iotdb.pipe.api.event.dml.insertion.TabletInsertionEvent;
import org.apache.iotdb.pipe.api.event.dml.insertion.TsFileInsertionEvent;
@@ -84,6 +85,8 @@ public class PipeTsFileInsertionEvent extends PipeInsertionEvent
// Whether this event should transfer mod files if they exist.
private boolean shouldTransferModFile;
+ // Whether this event should consult the live TsFileResource for mod visibility before pinning.
+ private boolean shouldRefreshModFileStateFromResource;
// This is true iff the modFile exists and should be transferred
private boolean isWithMod;
private File modFile;
@@ -117,6 +120,8 @@ public PipeTsFileInsertionEvent(
databaseNameFromDataRegion,
resource,
null,
+ null,
+ true,
true,
isLoaded,
false,
@@ -139,6 +144,8 @@ public PipeTsFileInsertionEvent(
final String databaseNameFromDataRegion,
final TsFileResource resource,
final File tsFile,
+ final File modFile,
+ final boolean shouldRefreshModFileStateFromResource,
final boolean isWithMod,
final boolean isLoaded,
final boolean isGeneratedByHistoricalExtractor,
@@ -178,7 +185,14 @@ public PipeTsFileInsertionEvent(
this.tsFile = Objects.isNull(tsFile) ? resource.getTsFile() : tsFile;
this.shouldTransferModFile = isWithMod;
- refreshModFileState();
+ this.shouldRefreshModFileStateFromResource =
+ this.shouldTransferModFile && shouldRefreshModFileStateFromResource;
+ this.modFile = modFile;
+ if (this.shouldRefreshModFileStateFromResource) {
+ refreshModFileState();
+ } else {
+ this.isWithMod = this.shouldTransferModFile && Objects.nonNull(this.modFile);
+ }
// TODO: process the shared mod file
this.sharedModFile =
resource.getSharedModFile() != null ? resource.getSharedModFile().getFile() : null;
@@ -327,16 +341,7 @@ public long getExtractTime() {
public boolean internallyIncreaseResourceReferenceCount(final String holderMessage) {
extractTime = System.nanoTime();
try {
- if (Objects.nonNull(resource)) {
- PipeTsFileDeletionBarrier.getInstance()
- .awaitPendingDeletionIfNecessary(resource.getTsFilePath());
- }
- refreshModFileState();
- tsFile = PipeDataNodeResourceManager.tsfile().increaseFileReference(tsFile, true, pipeName);
- if (isWithMod) {
- modFile =
- PipeDataNodeResourceManager.tsfile().increaseFileReference(modFile, false, pipeName);
- }
+ pinSnapshotForFirstReference();
return true;
} catch (final Exception e) {
LOGGER.warn(
@@ -432,6 +437,8 @@ public PipeTsFileInsertionEvent shallowCopySelfAndBindPipeTaskMetaForProgressRep
getSourceDatabaseNameFromDataRegion(),
resource,
tsFile,
+ modFile,
+ shouldRefreshModFileStateFromResource,
shouldTransferModFile,
isLoaded,
isGeneratedByHistoricalExtractor,
@@ -871,32 +878,18 @@ public PipeEventResource eventResourceBuilder() {
}
private void refreshModFileState() {
- if (!shouldTransferModFile || Objects.isNull(resource)) {
+ if (!shouldTransferModFile) {
isWithMod = false;
modFile = null;
return;
}
- // Once the event is pinned (referenceCount > 0), `modFile` should already be the
- // hardlinked/copied file managed by PipeTsFileResourceManager. We must not overwrite it by
- // reloading the original `resource.getExclusiveModFile()` path, otherwise reference tracking
- // will be broken. However, we still need to support the case where the mod file is created
- // after pinning: in this case we pin it lazily and then keep the pinned path stable.
- if (referenceCount.get() > 0) {
- if (Objects.isNull(modFile) && resource.getExclusiveModFile().exists()) {
- try {
- modFile =
- PipeDataNodeResourceManager.tsfile()
- .increaseFileReference(resource.getExclusiveModFile().getFile(), false, pipeName);
- } catch (final Exception e) {
- LOGGER.warn(
- "Failed to pin late-created mod file {} for tsfile {}, will transfer without mod.",
- resource.getExclusiveModFile().getFile(),
- tsFile,
- e);
- modFile = null;
- }
- }
+ if (!shouldRefreshModFileStateFromResource) {
+ isWithMod = Objects.nonNull(modFile);
+ return;
+ }
+
+ if (Objects.isNull(resource)) {
isWithMod = Objects.nonNull(modFile);
return;
}
@@ -905,6 +898,64 @@ private void refreshModFileState() {
modFile = isWithMod ? resource.getExclusiveModFile().getFile() : null;
}
+ private void pinSnapshotForFirstReference() throws IOException {
+ if (Objects.nonNull(resource)) {
+ PipeTsFileDeletionBarrier.getInstance()
+ .awaitPendingDeletionIfNecessary(resource.getTsFilePath());
+ resource.readLock();
+ try {
+ tsFile = PipeDataNodeResourceManager.tsfile().increaseFileReference(tsFile, true, pipeName);
+ pinModSnapshotUnderResourceReadLock();
+ shouldRefreshModFileStateFromResource = false;
+ } finally {
+ resource.readUnlock();
+ }
+ return;
+ }
+
+ tsFile = PipeDataNodeResourceManager.tsfile().increaseFileReference(tsFile, true, pipeName);
+ if (Objects.nonNull(modFile)) {
+ modFile = PipeDataNodeResourceManager.tsfile().increaseFileReference(modFile, false, pipeName);
+ isWithMod = true;
+ } else {
+ isWithMod = false;
+ }
+ shouldRefreshModFileStateFromResource = false;
+ }
+
+ private void pinModSnapshotUnderResourceReadLock() throws IOException {
+ if (!shouldTransferModFile) {
+ isWithMod = false;
+ modFile = null;
+ return;
+ }
+
+ if (!shouldRefreshModFileStateFromResource) {
+ if (Objects.nonNull(modFile)) {
+ modFile = PipeDataNodeResourceManager.tsfile().increaseFileReference(modFile, false, pipeName);
+ isWithMod = true;
+ } else {
+ isWithMod = false;
+ }
+ return;
+ }
+
+ final ModificationFile resourceExclusiveModFile = resource.getExclusiveModFile();
+ resourceExclusiveModFile.writeLock();
+ try {
+ if (resourceExclusiveModFile.exists()) {
+ modFile =
+ PipeDataNodeResourceManager.tsfile()
+ .increaseFileReference(resourceExclusiveModFile.getFile(), false, pipeName);
+ } else {
+ modFile = null;
+ }
+ isWithMod = Objects.nonNull(modFile);
+ } finally {
+ resourceExclusiveModFile.writeUnlock();
+ }
+ }
+
private static class PipeTsFileInsertionEventResource extends PipeEventResource {
private final File tsFile;
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionSource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionSource.java
index 809b40f6eba02..76b4dd1f30b30 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionSource.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionSource.java
@@ -856,6 +856,8 @@ private Event supplyTsFileEvent(final TsFileResource resource) {
resource.getDatabaseName(),
resource,
null,
+ null,
+ true,
shouldTransferModFile,
false,
true,
diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java
index 8d40122ed6b4b..e66aa98d49dae 100644
--- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java
+++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java
@@ -107,6 +107,8 @@ public void testAuthCheck() throws Exception {
"db",
resource,
null,
+ null,
+ true,
true,
false,
false,
@@ -132,6 +134,8 @@ public void testAuthCheck() throws Exception {
"root.db",
resource,
null,
+ null,
+ true,
true,
false,
false,
@@ -182,6 +186,8 @@ public void testLateCreatedModFileCanStillBeObservedAfterShallowCopy() throws Ex
"root.db",
resource,
null,
+ null,
+ true,
true,
false,
false,
@@ -239,6 +245,180 @@ public void testLateCreatedModFileCanStillBeObservedAfterShallowCopy() throws Ex
}
}
+ @Test
+ public void testShallowCopyKeepsPinnedModSnapshotAfterSourceModDisappears() throws Exception {
+ final File tsFile =
+ new File(
+ TsFileNameGenerator.generateNewTsFilePath(
+ TestConstant.BASE_OUTPUT_PATH + IoTDBConstant.SEQUENCE_FOLDER_NAME, 1, 1, 1, 4));
+ PipeTsFileInsertionEvent originalEvent = null;
+ PipeTsFileInsertionEvent copiedEvent = null;
+ try {
+ Assert.assertTrue(tsFile.getParentFile().mkdirs() || tsFile.getParentFile().exists());
+ Assert.assertTrue(tsFile.createNewFile() || tsFile.exists());
+
+ final TsFileResource resource = new TsFileResource(tsFile);
+ resource.setStatus(TsFileResourceStatus.NORMAL);
+ resource
+ .getExclusiveModFile()
+ .write(new TreeDeletionEntry(new MeasurementPath("root.db.d1.s1"), 0, 1));
+ final File originalModFile = resource.getExclusiveModFile().getFile();
+ Assert.assertTrue(originalModFile.exists());
+
+ originalEvent =
+ new PipeTsFileInsertionEvent(
+ false,
+ "root.db",
+ resource,
+ null,
+ null,
+ true,
+ true,
+ false,
+ false,
+ null,
+ null,
+ 1L,
+ null,
+ buildUnionPattern(true, Collections.singletonList(new IoTDBTreePattern(true, null))),
+ new TablePattern(false, null, null),
+ null,
+ null,
+ null,
+ true,
+ Long.MIN_VALUE,
+ Long.MAX_VALUE);
+ Assert.assertTrue(originalEvent.increaseReferenceCount("assigner"));
+
+ final File assignerPinnedModFile = originalEvent.getModFile();
+ Assert.assertNotNull(assignerPinnedModFile);
+ Assert.assertNotEquals(
+ originalModFile.getAbsolutePath(), assignerPinnedModFile.getAbsolutePath());
+
+ Assert.assertTrue(originalModFile.delete());
+ Assert.assertFalse(originalModFile.exists());
+
+ copiedEvent =
+ originalEvent.shallowCopySelfAndBindPipeTaskMetaForProgressReport(
+ "testPipeCopy",
+ 2L,
+ null,
+ buildUnionPattern(true, Collections.singletonList(new IoTDBTreePattern(true, null))),
+ new TablePattern(false, null, null),
+ null,
+ null,
+ null,
+ true,
+ Long.MIN_VALUE,
+ Long.MAX_VALUE);
+
+ Assert.assertTrue(copiedEvent.isWithMod());
+ Assert.assertEquals(
+ assignerPinnedModFile.getAbsolutePath(), copiedEvent.getModFile().getAbsolutePath());
+
+ Assert.assertTrue(copiedEvent.increaseReferenceCount("source"));
+ final File pipePinnedModFile = copiedEvent.getModFile();
+ Assert.assertNotNull(pipePinnedModFile);
+ Assert.assertTrue(pipePinnedModFile.exists());
+ Assert.assertNotEquals(originalModFile.getAbsolutePath(), pipePinnedModFile.getAbsolutePath());
+ } finally {
+ if (copiedEvent != null) {
+ copiedEvent.clearReferenceCount("source");
+ copiedEvent.close();
+ }
+ if (originalEvent != null) {
+ originalEvent.clearReferenceCount("assigner");
+ originalEvent.close();
+ }
+ FileUtils.deleteFileOrDirectory(new File(TestConstant.BASE_OUTPUT_PATH));
+ }
+ }
+
+ @Test
+ public void testPinnedEventDoesNotAdoptFutureModFile() throws Exception {
+ final File tsFile =
+ new File(
+ TsFileNameGenerator.generateNewTsFilePath(
+ TestConstant.BASE_OUTPUT_PATH + IoTDBConstant.SEQUENCE_FOLDER_NAME, 1, 1, 1, 5));
+ PipeTsFileInsertionEvent originalEvent = null;
+ PipeTsFileInsertionEvent copiedEvent = null;
+ try {
+ Assert.assertTrue(tsFile.getParentFile().mkdirs() || tsFile.getParentFile().exists());
+ Assert.assertTrue(tsFile.createNewFile() || tsFile.exists());
+
+ final TsFileResource resource = new TsFileResource(tsFile);
+ resource.setStatus(TsFileResourceStatus.NORMAL);
+
+ originalEvent =
+ new PipeTsFileInsertionEvent(
+ false,
+ "root.db",
+ resource,
+ null,
+ null,
+ true,
+ true,
+ false,
+ false,
+ null,
+ "testPipe",
+ 1L,
+ null,
+ buildUnionPattern(true, Collections.singletonList(new IoTDBTreePattern(true, null))),
+ new TablePattern(false, null, null),
+ null,
+ null,
+ null,
+ true,
+ Long.MIN_VALUE,
+ Long.MAX_VALUE);
+
+ Assert.assertFalse(originalEvent.isWithMod());
+ Assert.assertTrue(originalEvent.increaseReferenceCount("assigner"));
+ Assert.assertFalse(originalEvent.isWithMod());
+ Assert.assertNull(originalEvent.getModFile());
+
+ resource
+ .getExclusiveModFile()
+ .write(new TreeDeletionEntry(new MeasurementPath("root.db.d1.s1"), 0, 1));
+ Assert.assertTrue(resource.getExclusiveModFile().getFile().exists());
+
+ Assert.assertFalse(originalEvent.isWithMod());
+ Assert.assertNull(originalEvent.getModFile());
+
+ copiedEvent =
+ originalEvent.shallowCopySelfAndBindPipeTaskMetaForProgressReport(
+ "testPipeCopy",
+ 2L,
+ null,
+ buildUnionPattern(true, Collections.singletonList(new IoTDBTreePattern(true, null))),
+ new TablePattern(false, null, null),
+ null,
+ null,
+ null,
+ true,
+ Long.MIN_VALUE,
+ Long.MAX_VALUE);
+
+ Assert.assertFalse(copiedEvent.isWithMod());
+ Assert.assertNull(copiedEvent.getModFile());
+
+ Assert.assertTrue(copiedEvent.increaseReferenceCount("source"));
+ Assert.assertFalse(copiedEvent.isWithMod());
+ Assert.assertNull(copiedEvent.getModFile());
+ } finally {
+ if (copiedEvent != null) {
+ copiedEvent.clearReferenceCount("source");
+ copiedEvent.close();
+ }
+ if (originalEvent != null) {
+ originalEvent.clearReferenceCount("assigner");
+ originalEvent.close();
+ }
+ FileUtils.deleteFileOrDirectory(new File(TestConstant.BASE_OUTPUT_PATH));
+ }
+ }
+
@Test
public void testPinnedModFilePathIsStableAfterIncreaseReferenceCount() throws Exception {
final File tsFile =
@@ -259,6 +439,8 @@ public void testPinnedModFilePathIsStableAfterIncreaseReferenceCount() throws Ex
"root.db",
resource,
null,
+ null,
+ true,
true,
false,
false,
@@ -328,6 +510,8 @@ public void testIncreaseReferenceCountWaitsForPendingDeletionBarrier() throws Ex
"root.db",
resource,
null,
+ null,
+ true,
true,
false,
false,
@@ -377,7 +561,8 @@ public void testIncreaseReferenceCountWaitsForPendingDeletionBarrier() throws Ex
Assert.assertTrue(increased.get());
Assert.assertTrue(event.isWithMod());
Assert.assertNotNull(event.getModFile());
- Assert.assertNotEquals(originalModFile.getAbsolutePath(), event.getModFile().getAbsolutePath());
+ Assert.assertNotEquals(
+ originalModFile.getAbsolutePath(), event.getModFile().getAbsolutePath());
} finally {
// Ensure the singleton barrier is not leaked into other tests.
if (tsFilePath != null) {
diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/TsFileInsertionEventParserTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/TsFileInsertionEventParserTest.java
index 8814190755e27..259ec83650db6 100644
--- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/TsFileInsertionEventParserTest.java
+++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/TsFileInsertionEventParserTest.java
@@ -560,6 +560,8 @@ private void testTsFilePointNum(
"",
new TsFileResource(tsFile),
null,
+ null,
+ true,
true,
false,
false,
From 6f96abe173e3acea8312e08a36ff2b2920084454 Mon Sep 17 00:00:00 2001
From: Peng Junzhi <201250214@smail.nju.edu.cn>
Date: Mon, 23 Mar 2026 11:26:16 +0800
Subject: [PATCH 7/9] spotless
---
.../pipe/event/common/tsfile/PipeTsFileInsertionEvent.java | 6 ++++--
.../iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java | 3 ++-
2 files changed, 6 insertions(+), 3 deletions(-)
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java
index 6bd5c7859f3b2..c362eb1232570 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java
@@ -915,7 +915,8 @@ private void pinSnapshotForFirstReference() throws IOException {
tsFile = PipeDataNodeResourceManager.tsfile().increaseFileReference(tsFile, true, pipeName);
if (Objects.nonNull(modFile)) {
- modFile = PipeDataNodeResourceManager.tsfile().increaseFileReference(modFile, false, pipeName);
+ modFile =
+ PipeDataNodeResourceManager.tsfile().increaseFileReference(modFile, false, pipeName);
isWithMod = true;
} else {
isWithMod = false;
@@ -932,7 +933,8 @@ private void pinModSnapshotUnderResourceReadLock() throws IOException {
if (!shouldRefreshModFileStateFromResource) {
if (Objects.nonNull(modFile)) {
- modFile = PipeDataNodeResourceManager.tsfile().increaseFileReference(modFile, false, pipeName);
+ modFile =
+ PipeDataNodeResourceManager.tsfile().increaseFileReference(modFile, false, pipeName);
isWithMod = true;
} else {
isWithMod = false;
diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java
index e66aa98d49dae..455750e635ea8 100644
--- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java
+++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java
@@ -320,7 +320,8 @@ public void testShallowCopyKeepsPinnedModSnapshotAfterSourceModDisappears() thro
final File pipePinnedModFile = copiedEvent.getModFile();
Assert.assertNotNull(pipePinnedModFile);
Assert.assertTrue(pipePinnedModFile.exists());
- Assert.assertNotEquals(originalModFile.getAbsolutePath(), pipePinnedModFile.getAbsolutePath());
+ Assert.assertNotEquals(
+ originalModFile.getAbsolutePath(), pipePinnedModFile.getAbsolutePath());
} finally {
if (copiedEvent != null) {
copiedEvent.clearReferenceCount("source");
From ab77341325f027baa2453b1c88c2bdcbe565df46 Mon Sep 17 00:00:00 2001
From: Peng Junzhi <201250214@smail.nju.edu.cn>
Date: Mon, 23 Mar 2026 11:26:57 +0800
Subject: [PATCH 8/9] chore: drop local investigation notes from pr
---
...ensusv2_delete_tsfile_mod_inconsistency.md | 411 ------------------
1 file changed, 411 deletions(-)
delete mode 100644 iotconsensusv2_delete_tsfile_mod_inconsistency.md
diff --git a/iotconsensusv2_delete_tsfile_mod_inconsistency.md b/iotconsensusv2_delete_tsfile_mod_inconsistency.md
deleted file mode 100644
index 9c769d2354305..0000000000000
--- a/iotconsensusv2_delete_tsfile_mod_inconsistency.md
+++ /dev/null
@@ -1,411 +0,0 @@
-# IoTConsensusV2 并发 Delete + TsFile 同步导致副本不一致:证据链与修复说明
-
-## 0. 结论摘要
-
-在本次 benchmark 场景里,IoTConsensusV2 的副本不一致,根因已经收敛为:
-
-1. 本地 IoTV2 `DELETE` 的传播,与 sealed TsFile 上 `.mods` 的物化,不是一个原子动作;
-2. `PipeTsFileInsertionEvent` 的第一次 `increaseReferenceCount()` 会把 tsfile/mod 的传输快照 pin 下来;
-3. 如果这个第一次 pin 发生在 delete 把对应 `.mods` 写出来之前,那么后续即使 delete 已经成功同步,tsfile 事件也可能仍然带着“无 mod”的旧快照;
-4. follower 若先执行 `delete(N)`,后 `seal/load tsfile(N+1)`,这个 tsfile 上本应删除的数据就会“回弹”。
-
-这里要特别区分两件事:
-
-- “event 被创建 / TsFileEpoch 被 extracted”
-- “event 第一次被 pin / snapshot 用于传输”
-
-真正缺失同步的是第二件事,不是单纯的“event 创建得早”。
-
-另外,之前分支上做的两类修复仍然是有价值的,但它们不是根因闭环:
-
-- `PipeTsFileInsertionEvent` 动态 refresh mod,可避免把 mod 是否存在永久冻结在构造时;
-- pinned mod path 稳定化,可避免 reference tracking 与实际传输文件不一致;
-- 更关键的是,第一次 pin 完成后必须冻结 snapshot,后续不能再从 live resource 吸收“未来才出现”的 mod。
-
-但仅靠 refresh 仍然不够,因为 tsfile 可能已经更早被 pin 住;此时再 refresh,也改不了已经固定下来的传输快照。最终根修复需要把“delete 的 sealed-file materialization”与“tsfile 的第一次 pin”串起来。
-
-在把 barrier 补上之后,又进一步发现一个 residual window:
-
-1. realtime 原始 event 在 assigner 第一次 pin 后,`tsFile` 已经变成 pipe 目录里的 hardlink;
-2. 但后续 `shallowCopy` 给各个 pipe source 创建副本时,旧代码并没有继承已经 pin 好的 `modFile`;
-3. 这些副本仍然会重新读取 live `resource.getExclusiveModFile()`;
-4. 如果此时 source mod 已经被 compaction / settle / replace 删除或替换,副本仍然可能丢 mod。
-
-因此最终修复除了 delete barrier 之外,还要保证:
-
-- 首次 snapshot/pin 在资源锁保护下完成;
-- 第一次 pin 完成后,事件的 mod snapshot 立即冻结;
-- `shallowCopy` 继承已经 pin 好的 mod snapshot 状态,而不是回头读取 live resource。
-
----
-
-## 1. 测试场景与异常现象
-
-### 1.1 场景
-
-- 共识协议:IoTConsensusV2(默认参数)
-- 负载:benchmark 并发读写,过程中 CLI 执行 delete
-- 操作:未执行任何节点故障/启停
-
-### 1.2 校验 SQL
-
-```sql
-select
- count(s_0),count(s_1000),count(s_2000),count(s_3000),count(s_4000),
- count(s_5000),count(s_6000),count(s_7000),count(s_8000),count(s_9999)
-from root.treedb.g_0.** align by device;
-```
-
-### 1.3 结果差异(Tree)
-
-证据文件:
-
-- `/Users/pengjunzhi/Code/iotdb-11/bug-report-and-log/v2082rc6_iotv2_query_result/q_all_online_tree.out`
-- `/Users/pengjunzhi/Code/iotdb-11/bug-report-and-log/v2082rc6_iotv2_query_result/q_stop_ip2_tree.out`
-
-关键行均为第 58 行:
-
-- all-online:`root.treedb.g_0.nonaligned_12` 的 `count(s_1000)=26960`
-- stop-ip2:`root.treedb.g_0.nonaligned_12` 的 `count(s_1000)=27350`
-
-差值为 `+390`,说明某个副本上 `s_1000` 的 delete 没有完整体现在查询结果里。
-
-### 1.4 删除语句确实覆盖问题列
-
-证据文件:
-
-- `/Users/pengjunzhi/Code/iotdb-11/bug-report-and-log/delete-sql.txt`
-
-第 4 行包含:
-
-```text
-delete from root.treedb.g_0.nonaligned_12.s_1000 ...
-```
-
----
-
-## 2. 关键证据链
-
-本节只关注同一个问题 tsfile:
-
-- Region:`DataRegion 21`
-- tsfile:`1773977715979-129-0-0.tsfile`
-- 路径:`root.treedb.g_0.nonaligned_12.s_1000`
-
-### 2.1 Leader(dn_ip2)侧证据
-
-日志:
-
-- `/Users/pengjunzhi/Code/iotdb-11/bug-report-and-log/v2082rc6_iotv2_deletion_logs/dn_ip2/log_datanode_all.log`
-
-片段可用:
-
-```bash
-nl -ba /Users/pengjunzhi/Code/iotdb-11/bug-report-and-log/v2082rc6_iotv2_deletion_logs/dn_ip2/log_datanode_all.log | sed -n '19935,19946p'
-```
-
-关键日志:
-
-```text
-19937 2026-03-20 11:36:02,284 ... TsFileEpoch not found ... creating a new one
-19938 2026-03-20 11:36:02,285 ... All data in TsFileEpoch ... was extracted
-19939 2026-03-20 11:36:02,293 ... [Deletion] ... nonaligned_12.s_1000 ... written into 19 mod files
-19940 2026-03-20 11:36:02,494 ... DeleteNodeTransfer: no.160 event successfully processed!
-19945 2026-03-20 11:36:04,356 ... transferred file ... replicate index=161
-```
-
-这段日志能证明:
-
-1. `11:36:02.285`,leader 已经把这个 tsfile 对应的 epoch 提取出来;
-2. `11:36:02.293`,delete 随后才把 `nonaligned_12.s_1000` 写入 sealed tsfile 的 `.mods`;
-3. `11:36:02.494`,delete 事件以 `replicateIndex=160` 处理成功;
-4. `11:36:04.356`,对应 tsfile 事件以 `replicateIndex=161` 完成传输。
-
-这正是“leader 侧先出现 tsfile 提取,再出现 deletion 写 mod,再出现 delete 复制成功,最后才完成 tsfile 传输”的完整证据链。
-
-### 2.2 Follower(dn_ip4)侧证据
-
-日志:
-
-- `/Users/pengjunzhi/Code/iotdb-11/bug-report-and-log/v2082rc6_iotv2_deletion_logs/dn_ip4/log_datanode_all.log`
-
-片段可用:
-
-```bash
-nl -ba /Users/pengjunzhi/Code/iotdb-11/bug-report-and-log/v2082rc6_iotv2_deletion_logs/dn_ip4/log_datanode_all.log | sed -n '18304,18338p'
-```
-
-关键日志:
-
-```text
-18306 2026-03-20 11:36:02,488 ... start to receive ... replicateIndex:160
-18307 2026-03-20 11:36:02,493 ... [Deletion] ... nonaligned_12.s_1000 ... written into 6 mod files
-18308 2026-03-20 11:36:02,493 ... process ... replicateIndex:160 ... successfully
-
-18309 2026-03-20 11:36:02,520 ... start to receive ... replicateIndex:161
-18326 2026-03-20 11:36:03,677 ... starting to receive tsFile seal
-18328 2026-03-20 11:36:04,352 ... Load tsfile in unsequence list ...
-18331 2026-03-20 11:36:04,353 ... TsFile ... successfully loaded ...
-18338 2026-03-20 11:36:04,354 ... process ... replicateIndex:161 ... successfully
-```
-
-这段日志能证明:
-
-1. follower 先完整执行了 `delete(160)`;
-2. 然后才接收并 seal/load 了 `tsfile(161)`;
-3. 如果这个 tsfile 传过来时不带 `.mods`,那么 follower 上该 tsfile 的删除效果就不会被补上,最终查询结果会偏大。
-
----
-
-## 3. 关于 replicateIndex 与 extracted 时序的澄清
-
-之前一个很容易混淆的点是:
-
-- `TsFileEpoch` 被 extracted 的时间
-- `replicateIndex` 被赋值的时间
-- `PipeTsFileInsertionEvent` 第一次被 pin 的时间
-
-它们不是同一件事。
-
-`replicateIndex` 的赋值发生在 Pipe 分发阶段,而不是 `TsFileEpoch` 被 extracted 的那个瞬间。代码位置:
-
-- `/Users/pengjunzhi/Code/iotdb-11/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/PipeDataRegionAssigner.java`
-
-关键逻辑:
-
-```java
-if (DataRegionConsensusImpl.getInstance() instanceof IoTConsensusV2
- && IoTConsensusV2Processor.isShouldReplicate(innerEvent)) {
- innerEvent.setReplicateIndexForIoTV2(
- ReplicateProgressDataNodeManager.assignReplicateIndexForIoTV2(source.getPipeName()));
-}
-```
-
-所以,leader 日志里“tsfile 更早 extracted”只说明它更早进入了 realtime 提取链路,并不能单独推出“它的 replicateIndex 一定更小或更大”。
-
-本次最终收敛后的判断是:
-
-- follower 日志已经足够证明最终外部执行顺序是 `delete(160)` 在前、`tsfile(161)` 在后;
-- leader 日志已经足够证明 `.mods` 是在 tsfile 被 extracted 之后才物化出来;
-- 真正决定是否丢 delete 的,是 tsfile 第一次 pin/snapshot 是否早于 delete 的 sealed-file materialization。
-
-也就是说,根因并不是“必须存在 replicateIndex 逆序赋值 bug”,而是“pin 时刻缺少与 delete materialization 的同步”。
-
----
-
-## 4. 为什么“仅在传输前 refresh mod”仍然不够
-
-分支之前的修复做了两件正确的事:
-
-1. `PipeTsFileInsertionEvent` 不再把 `isWithMod/modFile` 永久冻结在构造时,而是在使用前 refresh;
-2. 一旦 mod 已被 pin,就保持 pinned mod path 稳定,不再被后续 refresh 覆盖。
-
-这些修复解决了两类真实问题:
-
-- event 创建后、真正传输前,晚到的 `.mods` 能被看见;
-- 避免 reference count 绑定的是一个 mod 文件,而实际传输又换成另一个 mod 文件。
-
-但它仍然不能覆盖下面这个窗口:
-
-1. tsfile event 已经在上游某个环节调用了第一次 `increaseReferenceCount()`;
-2. 这次调用把 tsfile 以及当时能看到的 mod 快照 pin 下来;
-3. 随后 delete 才去 sealed tsfile 上写 `.mods`;
-4. 再晚一点即使 `refreshModFileState()` 能看到新 mod,也不一定还能安全替换已经 pinned 的传输快照。
-
-换句话说:
-
-- refresh 解决的是“看不看得见晚到 mod”
-- barrier 解决的是“能不能保证第一次 pin 不早于 delete materialization”
-
-后者才是这次副本不一致的根因闭环。
-
----
-
-## 5. 真实根因(代码级)
-
-### 5.1 缺失同步的位置
-
-delete 路径里,本地 IoTV2 删除会经历:
-
-1. WAL / delete node 处理;
-2. 通过 `PipeInsertionDataNodeListener.listenToDeleteData(...)` 把 delete 事件发布到 Pipe / IoTV2;
-3. 再把对应 sealed tsfile 上的 `.mods` 真正写出来。
-
-而 tsfile 事件在另一条链路里,会在第一次 `increaseReferenceCount()` 时 pin 住将要传输的 tsfile/mod。
-
-原来这两条链路之间没有 per-tsfile 的同步机制,所以会出现:
-
-- delete 已经发出;
-- 但对应 sealed tsfile 的 `.mods` 还没写完;
-- tsfile 事件已经先一步完成了第一次 pin。
-
-### 5.2 这会怎样导致 follower 不一致
-
-当 follower 的外部执行顺序是:
-
-1. `delete(160)` 先执行;
-2. `tsfile(161)` 后 seal/load;
-
-如果 `tsfile(161)` 传来的仍然是 delete 物化前 pin 下来的“无 mod 快照”,那么 follower 载入这个 tsfile 后,对应删除就会缺失,表现出来就是 count 偏大。
-
----
-
-## 6. 修复方案
-
-### 6.1 既有加固:继续保留动态 mod refresh 与 pinned path 稳定化
-
-文件:
-
-- `/Users/pengjunzhi/Code/iotdb-11/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java`
-
-这部分是分支上已有修复,但最终语义已经收敛为“pin 前可刷新,pin 后冻结”:
-
-1. 在第一次 pin 之前,`refreshModFileState()` 仍然会读取 live `resource.anyModFileExists()`,因此 event 创建后、pin 前晚到的 `.mods` 仍可被观察到;
-2. 一旦第一次 pin 完成,`PipeTsFileInsertionEvent` 会立刻冻结 mod snapshot,不再回头读取 live `resource.getExclusiveModFile()`;
-3. 已经 pin 好的 `modFile` 路径会保持稳定,不会再被后续 refresh 覆盖。
-
-这一步解决的是三件事:
-
-- pin 前的晚到 mod 可见性;
-- pinned mod path 正确性;
-- 防止旧 tsfile event 在 pin 之后误吸收未来 delete 产生的 mod。
-
-### 6.2 根修复:给 sealed tsfile 的 delete materialization 增加 barrier
-
-新增文件:
-
-- `/Users/pengjunzhi/Code/iotdb-11/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/PipeTsFileDeletionBarrier.java`
-
-核心思路:
-
-1. delete 路径先算出这次本地 IoTV2 delete 会影响哪些 sealed tsfile;
-2. 在真正发布 delete 并物化 `.mods` 之前,把这些 tsfile path 注册到 barrier;
-3. tsfile event 在第一次 `increaseReferenceCount()` 前,先检查该 tsfile 是否存在 pending deletion;
-4. 若存在,则等待 delete 完成 sealed-file `.mods` 物化后再继续 pin/snapshot;
-5. delete 完成后释放 barrier。
-
-落地位置:
-
-- `DataRegion.deleteByDevice(...)`
-- `DataRegion.deleteByTable(...)`
-- `DataRegion.deleteDataDirectly(...)`
-- `PipeTsFileInsertionEvent.internallyIncreaseResourceReferenceCount(...)`
-
-对应代码文件:
-
-- `/Users/pengjunzhi/Code/iotdb-11/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java`
-- `/Users/pengjunzhi/Code/iotdb-11/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java`
-
-### 6.3 补齐剩余窗口:首次 snapshot 加锁,shallow copy 继承 frozen snapshot
-
-在补上 barrier 之后,还需要再解决一个很隐蔽的 realtime-only 问题:
-
-1. 原始 realtime `PipeTsFileInsertionEvent` 在 assigner 的第一次 `increaseReferenceCount()` 时,会把 tsfile/mod snapshot 到 common pipe 目录;
-2. 随后 assigner 会对 event 做 `shallowCopy`,为每个 pipe source 生成副本;
-3. 旧代码里,`shallowCopy` 只继承了已经 pin 好的 `tsFile`,却没有继承“mod snapshot 已冻结”这个状态;
-4. 当原始 event 已经 pin 到有 mod 时,副本可能丢掉已 pin 的 `modFile`,重新读取 live `resource.getExclusiveModFile()`;
-5. 当原始 event 已经 pin 到“无 mod”时,副本也可能重新看见未来才出现的 mod;
-6. 如果此时源 `.mods` 已被 compaction / settle / replace 删除或替换,副本还可能再次丢 mod。
-
-本次补充修复做了两件事:
-
-1. `PipeTsFileInsertionEvent` 的第一次 snapshot/pin 改为在 `TsFileResource.readLock()` 下完成;
-2. 若需要 pin live exclusive mod,则在同一阶段持有对应 `ModificationFile.writeLock()` 复制 mod;
-3. 第一次 pin 完成后,event 会把“mod snapshot 是否还允许从 live resource refresh”切成 false,从而冻结 snapshot;
-4. `shallowCopy` 会直接继承这份 frozen snapshot 状态,包括:
- - 已经 pin 好的 `modFile` 路径;
- - 以及“已经确认当前 snapshot 没有 mod”这个空快照状态;
-5. 后续 pipe source 副本再做第一次 pin 时,会基于这份 frozen snapshot 继续复制,而不是回头读 live resource。
-
-这样可以避免:
-
-- 首次 snapshot 过程中,source tsfile 被 compaction/remove 删除;
-- 原始 event 已经拿到 pinned snapshot,但 shallow copy 仍然回头依赖 live mod;
-- 原始 event 已经 pin 到“无 mod”快照,但 shallow copy 又错误吸收未来 mod;
-- source mod 被 merge/replace 后,pipe 副本看不到已经 pin 好的 mod。
-
-### 6.4 为什么这套组合修复才是当前闭环
-
-它把系统收敛到两个都安全的分支:
-
-1. tsfile event 的第一次 pin 早于 delete 注册 barrier
-
-这说明 tsfile 已经先进入复制顺序,它会保持更早的外部顺序;follower 先 load tsfile、后 apply delete,是安全的。
-
-2. delete 先注册 barrier,再发生 tsfile event 的第一次 pin
-
-这时 tsfile event 会等待,直到 delete 完成 sealed-file `.mods` 物化,再把带 mod 的快照 pin 下来;follower 即使先 apply delete、后 load tsfile,也仍然安全。
-
-真正不安全的只有第三种:
-
-- delete 已经开始影响该 tsfile,但 barrier 尚未建立;
-- 同时 tsfile 又先完成了第一次 pin。
-
-这正是本次修复消除的窗口。
-
----
-
-## 7. 回归测试
-
-测试文件:
-
-- `/Users/pengjunzhi/Code/iotdb-11/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java`
-
-本次新增:
-
-- `testIncreaseReferenceCountWaitsForPendingDeletionBarrier`
-- `testShallowCopyKeepsPinnedModSnapshotAfterSourceModDisappears`
-- `testPinnedEventDoesNotAdoptFutureModFile`
-
-覆盖点:
-
-1. 手工为某个 tsfile path 注册 barrier;
-2. 在另一个线程里调用 `event.increaseReferenceCount(...)`;
-3. 断言它会先阻塞;
-4. 阻塞期间创建 mod 文件;
-5. 释放 barrier 后,断言事件成功完成 pin,并且能观察到 mod。
-
-分支上原有的 late-created mod 可见性测试也继续保留,因此现在覆盖了三类行为:
-
-- mod 在 event 创建后、pin 前出现
-- delete 先占住 barrier,tsfile 的第一次 pin 必须等待
-- 原始 event 已经 pin 了 mod snapshot 后,即使 source mod 消失,shallow copy 仍然沿用 pinned snapshot
-- 原始 event 已经 pin 到“无 mod”快照后,即使未来真的出现 mod,旧事件及其 shallow copy 也不会再吸收它
-
----
-
-## 8. 验证结果
-
-执行命令:
-
-```bash
-./mvnw -pl iotdb-core/datanode -am \
- -Dtest=org.apache.iotdb.db.pipe.event.PipeTsFileInsertionEventTest,org.apache.iotdb.db.pipe.event.TsFileInsertionEventParserTest \
- -Dsurefire.failIfNoSpecifiedTests=false -DfailIfNoTests=false \
- test -DskipITs
-```
-
-结果:
-
-- `BUILD SUCCESS`
-
----
-
-## 9. 仍建议后续单独跟进的问题
-
-`historicalExtractor` 里 historical deletion 没有 assign replicateIndex,这个判断依然成立,而且值得单独修。
-
-它对应的是另一个独立风险:
-
-1. historical tsfile 路径会 assign IoTV2 replicateIndex;
-2. historical deletion 路径当前没有补齐 replicateIndex;
-3. `IoTConsensusV2Processor` 会过滤 `NO_COMMIT_ID` 事件;
-4. 因而 historical deletion 存在被 IoTV2 处理链过滤掉的风险。
-
-但它不是这次 benchmark 事故的主因,因为本次事故里的 deletion 已经明确在 leader/follower 两侧都被成功处理了。
-
-建议另开 issue 跟进:
-
-1. 在 historical deletion 路径补齐 IoTV2 replicateIndex;
-2. 增加对应回归测试;
-3. 与本次副本不一致修复分开提交,降低回归面。
From 8c17a6a82a19991c3fba0de0034c84444898c07a Mon Sep 17 00:00:00 2001
From: Peng Junzhi <201250214@smail.nju.edu.cn>
Date: Mon, 23 Mar 2026 19:10:26 +0800
Subject: [PATCH 9/9] fix(iotv2): order delete materialization with tsfile
snapshot
---
.../deletion/PipeTsFileDeletionBarrier.java | 290 ++++++++++++----
.../tsfile/PipeTsFileInsertionEvent.java | 314 ++++++++++++++----
.../assigner/PipeDataRegionAssigner.java | 20 +-
.../storageengine/dataregion/DataRegion.java | 149 +++++++--
.../PipeTsFileDeletionBarrierTest.java | 162 +++++++++
.../event/PipeTsFileInsertionEventTest.java | 259 ++++++++++++---
6 files changed, 994 insertions(+), 200 deletions(-)
create mode 100644 iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/deletion/PipeTsFileDeletionBarrierTest.java
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/PipeTsFileDeletionBarrier.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/PipeTsFileDeletionBarrier.java
index 96b81eafd83aa..954821dea0278 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/PipeTsFileDeletionBarrier.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/deletion/PipeTsFileDeletionBarrier.java
@@ -19,121 +19,299 @@
package org.apache.iotdb.db.pipe.consensus.deletion;
+import org.apache.iotdb.commons.utils.TestOnly;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.NavigableSet;
import java.util.Objects;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.atomic.AtomicInteger;
/**
- * A lightweight barrier for coordinating delete materialization and TsFile transfer.
+ * An ordered barrier for coordinating delete materialization and TsFile transfer.
+ *
+ * Delete materialization and realtime TsFile transfer are observed by followers through
+ * replicateIndex order, but the leader locally materializes delete mods after publishing delete
+ * events. This barrier ensures that a TsFile snapshot:
*
- *
In IoTDB, a {@code DELETE} is first published to pipe/consensus and then its modifications are
- * materialized to sealed TsFiles as mod files. Under concurrent flush and delete, it is possible
- * that a TsFile event gets pinned/transferred before its corresponding mod file is generated,
- * causing followers to miss deletions if they apply delete before loading that TsFile.
+ *
1. includes every delete that entered the delete path before the TsFile got its replicate
+ * index, and
*
- *
This barrier allows the delete path to mark a set of TsFiles as "pending deletion
- * materialization". A TsFile event will wait (best effort) for the pending count to drop to zero
- * before pinning/snapshotting the TsFile and its mod.
+ *
2. excludes deletes that entered after that replicate index.
*/
public class PipeTsFileDeletionBarrier {
private static final Logger LOGGER = LoggerFactory.getLogger(PipeTsFileDeletionBarrier.class);
- private final ConcurrentMap tsFilePath2PendingCounter =
+ private static final long INITIAL_DELETE_SEQ = 0L;
+
+ private final ConcurrentMap regionId2DeletionState =
+ new ConcurrentHashMap<>();
+ private final ConcurrentMap tsFilePath2BarrierState =
new ConcurrentHashMap<>();
- public void registerPendingDeletion(final Collection tsFilePaths) {
- if (Objects.isNull(tsFilePaths) || tsFilePaths.isEmpty()) {
+ public long beginDeletion(final int regionId) {
+ return getOrCreateRegionDeletionState(regionId).beginDeletion();
+ }
+
+ public void resolveDeletionTargets(
+ final int regionId, final long deleteSeq, final Collection tsFilePaths) {
+ if (deleteSeq <= INITIAL_DELETE_SEQ) {
return;
}
- tsFilePaths.forEach(this::registerPendingDeletion);
+
+ final RegionDeletionState regionDeletionState = getOrCreateRegionDeletionState(regionId);
+ final Set sanitizedTsFilePaths = sanitizeTsFilePaths(tsFilePaths);
+
+ synchronized (regionDeletionState.monitor) {
+ for (final String tsFilePath : sanitizedTsFilePaths) {
+ getOrCreateTsFileBarrierState(tsFilePath).registerPendingDeletion(deleteSeq);
+ }
+ regionDeletionState.resolveDeletion(deleteSeq);
+ }
}
- public void registerPendingDeletion(final String tsFilePath) {
+ public long beginSnapshot(final int regionId, final String tsFilePath) {
if (Objects.isNull(tsFilePath)) {
- return;
+ return INITIAL_DELETE_SEQ;
}
- tsFilePath2PendingCounter.compute(
- tsFilePath,
- (k, v) -> {
- final PendingCounter counter = v == null ? new PendingCounter() : v;
- counter.pendingCount.incrementAndGet();
- return counter;
- });
+ final RegionDeletionState regionDeletionState = getOrCreateRegionDeletionState(regionId);
+ synchronized (regionDeletionState.monitor) {
+ final long snapshotUpperBound = regionDeletionState.getMaxAllocatedDeletionSeq();
+ getOrCreateTsFileBarrierState(tsFilePath).registerSnapshot(snapshotUpperBound);
+ return snapshotUpperBound;
+ }
}
- public void releasePendingDeletion(final Collection tsFilePaths) {
- if (Objects.isNull(tsFilePaths) || tsFilePaths.isEmpty()) {
+ public void awaitDeletionResolutionUpTo(final int regionId, final long deleteSeqUpperBound)
+ throws InterruptedException {
+ if (deleteSeqUpperBound <= INITIAL_DELETE_SEQ) {
return;
}
- tsFilePaths.forEach(this::releasePendingDeletion);
+
+ getOrCreateRegionDeletionState(regionId).awaitDeletionResolutionUpTo(deleteSeqUpperBound);
}
- public void releasePendingDeletion(final String tsFilePath) {
- if (Objects.isNull(tsFilePath)) {
+ public void awaitPendingDeletionsUpTo(final String tsFilePath, final long deleteSeqUpperBound)
+ throws InterruptedException {
+ if (Objects.isNull(tsFilePath) || deleteSeqUpperBound <= INITIAL_DELETE_SEQ) {
return;
}
- final PendingCounter counter = tsFilePath2PendingCounter.get(tsFilePath);
- if (counter == null) {
+ final TsFileBarrierState barrierState = tsFilePath2BarrierState.get(tsFilePath);
+ if (Objects.isNull(barrierState)) {
return;
}
- final int remaining = counter.pendingCount.decrementAndGet();
- if (remaining > 0) {
+ barrierState.awaitPendingDeletionsUpTo(deleteSeqUpperBound);
+ }
+
+ public void awaitSnapshotsBeforeMaterialization(final String tsFilePath, final long deleteSeq)
+ throws InterruptedException {
+ if (Objects.isNull(tsFilePath) || deleteSeq <= INITIAL_DELETE_SEQ) {
return;
}
- if (remaining < 0) {
- LOGGER.warn(
- "PipeTsFileDeletionBarrier: pending deletion count becomes negative for {}, current {}. "
- + "This may indicate mismatched register/release.",
- tsFilePath,
- remaining);
+ final TsFileBarrierState barrierState = tsFilePath2BarrierState.get(tsFilePath);
+ if (Objects.isNull(barrierState)) {
+ return;
}
- tsFilePath2PendingCounter.remove(tsFilePath, counter);
- synchronized (counter.monitor) {
- counter.monitor.notifyAll();
- }
+ barrierState.awaitSnapshotsBeforeMaterialization(deleteSeq);
}
- public void awaitPendingDeletionIfNecessary(final String tsFilePath) {
+ public void finishSnapshot(final String tsFilePath, final long snapshotUpperBound) {
if (Objects.isNull(tsFilePath)) {
return;
}
- final PendingCounter counter = tsFilePath2PendingCounter.get(tsFilePath);
- if (counter == null) {
+ final TsFileBarrierState barrierState = tsFilePath2BarrierState.get(tsFilePath);
+ if (Objects.isNull(barrierState)) {
return;
}
- synchronized (counter.monitor) {
- while (counter.pendingCount.get() > 0) {
- try {
- counter.monitor.wait();
- } catch (final InterruptedException e) {
- Thread.currentThread().interrupt();
+ barrierState.finishSnapshot(snapshotUpperBound);
+ cleanupTsFileBarrierStateIfEmpty(tsFilePath, barrierState);
+ }
+
+ public void finishDeletion(final long deleteSeq, final Collection tsFilePaths) {
+ if (deleteSeq <= INITIAL_DELETE_SEQ) {
+ return;
+ }
+
+ final Set sanitizedTsFilePaths = sanitizeTsFilePaths(tsFilePaths);
+ for (final String tsFilePath : sanitizedTsFilePaths) {
+ final TsFileBarrierState barrierState = tsFilePath2BarrierState.get(tsFilePath);
+ if (Objects.isNull(barrierState)) {
+ continue;
+ }
+
+ barrierState.finishDeletion(deleteSeq);
+ cleanupTsFileBarrierStateIfEmpty(tsFilePath, barrierState);
+ }
+ }
+
+ @TestOnly
+ public void clear() {
+ regionId2DeletionState.clear();
+ tsFilePath2BarrierState.clear();
+ }
+
+ private RegionDeletionState getOrCreateRegionDeletionState(final int regionId) {
+ return regionId2DeletionState.computeIfAbsent(regionId, ignored -> new RegionDeletionState());
+ }
+
+ private TsFileBarrierState getOrCreateTsFileBarrierState(final String tsFilePath) {
+ return tsFilePath2BarrierState.computeIfAbsent(tsFilePath, ignored -> new TsFileBarrierState());
+ }
+
+ private Set sanitizeTsFilePaths(final Collection tsFilePaths) {
+ if (Objects.isNull(tsFilePaths) || tsFilePaths.isEmpty()) {
+ return Collections.emptySet();
+ }
+
+ final Set sanitizedTsFilePaths = new HashSet<>();
+ for (final String tsFilePath : tsFilePaths) {
+ if (Objects.nonNull(tsFilePath)) {
+ sanitizedTsFilePaths.add(tsFilePath);
+ }
+ }
+ return sanitizedTsFilePaths;
+ }
+
+ private void cleanupTsFileBarrierStateIfEmpty(
+ final String tsFilePath, final TsFileBarrierState barrierState) {
+ tsFilePath2BarrierState.computeIfPresent(
+ tsFilePath,
+ (ignored, state) -> {
+ if (state != barrierState) {
+ return state;
+ }
+ synchronized (state.monitor) {
+ return state.isEmptyUnsafe() ? null : state;
+ }
+ });
+ }
+
+ private static class RegionDeletionState {
+ private final Object monitor = new Object();
+ private final NavigableSet unresolvedDeletionSeqs = new TreeSet<>();
+
+ private long maxAllocatedDeletionSeq = INITIAL_DELETE_SEQ;
+ private long maxResolvedDeletionSeq = INITIAL_DELETE_SEQ;
+
+ private long beginDeletion() {
+ synchronized (monitor) {
+ final long deleteSeq = ++maxAllocatedDeletionSeq;
+ unresolvedDeletionSeqs.add(deleteSeq);
+ return deleteSeq;
+ }
+ }
+
+ private long getMaxAllocatedDeletionSeq() {
+ return maxAllocatedDeletionSeq;
+ }
+
+ private void resolveDeletion(final long deleteSeq) {
+ synchronized (monitor) {
+ if (deleteSeq > maxAllocatedDeletionSeq || deleteSeq <= INITIAL_DELETE_SEQ) {
LOGGER.warn(
- "PipeTsFileDeletionBarrier: interrupted while waiting pending deletions for {}, "
- + "will continue without waiting.",
- tsFilePath,
- e);
+ "PipeTsFileDeletionBarrier: try to resolve invalid delete seq {}, max allocated {}.",
+ deleteSeq,
+ maxAllocatedDeletionSeq);
return;
}
+
+ unresolvedDeletionSeqs.remove(deleteSeq);
+ while (maxResolvedDeletionSeq < maxAllocatedDeletionSeq
+ && !unresolvedDeletionSeqs.contains(maxResolvedDeletionSeq + 1)) {
+ maxResolvedDeletionSeq++;
+ }
+ monitor.notifyAll();
+ }
+ }
+
+ private void awaitDeletionResolutionUpTo(final long deleteSeqUpperBound)
+ throws InterruptedException {
+ synchronized (monitor) {
+ while (maxResolvedDeletionSeq < deleteSeqUpperBound) {
+ monitor.wait();
+ }
}
}
}
- private static class PendingCounter {
- private final AtomicInteger pendingCount = new AtomicInteger(0);
+ private static class TsFileBarrierState {
private final Object monitor = new Object();
+ private final NavigableSet pendingDeletionSeqs = new TreeSet<>();
+ private final NavigableMap snapshotUpperBound2ReferenceCount = new TreeMap<>();
+
+ private void registerPendingDeletion(final long deleteSeq) {
+ synchronized (monitor) {
+ pendingDeletionSeqs.add(deleteSeq);
+ monitor.notifyAll();
+ }
+ }
+
+ private void finishDeletion(final long deleteSeq) {
+ synchronized (monitor) {
+ pendingDeletionSeqs.remove(deleteSeq);
+ monitor.notifyAll();
+ }
+ }
+
+ private void registerSnapshot(final long snapshotUpperBound) {
+ synchronized (monitor) {
+ snapshotUpperBound2ReferenceCount.merge(snapshotUpperBound, 1, Integer::sum);
+ monitor.notifyAll();
+ }
+ }
+
+ private void finishSnapshot(final long snapshotUpperBound) {
+ synchronized (monitor) {
+ snapshotUpperBound2ReferenceCount.computeIfPresent(
+ snapshotUpperBound, (ignored, count) -> count == 1 ? null : count - 1);
+ monitor.notifyAll();
+ }
+ }
+
+ private void awaitPendingDeletionsUpTo(final long deleteSeqUpperBound)
+ throws InterruptedException {
+ synchronized (monitor) {
+ while (!pendingDeletionSeqs.headSet(deleteSeqUpperBound, true).isEmpty()) {
+ monitor.wait();
+ }
+ }
+ }
+
+ private void awaitSnapshotsBeforeMaterialization(final long deleteSeq)
+ throws InterruptedException {
+ synchronized (monitor) {
+ while (hasSnapshotEarlierThan(deleteSeq)) {
+ monitor.wait();
+ }
+ }
+ }
+
+ private boolean hasSnapshotEarlierThan(final long deleteSeq) {
+ final Map.Entry firstSnapshot = snapshotUpperBound2ReferenceCount.firstEntry();
+ return Objects.nonNull(firstSnapshot) && firstSnapshot.getKey() < deleteSeq;
+ }
+
+ private boolean isEmptyUnsafe() {
+ return pendingDeletionSeqs.isEmpty() && snapshotUpperBound2ReferenceCount.isEmpty();
+ }
}
/////////////////////////////// singleton ///////////////////////////////
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java
index c362eb1232570..5fdf8e22ec164 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/event/common/tsfile/PipeTsFileInsertionEvent.java
@@ -85,12 +85,18 @@ public class PipeTsFileInsertionEvent extends PipeInsertionEvent
// Whether this event should transfer mod files if they exist.
private boolean shouldTransferModFile;
+ // Whether the assigner-side first retain should delay mod snapshotting until replicateIndex is
+ // assigned to the pipe-specific copy.
+ private boolean shouldDelayModSnapshotUntilReplicateIndex;
// Whether this event should consult the live TsFileResource for mod visibility before pinning.
private boolean shouldRefreshModFileStateFromResource;
+ // Whether the current modFile is already pinned and should no longer be overwritten by refresh.
+ private boolean isModPinned;
// This is true iff the modFile exists and should be transferred
private boolean isWithMod;
private File modFile;
private final File sharedModFile;
+ private long snapshotUpperBoundForDelayedMod = NO_COMMIT_ID;
protected final boolean isLoaded;
protected final boolean isGeneratedByPipe;
@@ -136,7 +142,8 @@ public PipeTsFileInsertionEvent(
null,
true,
Long.MIN_VALUE,
- Long.MAX_VALUE);
+ Long.MAX_VALUE,
+ false);
}
public PipeTsFileInsertionEvent(
@@ -161,6 +168,54 @@ public PipeTsFileInsertionEvent(
final boolean skipIfNoPrivileges,
final long startTime,
final long endTime) {
+ this(
+ isTableModelEvent,
+ databaseNameFromDataRegion,
+ resource,
+ tsFile,
+ modFile,
+ shouldRefreshModFileStateFromResource,
+ isWithMod,
+ isLoaded,
+ isGeneratedByHistoricalExtractor,
+ tableNames,
+ pipeName,
+ creationTime,
+ pipeTaskMeta,
+ treePattern,
+ tablePattern,
+ userId,
+ userName,
+ cliHostname,
+ skipIfNoPrivileges,
+ startTime,
+ endTime,
+ false);
+ }
+
+ private PipeTsFileInsertionEvent(
+ final Boolean isTableModelEvent,
+ final String databaseNameFromDataRegion,
+ final TsFileResource resource,
+ final File tsFile,
+ final File modFile,
+ final boolean shouldRefreshModFileStateFromResource,
+ final boolean isWithMod,
+ final boolean isLoaded,
+ final boolean isGeneratedByHistoricalExtractor,
+ final Set tableNames,
+ final String pipeName,
+ final long creationTime,
+ final PipeTaskMeta pipeTaskMeta,
+ final TreePattern treePattern,
+ final TablePattern tablePattern,
+ final String userId,
+ final String userName,
+ final String cliHostname,
+ final boolean skipIfNoPrivileges,
+ final long startTime,
+ final long endTime,
+ final boolean shouldDelayModSnapshotUntilReplicateIndex) {
super(
pipeName,
creationTime,
@@ -185,9 +240,14 @@ public PipeTsFileInsertionEvent(
this.tsFile = Objects.isNull(tsFile) ? resource.getTsFile() : tsFile;
this.shouldTransferModFile = isWithMod;
+ this.shouldDelayModSnapshotUntilReplicateIndex = shouldDelayModSnapshotUntilReplicateIndex;
this.shouldRefreshModFileStateFromResource =
this.shouldTransferModFile && shouldRefreshModFileStateFromResource;
this.modFile = modFile;
+ this.isModPinned =
+ this.shouldTransferModFile
+ && !this.shouldRefreshModFileStateFromResource
+ && Objects.nonNull(this.modFile);
if (this.shouldRefreshModFileStateFromResource) {
refreshModFileState();
} else {
@@ -311,6 +371,14 @@ public void disableMod4NonTransferPipes(final boolean shouldTransferModFile) {
refreshModFileState();
}
+ public void enableDelayModSnapshotUntilReplicateIndex() {
+ shouldDelayModSnapshotUntilReplicateIndex = true;
+ }
+
+ public boolean isModPinned() {
+ return isModPinned;
+ }
+
public boolean isLoaded() {
return isLoaded;
}
@@ -337,6 +405,23 @@ public long getExtractTime() {
/////////////////////////// EnrichedEvent ///////////////////////////
+ @Override
+ public void setReplicateIndexForIoTV2(final long replicateIndexForIoTV2) {
+ super.setReplicateIndexForIoTV2(replicateIndexForIoTV2);
+
+ if (!shouldDelayModSnapshotUntilReplicateIndex
+ || !shouldTransferModFile
+ || Objects.isNull(resource)
+ || Objects.isNull(pipeName)
+ || snapshotUpperBoundForDelayedMod != NO_COMMIT_ID) {
+ return;
+ }
+
+ snapshotUpperBoundForDelayedMod =
+ PipeTsFileDeletionBarrier.getInstance()
+ .beginSnapshot(Integer.parseInt(resource.getDataRegionId()), resource.getTsFilePath());
+ }
+
@Override
public boolean internallyIncreaseResourceReferenceCount(final String holderMessage) {
extractTime = System.nanoTime();
@@ -344,6 +429,9 @@ public boolean internallyIncreaseResourceReferenceCount(final String holderMessa
pinSnapshotForFirstReference();
return true;
} catch (final Exception e) {
+ if (e instanceof InterruptedException) {
+ Thread.currentThread().interrupt();
+ }
LOGGER.warn(
String.format(
"Increase reference count for TsFile %s or modFile %s error. Holder Message: %s",
@@ -362,7 +450,7 @@ public boolean internallyIncreaseResourceReferenceCount(final String holderMessa
public boolean internallyDecreaseResourceReferenceCount(final String holderMessage) {
try {
PipeDataNodeResourceManager.tsfile().decreaseFileReference(tsFile, pipeName);
- if (isWithMod) {
+ if (isModPinned && Objects.nonNull(modFile)) {
PipeDataNodeResourceManager.tsfile().decreaseFileReference(modFile, pipeName);
}
close();
@@ -453,7 +541,8 @@ public PipeTsFileInsertionEvent shallowCopySelfAndBindPipeTaskMetaForProgressRep
cliHostname,
skipIfNoPrivileges,
startTime,
- endTime);
+ endTime,
+ shouldDelayModSnapshotUntilReplicateIndex);
}
@Override
@@ -871,20 +960,21 @@ public PipeEventResource eventResourceBuilder() {
this.referenceCount,
this.pipeName,
this.tsFile,
- this.isWithMod,
+ this.isModPinned,
this.modFile,
- this.sharedModFile,
this.eventParser);
}
private void refreshModFileState() {
if (!shouldTransferModFile) {
isWithMod = false;
- modFile = null;
+ if (!isModPinned) {
+ modFile = null;
+ }
return;
}
- if (!shouldRefreshModFileStateFromResource) {
+ if (isModPinned || !shouldRefreshModFileStateFromResource) {
isWithMod = Objects.nonNull(modFile);
return;
}
@@ -898,72 +988,186 @@ private void refreshModFileState() {
modFile = isWithMod ? resource.getExclusiveModFile().getFile() : null;
}
- private void pinSnapshotForFirstReference() throws IOException {
- if (Objects.nonNull(resource)) {
- PipeTsFileDeletionBarrier.getInstance()
- .awaitPendingDeletionIfNecessary(resource.getTsFilePath());
- resource.readLock();
- try {
- tsFile = PipeDataNodeResourceManager.tsfile().increaseFileReference(tsFile, true, pipeName);
- pinModSnapshotUnderResourceReadLock();
- shouldRefreshModFileStateFromResource = false;
- } finally {
- resource.readUnlock();
+ private void pinSnapshotForFirstReference() throws Exception {
+ final SnapshotPinResult snapshotPinResult =
+ Objects.nonNull(resource)
+ ? pinResourceBackedSnapshotForFirstReference()
+ : pinStandaloneSnapshotForFirstReference();
+
+ tsFile = snapshotPinResult.tsFile;
+ modFile = snapshotPinResult.modFile;
+ isWithMod = snapshotPinResult.isWithMod;
+ isModPinned = snapshotPinResult.isModPinned;
+ shouldRefreshModFileStateFromResource = snapshotPinResult.shouldRefreshModFileStateFromResource;
+ }
+
+ private SnapshotPinResult pinResourceBackedSnapshotForFirstReference() throws Exception {
+ final File pinnedTsFile = pinTsFileUnderResourceReadLock();
+ final boolean shouldFinishSnapshotBarrier = shouldDelayModSnapshotForPipeCopy();
+ boolean modSnapshotAttempted = false;
+ try {
+ if (shouldOnlyPinTsFileOnFirstReference()) {
+ refreshModFileState();
+ return new SnapshotPinResult(
+ pinnedTsFile, modFile, isWithMod, false, shouldRefreshModFileStateFromResource);
}
- return;
+
+ if (shouldFinishSnapshotBarrier) {
+ awaitDelayedDeletionMaterializationBeforeModSnapshot();
+ }
+
+ modSnapshotAttempted = true;
+ return pinModSnapshotUnderResourceReadLock(pinnedTsFile);
+ } catch (final Exception e) {
+ if (!modSnapshotAttempted) {
+ releasePinnedSnapshotQuietly(pinnedTsFile, null);
+ }
+ throw e;
+ } finally {
+ if (shouldFinishSnapshotBarrier) {
+ finishDelayedSnapshotBarrier();
+ }
+ }
+ }
+
+ private SnapshotPinResult pinStandaloneSnapshotForFirstReference() throws IOException {
+ final File pinnedTsFile =
+ PipeDataNodeResourceManager.tsfile().increaseFileReference(tsFile, true, pipeName);
+
+ if (Objects.isNull(modFile)) {
+ return new SnapshotPinResult(
+ pinnedTsFile, null, false, false, shouldRefreshModFileStateFromResource);
}
- tsFile = PipeDataNodeResourceManager.tsfile().increaseFileReference(tsFile, true, pipeName);
- if (Objects.nonNull(modFile)) {
- modFile =
+ File pinnedModFile = null;
+ try {
+ pinnedModFile =
PipeDataNodeResourceManager.tsfile().increaseFileReference(modFile, false, pipeName);
- isWithMod = true;
- } else {
- isWithMod = false;
+ return new SnapshotPinResult(pinnedTsFile, pinnedModFile, true, true, false);
+ } catch (final Exception e) {
+ releasePinnedSnapshotQuietly(pinnedTsFile, pinnedModFile);
+ throw e;
}
- shouldRefreshModFileStateFromResource = false;
}
- private void pinModSnapshotUnderResourceReadLock() throws IOException {
- if (!shouldTransferModFile) {
- isWithMod = false;
- modFile = null;
- return;
+ private File pinTsFileUnderResourceReadLock() throws IOException {
+ resource.readLock();
+ try {
+ return PipeDataNodeResourceManager.tsfile().increaseFileReference(tsFile, true, pipeName);
+ } finally {
+ resource.readUnlock();
}
+ }
+
+ private void awaitDelayedDeletionMaterializationBeforeModSnapshot() throws InterruptedException {
+ final int regionId = Integer.parseInt(resource.getDataRegionId());
+ final PipeTsFileDeletionBarrier barrier = PipeTsFileDeletionBarrier.getInstance();
+ barrier.awaitDeletionResolutionUpTo(regionId, snapshotUpperBoundForDelayedMod);
+ barrier.awaitPendingDeletionsUpTo(resource.getTsFilePath(), snapshotUpperBoundForDelayedMod);
+ }
- if (!shouldRefreshModFileStateFromResource) {
- if (Objects.nonNull(modFile)) {
- modFile =
- PipeDataNodeResourceManager.tsfile().increaseFileReference(modFile, false, pipeName);
- isWithMod = true;
- } else {
- isWithMod = false;
+ private SnapshotPinResult pinModSnapshotUnderResourceReadLock(final File pinnedTsFile)
+ throws IOException {
+ File pinnedModFile = null;
+ boolean withMod = false;
+
+ resource.readLock();
+ try {
+ final ModificationFile resourceExclusiveModFile = resource.getExclusiveModFile();
+ resourceExclusiveModFile.writeLock();
+ try {
+ if (!shouldTransferModFile) {
+ return new SnapshotPinResult(pinnedTsFile, null, false, false, false);
+ }
+
+ if (!shouldRefreshModFileStateFromResource && Objects.nonNull(modFile)) {
+ pinnedModFile =
+ PipeDataNodeResourceManager.tsfile().increaseFileReference(modFile, false, pipeName);
+ withMod = true;
+ } else if (resourceExclusiveModFile.exists()) {
+ pinnedModFile =
+ PipeDataNodeResourceManager.tsfile()
+ .increaseFileReference(resourceExclusiveModFile.getFile(), false, pipeName);
+ withMod = true;
+ }
+ } finally {
+ resourceExclusiveModFile.writeUnlock();
}
+ } catch (final Exception e) {
+ releasePinnedSnapshotQuietly(pinnedTsFile, pinnedModFile);
+ throw e;
+ } finally {
+ resource.readUnlock();
+ }
+
+ return new SnapshotPinResult(
+ pinnedTsFile, pinnedModFile, withMod, Objects.nonNull(pinnedModFile), false);
+ }
+
+ private boolean shouldOnlyPinTsFileOnFirstReference() {
+ return shouldDelayModSnapshotUntilReplicateIndex && Objects.isNull(pipeName);
+ }
+
+ private boolean shouldDelayModSnapshotForPipeCopy() {
+ return shouldDelayModSnapshotUntilReplicateIndex
+ && Objects.nonNull(pipeName)
+ && snapshotUpperBoundForDelayedMod != NO_COMMIT_ID;
+ }
+
+ private void finishDelayedSnapshotBarrier() {
+ if (!shouldDelayModSnapshotForPipeCopy()) {
return;
}
- final ModificationFile resourceExclusiveModFile = resource.getExclusiveModFile();
- resourceExclusiveModFile.writeLock();
+ PipeTsFileDeletionBarrier.getInstance()
+ .finishSnapshot(resource.getTsFilePath(), snapshotUpperBoundForDelayedMod);
+ snapshotUpperBoundForDelayedMod = NO_COMMIT_ID;
+ }
+
+ private void releasePinnedSnapshotQuietly(final File pinnedTsFile, final File pinnedModFile) {
try {
- if (resourceExclusiveModFile.exists()) {
- modFile =
- PipeDataNodeResourceManager.tsfile()
- .increaseFileReference(resourceExclusiveModFile.getFile(), false, pipeName);
- } else {
- modFile = null;
+ if (Objects.nonNull(pinnedModFile)) {
+ PipeDataNodeResourceManager.tsfile().decreaseFileReference(pinnedModFile, pipeName);
}
- isWithMod = Objects.nonNull(modFile);
- } finally {
- resourceExclusiveModFile.writeUnlock();
+ } catch (final Exception e) {
+ LOGGER.warn("Decrease reference count for modFile {} error.", pinnedModFile, e);
+ }
+
+ try {
+ if (Objects.nonNull(pinnedTsFile)) {
+ PipeDataNodeResourceManager.tsfile().decreaseFileReference(pinnedTsFile, pipeName);
+ }
+ } catch (final Exception e) {
+ LOGGER.warn("Decrease reference count for TsFile {} error.", pinnedTsFile, e);
+ }
+ }
+
+ private static class SnapshotPinResult {
+ private final File tsFile;
+ private final File modFile;
+ private final boolean isWithMod;
+ private final boolean isModPinned;
+ private final boolean shouldRefreshModFileStateFromResource;
+
+ private SnapshotPinResult(
+ final File tsFile,
+ final File modFile,
+ final boolean isWithMod,
+ final boolean isModPinned,
+ final boolean shouldRefreshModFileStateFromResource) {
+ this.tsFile = tsFile;
+ this.modFile = modFile;
+ this.isWithMod = isWithMod;
+ this.isModPinned = isModPinned;
+ this.shouldRefreshModFileStateFromResource = shouldRefreshModFileStateFromResource;
}
}
private static class PipeTsFileInsertionEventResource extends PipeEventResource {
private final File tsFile;
- private final boolean isWithMod;
+ private final boolean isModPinned;
private final File modFile;
- private final File sharedModFile; // unused now
private final AtomicReference eventParser;
private final String pipeName;
@@ -972,16 +1176,14 @@ private PipeTsFileInsertionEventResource(
final AtomicInteger referenceCount,
final String pipeName,
final File tsFile,
- final boolean isWithMod,
+ final boolean isModPinned,
final File modFile,
- final File sharedModFile,
final AtomicReference eventParser) {
super(isReleased, referenceCount);
this.pipeName = pipeName;
this.tsFile = tsFile;
- this.isWithMod = isWithMod;
+ this.isModPinned = isModPinned;
this.modFile = modFile;
- this.sharedModFile = sharedModFile;
this.eventParser = eventParser;
}
@@ -990,7 +1192,7 @@ protected void finalizeResource() {
try {
// decrease reference count
PipeDataNodeResourceManager.tsfile().decreaseFileReference(tsFile, pipeName);
- if (isWithMod) {
+ if (isModPinned && Objects.nonNull(modFile)) {
PipeDataNodeResourceManager.tsfile().decreaseFileReference(modFile, pipeName);
}
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/PipeDataRegionAssigner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/PipeDataRegionAssigner.java
index 9c7182f051c74..7041d723bd4d4 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/PipeDataRegionAssigner.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/assigner/PipeDataRegionAssigner.java
@@ -93,13 +93,19 @@ public PipeDataRegionAssigner(final int dataRegionId) {
}
public void publishToAssign(final PipeRealtimeEvent event) {
+ final EnrichedEvent innerEvent = event.getEvent();
+ if (innerEvent instanceof PipeTsFileInsertionEvent
+ && DataRegionConsensusImpl.getInstance() instanceof IoTConsensusV2
+ && IoTConsensusV2Processor.isShouldReplicate(innerEvent)) {
+ ((PipeTsFileInsertionEvent) innerEvent).enableDelayModSnapshotUntilReplicateIndex();
+ }
+
if (!event.increaseReferenceCount(PipeDataRegionAssigner.class.getName())) {
LOGGER.warn(
"The reference count of the realtime event {} cannot be increased, skipping it.", event);
return;
}
- final EnrichedEvent innerEvent = event.getEvent();
eventCounter.increaseEventCount(innerEvent);
if (innerEvent instanceof PipeHeartbeatEvent) {
((PipeHeartbeatEvent) innerEvent).onPublished();
@@ -174,6 +180,12 @@ private void assignToSource(
source.getRealtimeDataExtractionStartTime(),
source.getRealtimeDataExtractionEndTime());
final EnrichedEvent innerEvent = copiedEvent.getEvent();
+ if (innerEvent instanceof PipeTsFileInsertionEvent) {
+ final PipeTsFileInsertionEvent tsFileInsertionEvent =
+ (PipeTsFileInsertionEvent) innerEvent;
+ tsFileInsertionEvent.disableMod4NonTransferPipes(source.isShouldTransferModFile());
+ }
+
// if using IoTV2, assign a replicateIndex for this realtime event
if (DataRegionConsensusImpl.getInstance() instanceof IoTConsensusV2
&& IoTConsensusV2Processor.isShouldReplicate(innerEvent)) {
@@ -187,12 +199,6 @@ private void assignToSource(
innerEvent);
}
- if (innerEvent instanceof PipeTsFileInsertionEvent) {
- final PipeTsFileInsertionEvent tsFileInsertionEvent =
- (PipeTsFileInsertionEvent) innerEvent;
- tsFileInsertionEvent.disableMod4NonTransferPipes(source.isShouldTransferModFile());
- }
-
if (innerEvent instanceof PipeDeleteDataNodeEvent) {
final PipeDeleteDataNodeEvent deleteDataNodeEvent =
(PipeDeleteDataNodeEvent) innerEvent;
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 b0e9be5eb2acd..3c09dd37d17f9 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
@@ -2831,6 +2831,88 @@ private void getTwoKindsOfTsFiles(
.forEach(unsealedResource::add);
}
+ private DeleteMaterializationBarrierContext beginDeleteMaterializationBarrierIfNecessary(
+ final boolean shouldUseBarrier) {
+ if (!shouldUseBarrier) {
+ return null;
+ }
+
+ return new DeleteMaterializationBarrierContext(
+ dataRegionId.getId(),
+ PipeTsFileDeletionBarrier.getInstance().beginDeletion(dataRegionId.getId()));
+ }
+
+ private Set collectImpactedTsFilePaths(
+ final Collection sealedTsFiles, final ModEntry deletion) {
+ return sealedTsFiles.stream()
+ .filter(resource -> !canSkipDelete(resource, deletion))
+ .map(TsFileResource::getTsFilePath)
+ .collect(Collectors.toSet());
+ }
+
+ private void resolveDeleteMaterializationTargets(
+ final DeleteMaterializationBarrierContext barrierContext,
+ final Collection tsFilePaths) {
+ if (Objects.isNull(barrierContext) || barrierContext.resolved) {
+ return;
+ }
+
+ barrierContext.tsFilePaths.addAll(tsFilePaths);
+ PipeTsFileDeletionBarrier.getInstance()
+ .resolveDeletionTargets(
+ barrierContext.regionId, barrierContext.deleteSeq, barrierContext.tsFilePaths);
+ barrierContext.resolved = true;
+ }
+
+ private void awaitSnapshotsBeforeMaterializing(
+ final DeleteMaterializationBarrierContext barrierContext,
+ final Collection sealedTsFiles,
+ final ModEntry deletion)
+ throws InterruptedException {
+ if (Objects.isNull(barrierContext)) {
+ return;
+ }
+
+ final PipeTsFileDeletionBarrier barrier = PipeTsFileDeletionBarrier.getInstance();
+ for (final TsFileResource sealedTsFile : sealedTsFiles) {
+ if (!canSkipDelete(sealedTsFile, deletion)) {
+ barrier.awaitSnapshotsBeforeMaterialization(
+ sealedTsFile.getTsFilePath(), barrierContext.deleteSeq);
+ }
+ }
+ }
+
+ private void finishDeleteMaterializationBarrier(
+ final DeleteMaterializationBarrierContext barrierContext) {
+ if (Objects.isNull(barrierContext) || barrierContext.finished) {
+ return;
+ }
+
+ if (!barrierContext.resolved) {
+ PipeTsFileDeletionBarrier.getInstance()
+ .resolveDeletionTargets(
+ barrierContext.regionId, barrierContext.deleteSeq, Collections.emptySet());
+ barrierContext.resolved = true;
+ }
+
+ PipeTsFileDeletionBarrier.getInstance()
+ .finishDeletion(barrierContext.deleteSeq, barrierContext.tsFilePaths);
+ barrierContext.finished = true;
+ }
+
+ private static class DeleteMaterializationBarrierContext {
+ private final int regionId;
+ private final long deleteSeq;
+ private final Set tsFilePaths = new HashSet<>();
+ private boolean resolved = false;
+ private boolean finished = false;
+
+ private DeleteMaterializationBarrierContext(final int regionId, final long deleteSeq) {
+ this.regionId = regionId;
+ this.deleteSeq = deleteSeq;
+ }
+ }
+
public void deleteByDevice(final MeasurementPath pattern, final DeleteDataNode node)
throws IOException {
if (SettleService.getINSTANCE().getFilesToBeSettledCount().get() != 0) {
@@ -2864,6 +2946,9 @@ public void deleteByDevice(final MeasurementPath pattern, final DeleteDataNode n
}
ModEntry deletion = new TreeDeletionEntry(pattern, startTime, endTime);
+ final DeleteMaterializationBarrierContext barrierContext =
+ beginDeleteMaterializationBarrierIfNecessary(
+ DeletionResource.isDeleteNodeGeneratedInLocalByIoTV2(node));
List sealedTsFileResource = new ArrayList<>();
List unsealedTsFileResource = new ArrayList<>();
@@ -2871,17 +2956,8 @@ public void deleteByDevice(final MeasurementPath pattern, final DeleteDataNode n
// deviceMatchInfo is used for filter the matched deviceId in TsFileResource
// deviceMatchInfo contains the DeviceId means this device matched the pattern
deleteDataInUnsealedFiles(unsealedTsFileResource, deletion, sealedTsFileResource);
-
- // Prevent a TsFile event from being pinned/transferred before this deletion is materialized
- // to its corresponding mod files.
- final Set tsFilePathsPendingDeletion =
- DeletionResource.isDeleteNodeGeneratedInLocalByIoTV2(node)
- ? sealedTsFileResource.stream()
- .filter(resource -> !canSkipDelete(resource, deletion))
- .map(TsFileResource::getTsFilePath)
- .collect(Collectors.toSet())
- : Collections.emptySet();
- PipeTsFileDeletionBarrier.getInstance().registerPendingDeletion(tsFilePathsPendingDeletion);
+ resolveDeleteMaterializationTargets(
+ barrierContext, collectImpactedTsFilePaths(sealedTsFileResource, deletion));
try {
// capture deleteDataNode and wait it to be persisted to DAL.
final DeletionResource deletionResource =
@@ -2894,11 +2970,15 @@ public void deleteByDevice(final MeasurementPath pattern, final DeleteDataNode n
writeUnlock();
hasReleasedLock = true;
+ awaitSnapshotsBeforeMaterializing(barrierContext, sealedTsFileResource, deletion);
deleteDataInSealedFiles(sealedTsFileResource, deletion);
} finally {
- PipeTsFileDeletionBarrier.getInstance().releasePendingDeletion(tsFilePathsPendingDeletion);
+ finishDeleteMaterializationBarrier(barrierContext);
}
} catch (Exception e) {
+ if (e instanceof InterruptedException) {
+ Thread.currentThread().interrupt();
+ }
throw new IOException(e);
} finally {
if (!hasReleasedLock) {
@@ -2975,6 +3055,9 @@ public void deleteByTable(RelationalDeleteDataNode node) throws IOException {
}
}
+ final DeleteMaterializationBarrierContext barrierContext =
+ beginDeleteMaterializationBarrierIfNecessary(
+ DeletionResource.isDeleteNodeGeneratedInLocalByIoTV2(node));
List> sealedTsFileResourceLists = new ArrayList<>(modEntries.size());
for (TableDeletionEntry modEntry : modEntries) {
List sealedTsFileResource = new ArrayList<>();
@@ -2990,18 +3073,12 @@ public void deleteByTable(RelationalDeleteDataNode node) throws IOException {
sealedTsFileResourceLists.add(sealedTsFileResource);
}
- final Set tsFilePathsPendingDeletion = new HashSet<>();
- if (DeletionResource.isDeleteNodeGeneratedInLocalByIoTV2(node)) {
- for (int i = 0; i < modEntries.size(); i++) {
- final TableDeletionEntry modEntry = modEntries.get(i);
- tsFilePathsPendingDeletion.addAll(
- sealedTsFileResourceLists.get(i).stream()
- .filter(resource -> !canSkipDelete(resource, modEntry))
- .map(TsFileResource::getTsFilePath)
- .collect(Collectors.toSet()));
- }
+ final Set impactedTsFilePaths = new HashSet<>();
+ for (int i = 0; i < modEntries.size(); i++) {
+ impactedTsFilePaths.addAll(
+ collectImpactedTsFilePaths(sealedTsFileResourceLists.get(i), modEntries.get(i)));
}
- PipeTsFileDeletionBarrier.getInstance().registerPendingDeletion(tsFilePathsPendingDeletion);
+ resolveDeleteMaterializationTargets(barrierContext, impactedTsFilePaths);
try {
// capture deleteDataNode and wait it to be persisted to DAL.
final DeletionResource deletionResource =
@@ -3016,12 +3093,17 @@ public void deleteByTable(RelationalDeleteDataNode node) throws IOException {
hasReleasedLock = true;
for (int i = 0; i < modEntries.size(); i++) {
+ awaitSnapshotsBeforeMaterializing(
+ barrierContext, sealedTsFileResourceLists.get(i), modEntries.get(i));
deleteDataInSealedFiles(sealedTsFileResourceLists.get(i), modEntries.get(i));
}
} finally {
- PipeTsFileDeletionBarrier.getInstance().releasePendingDeletion(tsFilePathsPendingDeletion);
+ finishDeleteMaterializationBarrier(barrierContext);
}
} catch (Exception e) {
+ if (e instanceof InterruptedException) {
+ Thread.currentThread().interrupt();
+ }
throw new IOException(e);
} finally {
if (!hasReleasedLock) {
@@ -3060,18 +3142,15 @@ public void deleteDataDirectly(MeasurementPath pathToDelete, DeleteDataNode node
}
}
TreeDeletionEntry deletion = new TreeDeletionEntry(pathToDelete, startTime, endTime);
+ final DeleteMaterializationBarrierContext barrierContext =
+ beginDeleteMaterializationBarrierIfNecessary(
+ DeletionResource.isDeleteNodeGeneratedInLocalByIoTV2(node));
List sealedTsFileResource = new ArrayList<>();
List unsealedTsFileResource = new ArrayList<>();
getTwoKindsOfTsFiles(sealedTsFileResource, unsealedTsFileResource, startTime, endTime);
deleteDataDirectlyInFile(unsealedTsFileResource, deletion);
- final Set tsFilePathsPendingDeletion =
- DeletionResource.isDeleteNodeGeneratedInLocalByIoTV2(node)
- ? sealedTsFileResource.stream()
- .filter(resource -> !canSkipDelete(resource, deletion))
- .map(TsFileResource::getTsFilePath)
- .collect(Collectors.toSet())
- : Collections.emptySet();
- PipeTsFileDeletionBarrier.getInstance().registerPendingDeletion(tsFilePathsPendingDeletion);
+ resolveDeleteMaterializationTargets(
+ barrierContext, collectImpactedTsFilePaths(sealedTsFileResource, deletion));
try {
// capture deleteDataNode and wait it to be persisted to DAL.
final DeletionResource deletionResource =
@@ -3083,11 +3162,15 @@ public void deleteDataDirectly(MeasurementPath pathToDelete, DeleteDataNode node
}
writeUnlock();
releasedLock = true;
+ awaitSnapshotsBeforeMaterializing(barrierContext, sealedTsFileResource, deletion);
deleteDataDirectlyInFile(sealedTsFileResource, deletion);
} finally {
- PipeTsFileDeletionBarrier.getInstance().releasePendingDeletion(tsFilePathsPendingDeletion);
+ finishDeleteMaterializationBarrier(barrierContext);
}
} catch (Exception e) {
+ if (e instanceof InterruptedException) {
+ Thread.currentThread().interrupt();
+ }
throw new IOException(e);
} finally {
if (!releasedLock) {
diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/deletion/PipeTsFileDeletionBarrierTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/deletion/PipeTsFileDeletionBarrierTest.java
new file mode 100644
index 0000000000000..5a7a4e8a0a115
--- /dev/null
+++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/consensus/deletion/PipeTsFileDeletionBarrierTest.java
@@ -0,0 +1,162 @@
+/*
+ * 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.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+public class PipeTsFileDeletionBarrierTest {
+
+ private static final int TEST_REGION_ID = 1;
+ private static final String TEST_TSFILE_PATH = "/tmp/test.tsfile";
+
+ @After
+ public void tearDown() {
+ PipeTsFileDeletionBarrier.getInstance().clear();
+ }
+
+ @Test
+ public void testSnapshotWaitsForEarlierDeletionResolution() throws Exception {
+ final PipeTsFileDeletionBarrier barrier = PipeTsFileDeletionBarrier.getInstance();
+ final long deleteSeq = barrier.beginDeletion(TEST_REGION_ID);
+
+ final CountDownLatch started = new CountDownLatch(1);
+ final CountDownLatch finished = new CountDownLatch(1);
+ final AtomicReference failure = new AtomicReference<>(null);
+
+ final Thread thread =
+ new Thread(
+ () -> {
+ try {
+ started.countDown();
+ barrier.awaitDeletionResolutionUpTo(TEST_REGION_ID, deleteSeq);
+ } catch (final Throwable throwable) {
+ failure.set(throwable);
+ } finally {
+ finished.countDown();
+ }
+ },
+ "test-wait-delete-resolution");
+ thread.start();
+
+ Assert.assertTrue(started.await(5, TimeUnit.SECONDS));
+ Assert.assertFalse(finished.await(200, TimeUnit.MILLISECONDS));
+
+ barrier.resolveDeletionTargets(TEST_REGION_ID, deleteSeq, Collections.emptySet());
+
+ Assert.assertTrue(finished.await(5, TimeUnit.SECONDS));
+ if (failure.get() != null) {
+ throw new AssertionError("Snapshot wait should finish successfully", failure.get());
+ }
+ thread.join(TimeUnit.SECONDS.toMillis(5));
+ }
+
+ @Test
+ public void testSnapshotWaitsForPendingDeletionUpToUpperBound() throws Exception {
+ final PipeTsFileDeletionBarrier barrier = PipeTsFileDeletionBarrier.getInstance();
+ final long deleteSeq = barrier.beginDeletion(TEST_REGION_ID);
+ barrier.resolveDeletionTargets(
+ TEST_REGION_ID, deleteSeq, Collections.singleton(TEST_TSFILE_PATH));
+
+ final long snapshotUpperBound = barrier.beginSnapshot(TEST_REGION_ID, TEST_TSFILE_PATH);
+ Assert.assertEquals(deleteSeq, snapshotUpperBound);
+
+ final CountDownLatch started = new CountDownLatch(1);
+ final CountDownLatch finished = new CountDownLatch(1);
+ final AtomicReference failure = new AtomicReference<>(null);
+
+ final Thread thread =
+ new Thread(
+ () -> {
+ try {
+ started.countDown();
+ barrier.awaitPendingDeletionsUpTo(TEST_TSFILE_PATH, snapshotUpperBound);
+ } catch (final Throwable throwable) {
+ failure.set(throwable);
+ } finally {
+ finished.countDown();
+ }
+ },
+ "test-wait-pending-delete");
+ thread.start();
+
+ Assert.assertTrue(started.await(5, TimeUnit.SECONDS));
+ Assert.assertFalse(finished.await(200, TimeUnit.MILLISECONDS));
+
+ barrier.finishDeletion(deleteSeq, Collections.singleton(TEST_TSFILE_PATH));
+
+ Assert.assertTrue(finished.await(5, TimeUnit.SECONDS));
+ if (failure.get() != null) {
+ throw new AssertionError("Pending delete wait should finish successfully", failure.get());
+ }
+
+ barrier.finishSnapshot(TEST_TSFILE_PATH, snapshotUpperBound);
+ thread.join(TimeUnit.SECONDS.toMillis(5));
+ }
+
+ @Test
+ public void testLaterDeletionWaitsForEarlierSnapshot() throws Exception {
+ final PipeTsFileDeletionBarrier barrier = PipeTsFileDeletionBarrier.getInstance();
+ final long snapshotUpperBound = barrier.beginSnapshot(TEST_REGION_ID, TEST_TSFILE_PATH);
+ Assert.assertEquals(0L, snapshotUpperBound);
+
+ final long deleteSeq = barrier.beginDeletion(TEST_REGION_ID);
+ barrier.resolveDeletionTargets(
+ TEST_REGION_ID, deleteSeq, Collections.singleton(TEST_TSFILE_PATH));
+
+ final CountDownLatch started = new CountDownLatch(1);
+ final CountDownLatch finished = new CountDownLatch(1);
+ final AtomicReference failure = new AtomicReference<>(null);
+
+ final Thread thread =
+ new Thread(
+ () -> {
+ try {
+ started.countDown();
+ barrier.awaitSnapshotsBeforeMaterialization(TEST_TSFILE_PATH, deleteSeq);
+ } catch (final Throwable throwable) {
+ failure.set(throwable);
+ } finally {
+ finished.countDown();
+ }
+ },
+ "test-wait-earlier-snapshot");
+ thread.start();
+
+ Assert.assertTrue(started.await(5, TimeUnit.SECONDS));
+ Assert.assertFalse(finished.await(200, TimeUnit.MILLISECONDS));
+
+ barrier.finishSnapshot(TEST_TSFILE_PATH, snapshotUpperBound);
+
+ Assert.assertTrue(finished.await(5, TimeUnit.SECONDS));
+ if (failure.get() != null) {
+ throw new AssertionError("Later deletion wait should finish successfully", failure.get());
+ }
+
+ barrier.finishDeletion(deleteSeq, Collections.singleton(TEST_TSFILE_PATH));
+ thread.join(TimeUnit.SECONDS.toMillis(5));
+ }
+}
diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java
index 455750e635ea8..90f3d3b002f28 100644
--- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java
+++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/pipe/event/PipeTsFileInsertionEventTest.java
@@ -51,6 +51,7 @@
import org.apache.tsfile.file.metadata.enums.CompressionType;
import org.apache.tsfile.file.metadata.enums.TSEncoding;
import org.apache.tsfile.read.common.TimeRange;
+import org.junit.After;
import org.junit.Assert;
import org.junit.Test;
@@ -62,6 +63,7 @@
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
@@ -70,6 +72,11 @@
import static org.apache.iotdb.db.auth.AuthorityChecker.NO_PERMISSION_PROMOTION;
public class PipeTsFileInsertionEventTest {
+ @After
+ public void tearDown() {
+ PipeTsFileDeletionBarrier.getInstance().clear();
+ }
+
@Test
public void testAuthCheck() throws Exception {
final AccessControl oldControl = AuthorityChecker.getAccessControl();
@@ -337,10 +344,7 @@ public void testShallowCopyKeepsPinnedModSnapshotAfterSourceModDisappears() thro
@Test
public void testPinnedEventDoesNotAdoptFutureModFile() throws Exception {
- final File tsFile =
- new File(
- TsFileNameGenerator.generateNewTsFilePath(
- TestConstant.BASE_OUTPUT_PATH + IoTDBConstant.SEQUENCE_FOLDER_NAME, 1, 1, 1, 5));
+ final File tsFile = createTestTsFileUnderDataRegion(1, 1, 1, 5);
PipeTsFileInsertionEvent originalEvent = null;
PipeTsFileInsertionEvent copiedEvent = null;
try {
@@ -405,8 +409,11 @@ public void testPinnedEventDoesNotAdoptFutureModFile() throws Exception {
Assert.assertNull(copiedEvent.getModFile());
Assert.assertTrue(copiedEvent.increaseReferenceCount("source"));
- Assert.assertFalse(copiedEvent.isWithMod());
- Assert.assertNull(copiedEvent.getModFile());
+ Assert.assertTrue(copiedEvent.isWithMod());
+ Assert.assertNotNull(copiedEvent.getModFile());
+ Assert.assertNotEquals(
+ resource.getExclusiveModFile().getFile().getAbsolutePath(),
+ copiedEvent.getModFile().getAbsolutePath());
} finally {
if (copiedEvent != null) {
copiedEvent.clearReferenceCount("source");
@@ -489,14 +496,50 @@ public void testPinnedModFilePathIsStableAfterIncreaseReferenceCount() throws Ex
}
@Test
- public void testIncreaseReferenceCountWaitsForPendingDeletionBarrier() throws Exception {
+ public void testDelayedAssignerRetainDoesNotFreezeFutureMod() throws Exception {
final File tsFile =
new File(
TsFileNameGenerator.generateNewTsFilePath(
TestConstant.BASE_OUTPUT_PATH + IoTDBConstant.SEQUENCE_FOLDER_NAME, 1, 1, 1, 3));
PipeTsFileInsertionEvent event = null;
- final String holder = "test";
- String tsFilePath = null;
+ try {
+ Assert.assertTrue(tsFile.getParentFile().mkdirs() || tsFile.getParentFile().exists());
+ Assert.assertTrue(tsFile.createNewFile() || tsFile.exists());
+
+ final TsFileResource resource = new TsFileResource(tsFile);
+ resource.setStatus(TsFileResourceStatus.NORMAL);
+
+ event = createTestTsFileInsertionEvent(resource, null);
+ event.enableDelayModSnapshotUntilReplicateIndex();
+
+ Assert.assertTrue(event.increaseReferenceCount("assigner"));
+ Assert.assertFalse(event.isWithMod());
+ Assert.assertFalse(event.isModPinned());
+ Assert.assertNotEquals(tsFile.getAbsolutePath(), event.getTsFile().getAbsolutePath());
+
+ resource
+ .getExclusiveModFile()
+ .write(new TreeDeletionEntry(new MeasurementPath("root.db.d1.s1"), 0, 1));
+ final File originalModFile = resource.getExclusiveModFile().getFile();
+ Assert.assertTrue(originalModFile.exists());
+
+ Assert.assertTrue(event.isWithMod());
+ Assert.assertEquals(originalModFile.getAbsolutePath(), event.getModFile().getAbsolutePath());
+ Assert.assertFalse(event.isModPinned());
+ } finally {
+ if (event != null) {
+ event.clearReferenceCount("assigner");
+ event.close();
+ }
+ FileUtils.deleteFileOrDirectory(new File(TestConstant.BASE_OUTPUT_PATH));
+ }
+ }
+
+ @Test
+ public void testDelayedPipeCopyWaitsForEarlierDeletionBeforeFreezingMod() throws Exception {
+ final File tsFile = createTestTsFileUnderDataRegion(1, 1, 1, 6);
+ PipeTsFileInsertionEvent originalEvent = null;
+ PipeTsFileInsertionEvent copiedEvent = null;
Thread t = null;
try {
Assert.assertTrue(tsFile.getParentFile().mkdirs() || tsFile.getParentFile().exists());
@@ -504,46 +547,32 @@ public void testIncreaseReferenceCountWaitsForPendingDeletionBarrier() throws Ex
final TsFileResource resource = new TsFileResource(tsFile);
resource.setStatus(TsFileResourceStatus.NORMAL);
+ final PipeTsFileDeletionBarrier barrier = PipeTsFileDeletionBarrier.getInstance();
+ final int regionId = Integer.parseInt(resource.getDataRegionId());
+ final String tsFilePath = resource.getTsFilePath();
- event =
- new PipeTsFileInsertionEvent(
- false,
- "root.db",
- resource,
- null,
- null,
- true,
- true,
- false,
- false,
- null,
- "testPipe",
- 1L,
- null,
- buildUnionPattern(true, Collections.singletonList(new IoTDBTreePattern(true, null))),
- new TablePattern(false, null, null),
- null,
- null,
- null,
- true,
- Long.MIN_VALUE,
- Long.MAX_VALUE);
- final PipeTsFileInsertionEvent finalEvent = event;
+ originalEvent = createTestTsFileInsertionEvent(resource, null);
+ originalEvent.enableDelayModSnapshotUntilReplicateIndex();
+ Assert.assertTrue(originalEvent.increaseReferenceCount("assigner"));
+
+ copiedEvent = createTestPipeCopy(originalEvent, "testPipeCopy");
+ final long deleteSeq = barrier.beginDeletion(regionId);
+ barrier.resolveDeletionTargets(regionId, deleteSeq, Collections.singleton(tsFilePath));
+ copiedEvent.setReplicateIndexForIoTV2(100L);
- tsFilePath = resource.getTsFilePath();
- PipeTsFileDeletionBarrier.getInstance().registerPendingDeletion(tsFilePath);
final CountDownLatch started = new CountDownLatch(1);
final CountDownLatch finished = new CountDownLatch(1);
final AtomicBoolean increased = new AtomicBoolean(false);
+ final PipeTsFileInsertionEvent finalCopiedEvent = copiedEvent;
t =
new Thread(
() -> {
started.countDown();
- increased.set(finalEvent.increaseReferenceCount(holder));
+ increased.set(finalCopiedEvent.increaseReferenceCount("source"));
finished.countDown();
},
- "test-tsfile-delete-barrier");
+ "test-tsfile-delete-barrier-wait-earlier-delete");
t.start();
Assert.assertTrue(started.await(5, TimeUnit.SECONDS));
@@ -555,31 +584,165 @@ public void testIncreaseReferenceCountWaitsForPendingDeletionBarrier() throws Ex
final File originalModFile = resource.getExclusiveModFile().getFile();
Assert.assertTrue(originalModFile.exists());
- PipeTsFileDeletionBarrier.getInstance().releasePendingDeletion(tsFilePath);
- tsFilePath = null;
+ barrier.finishDeletion(deleteSeq, Collections.singleton(tsFilePath));
Assert.assertTrue(finished.await(5, TimeUnit.SECONDS));
Assert.assertTrue(increased.get());
- Assert.assertTrue(event.isWithMod());
- Assert.assertNotNull(event.getModFile());
+ Assert.assertTrue(copiedEvent.isWithMod());
+ Assert.assertTrue(copiedEvent.isModPinned());
+ Assert.assertNotNull(copiedEvent.getModFile());
Assert.assertNotEquals(
- originalModFile.getAbsolutePath(), event.getModFile().getAbsolutePath());
+ originalModFile.getAbsolutePath(), copiedEvent.getModFile().getAbsolutePath());
} finally {
- // Ensure the singleton barrier is not leaked into other tests.
- if (tsFilePath != null) {
- PipeTsFileDeletionBarrier.getInstance().releasePendingDeletion(tsFilePath);
+ if (t != null) {
+ t.join(TimeUnit.SECONDS.toMillis(5));
+ }
+ if (copiedEvent != null && copiedEvent.getReferenceCount() > 0) {
+ copiedEvent.clearReferenceCount("source");
+ copiedEvent.close();
+ }
+ if (originalEvent != null && originalEvent.getReferenceCount() > 0) {
+ originalEvent.clearReferenceCount("assigner");
+ originalEvent.close();
+ }
+ FileUtils.deleteFileOrDirectory(new File(TestConstant.BASE_OUTPUT_PATH));
+ }
+ }
+
+ @Test
+ public void testDelayedPipeCopyDoesNotFreezeLaterDeletion() throws Exception {
+ final File tsFile = createTestTsFileUnderDataRegion(1, 1, 1, 7);
+ PipeTsFileInsertionEvent originalEvent = null;
+ PipeTsFileInsertionEvent copiedEvent = null;
+ Thread t = null;
+ try {
+ Assert.assertTrue(tsFile.getParentFile().mkdirs() || tsFile.getParentFile().exists());
+ Assert.assertTrue(tsFile.createNewFile() || tsFile.exists());
+
+ final TsFileResource resource = new TsFileResource(tsFile);
+ resource.setStatus(TsFileResourceStatus.NORMAL);
+ final PipeTsFileDeletionBarrier barrier = PipeTsFileDeletionBarrier.getInstance();
+ final int regionId = Integer.parseInt(resource.getDataRegionId());
+ final String tsFilePath = resource.getTsFilePath();
+
+ originalEvent = createTestTsFileInsertionEvent(resource, null);
+ originalEvent.enableDelayModSnapshotUntilReplicateIndex();
+ Assert.assertTrue(originalEvent.increaseReferenceCount("assigner"));
+
+ copiedEvent = createTestPipeCopy(originalEvent, "testPipeCopy");
+ copiedEvent.setReplicateIndexForIoTV2(100L);
+
+ final long deleteSeq = barrier.beginDeletion(regionId);
+ barrier.resolveDeletionTargets(regionId, deleteSeq, Collections.singleton(tsFilePath));
+
+ final CountDownLatch started = new CountDownLatch(1);
+ final CountDownLatch finished = new CountDownLatch(1);
+ final AtomicReference failure = new AtomicReference<>(null);
+ t =
+ new Thread(
+ () -> {
+ try {
+ started.countDown();
+ barrier.awaitSnapshotsBeforeMaterialization(tsFilePath, deleteSeq);
+ resource
+ .getExclusiveModFile()
+ .write(new TreeDeletionEntry(new MeasurementPath("root.db.d1.s1"), 0, 1));
+ barrier.finishDeletion(deleteSeq, Collections.singleton(tsFilePath));
+ } catch (final Throwable throwable) {
+ failure.set(throwable);
+ } finally {
+ finished.countDown();
+ }
+ },
+ "test-tsfile-delete-barrier-wait-later-delete");
+ t.start();
+
+ Assert.assertTrue(started.await(5, TimeUnit.SECONDS));
+ Assert.assertFalse(finished.await(200, TimeUnit.MILLISECONDS));
+
+ Assert.assertTrue(copiedEvent.increaseReferenceCount("source"));
+ Assert.assertFalse(copiedEvent.isWithMod());
+ Assert.assertNull(copiedEvent.getModFile());
+
+ Assert.assertTrue(finished.await(5, TimeUnit.SECONDS));
+ if (failure.get() != null) {
+ throw new AssertionError("Later deletion thread should finish successfully", failure.get());
}
+ Assert.assertTrue(resource.getExclusiveModFile().getFile().exists());
+ } finally {
if (t != null) {
t.join(TimeUnit.SECONDS.toMillis(5));
}
- if (event != null) {
- event.clearReferenceCount(holder);
- event.close();
+ if (copiedEvent != null && copiedEvent.getReferenceCount() > 0) {
+ copiedEvent.clearReferenceCount("source");
+ copiedEvent.close();
+ }
+ if (originalEvent != null && originalEvent.getReferenceCount() > 0) {
+ originalEvent.clearReferenceCount("assigner");
+ originalEvent.close();
}
FileUtils.deleteFileOrDirectory(new File(TestConstant.BASE_OUTPUT_PATH));
}
}
+ private PipeTsFileInsertionEvent createTestTsFileInsertionEvent(
+ final TsFileResource resource, final String pipeName) {
+ return new PipeTsFileInsertionEvent(
+ false,
+ "root.db",
+ resource,
+ null,
+ null,
+ true,
+ true,
+ false,
+ false,
+ null,
+ pipeName,
+ 1L,
+ null,
+ buildUnionPattern(true, Collections.singletonList(new IoTDBTreePattern(true, null))),
+ new TablePattern(false, null, null),
+ null,
+ null,
+ null,
+ true,
+ Long.MIN_VALUE,
+ Long.MAX_VALUE);
+ }
+
+ private PipeTsFileInsertionEvent createTestPipeCopy(
+ final PipeTsFileInsertionEvent sourceEvent, final String pipeName) {
+ return sourceEvent.shallowCopySelfAndBindPipeTaskMetaForProgressReport(
+ pipeName,
+ 2L,
+ null,
+ buildUnionPattern(true, Collections.singletonList(new IoTDBTreePattern(true, null))),
+ new TablePattern(false, null, null),
+ null,
+ null,
+ null,
+ true,
+ Long.MIN_VALUE,
+ Long.MAX_VALUE);
+ }
+
+ private File createTestTsFileUnderDataRegion(
+ final long time,
+ final long version,
+ final int innerSpaceCompactionCount,
+ final int crossSpaceCompactionCount) {
+ return new File(
+ new File(
+ new File(
+ new File(
+ TestConstant.BASE_OUTPUT_PATH + IoTDBConstant.SEQUENCE_FOLDER_NAME, "root.db"),
+ "1"),
+ "0"),
+ TsFileNameGenerator.generateNewTsFileName(
+ time, version, innerSpaceCompactionCount, crossSpaceCompactionCount));
+ }
+
static class TestAccessControl implements AccessControl {
@Override