You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by tk...@apache.org on 2024/02/02 09:37:00 UTC

(ignite-3) branch main updated: IGNITE-21398 Correct IndexBuilderTxRwOperationTracker update in PartitionReplicaListener (#3136)

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

tkalkirill 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 be7c2bc2e8 IGNITE-21398 Correct IndexBuilderTxRwOperationTracker update in PartitionReplicaListener (#3136)
be7c2bc2e8 is described below

commit be7c2bc2e8b0cdf25dab64b9fe8cb8b32f6df895
Author: Kirill Tkalenko <tk...@yandex.ru>
AuthorDate: Fri Feb 2 12:36:55 2024 +0300

    IGNITE-21398 Correct IndexBuilderTxRwOperationTracker update in PartitionReplicaListener (#3136)
---
 .../IndexBuilderTxRwOperationTracker.java          |  11 +-
 .../replicator/PartitionReplicaListener.java       |  62 ++++++--
 .../distributed/replicator/ReplicatorUtils.java    |  77 ++++++++++
 .../replicator/ReplicatorUtilsTest.java            | 157 +++++++++++++++++++++
 .../ignite/internal/table/TableTestUtils.java      |  69 ++++++++-
 5 files changed, 358 insertions(+), 18 deletions(-)

diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/IndexBuilderTxRwOperationTracker.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/IndexBuilderTxRwOperationTracker.java
index 72070eeeb1..5de76489b0 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/IndexBuilderTxRwOperationTracker.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/IndexBuilderTxRwOperationTracker.java
@@ -72,19 +72,16 @@ class IndexBuilderTxRwOperationTracker implements ManuallyCloseable {
     /**
      * Updating the minimum available catalog version on which new RW transaction operations can be started.
      *
-     * <p>It is expected that it will be updated when the index transitions to the backfield state and will increase monotonically.</p>
+     * <p>NOTE: Value will only increase.</p>
      *
      * @param catalogVersion Catalog version in which the new index appeared. New operations for RW transactions started on versions
      *      strictly before this one will not be allowed to start.
      */
     void updateMinAllowedCatalogVersionForStartOperation(int catalogVersion) {
         inBusyLock(busyLock, () -> {
-            minAllowedCatalogVersionForStartOperation.updateAndGet(previousCatalogVersion -> {
-                assert catalogVersion > previousCatalogVersion :
-                        "It should only grow: [previous=" + previousCatalogVersion + ", new=" + catalogVersion + "]";
-
-                return catalogVersion;
-            });
+            minAllowedCatalogVersionForStartOperation.updateAndGet(
+                    previousCatalogVersion -> Math.max(catalogVersion, previousCatalogVersion)
+            );
 
             Collection<CompletableFuture<Void>> futures = minAllowedVersionRaiseFutures.headMap(catalogVersion, true)
                     .values();
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java
index c1b3bbcafe..3a67be0fab 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java
@@ -25,6 +25,9 @@ import static java.util.stream.Collectors.toList;
 import static java.util.stream.Collectors.toMap;
 import static org.apache.ignite.internal.hlc.HybridTimestamp.hybridTimestampToLong;
 import static org.apache.ignite.internal.lang.IgniteStringFormatter.format;
+import static org.apache.ignite.internal.table.distributed.replicator.ReplicatorUtils.beginRwTxTs;
+import static org.apache.ignite.internal.table.distributed.replicator.ReplicatorUtils.latestIndexDescriptorInBuildingStatus;
+import static org.apache.ignite.internal.table.distributed.replicator.ReplicatorUtils.rwTxActiveCatalogVersion;
 import static org.apache.ignite.internal.tx.TxState.ABANDONED;
 import static org.apache.ignite.internal.tx.TxState.ABORTED;
 import static org.apache.ignite.internal.tx.TxState.COMMITTED;
@@ -36,6 +39,7 @@ import static org.apache.ignite.internal.util.CompletableFutures.emptyCollection
 import static org.apache.ignite.internal.util.CompletableFutures.emptyListCompletedFuture;
 import static org.apache.ignite.internal.util.CompletableFutures.falseCompletedFuture;
 import static org.apache.ignite.internal.util.CompletableFutures.nullCompletedFuture;
+import static org.apache.ignite.internal.util.CompletableFutures.trueCompletedFuture;
 import static org.apache.ignite.internal.util.IgniteUtils.findAny;
 import static org.apache.ignite.internal.util.IgniteUtils.findFirst;
 import static org.apache.ignite.internal.util.IgniteUtils.inBusyLock;
@@ -68,6 +72,11 @@ import java.util.function.Predicate;
 import java.util.function.Supplier;
 import org.apache.ignite.internal.binarytuple.BinaryTupleCommon;
 import org.apache.ignite.internal.catalog.CatalogService;
+import org.apache.ignite.internal.catalog.descriptors.CatalogIndexDescriptor;
+import org.apache.ignite.internal.catalog.events.CatalogEvent;
+import org.apache.ignite.internal.catalog.events.CatalogEventParameters;
+import org.apache.ignite.internal.catalog.events.StartBuildingIndexEventParameters;
+import org.apache.ignite.internal.event.EventListener;
 import org.apache.ignite.internal.hlc.HybridClock;
 import org.apache.ignite.internal.hlc.HybridTimestamp;
 import org.apache.ignite.internal.lang.IgniteBiTuple;
@@ -153,7 +162,6 @@ import org.apache.ignite.internal.tx.LockKey;
 import org.apache.ignite.internal.tx.LockManager;
 import org.apache.ignite.internal.tx.LockMode;
 import org.apache.ignite.internal.tx.MismatchingTransactionOutcomeException;
-import org.apache.ignite.internal.tx.TransactionIds;
 import org.apache.ignite.internal.tx.TransactionMeta;
 import org.apache.ignite.internal.tx.TransactionResult;
 import org.apache.ignite.internal.tx.TxManager;
@@ -277,6 +285,9 @@ public class PartitionReplicaListener implements ReplicaListener {
     /** Read-write transaction operation tracker for building indexes. */
     private final IndexBuilderTxRwOperationTracker txRwOperationTracker = new IndexBuilderTxRwOperationTracker();
 
+    /** Listener for {@link CatalogEvent#INDEX_BUILDING}. */
+    private final EventListener<CatalogEventParameters> indexBuildingCatalogEventListener = this::onIndexBuilding;
+
     /**
      * The constructor.
      *
@@ -345,6 +356,8 @@ public class PartitionReplicaListener implements ReplicaListener {
         cursors = new ConcurrentSkipListMap<>(IgniteUuid.globalOrderComparator());
 
         schemaCompatValidator = new SchemaCompatibilityValidator(validationSchemasSource, catalogService, schemaSyncService);
+
+        prepareIndexBuilderTxRwOperationTracker();
     }
 
     private CompletableFuture<?> durableCleanup(UUID txId, TxMeta txMeta) {
@@ -3723,6 +3736,8 @@ public class PartitionReplicaListener implements ReplicaListener {
 
         busyLock.block();
 
+        catalogService.removeListener(CatalogEvent.INDEX_BUILDING, indexBuildingCatalogEventListener);
+
         txRwOperationTracker.close();
     }
 
@@ -3805,24 +3820,55 @@ public class PartitionReplicaListener implements ReplicaListener {
         if (request instanceof ReadWriteReplicaRequest) {
             // It is very important that the counter is increased only after the schema sync at the begin timestamp of RW transaction,
             // otherwise there may be races/errors and the index will not be able to start building.
-            txRwOperationTracker.incrementOperationCount(rwTxActiveCatalogVersion((ReadWriteReplicaRequest) request));
+            txRwOperationTracker.incrementOperationCount(rwTxActiveCatalogVersion(catalogService, (ReadWriteReplicaRequest) request));
         }
 
         return processOperationRequest(request, isPrimary, senderId, opStartTsIfDirectRo)
                 .whenComplete((unused, throwable) -> {
                     if (request instanceof ReadWriteReplicaRequest) {
-                        txRwOperationTracker.decrementOperationCount(rwTxActiveCatalogVersion((ReadWriteReplicaRequest) request));
+                        txRwOperationTracker.decrementOperationCount(
+                                rwTxActiveCatalogVersion(catalogService, (ReadWriteReplicaRequest) request)
+                        );
                     }
                 });
     }
 
-    private static HybridTimestamp beginRwTxTs(ReadWriteReplicaRequest request) {
-        return TransactionIds.beginTimestamp(request.transactionId());
+    private void prepareIndexBuilderTxRwOperationTracker() {
+        // Expected to be executed on the metastore thread.
+        CatalogIndexDescriptor indexDescriptor = latestIndexDescriptorInBuildingStatus(catalogService, tableId());
+
+        if (indexDescriptor != null) {
+            txRwOperationTracker.updateMinAllowedCatalogVersionForStartOperation(indexDescriptor.creationCatalogVersion());
+        }
+
+        catalogService.listen(CatalogEvent.INDEX_BUILDING, indexBuildingCatalogEventListener);
     }
 
-    private int rwTxActiveCatalogVersion(ReadWriteReplicaRequest request) {
-        HybridTimestamp beginRwTxTs = beginRwTxTs(request);
+    private CompletableFuture<Boolean> onIndexBuilding(CatalogEventParameters parameters, @Nullable Throwable exception) {
+        if (exception != null) {
+            return failedFuture(exception);
+        }
+
+        if (!busyLock.enterBusy()) {
+            return trueCompletedFuture();
+        }
+
+        try {
+            int indexId = ((StartBuildingIndexEventParameters) parameters).indexId();
+
+            CatalogIndexDescriptor indexDescriptor = catalogService.index(indexId, parameters.catalogVersion());
+
+            assert indexDescriptor != null : "indexId=" + indexId + ", catalogVersion=" + parameters.catalogVersion();
 
-        return catalogService.activeCatalogVersion(beginRwTxTs.longValue());
+            if (indexDescriptor.tableId() == tableId()) {
+                txRwOperationTracker.updateMinAllowedCatalogVersionForStartOperation(indexDescriptor.creationCatalogVersion());
+            }
+
+            return falseCompletedFuture();
+        } catch (Throwable t) {
+            return failedFuture(t);
+        } finally {
+            busyLock.leaveBusy();
+        }
     }
 }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/ReplicatorUtils.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/ReplicatorUtils.java
new file mode 100644
index 0000000000..9380b43264
--- /dev/null
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/ReplicatorUtils.java
@@ -0,0 +1,77 @@
+/*
+ * 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.ignite.internal.table.distributed.replicator;
+
+import static org.apache.ignite.internal.catalog.descriptors.CatalogIndexStatus.BUILDING;
+
+import org.apache.ignite.internal.catalog.CatalogService;
+import org.apache.ignite.internal.catalog.descriptors.CatalogIndexDescriptor;
+import org.apache.ignite.internal.catalog.descriptors.CatalogIndexStatus;
+import org.apache.ignite.internal.hlc.HybridTimestamp;
+import org.apache.ignite.internal.table.distributed.replication.request.ReadWriteReplicaRequest;
+import org.apache.ignite.internal.tx.TransactionIds;
+import org.jetbrains.annotations.Nullable;
+
+/** Auxiliary class. */
+class ReplicatorUtils {
+    /**
+     * Looks for the latest index with {@link CatalogIndexStatus#BUILDING} for the table, {@code null} if missing.
+     *
+     * <p>NOTE: It is expected that the method will be executed in the metastore thread so that the catalog does not change
+     * concurrently.</p>
+     *
+     * @param catalogService Catalog service.
+     * @param tableId Table ID.
+     */
+    static @Nullable CatalogIndexDescriptor latestIndexDescriptorInBuildingStatus(CatalogService catalogService, int tableId) {
+        // Since we expect to be executed on the metastore thread, it is safe to use these versions.
+        int latestCatalogVersion = catalogService.latestCatalogVersion();
+        int earliestCatalogVersion = catalogService.earliestCatalogVersion();
+
+        for (int catalogVersion = latestCatalogVersion; catalogVersion >= earliestCatalogVersion; catalogVersion--) {
+            for (CatalogIndexDescriptor indexDescriptor : catalogService.indexes(catalogVersion, tableId)) {
+                if (indexDescriptor.status() == BUILDING) {
+                    return indexDescriptor;
+                }
+            }
+        }
+
+        return null;
+    }
+
+    /**
+     * Extracts begin timestamp of a read-write transaction from a request.
+     *
+     * @param request Read-write replica request.
+     */
+    static HybridTimestamp beginRwTxTs(ReadWriteReplicaRequest request) {
+        return TransactionIds.beginTimestamp(request.transactionId());
+    }
+
+    /**
+     * Returns the active catalog version by begin timestamp of a read-write transaction from a request.
+     *
+     * @param catalogService Catalog service.
+     * @param request Read-write replica request.
+     */
+    static int rwTxActiveCatalogVersion(CatalogService catalogService, ReadWriteReplicaRequest request) {
+        HybridTimestamp beginRwTxTs = beginRwTxTs(request);
+
+        return catalogService.activeCatalogVersion(beginRwTxTs.longValue());
+    }
+}
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replicator/ReplicatorUtilsTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replicator/ReplicatorUtilsTest.java
new file mode 100644
index 0000000000..84a1ba4be6
--- /dev/null
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replicator/ReplicatorUtilsTest.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.ignite.internal.table.distributed.replicator;
+
+import static org.apache.ignite.internal.table.TableTestUtils.INDEX_NAME;
+import static org.apache.ignite.internal.table.TableTestUtils.TABLE_NAME;
+import static org.apache.ignite.internal.table.TableTestUtils.createSimpleHashIndex;
+import static org.apache.ignite.internal.table.TableTestUtils.createSimpleTable;
+import static org.apache.ignite.internal.table.TableTestUtils.getIndexIdStrict;
+import static org.apache.ignite.internal.table.TableTestUtils.getTableIdStrict;
+import static org.apache.ignite.internal.table.TableTestUtils.makeIndexAvailable;
+import static org.apache.ignite.internal.table.TableTestUtils.startBuildingIndex;
+import static org.apache.ignite.internal.table.distributed.replicator.ReplicatorUtils.beginRwTxTs;
+import static org.apache.ignite.internal.table.distributed.replicator.ReplicatorUtils.latestIndexDescriptorInBuildingStatus;
+import static org.apache.ignite.internal.table.distributed.replicator.ReplicatorUtils.rwTxActiveCatalogVersion;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.apache.ignite.internal.tx.TransactionIds.transactionId;
+import static org.apache.ignite.internal.util.IgniteUtils.closeAll;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.mockito.ArgumentMatchers.anyLong;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.util.UUID;
+import java.util.function.Consumer;
+import org.apache.ignite.internal.catalog.CatalogManager;
+import org.apache.ignite.internal.catalog.CatalogService;
+import org.apache.ignite.internal.catalog.CatalogTestUtils;
+import org.apache.ignite.internal.hlc.HybridClock;
+import org.apache.ignite.internal.hlc.HybridClockImpl;
+import org.apache.ignite.internal.hlc.HybridTimestamp;
+import org.apache.ignite.internal.table.distributed.replication.request.ReadWriteReplicaRequest;
+import org.apache.ignite.internal.testframework.IgniteAbstractTest;
+import org.junit.jupiter.api.Test;
+
+/** For {@link ReplicatorUtils} testing. */
+public class ReplicatorUtilsTest extends IgniteAbstractTest {
+    private final HybridClock clock = new HybridClockImpl();
+
+    @Test
+    void testBeginRwTxTs() {
+        HybridTimestamp beginTs = clock.now();
+
+        UUID txId = transactionId(beginTs, 10);
+
+        assertEquals(beginTs, beginRwTxTs(readWriteReplicaRequest(txId)));
+    }
+
+    @Test
+    void testRwTxActiveCatalogVersion() {
+        HybridTimestamp beginTs = clock.now();
+
+        UUID txId = transactionId(beginTs, 10);
+
+        CatalogService catalogService = mock(CatalogService.class);
+
+        when(catalogService.activeCatalogVersion(anyLong())).thenReturn(666);
+
+        assertEquals(666, rwTxActiveCatalogVersion(catalogService, readWriteReplicaRequest(txId)));
+
+        verify(catalogService).activeCatalogVersion(eq(beginTs.longValue()));
+    }
+
+    @Test
+    void testLatestIndexDescriptorInBuildingStatus() throws Exception {
+        withCatalogManager(catalogManager -> {
+            createSimpleTable(catalogManager, TABLE_NAME);
+
+            int tableId = tableId(catalogManager, TABLE_NAME);
+
+            assertNull(latestIndexDescriptorInBuildingStatus(catalogManager, tableId));
+
+            createSimpleHashIndex(catalogManager, TABLE_NAME, INDEX_NAME);
+            assertNull(latestIndexDescriptorInBuildingStatus(catalogManager, tableId));
+
+            int indexId = indexId(catalogManager, INDEX_NAME);
+
+            startBuildingIndex(catalogManager, indexId);
+            assertEquals(indexId, latestIndexDescriptorInBuildingStatus(catalogManager, tableId).id());
+
+            makeIndexAvailable(catalogManager, indexId);
+            assertEquals(indexId, latestIndexDescriptorInBuildingStatus(catalogManager, tableId).id());
+
+            String otherIndexName = INDEX_NAME + 1;
+
+            createSimpleHashIndex(catalogManager, TABLE_NAME, otherIndexName);
+            assertEquals(indexId, latestIndexDescriptorInBuildingStatus(catalogManager, tableId).id());
+
+            int otherIndexId = indexId(catalogManager, otherIndexName);
+
+            startBuildingIndex(catalogManager, otherIndexId);
+            assertEquals(otherIndexId, latestIndexDescriptorInBuildingStatus(catalogManager, tableId).id());
+        });
+    }
+
+    @Test
+    void testLatestIndexDescriptorInBuildingStatusForOtherTable() throws Exception {
+        withCatalogManager(catalogManager -> {
+            String otherTableName = TABLE_NAME + 1;
+
+            createSimpleTable(catalogManager, TABLE_NAME);
+            createSimpleTable(catalogManager, otherTableName);
+
+            createSimpleHashIndex(catalogManager, TABLE_NAME, INDEX_NAME);
+            startBuildingIndex(catalogManager, indexId(catalogManager, INDEX_NAME));
+
+            assertNull(latestIndexDescriptorInBuildingStatus(catalogManager, tableId(catalogManager, otherTableName)));
+        });
+    }
+
+    private static ReadWriteReplicaRequest readWriteReplicaRequest(UUID txId) {
+        ReadWriteReplicaRequest request = mock(ReadWriteReplicaRequest.class);
+
+        when(request.transactionId()).thenReturn(txId);
+
+        return request;
+    }
+
+    private void withCatalogManager(Consumer<CatalogManager> consumer) throws Exception {
+        CatalogManager catalogManager = CatalogTestUtils.createTestCatalogManager("test-node", clock);
+
+        assertThat(catalogManager.start(), willCompleteSuccessfully());
+
+        try {
+            consumer.accept(catalogManager);
+        } finally {
+            closeAll(catalogManager::beforeNodeStop, catalogManager::stop);
+        }
+    }
+
+    private int indexId(CatalogService catalogService, String indexName) {
+        return getIndexIdStrict(catalogService, indexName, clock.nowLong());
+    }
+
+    private int tableId(CatalogService catalogService, String tableName) {
+        return getTableIdStrict(catalogService, tableName, clock.nowLong());
+    }
+}
diff --git a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TableTestUtils.java b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TableTestUtils.java
index 3f94c30f58..bc0e92bb55 100644
--- a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TableTestUtils.java
+++ b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/TableTestUtils.java
@@ -17,7 +17,10 @@
 
 package org.apache.ignite.internal.table;
 
+import static org.apache.ignite.internal.catalog.CatalogService.DEFAULT_SCHEMA_NAME;
+import static org.apache.ignite.internal.catalog.CatalogService.DEFAULT_ZONE_NAME;
 import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.apache.ignite.sql.ColumnType.INT32;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 
@@ -30,15 +33,26 @@ import org.apache.ignite.internal.catalog.commands.CreateHashIndexCommand;
 import org.apache.ignite.internal.catalog.commands.CreateTableCommand;
 import org.apache.ignite.internal.catalog.commands.DropIndexCommand;
 import org.apache.ignite.internal.catalog.commands.DropTableCommand;
+import org.apache.ignite.internal.catalog.commands.MakeIndexAvailableCommand;
 import org.apache.ignite.internal.catalog.commands.RemoveIndexCommand;
+import org.apache.ignite.internal.catalog.commands.StartBuildingIndexCommand;
 import org.apache.ignite.internal.catalog.descriptors.CatalogIndexDescriptor;
+import org.apache.ignite.internal.catalog.descriptors.CatalogIndexStatus;
 import org.apache.ignite.internal.catalog.descriptors.CatalogTableDescriptor;
+import org.apache.ignite.sql.ColumnType;
 import org.jetbrains.annotations.Nullable;
 
-/**
- * Utils to manage tables inside tests.
- */
+/** Utils to manage tables inside tests. */
 public class TableTestUtils {
+    /** Table name. */
+    public static final String TABLE_NAME = "TEST_TABLE";
+
+    /** Index name. */
+    public static final String INDEX_NAME = "TEST_INDEX";
+
+    /** Column name. */
+    public static final String COLUMN_NAME = "TEST_COLUMN";
+
     /**
      * Creates table in the catalog.
      *
@@ -259,4 +273,53 @@ public class TableTestUtils {
 
         return index;
     }
+
+    /**
+     * Creates a simple table in {@link CatalogService#DEFAULT_SCHEMA_NAME} and {@link CatalogService#DEFAULT_ZONE_NAME} and single
+     * {@link #COLUMN_NAME column} of type {@link ColumnType#INT32}.
+     *
+     * @param catalogManager Catalog name.
+     * @param tableName Table name.
+     */
+    public static void createSimpleTable(CatalogManager catalogManager, String tableName) {
+        createTable(
+                catalogManager,
+                DEFAULT_SCHEMA_NAME,
+                DEFAULT_ZONE_NAME,
+                tableName,
+                List.of(ColumnParams.builder().name(COLUMN_NAME).type(INT32).build()),
+                List.of(COLUMN_NAME)
+        );
+    }
+
+    /**
+     * Creates a simple index on the table from {@link #createSimpleTable(CatalogManager, String)}.
+     *
+     * @param catalogManager Catalog name.
+     * @param tableName Table name.
+     * @param indexName Index name.
+     */
+    public static void createSimpleHashIndex(CatalogManager catalogManager, String tableName, String indexName) {
+        createHashIndex(catalogManager, DEFAULT_SCHEMA_NAME, tableName, indexName, List.of(COLUMN_NAME), false);
+    }
+
+    /**
+     * Sets the index status to {@link CatalogIndexStatus#BUILDING}.
+     *
+     * @param catalogManager Catalog manager.
+     * @param indexId Index ID.
+     */
+    public static void startBuildingIndex(CatalogManager catalogManager, int indexId) {
+        assertThat(catalogManager.execute(StartBuildingIndexCommand.builder().indexId(indexId).build()), willCompleteSuccessfully());
+    }
+
+    /**
+     * Sets the index to {@link CatalogIndexStatus#AVAILABLE}.
+     *
+     * @param catalogManager Catalog manager.
+     * @param indexId Index ID.
+     */
+    public static void makeIndexAvailable(CatalogManager catalogManager, int indexId) {
+        assertThat(catalogManager.execute(MakeIndexAvailableCommand.builder().indexId(indexId).build()), willCompleteSuccessfully());
+    }
 }