You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sd...@apache.org on 2023/03/02 16:04:18 UTC

[ignite-3] branch main updated: IGNITE-18882 Don't store tombstones if there is no other version of row (#1739)

This is an automated email from the ASF dual-hosted git repository.

sdanilov pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
     new 31a82d91fa IGNITE-18882 Don't store tombstones if there is no other version of row (#1739)
31a82d91fa is described below

commit 31a82d91fad546e33de00f76cd73a5e411d14676
Author: Semyon Danilov <sa...@yandex.ru>
AuthorDate: Thu Mar 2 20:04:12 2023 +0400

    IGNITE-18882 Don't store tombstones if there is no other version of row (#1739)
---
 .../storage/AbstractMvPartitionStorageGcTest.java  | 15 ++++++++--
 .../storage/impl/TestMvPartitionStorage.java       |  7 ++++-
 .../mv/AddWriteCommittedInvokeClosure.java         |  7 +++++
 .../pagememory/mv/CommitWriteInvokeClosure.java    |  7 +++++
 .../internal/storage/rocksdb/GarbageCollector.java | 34 +++++++++++-----------
 5 files changed, 50 insertions(+), 20 deletions(-)

diff --git a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageGcTest.java b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageGcTest.java
index f98e1822f3..bec02e5791 100644
--- a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageGcTest.java
+++ b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageGcTest.java
@@ -21,7 +21,6 @@ import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertNull;
 
 import org.apache.ignite.internal.hlc.HybridTimestamp;
-import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
 
 /**
@@ -132,7 +131,6 @@ public abstract class AbstractMvPartitionStorageGcTest extends BaseMvPartitionSt
     }
 
     @Test
-    @Disabled("https://issues.apache.org/jira/browse/IGNITE-18882")
     void testVacuumsSecondRowIfTombstoneIsFirst() {
         addAndCommit(null);
 
@@ -144,4 +142,17 @@ public abstract class AbstractMvPartitionStorageGcTest extends BaseMvPartitionSt
 
         assertRowMatches(row.binaryRow(), TABLE_ROW);
     }
+
+    @Test
+    void testVacuumsSecondRowIfTombstoneIsFirstAddCommitted() {
+        addWriteCommitted(ROW_ID, null, clock.now());
+
+        addWriteCommitted(ROW_ID, TABLE_ROW, clock.now());
+
+        addWriteCommitted(ROW_ID, TABLE_ROW2, clock.now());
+
+        BinaryRowAndRowId row = pollForVacuum(HybridTimestamp.MAX_VALUE);
+
+        assertRowMatches(row.binaryRow(), TABLE_ROW);
+    }
 }
diff --git a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/impl/TestMvPartitionStorage.java b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/impl/TestMvPartitionStorage.java
index 6afd07d7dd..a5c435df51 100644
--- a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/impl/TestMvPartitionStorage.java
+++ b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/impl/TestMvPartitionStorage.java
@@ -259,7 +259,7 @@ public class TestMvPartitionStorage implements MvPartitionStorage {
         });
     }
 
-    private VersionChain resolveCommittedVersionChain(VersionChain committedVersionChain) {
+    private @Nullable VersionChain resolveCommittedVersionChain(VersionChain committedVersionChain) {
         VersionChain nextChain = committedVersionChain.next;
 
         if (nextChain != null) {
@@ -271,6 +271,11 @@ public class TestMvPartitionStorage implements MvPartitionStorage {
             // Calling it from the compute is fine. Concurrent writes of the same row are impossible, and if we call the compute closure
             // several times, the same tuple will be inserted into the GC queue (timestamp and rowId don't change in this case).
             gcQueue.add(committedVersionChain);
+        } else {
+            if (committedVersionChain.row == null) {
+                // If there is only one version, and it is a tombstone, then remove the chain.
+                return null;
+            }
         }
 
         return committedVersionChain;
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/AddWriteCommittedInvokeClosure.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/AddWriteCommittedInvokeClosure.java
index 3fddbc1249..02876a9ce1 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/AddWriteCommittedInvokeClosure.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/AddWriteCommittedInvokeClosure.java
@@ -81,6 +81,13 @@ class AddWriteCommittedInvokeClosure implements InvokeClosure<VersionChain> {
             throw new StorageException("Write intent exists: [rowId={}, {}]", oldRow.rowId(), storage.createStorageInfo());
         }
 
+        if (row == null && oldRow == null) {
+            // If there is only one version, and it is a tombstone, then don't save the chain.
+            operationType = OperationType.NOOP;
+
+            return;
+        }
+
         if (oldRow == null) {
             operationType = OperationType.PUT;
 
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/CommitWriteInvokeClosure.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/CommitWriteInvokeClosure.java
index 3a9b6e28e5..91935666c1 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/CommitWriteInvokeClosure.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/CommitWriteInvokeClosure.java
@@ -80,6 +80,13 @@ class CommitWriteInvokeClosure implements InvokeClosure<VersionChain> {
         RowVersion current = storage.readRowVersion(oldRow.headLink(), DONT_LOAD_VALUE);
         RowVersion next = oldRow.hasNextLink() ? storage.readRowVersion(oldRow.nextLink(), DONT_LOAD_VALUE) : null;
 
+        if (next == null && current.isTombstone()) {
+            // If there is only one version, and it is a tombstone, then remove the chain.
+            operationType = OperationType.REMOVE;
+
+            return;
+        }
+
         // If the previous and current version are tombstones, then delete the current version.
         if (next != null && current.isTombstone() && next.isTombstone()) {
             toRemove = current;
diff --git a/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/GarbageCollector.java b/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/GarbageCollector.java
index d150deac98..326375519a 100644
--- a/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/GarbageCollector.java
+++ b/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/GarbageCollector.java
@@ -105,8 +105,6 @@ class GarbageCollector {
             throws RocksDBException {
         ColumnFamilyHandle partCf = helper.partCf;
 
-        boolean newAndPrevTombstones = false;
-
         // Try find previous value for the row id.
         ByteBuffer keyBuffer = MV_KEY_BUFFER.get();
         keyBuffer.clear();
@@ -117,7 +115,7 @@ class GarbageCollector {
             it.seek(keyBuffer);
 
             if (invalid(it)) {
-                return false;
+                return isNewValueTombstone;
             }
 
             keyBuffer.clear();
@@ -126,28 +124,30 @@ class GarbageCollector {
 
             RowId readRowId = helper.getRowId(keyBuffer, ROW_ID_OFFSET);
 
-            if (readRowId.equals(rowId)) {
-                // Found previous value.
-                assert keyLen == MAX_KEY_SIZE; // Can not be write-intent.
+            if (!readRowId.equals(rowId)) {
+                return isNewValueTombstone;
+            }
+
+            // Found previous value.
+            assert keyLen == MAX_KEY_SIZE; // Can not be write-intent.
 
-                if (isNewValueTombstone) {
-                    // If new value is a tombstone, lets check if previous value was also a tombstone.
-                    int valueSize = it.value(EMPTY_DIRECT_BUFFER);
+            if (isNewValueTombstone) {
+                // If new value is a tombstone, lets check if previous value was also a tombstone.
+                int valueSize = it.value(EMPTY_DIRECT_BUFFER);
 
-                    newAndPrevTombstones = valueSize == 0;
+                if (valueSize == 0) {
+                    return true;
                 }
+            }
 
-                if (!newAndPrevTombstones) {
-                    keyBuffer.clear();
+            keyBuffer.clear();
 
-                    helper.putGcKey(keyBuffer, rowId, timestamp);
+            helper.putGcKey(keyBuffer, rowId, timestamp);
 
-                    writeBatch.put(gcQueueCf, keyBuffer, EMPTY_DIRECT_BUFFER);
-                }
-            }
+            writeBatch.put(gcQueueCf, keyBuffer, EMPTY_DIRECT_BUFFER);
         }
 
-        return newAndPrevTombstones;
+        return false;
     }
 
     /**