You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by "tkalkirill (via GitHub)" <gi...@apache.org> on 2023/05/05 13:30:36 UTC

[GitHub] [ignite-3] tkalkirill opened a new pull request, #2030: IGNITE-19177 Add assignments checks when building indexes

tkalkirill opened a new pull request, #2030:
URL: https://github.com/apache/ignite-3/pull/2030

   https://issues.apache.org/jira/browse/IGNITE-19177


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sashapolo commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "sashapolo (via GitHub)" <gi...@apache.org>.
sashapolo commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1197692268


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuilder.java:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.index;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.close.ManuallyCloseable;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.index.IndexStorage;
+import org.apache.ignite.internal.table.distributed.command.BuildIndexCommand;
+import org.apache.ignite.internal.thread.NamedThreadFactory;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.internal.util.IgniteUtils;
+
+/**
+ * Class for managing the building of table indexes.
+ */
+public class IndexBuilder implements ManuallyCloseable {
+    private static final IgniteLogger LOG = Loggers.forClass(IndexBuilder.class);
+
+    private static final int BATCH_SIZE = 100;
+
+    private final ExecutorService executor;
+
+    private final Map<IndexBuildTaskId, IndexBuildTask> indexBuildTaskById = new ConcurrentHashMap<>();
+
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    private final AtomicBoolean closeGuard = new AtomicBoolean();
+
+    /**
+     * Constructor.
+     *
+     * @param nodeName Node name.
+     * @param threadCount Number of threads to build indexes.
+     */
+    public IndexBuilder(String nodeName, int threadCount) {
+        executor = new ThreadPoolExecutor(
+                threadCount,
+                threadCount,
+                30,
+                TimeUnit.SECONDS,
+                new LinkedBlockingQueue<>(),
+                NamedThreadFactory.create(nodeName, "build-index", LOG)
+        );
+    }
+
+    /**
+     * Starts building the index if it is not already built or is not yet in progress.
+     *
+     * <p>Index is built in batches using {@link BuildIndexCommand} (via raft), batches are sent sequentially.
+     *
+     * <p>It is expected that the index building is triggered by the leader of the raft group.
+     *
+     * @param tableId Table ID.
+     * @param partitionId Partition ID.
+     * @param indexId Index ID.
+     * @param indexStorage Index storage to build.
+     * @param partitionStorage Multi-versioned partition storage.
+     * @param raftClient Raft client.
+     */
+    // TODO: IGNITE-19498 Perhaps we need to start building the index only once
+    public void startBuildIndex(
+            UUID tableId,
+            int partitionId,
+            UUID indexId,
+            IndexStorage indexStorage,
+            MvPartitionStorage partitionStorage,
+            RaftGroupService raftClient
+    ) {
+        inBusyLock(() -> {
+            if (indexStorage.getNextRowIdToBuild() == null) {
+                return;
+            }
+
+            IndexBuildTaskId taskId = new IndexBuildTaskId(tableId, partitionId, indexId);
+
+            IndexBuildTask newTask = new IndexBuildTask(taskId, indexStorage, partitionStorage, raftClient, executor, busyLock, BATCH_SIZE);
+
+            IndexBuildTask previousTask = indexBuildTaskById.putIfAbsent(taskId, newTask);

Review Comment:
   ok, I agree



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1197599838


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuildTaskId.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.index;
+
+import java.util.UUID;
+import org.apache.ignite.internal.tostring.S;
+
+/**
+ * {@link IndexBuildTask} ID.
+ */
+class IndexBuildTaskId {
+    private final UUID tableId;
+
+    private final int partitionId;
+
+    private final UUID indexId;
+
+    IndexBuildTaskId(UUID tableId, int partitionId, UUID indexId) {
+        this.tableId = tableId;
+        this.partitionId = partitionId;
+        this.indexId = indexId;
+    }
+
+    public UUID getTableId() {

Review Comment:
   This rule is valid for 2.0 in 3.0 we don't have it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1200130697


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -382,8 +382,17 @@ public boolean stopReplica(ReplicationGroupId replicaGrpId) throws NodeStoppingE
      * @param replicaGrpId Replication group id.
      * @return True if the replica is found and closed, false otherwise.
      */
+    // TODO: IGNITE-19494 We need to correctly stop the replica

Review Comment:
   Yep. We(TX teem) will do this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] vldpyatkov commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "vldpyatkov (via GitHub)" <gi...@apache.org>.
vldpyatkov commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1187256731


##########
modules/index/src/main/java/org/apache/ignite/internal/index/IndexBuilder.java:
##########
@@ -100,10 +111,14 @@ void stop() {
      */
     void startIndexBuild(TableIndexView tableIndexView, TableImpl table) {
         for (int partitionId = 0; partitionId < table.internalTable().partitions(); partitionId++) {
+            // Let's check if there is a node in the assignments for the partition.
+            if (!replicaManager.startedGroups().contains(new TablePartitionId(table.tableId(), partitionId))) {

Review Comment:
   The checking based on started replica does not look like a permanent solution. Because the replica is a consequence of assignments.
   If it is a temporary, that required TODO here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1193461076


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItBuildIndexTest.java:
##########
@@ -63,8 +70,20 @@ TABLE_NAME, toValuesString(List.of(1, 1), List.of(2, 2), List.of(3, 3), List.of(
         sql(IgniteStringFormatter.format("CREATE INDEX {} ON {} (i1)", INDEX_NAME, TABLE_NAME));
 
         // TODO: IGNITE-19150 We are waiting for schema synchronization to avoid races to create and destroy indexes
-        waitForIndexBuild(TABLE_NAME, INDEX_NAME);
+        Map<Integer, List<Ignite>> nodesWithBuiltIndexesByPartitionId = waitForIndexBuild(TABLE_NAME, INDEX_NAME);
+
+        // Check that the number of nodes with built indexes is equal to the number of replicas.
+        assertEquals(partitions, nodesWithBuiltIndexesByPartitionId.size());
+
+        for (Entry<Integer, List<Ignite>> entry : nodesWithBuiltIndexesByPartitionId.entrySet()) {
+            assertEquals(
+                    replicas,
+                    entry.getValue().size(),
+                    IgniteStringFormatter.format("p={}, nodes={}", entry.getKey(), entry.getValue())
+            );
+        }
 
+        // Let's check the sql query.

Review Comment:
   Fix it



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sashapolo commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "sashapolo (via GitHub)" <gi...@apache.org>.
sashapolo commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1197733914


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuilder.java:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.index;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.close.ManuallyCloseable;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.index.IndexStorage;
+import org.apache.ignite.internal.table.distributed.command.BuildIndexCommand;
+import org.apache.ignite.internal.thread.NamedThreadFactory;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.internal.util.IgniteUtils;
+
+/**
+ * Class for managing the building of table indexes.
+ */
+public class IndexBuilder implements ManuallyCloseable {
+    private static final IgniteLogger LOG = Loggers.forClass(IndexBuilder.class);
+
+    private static final int BATCH_SIZE = 100;
+
+    private final ExecutorService executor;
+
+    private final Map<IndexBuildTaskId, IndexBuildTask> indexBuildTaskById = new ConcurrentHashMap<>();
+
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    private final AtomicBoolean closeGuard = new AtomicBoolean();
+
+    /**
+     * Constructor.
+     *
+     * @param nodeName Node name.
+     * @param threadCount Number of threads to build indexes.
+     */
+    public IndexBuilder(String nodeName, int threadCount) {
+        executor = new ThreadPoolExecutor(
+                threadCount,
+                threadCount,
+                30,
+                TimeUnit.SECONDS,
+                new LinkedBlockingQueue<>(),
+                NamedThreadFactory.create(nodeName, "build-index", LOG)
+        );
+    }
+
+    /**
+     * Starts building the index if it is not already built or is not yet in progress.
+     *
+     * <p>Index is built in batches using {@link BuildIndexCommand} (via raft), batches are sent sequentially.
+     *
+     * <p>It is expected that the index building is triggered by the leader of the raft group.
+     *
+     * @param tableId Table ID.
+     * @param partitionId Partition ID.
+     * @param indexId Index ID.
+     * @param indexStorage Index storage to build.
+     * @param partitionStorage Multi-versioned partition storage.
+     * @param raftClient Raft client.
+     */
+    // TODO: IGNITE-19498 Perhaps we need to start building the index only once
+    public void startBuildIndex(
+            UUID tableId,
+            int partitionId,
+            UUID indexId,
+            IndexStorage indexStorage,
+            MvPartitionStorage partitionStorage,
+            RaftGroupService raftClient
+    ) {
+        inBusyLock(() -> {
+            if (indexStorage.getNextRowIdToBuild() == null) {
+                return;
+            }
+
+            IndexBuildTaskId taskId = new IndexBuildTaskId(tableId, partitionId, indexId);
+
+            IndexBuildTask newTask = new IndexBuildTask(taskId, indexStorage, partitionStorage, raftClient, executor, busyLock, BATCH_SIZE);
+
+            IndexBuildTask previousTask = indexBuildTaskById.putIfAbsent(taskId, newTask);
+
+            if (previousTask != null) {
+                // Index building is already in progress.
+                return;
+            }
+
+            newTask.start();
+
+            newTask.getTaskFuture().whenComplete((unused, throwable) -> indexBuildTaskById.remove(taskId));
+        });
+    }
+
+    /**
+     * Stops index building if it is in progress.
+     *
+     * @param tableId Table ID.
+     * @param partitionId Partition ID.
+     * @param indexId Index ID.
+     */
+    public void stopBuildIndex(UUID tableId, int partitionId, UUID indexId) {
+        inBusyLock(() -> {
+            IndexBuildTask removed = indexBuildTaskById.remove(new IndexBuildTaskId(tableId, partitionId, indexId));
+
+            if (removed != null) {
+                removed.stop();
+            }
+        });
+    }
+
+    /**
+     * Stops building all indexes (for a table partition) if they are in progress.
+     *
+     * @param tableId Table ID.
+     * @param partitionId Partition ID.
+     */
+    public void stopBuildIndexes(UUID tableId, int partitionId) {
+        for (Iterator<Entry<IndexBuildTaskId, IndexBuildTask>> it = indexBuildTaskById.entrySet().iterator(); it.hasNext(); ) {
+            if (!busyLock.enterBusy()) {
+                return;
+            }
+
+            try {
+                Entry<IndexBuildTaskId, IndexBuildTask> entry = it.next();
+
+                IndexBuildTaskId taskId = entry.getKey();
+
+                if (tableId.equals(taskId.getTableId()) && partitionId == taskId.getPartitionId()) {
+                    it.remove();
+
+                    entry.getValue().stop();
+                }
+            } finally {
+                busyLock.leaveBusy();
+            }
+        }
+    }
+
+    @Override
+    public void close() {
+        if (!closeGuard.compareAndSet(false, true)) {
+            return;
+        }
+
+        busyLock.block();

Review Comment:
   Shall we still clear the map?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1199577985


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -382,8 +382,17 @@ public boolean stopReplica(ReplicationGroupId replicaGrpId) throws NodeStoppingE
      * @param replicaGrpId Replication group id.
      * @return True if the replica is found and closed, false otherwise.
      */
+    // TODO: IGNITE-19494 We need to correctly stop the replica

Review Comment:
   Instead of waiting we should complete the future exceptionally in order to interrupt replica await process. Semantically, only if it's not completed already, but who cares completeExceptionally will do the trick. We may introduce ReplicaStoppingException or similar and verify that such kind of exception is properly handled in `ReplicaService#sendToReplica`.
   @vldpyatkov @sergeyuttsel Guys you've touched replica await code recently, do you have any objections?
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1200045358


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/listener/ReplicaListener.java:
##########
@@ -31,4 +33,20 @@ public interface ReplicaListener {
      * @return Listener response.
      */
     CompletableFuture<?> invoke(ReplicaRequest request);
+
+    /**
+     * Callback on leader election.
+     *
+     * @param clusterNode Leader node.
+     */
+    default void onLeaderElected(ClusterNode clusterNode) {

Review Comment:
   fix it



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1197607156


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuildTask.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.index;
+
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static java.util.concurrent.CompletableFuture.supplyAsync;
+import static java.util.stream.Collectors.toList;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.RowId;
+import org.apache.ignite.internal.storage.index.IndexStorage;
+import org.apache.ignite.internal.table.distributed.TableMessagesFactory;
+import org.apache.ignite.internal.table.distributed.command.BuildIndexCommand;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteStringFormatter;
+
+/**
+ * Task of building a table index.
+ */
+class IndexBuildTask {
+    private static final IgniteLogger LOG = Loggers.forClass(IndexBuildTask.class);
+
+    private static final TableMessagesFactory TABLE_MESSAGES_FACTORY = new TableMessagesFactory();
+
+    private final IndexBuildTaskId taskId;
+
+    private final IndexStorage indexStorage;
+
+    private final MvPartitionStorage partitionStorage;
+
+    private final RaftGroupService raftClient;
+
+    private final ExecutorService executor;
+
+    private final IgniteSpinBusyLock busyLock;
+
+    private final int batchSize;
+
+    private final IgniteSpinBusyLock taskBusyLock = new IgniteSpinBusyLock();
+
+    private final AtomicBoolean taskStopGuard = new AtomicBoolean();
+
+    private final CompletableFuture<Void> taskFuture = new CompletableFuture<>();
+
+    IndexBuildTask(
+            IndexBuildTaskId taskId,
+            IndexStorage indexStorage,
+            MvPartitionStorage partitionStorage,
+            RaftGroupService raftClient,
+            ExecutorService executor,
+            IgniteSpinBusyLock busyLock,
+            int batchSize
+    ) {
+        this.taskId = taskId;
+        this.indexStorage = indexStorage;
+        this.partitionStorage = partitionStorage;
+        this.raftClient = raftClient;
+        this.executor = executor;
+        this.busyLock = busyLock;
+        this.batchSize = batchSize;
+    }
+
+    /**
+     * Starts building the index.
+     */
+    void start() {
+        if (!enterBusy()) {
+            taskFuture.complete(null);
+
+            return;
+        }
+
+        LOG.info("Start building the index: [{}]", getCommonIndexInfo());

Review Comment:
   fix it



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1197596673


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuilder.java:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.index;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.close.ManuallyCloseable;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.index.IndexStorage;
+import org.apache.ignite.internal.table.distributed.command.BuildIndexCommand;
+import org.apache.ignite.internal.thread.NamedThreadFactory;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.internal.util.IgniteUtils;
+
+/**
+ * Class for managing the building of table indexes.
+ */
+public class IndexBuilder implements ManuallyCloseable {
+    private static final IgniteLogger LOG = Loggers.forClass(IndexBuilder.class);
+
+    private static final int BATCH_SIZE = 100;
+
+    private final ExecutorService executor;
+
+    private final Map<IndexBuildTaskId, IndexBuildTask> indexBuildTaskById = new ConcurrentHashMap<>();
+
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    private final AtomicBoolean closeGuard = new AtomicBoolean();
+
+    /**
+     * Constructor.
+     *
+     * @param nodeName Node name.
+     * @param threadCount Number of threads to build indexes.
+     */
+    public IndexBuilder(String nodeName, int threadCount) {
+        executor = new ThreadPoolExecutor(
+                threadCount,
+                threadCount,
+                30,
+                TimeUnit.SECONDS,
+                new LinkedBlockingQueue<>(),
+                NamedThreadFactory.create(nodeName, "build-index", LOG)
+        );
+    }
+
+    /**
+     * Starts building the index if it is not already built or is not yet in progress.
+     *
+     * <p>Index is built in batches using {@link BuildIndexCommand} (via raft), batches are sent sequentially.
+     *
+     * <p>It is expected that the index building is triggered by the leader of the raft group.
+     *
+     * @param tableId Table ID.
+     * @param partitionId Partition ID.
+     * @param indexId Index ID.
+     * @param indexStorage Index storage to build.
+     * @param partitionStorage Multi-versioned partition storage.
+     * @param raftClient Raft client.
+     */
+    // TODO: IGNITE-19498 Perhaps we need to start building the index only once
+    public void startBuildIndex(
+            UUID tableId,
+            int partitionId,
+            UUID indexId,
+            IndexStorage indexStorage,
+            MvPartitionStorage partitionStorage,
+            RaftGroupService raftClient
+    ) {
+        inBusyLock(() -> {
+            if (indexStorage.getNextRowIdToBuild() == null) {
+                return;
+            }
+
+            IndexBuildTaskId taskId = new IndexBuildTaskId(tableId, partitionId, indexId);
+
+            IndexBuildTask newTask = new IndexBuildTask(taskId, indexStorage, partitionStorage, raftClient, executor, busyLock, BATCH_SIZE);
+
+            IndexBuildTask previousTask = indexBuildTaskById.putIfAbsent(taskId, newTask);
+
+            if (previousTask != null) {
+                // Index building is already in progress.
+                return;
+            }
+
+            newTask.start();
+
+            newTask.getTaskFuture().whenComplete((unused, throwable) -> indexBuildTaskById.remove(taskId));
+        });
+    }
+
+    /**
+     * Stops index building if it is in progress.
+     *
+     * @param tableId Table ID.
+     * @param partitionId Partition ID.
+     * @param indexId Index ID.
+     */
+    public void stopBuildIndex(UUID tableId, int partitionId, UUID indexId) {
+        inBusyLock(() -> {
+            IndexBuildTask removed = indexBuildTaskById.remove(new IndexBuildTaskId(tableId, partitionId, indexId));
+
+            if (removed != null) {
+                removed.stop();
+            }
+        });
+    }
+
+    /**
+     * Stops building all indexes (for a table partition) if they are in progress.
+     *
+     * @param tableId Table ID.
+     * @param partitionId Partition ID.
+     */
+    public void stopBuildIndexes(UUID tableId, int partitionId) {
+        for (Iterator<Entry<IndexBuildTaskId, IndexBuildTask>> it = indexBuildTaskById.entrySet().iterator(); it.hasNext(); ) {
+            if (!busyLock.enterBusy()) {
+                return;
+            }
+
+            try {
+                Entry<IndexBuildTaskId, IndexBuildTask> entry = it.next();
+
+                IndexBuildTaskId taskId = entry.getKey();
+
+                if (tableId.equals(taskId.getTableId()) && partitionId == taskId.getPartitionId()) {
+                    it.remove();
+
+                    entry.getValue().stop();
+                }
+            } finally {
+                busyLock.leaveBusy();
+            }
+        }
+    }
+
+    @Override
+    public void close() {
+        if (!closeGuard.compareAndSet(false, true)) {
+            return;
+        }
+
+        busyLock.block();

Review Comment:
   If the node stops then we must stop building all indexes.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1187264555


##########
modules/index/src/main/java/org/apache/ignite/internal/index/IndexBuilder.java:
##########
@@ -100,10 +111,14 @@ void stop() {
      */
     void startIndexBuild(TableIndexView tableIndexView, TableImpl table) {
         for (int partitionId = 0; partitionId < table.internalTable().partitions(); partitionId++) {
+            // Let's check if there is a node in the assignments for the partition.
+            if (!replicaManager.startedGroups().contains(new TablePartitionId(table.tableId(), partitionId))) {

Review Comment:
   I used replicas on the advice of @sanpwc.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sashapolo commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "sashapolo (via GitHub)" <gi...@apache.org>.
sashapolo commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1186141665


##########
modules/index/src/main/java/org/apache/ignite/internal/index/IndexBuilder.java:
##########
@@ -100,10 +111,14 @@ void stop() {
      */
     void startIndexBuild(TableIndexView tableIndexView, TableImpl table) {
         for (int partitionId = 0; partitionId < table.internalTable().partitions(); partitionId++) {
+            // Let's check if there is a node in the assignments for the partition.

Review Comment:
   ```suggestion
               // Check if this node is present in the assignments for the partition.
   ```



##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItBuildIndexTest.java:
##########
@@ -63,8 +70,20 @@ TABLE_NAME, toValuesString(List.of(1, 1), List.of(2, 2), List.of(3, 3), List.of(
         sql(IgniteStringFormatter.format("CREATE INDEX {} ON {} (i1)", INDEX_NAME, TABLE_NAME));
 
         // TODO: IGNITE-19150 We are waiting for schema synchronization to avoid races to create and destroy indexes
-        waitForIndexBuild(TABLE_NAME, INDEX_NAME);
+        Map<Integer, List<Ignite>> nodesWithBuiltIndexesByPartitionId = waitForIndexBuild(TABLE_NAME, INDEX_NAME);
+
+        // Check that the number of nodes with built indexes is equal to the number of replicas.
+        assertEquals(partitions, nodesWithBuiltIndexesByPartitionId.size());
+
+        for (Entry<Integer, List<Ignite>> entry : nodesWithBuiltIndexesByPartitionId.entrySet()) {
+            assertEquals(
+                    replicas,
+                    entry.getValue().size(),
+                    IgniteStringFormatter.format("p={}, nodes={}", entry.getKey(), entry.getValue())
+            );
+        }
 
+        // Let's check the sql query.

Review Comment:
   This comment is redundant



##########
modules/index/src/main/java/org/apache/ignite/internal/index/IndexBuilder.java:
##########
@@ -100,10 +111,14 @@ void stop() {
      */
     void startIndexBuild(TableIndexView tableIndexView, TableImpl table) {
         for (int partitionId = 0; partitionId < table.internalTable().partitions(); partitionId++) {
+            // Let's check if there is a node in the assignments for the partition.
+            if (!replicaManager.startedGroups().contains(new TablePartitionId(table.tableId(), partitionId))) {

Review Comment:
   Why do we need to iterate over all partitions and check if `startedGroups` contains a particular one? Can we simply iterate over `startedGroups` and get partitions from there?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1197599386


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuilder.java:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.index;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.close.ManuallyCloseable;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.index.IndexStorage;
+import org.apache.ignite.internal.table.distributed.command.BuildIndexCommand;
+import org.apache.ignite.internal.thread.NamedThreadFactory;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.internal.util.IgniteUtils;
+
+/**
+ * Class for managing the building of table indexes.
+ */
+public class IndexBuilder implements ManuallyCloseable {
+    private static final IgniteLogger LOG = Loggers.forClass(IndexBuilder.class);
+
+    private static final int BATCH_SIZE = 100;
+
+    private final ExecutorService executor;
+
+    private final Map<IndexBuildTaskId, IndexBuildTask> indexBuildTaskById = new ConcurrentHashMap<>();
+
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    private final AtomicBoolean closeGuard = new AtomicBoolean();
+
+    /**
+     * Constructor.
+     *
+     * @param nodeName Node name.
+     * @param threadCount Number of threads to build indexes.
+     */
+    public IndexBuilder(String nodeName, int threadCount) {
+        executor = new ThreadPoolExecutor(
+                threadCount,
+                threadCount,
+                30,
+                TimeUnit.SECONDS,
+                new LinkedBlockingQueue<>(),
+                NamedThreadFactory.create(nodeName, "build-index", LOG)
+        );
+    }
+
+    /**
+     * Starts building the index if it is not already built or is not yet in progress.
+     *
+     * <p>Index is built in batches using {@link BuildIndexCommand} (via raft), batches are sent sequentially.
+     *
+     * <p>It is expected that the index building is triggered by the leader of the raft group.
+     *
+     * @param tableId Table ID.
+     * @param partitionId Partition ID.
+     * @param indexId Index ID.
+     * @param indexStorage Index storage to build.
+     * @param partitionStorage Multi-versioned partition storage.
+     * @param raftClient Raft client.
+     */
+    // TODO: IGNITE-19498 Perhaps we need to start building the index only once
+    public void startBuildIndex(
+            UUID tableId,
+            int partitionId,
+            UUID indexId,
+            IndexStorage indexStorage,
+            MvPartitionStorage partitionStorage,
+            RaftGroupService raftClient
+    ) {
+        inBusyLock(() -> {
+            if (indexStorage.getNextRowIdToBuild() == null) {
+                return;
+            }
+
+            IndexBuildTaskId taskId = new IndexBuildTaskId(tableId, partitionId, indexId);
+
+            IndexBuildTask newTask = new IndexBuildTask(taskId, indexStorage, partitionStorage, raftClient, executor, busyLock, BATCH_SIZE);
+
+            IndexBuildTask previousTask = indexBuildTaskById.putIfAbsent(taskId, newTask);
+
+            if (previousTask != null) {
+                // Index building is already in progress.
+                return;
+            }
+
+            newTask.start();
+
+            newTask.getTaskFuture().whenComplete((unused, throwable) -> indexBuildTaskById.remove(taskId));
+        });
+    }
+
+    /**
+     * Stops index building if it is in progress.
+     *
+     * @param tableId Table ID.
+     * @param partitionId Partition ID.
+     * @param indexId Index ID.
+     */
+    public void stopBuildIndex(UUID tableId, int partitionId, UUID indexId) {
+        inBusyLock(() -> {
+            IndexBuildTask removed = indexBuildTaskById.remove(new IndexBuildTaskId(tableId, partitionId, indexId));
+
+            if (removed != null) {
+                removed.stop();
+            }
+        });
+    }
+
+    /**
+     * Stops building all indexes (for a table partition) if they are in progress.
+     *
+     * @param tableId Table ID.
+     * @param partitionId Partition ID.
+     */
+    public void stopBuildIndexes(UUID tableId, int partitionId) {
+        for (Iterator<Entry<IndexBuildTaskId, IndexBuildTask>> it = indexBuildTaskById.entrySet().iterator(); it.hasNext(); ) {
+            if (!busyLock.enterBusy()) {

Review Comment:
   This is an optimization, if we stop the node, then it no longer makes sense to stop building the partition indexes.
   
   If you don't like it, I can remove this optimization.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1197610992


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuildTask.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.index;
+
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static java.util.concurrent.CompletableFuture.supplyAsync;
+import static java.util.stream.Collectors.toList;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.RowId;
+import org.apache.ignite.internal.storage.index.IndexStorage;
+import org.apache.ignite.internal.table.distributed.TableMessagesFactory;
+import org.apache.ignite.internal.table.distributed.command.BuildIndexCommand;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteStringFormatter;
+
+/**
+ * Task of building a table index.
+ */
+class IndexBuildTask {
+    private static final IgniteLogger LOG = Loggers.forClass(IndexBuildTask.class);
+
+    private static final TableMessagesFactory TABLE_MESSAGES_FACTORY = new TableMessagesFactory();
+
+    private final IndexBuildTaskId taskId;
+
+    private final IndexStorage indexStorage;
+
+    private final MvPartitionStorage partitionStorage;
+
+    private final RaftGroupService raftClient;
+
+    private final ExecutorService executor;
+
+    private final IgniteSpinBusyLock busyLock;
+
+    private final int batchSize;
+
+    private final IgniteSpinBusyLock taskBusyLock = new IgniteSpinBusyLock();
+
+    private final AtomicBoolean taskStopGuard = new AtomicBoolean();
+
+    private final CompletableFuture<Void> taskFuture = new CompletableFuture<>();
+
+    IndexBuildTask(
+            IndexBuildTaskId taskId,
+            IndexStorage indexStorage,
+            MvPartitionStorage partitionStorage,
+            RaftGroupService raftClient,
+            ExecutorService executor,
+            IgniteSpinBusyLock busyLock,
+            int batchSize
+    ) {
+        this.taskId = taskId;
+        this.indexStorage = indexStorage;
+        this.partitionStorage = partitionStorage;
+        this.raftClient = raftClient;
+        this.executor = executor;
+        this.busyLock = busyLock;
+        this.batchSize = batchSize;
+    }
+
+    /**
+     * Starts building the index.
+     */
+    void start() {
+        if (!enterBusy()) {
+            taskFuture.complete(null);
+
+            return;
+        }
+
+        LOG.info("Start building the index: [{}]", getCommonIndexInfo());
+
+        try {
+            supplyAsync(this::handleNextBatch, executor)
+                    .thenCompose(Function.identity())
+                    .whenComplete((unused, throwable) -> {
+                        if (throwable != null) {
+                            LOG.error("Index build error: [{}]", throwable, getCommonIndexInfo());
+
+                            taskFuture.completeExceptionally(throwable);
+                        } else {
+                            taskFuture.complete(null);
+                        }
+                    });
+        } catch (Throwable t) {
+            taskFuture.completeExceptionally(t);
+
+            throw t;
+        } finally {
+            leaveBusy();
+        }
+    }
+
+    /**
+     * Stops index building.
+     */
+    void stop() {
+        if (!taskStopGuard.compareAndSet(false, true)) {
+            return;
+        }
+
+        taskBusyLock.block();
+    }
+
+    /**
+     * Returns the index build future.
+     */
+    CompletableFuture<Void> getTaskFuture() {
+        return taskFuture;
+    }
+
+    private CompletableFuture<Void> handleNextBatch() {
+        if (!enterBusy()) {
+            return completedFuture(null);
+        }
+
+        try {
+            List<RowId> batchRowIds = createBatchRowIds();
+
+            return raftClient.run(createBuildIndexCommand(batchRowIds))
+                    .thenComposeAsync(unused -> {
+                        if (indexStorage.getNextRowIdToBuild() == null) {
+                            // Index has been built.
+                            return completedFuture(null);
+                        }
+
+                        return handleNextBatch();
+                    }, executor);
+        } catch (Throwable t) {
+            return failedFuture(t);
+        } finally {
+            leaveBusy();
+        }
+    }
+
+    private List<RowId> createBatchRowIds() {
+        RowId nextRowIdToBuild = indexStorage.getNextRowIdToBuild();
+
+        List<RowId> batch = new ArrayList<>(batchSize);
+
+        for (int i = 0; i < batchSize && nextRowIdToBuild != null; i++) {
+            nextRowIdToBuild = partitionStorage.closestRowId(nextRowIdToBuild);
+
+            if (nextRowIdToBuild == null) {
+                break;
+            }
+
+            batch.add(nextRowIdToBuild);
+
+            nextRowIdToBuild = nextRowIdToBuild.increment();
+        }
+
+        return batch;
+    }
+
+    private BuildIndexCommand createBuildIndexCommand(List<RowId> rowIds) {
+        boolean finish = rowIds.size() < batchSize;
+
+        return TABLE_MESSAGES_FACTORY.buildIndexCommand()
+                .tablePartitionId(TABLE_MESSAGES_FACTORY.tablePartitionIdMessage()
+                        .tableId(taskId.getTableId())
+                        .partitionId(taskId.getPartitionId())
+                        .build()
+                )
+                .indexId(taskId.getIndexId())
+                .rowIds(rowIds.stream().map(RowId::uuid).collect(toList()))
+                .finish(finish)
+                .build();
+    }
+
+    private boolean enterBusy() {
+        if (!taskBusyLock.enterBusy()) {
+            return false;
+        }
+
+        if (!busyLock.enterBusy()) {

Review Comment:
   It doesn't matter to me - I'll change it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1200024376


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -382,8 +382,17 @@ public boolean stopReplica(ReplicationGroupId replicaGrpId) throws NodeStoppingE
      * @param replicaGrpId Replication group id.
      * @return True if the replica is found and closed, false otherwise.
      */
+    // TODO: IGNITE-19494 We need to correctly stop the replica

Review Comment:
   I have already created IGNITE-19494 in which it will be possible to implement any behavior, I propose this within wdyt?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1197595968


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuilder.java:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.index;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.close.ManuallyCloseable;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.index.IndexStorage;
+import org.apache.ignite.internal.table.distributed.command.BuildIndexCommand;
+import org.apache.ignite.internal.thread.NamedThreadFactory;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.internal.util.IgniteUtils;
+
+/**
+ * Class for managing the building of table indexes.
+ */
+public class IndexBuilder implements ManuallyCloseable {
+    private static final IgniteLogger LOG = Loggers.forClass(IndexBuilder.class);
+
+    private static final int BATCH_SIZE = 100;
+
+    private final ExecutorService executor;
+
+    private final Map<IndexBuildTaskId, IndexBuildTask> indexBuildTaskById = new ConcurrentHashMap<>();
+
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    private final AtomicBoolean closeGuard = new AtomicBoolean();
+
+    /**
+     * Constructor.
+     *
+     * @param nodeName Node name.
+     * @param threadCount Number of threads to build indexes.
+     */
+    public IndexBuilder(String nodeName, int threadCount) {
+        executor = new ThreadPoolExecutor(
+                threadCount,
+                threadCount,
+                30,
+                TimeUnit.SECONDS,
+                new LinkedBlockingQueue<>(),
+                NamedThreadFactory.create(nodeName, "build-index", LOG)
+        );
+    }
+
+    /**
+     * Starts building the index if it is not already built or is not yet in progress.
+     *
+     * <p>Index is built in batches using {@link BuildIndexCommand} (via raft), batches are sent sequentially.
+     *
+     * <p>It is expected that the index building is triggered by the leader of the raft group.
+     *
+     * @param tableId Table ID.
+     * @param partitionId Partition ID.
+     * @param indexId Index ID.
+     * @param indexStorage Index storage to build.
+     * @param partitionStorage Multi-versioned partition storage.
+     * @param raftClient Raft client.
+     */
+    // TODO: IGNITE-19498 Perhaps we need to start building the index only once
+    public void startBuildIndex(
+            UUID tableId,
+            int partitionId,
+            UUID indexId,
+            IndexStorage indexStorage,
+            MvPartitionStorage partitionStorage,
+            RaftGroupService raftClient
+    ) {
+        inBusyLock(() -> {
+            if (indexStorage.getNextRowIdToBuild() == null) {
+                return;
+            }
+
+            IndexBuildTaskId taskId = new IndexBuildTaskId(tableId, partitionId, indexId);
+
+            IndexBuildTask newTask = new IndexBuildTask(taskId, indexStorage, partitionStorage, raftClient, executor, busyLock, BATCH_SIZE);
+
+            IndexBuildTask previousTask = indexBuildTaskById.putIfAbsent(taskId, newTask);

Review Comment:
   think that your suggestion does not quite suit me, because then the code for starting the task and subscribing to the future will be inside the closure, which can lead to a lengthening of the closure execution, although it should be fast, and the deletion of the task can immediately be performed in this closure and we can freeze.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1197589653


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -382,8 +382,17 @@ public boolean stopReplica(ReplicationGroupId replicaGrpId) throws NodeStoppingE
      * @param replicaGrpId Replication group id.
      * @return True if the replica is found and closed, false otherwise.
      */
+    // TODO: IGNITE-19494 We need to correctly stop the replica
     private boolean stopReplicaInternal(ReplicationGroupId replicaGrpId) {
-        return replicas.remove(replicaGrpId) != null;
+        CompletableFuture<Replica> removed = replicas.remove(replicaGrpId);
+
+        if (removed != null) {
+            removed.whenComplete((replica, throwable) -> replica.shutdown());

Review Comment:
   Fix it



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1197740645


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuilder.java:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.index;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.close.ManuallyCloseable;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.index.IndexStorage;
+import org.apache.ignite.internal.table.distributed.command.BuildIndexCommand;
+import org.apache.ignite.internal.thread.NamedThreadFactory;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.internal.util.IgniteUtils;
+
+/**
+ * Class for managing the building of table indexes.
+ */
+public class IndexBuilder implements ManuallyCloseable {
+    private static final IgniteLogger LOG = Loggers.forClass(IndexBuilder.class);
+
+    private static final int BATCH_SIZE = 100;
+
+    private final ExecutorService executor;
+
+    private final Map<IndexBuildTaskId, IndexBuildTask> indexBuildTaskById = new ConcurrentHashMap<>();
+
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    private final AtomicBoolean closeGuard = new AtomicBoolean();
+
+    /**
+     * Constructor.
+     *
+     * @param nodeName Node name.
+     * @param threadCount Number of threads to build indexes.
+     */
+    public IndexBuilder(String nodeName, int threadCount) {
+        executor = new ThreadPoolExecutor(
+                threadCount,
+                threadCount,
+                30,
+                TimeUnit.SECONDS,
+                new LinkedBlockingQueue<>(),
+                NamedThreadFactory.create(nodeName, "build-index", LOG)
+        );
+    }
+
+    /**
+     * Starts building the index if it is not already built or is not yet in progress.
+     *
+     * <p>Index is built in batches using {@link BuildIndexCommand} (via raft), batches are sent sequentially.
+     *
+     * <p>It is expected that the index building is triggered by the leader of the raft group.
+     *
+     * @param tableId Table ID.
+     * @param partitionId Partition ID.
+     * @param indexId Index ID.
+     * @param indexStorage Index storage to build.
+     * @param partitionStorage Multi-versioned partition storage.
+     * @param raftClient Raft client.
+     */
+    // TODO: IGNITE-19498 Perhaps we need to start building the index only once
+    public void startBuildIndex(
+            UUID tableId,
+            int partitionId,
+            UUID indexId,
+            IndexStorage indexStorage,
+            MvPartitionStorage partitionStorage,
+            RaftGroupService raftClient
+    ) {
+        inBusyLock(() -> {
+            if (indexStorage.getNextRowIdToBuild() == null) {
+                return;
+            }
+
+            IndexBuildTaskId taskId = new IndexBuildTaskId(tableId, partitionId, indexId);
+
+            IndexBuildTask newTask = new IndexBuildTask(taskId, indexStorage, partitionStorage, raftClient, executor, busyLock, BATCH_SIZE);
+
+            IndexBuildTask previousTask = indexBuildTaskById.putIfAbsent(taskId, newTask);
+
+            if (previousTask != null) {
+                // Index building is already in progress.
+                return;
+            }
+
+            newTask.start();
+
+            newTask.getTaskFuture().whenComplete((unused, throwable) -> indexBuildTaskById.remove(taskId));
+        });
+    }
+
+    /**
+     * Stops index building if it is in progress.
+     *
+     * @param tableId Table ID.
+     * @param partitionId Partition ID.
+     * @param indexId Index ID.
+     */
+    public void stopBuildIndex(UUID tableId, int partitionId, UUID indexId) {
+        inBusyLock(() -> {
+            IndexBuildTask removed = indexBuildTaskById.remove(new IndexBuildTaskId(tableId, partitionId, indexId));
+
+            if (removed != null) {
+                removed.stop();
+            }
+        });
+    }
+
+    /**
+     * Stops building all indexes (for a table partition) if they are in progress.
+     *
+     * @param tableId Table ID.
+     * @param partitionId Partition ID.
+     */
+    public void stopBuildIndexes(UUID tableId, int partitionId) {
+        for (Iterator<Entry<IndexBuildTaskId, IndexBuildTask>> it = indexBuildTaskById.entrySet().iterator(); it.hasNext(); ) {
+            if (!busyLock.enterBusy()) {
+                return;
+            }
+
+            try {
+                Entry<IndexBuildTaskId, IndexBuildTask> entry = it.next();
+
+                IndexBuildTaskId taskId = entry.getKey();
+
+                if (tableId.equals(taskId.getTableId()) && partitionId == taskId.getPartitionId()) {
+                    it.remove();
+
+                    entry.getValue().stop();
+                }
+            } finally {
+                busyLock.leaveBusy();
+            }
+        }
+    }
+
+    @Override
+    public void close() {
+        if (!closeGuard.compareAndSet(false, true)) {
+            return;
+        }
+
+        busyLock.block();

Review Comment:
   Yes, when this **busyLock** is blocked, the tasks will complete their work and be removed from the map on their own.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] tkalkirill merged pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill merged PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "sergeyuttsel (via GitHub)" <gi...@apache.org>.
sergeyuttsel commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1200097549


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -382,8 +382,17 @@ public boolean stopReplica(ReplicationGroupId replicaGrpId) throws NodeStoppingE
      * @param replicaGrpId Replication group id.
      * @return True if the replica is found and closed, false otherwise.
      */
+    // TODO: IGNITE-19494 We need to correctly stop the replica

Review Comment:
   I have no objections. I agree that we should complete the future exceptionally in order to interrupt replica await process in ReplicaService#sendToReplica.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1197603994


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/PartitionAccessImpl.java:
##########
@@ -192,6 +192,8 @@ public long maxLastAppliedTerm() {
 
     @Override
     public CompletableFuture<Void> startRebalance() {
+        storageUpdateHandler.waitIndexes();

Review Comment:
   Fix it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1197610004


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuildTask.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.index;
+
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static java.util.concurrent.CompletableFuture.supplyAsync;
+import static java.util.stream.Collectors.toList;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.RowId;
+import org.apache.ignite.internal.storage.index.IndexStorage;
+import org.apache.ignite.internal.table.distributed.TableMessagesFactory;
+import org.apache.ignite.internal.table.distributed.command.BuildIndexCommand;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteStringFormatter;
+
+/**
+ * Task of building a table index.
+ */
+class IndexBuildTask {
+    private static final IgniteLogger LOG = Loggers.forClass(IndexBuildTask.class);
+
+    private static final TableMessagesFactory TABLE_MESSAGES_FACTORY = new TableMessagesFactory();
+
+    private final IndexBuildTaskId taskId;
+
+    private final IndexStorage indexStorage;
+
+    private final MvPartitionStorage partitionStorage;
+
+    private final RaftGroupService raftClient;
+
+    private final ExecutorService executor;
+
+    private final IgniteSpinBusyLock busyLock;
+
+    private final int batchSize;
+
+    private final IgniteSpinBusyLock taskBusyLock = new IgniteSpinBusyLock();
+
+    private final AtomicBoolean taskStopGuard = new AtomicBoolean();
+
+    private final CompletableFuture<Void> taskFuture = new CompletableFuture<>();
+
+    IndexBuildTask(
+            IndexBuildTaskId taskId,
+            IndexStorage indexStorage,
+            MvPartitionStorage partitionStorage,
+            RaftGroupService raftClient,
+            ExecutorService executor,
+            IgniteSpinBusyLock busyLock,
+            int batchSize
+    ) {
+        this.taskId = taskId;
+        this.indexStorage = indexStorage;
+        this.partitionStorage = partitionStorage;
+        this.raftClient = raftClient;
+        this.executor = executor;
+        this.busyLock = busyLock;
+        this.batchSize = batchSize;
+    }
+
+    /**
+     * Starts building the index.
+     */
+    void start() {
+        if (!enterBusy()) {
+            taskFuture.complete(null);
+
+            return;
+        }
+
+        LOG.info("Start building the index: [{}]", getCommonIndexInfo());
+
+        try {
+            supplyAsync(this::handleNextBatch, executor)

Review Comment:
   I would not like to load a thread that will start building the index, so I think it's right to do it in a separate pool.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sashapolo commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "sashapolo (via GitHub)" <gi...@apache.org>.
sashapolo commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1197547305


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -382,8 +382,17 @@ public boolean stopReplica(ReplicationGroupId replicaGrpId) throws NodeStoppingE
      * @param replicaGrpId Replication group id.
      * @return True if the replica is found and closed, false otherwise.
      */
+    // TODO: IGNITE-19494 We need to correctly stop the replica
     private boolean stopReplicaInternal(ReplicationGroupId replicaGrpId) {
-        return replicas.remove(replicaGrpId) != null;
+        CompletableFuture<Replica> removed = replicas.remove(replicaGrpId);
+
+        if (removed != null) {
+            removed.whenComplete((replica, throwable) -> replica.shutdown());

Review Comment:
   You need to check that `throwable` is `null`



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/StorageUpdateHandler.java:
##########
@@ -417,4 +417,12 @@ public void buildIndex(UUID indexId, List<UUID> rowUuids, boolean finish) {
     public PendingComparableValuesTracker<HybridTimestamp, Void> getSafeTimeTracker() {
         return safeTimeTracker;
     }
+
+    /**
+     * Waits for indexes to be created.
+     */
+    // TODO: IGNITE-19513 Fix it, we should have already waited for the indexes to be created

Review Comment:
   Again, the referred ticket description is basically nonexistent



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -382,8 +382,17 @@ public boolean stopReplica(ReplicationGroupId replicaGrpId) throws NodeStoppingE
      * @param replicaGrpId Replication group id.
      * @return True if the replica is found and closed, false otherwise.
      */
+    // TODO: IGNITE-19494 We need to correctly stop the replica

Review Comment:
   The ticket that is mentioned here has a terrible description, sorry



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuilder.java:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.index;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.close.ManuallyCloseable;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.index.IndexStorage;
+import org.apache.ignite.internal.table.distributed.command.BuildIndexCommand;
+import org.apache.ignite.internal.thread.NamedThreadFactory;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.internal.util.IgniteUtils;
+
+/**
+ * Class for managing the building of table indexes.
+ */
+public class IndexBuilder implements ManuallyCloseable {
+    private static final IgniteLogger LOG = Loggers.forClass(IndexBuilder.class);
+
+    private static final int BATCH_SIZE = 100;
+
+    private final ExecutorService executor;
+
+    private final Map<IndexBuildTaskId, IndexBuildTask> indexBuildTaskById = new ConcurrentHashMap<>();
+
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    private final AtomicBoolean closeGuard = new AtomicBoolean();
+
+    /**
+     * Constructor.
+     *
+     * @param nodeName Node name.
+     * @param threadCount Number of threads to build indexes.
+     */
+    public IndexBuilder(String nodeName, int threadCount) {
+        executor = new ThreadPoolExecutor(
+                threadCount,
+                threadCount,
+                30,
+                TimeUnit.SECONDS,
+                new LinkedBlockingQueue<>(),
+                NamedThreadFactory.create(nodeName, "build-index", LOG)
+        );
+    }
+
+    /**
+     * Starts building the index if it is not already built or is not yet in progress.
+     *
+     * <p>Index is built in batches using {@link BuildIndexCommand} (via raft), batches are sent sequentially.
+     *
+     * <p>It is expected that the index building is triggered by the leader of the raft group.
+     *
+     * @param tableId Table ID.
+     * @param partitionId Partition ID.
+     * @param indexId Index ID.
+     * @param indexStorage Index storage to build.
+     * @param partitionStorage Multi-versioned partition storage.
+     * @param raftClient Raft client.
+     */
+    // TODO: IGNITE-19498 Perhaps we need to start building the index only once
+    public void startBuildIndex(
+            UUID tableId,
+            int partitionId,
+            UUID indexId,
+            IndexStorage indexStorage,
+            MvPartitionStorage partitionStorage,
+            RaftGroupService raftClient
+    ) {
+        inBusyLock(() -> {
+            if (indexStorage.getNextRowIdToBuild() == null) {
+                return;
+            }
+
+            IndexBuildTaskId taskId = new IndexBuildTaskId(tableId, partitionId, indexId);
+
+            IndexBuildTask newTask = new IndexBuildTask(taskId, indexStorage, partitionStorage, raftClient, executor, busyLock, BATCH_SIZE);
+
+            IndexBuildTask previousTask = indexBuildTaskById.putIfAbsent(taskId, newTask);
+
+            if (previousTask != null) {
+                // Index building is already in progress.
+                return;
+            }
+
+            newTask.start();
+
+            newTask.getTaskFuture().whenComplete((unused, throwable) -> indexBuildTaskById.remove(taskId));
+        });
+    }
+
+    /**
+     * Stops index building if it is in progress.
+     *
+     * @param tableId Table ID.
+     * @param partitionId Partition ID.
+     * @param indexId Index ID.
+     */
+    public void stopBuildIndex(UUID tableId, int partitionId, UUID indexId) {
+        inBusyLock(() -> {
+            IndexBuildTask removed = indexBuildTaskById.remove(new IndexBuildTaskId(tableId, partitionId, indexId));
+
+            if (removed != null) {
+                removed.stop();
+            }
+        });
+    }
+
+    /**
+     * Stops building all indexes (for a table partition) if they are in progress.
+     *
+     * @param tableId Table ID.
+     * @param partitionId Partition ID.
+     */
+    public void stopBuildIndexes(UUID tableId, int partitionId) {
+        for (Iterator<Entry<IndexBuildTaskId, IndexBuildTask>> it = indexBuildTaskById.entrySet().iterator(); it.hasNext(); ) {
+            if (!busyLock.enterBusy()) {
+                return;
+            }
+
+            try {
+                Entry<IndexBuildTaskId, IndexBuildTask> entry = it.next();
+
+                IndexBuildTaskId taskId = entry.getKey();
+
+                if (tableId.equals(taskId.getTableId()) && partitionId == taskId.getPartitionId()) {
+                    it.remove();
+
+                    entry.getValue().stop();
+                }
+            } finally {
+                busyLock.leaveBusy();
+            }
+        }
+    }
+
+    @Override
+    public void close() {
+        if (!closeGuard.compareAndSet(false, true)) {
+            return;
+        }
+
+        busyLock.block();

Review Comment:
   Do we need to stop all index building tasks?



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuilder.java:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.index;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.close.ManuallyCloseable;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.index.IndexStorage;
+import org.apache.ignite.internal.table.distributed.command.BuildIndexCommand;
+import org.apache.ignite.internal.thread.NamedThreadFactory;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.internal.util.IgniteUtils;
+
+/**
+ * Class for managing the building of table indexes.
+ */
+public class IndexBuilder implements ManuallyCloseable {
+    private static final IgniteLogger LOG = Loggers.forClass(IndexBuilder.class);
+
+    private static final int BATCH_SIZE = 100;
+
+    private final ExecutorService executor;
+
+    private final Map<IndexBuildTaskId, IndexBuildTask> indexBuildTaskById = new ConcurrentHashMap<>();
+
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    private final AtomicBoolean closeGuard = new AtomicBoolean();
+
+    /**
+     * Constructor.
+     *
+     * @param nodeName Node name.
+     * @param threadCount Number of threads to build indexes.
+     */
+    public IndexBuilder(String nodeName, int threadCount) {
+        executor = new ThreadPoolExecutor(
+                threadCount,
+                threadCount,
+                30,
+                TimeUnit.SECONDS,
+                new LinkedBlockingQueue<>(),
+                NamedThreadFactory.create(nodeName, "build-index", LOG)
+        );
+    }
+
+    /**
+     * Starts building the index if it is not already built or is not yet in progress.
+     *
+     * <p>Index is built in batches using {@link BuildIndexCommand} (via raft), batches are sent sequentially.
+     *
+     * <p>It is expected that the index building is triggered by the leader of the raft group.
+     *
+     * @param tableId Table ID.
+     * @param partitionId Partition ID.
+     * @param indexId Index ID.
+     * @param indexStorage Index storage to build.
+     * @param partitionStorage Multi-versioned partition storage.
+     * @param raftClient Raft client.
+     */
+    // TODO: IGNITE-19498 Perhaps we need to start building the index only once
+    public void startBuildIndex(
+            UUID tableId,
+            int partitionId,
+            UUID indexId,
+            IndexStorage indexStorage,
+            MvPartitionStorage partitionStorage,
+            RaftGroupService raftClient
+    ) {
+        inBusyLock(() -> {
+            if (indexStorage.getNextRowIdToBuild() == null) {
+                return;
+            }
+
+            IndexBuildTaskId taskId = new IndexBuildTaskId(tableId, partitionId, indexId);
+
+            IndexBuildTask newTask = new IndexBuildTask(taskId, indexStorage, partitionStorage, raftClient, executor, busyLock, BATCH_SIZE);
+
+            IndexBuildTask previousTask = indexBuildTaskById.putIfAbsent(taskId, newTask);

Review Comment:
   you should use `computeIfAbsent` in order to avoid creating unnecessary task instances



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuilder.java:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.index;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.close.ManuallyCloseable;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.index.IndexStorage;
+import org.apache.ignite.internal.table.distributed.command.BuildIndexCommand;
+import org.apache.ignite.internal.thread.NamedThreadFactory;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.internal.util.IgniteUtils;
+
+/**
+ * Class for managing the building of table indexes.
+ */
+public class IndexBuilder implements ManuallyCloseable {
+    private static final IgniteLogger LOG = Loggers.forClass(IndexBuilder.class);
+
+    private static final int BATCH_SIZE = 100;
+
+    private final ExecutorService executor;
+
+    private final Map<IndexBuildTaskId, IndexBuildTask> indexBuildTaskById = new ConcurrentHashMap<>();
+
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    private final AtomicBoolean closeGuard = new AtomicBoolean();
+
+    /**
+     * Constructor.
+     *
+     * @param nodeName Node name.
+     * @param threadCount Number of threads to build indexes.
+     */
+    public IndexBuilder(String nodeName, int threadCount) {
+        executor = new ThreadPoolExecutor(
+                threadCount,
+                threadCount,
+                30,
+                TimeUnit.SECONDS,
+                new LinkedBlockingQueue<>(),
+                NamedThreadFactory.create(nodeName, "build-index", LOG)
+        );
+    }
+
+    /**
+     * Starts building the index if it is not already built or is not yet in progress.
+     *
+     * <p>Index is built in batches using {@link BuildIndexCommand} (via raft), batches are sent sequentially.
+     *
+     * <p>It is expected that the index building is triggered by the leader of the raft group.
+     *
+     * @param tableId Table ID.
+     * @param partitionId Partition ID.
+     * @param indexId Index ID.
+     * @param indexStorage Index storage to build.
+     * @param partitionStorage Multi-versioned partition storage.
+     * @param raftClient Raft client.
+     */
+    // TODO: IGNITE-19498 Perhaps we need to start building the index only once
+    public void startBuildIndex(
+            UUID tableId,
+            int partitionId,
+            UUID indexId,
+            IndexStorage indexStorage,
+            MvPartitionStorage partitionStorage,
+            RaftGroupService raftClient
+    ) {
+        inBusyLock(() -> {
+            if (indexStorage.getNextRowIdToBuild() == null) {
+                return;
+            }
+
+            IndexBuildTaskId taskId = new IndexBuildTaskId(tableId, partitionId, indexId);
+
+            IndexBuildTask newTask = new IndexBuildTask(taskId, indexStorage, partitionStorage, raftClient, executor, busyLock, BATCH_SIZE);
+
+            IndexBuildTask previousTask = indexBuildTaskById.putIfAbsent(taskId, newTask);
+
+            if (previousTask != null) {
+                // Index building is already in progress.
+                return;
+            }
+
+            newTask.start();
+
+            newTask.getTaskFuture().whenComplete((unused, throwable) -> indexBuildTaskById.remove(taskId));
+        });
+    }
+
+    /**
+     * Stops index building if it is in progress.
+     *
+     * @param tableId Table ID.
+     * @param partitionId Partition ID.
+     * @param indexId Index ID.
+     */
+    public void stopBuildIndex(UUID tableId, int partitionId, UUID indexId) {
+        inBusyLock(() -> {
+            IndexBuildTask removed = indexBuildTaskById.remove(new IndexBuildTaskId(tableId, partitionId, indexId));
+
+            if (removed != null) {
+                removed.stop();
+            }
+        });
+    }
+
+    /**
+     * Stops building all indexes (for a table partition) if they are in progress.
+     *
+     * @param tableId Table ID.
+     * @param partitionId Partition ID.
+     */
+    public void stopBuildIndexes(UUID tableId, int partitionId) {
+        for (Iterator<Entry<IndexBuildTaskId, IndexBuildTask>> it = indexBuildTaskById.entrySet().iterator(); it.hasNext(); ) {
+            if (!busyLock.enterBusy()) {

Review Comment:
   Why do we take the lock inside the loop?



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/PartitionAccessImpl.java:
##########
@@ -192,6 +192,8 @@ public long maxLastAppliedTerm() {
 
     @Override
     public CompletableFuture<Void> startRebalance() {
+        storageUpdateHandler.waitIndexes();

Review Comment:
   Please add a clarifying comment, why this line is here



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuildTask.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.index;
+
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static java.util.concurrent.CompletableFuture.supplyAsync;
+import static java.util.stream.Collectors.toList;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.RowId;
+import org.apache.ignite.internal.storage.index.IndexStorage;
+import org.apache.ignite.internal.table.distributed.TableMessagesFactory;
+import org.apache.ignite.internal.table.distributed.command.BuildIndexCommand;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteStringFormatter;
+
+/**
+ * Task of building a table index.
+ */
+class IndexBuildTask {
+    private static final IgniteLogger LOG = Loggers.forClass(IndexBuildTask.class);
+
+    private static final TableMessagesFactory TABLE_MESSAGES_FACTORY = new TableMessagesFactory();
+
+    private final IndexBuildTaskId taskId;
+
+    private final IndexStorage indexStorage;
+
+    private final MvPartitionStorage partitionStorage;
+
+    private final RaftGroupService raftClient;
+
+    private final ExecutorService executor;
+
+    private final IgniteSpinBusyLock busyLock;
+
+    private final int batchSize;
+
+    private final IgniteSpinBusyLock taskBusyLock = new IgniteSpinBusyLock();
+
+    private final AtomicBoolean taskStopGuard = new AtomicBoolean();
+
+    private final CompletableFuture<Void> taskFuture = new CompletableFuture<>();
+
+    IndexBuildTask(
+            IndexBuildTaskId taskId,
+            IndexStorage indexStorage,
+            MvPartitionStorage partitionStorage,
+            RaftGroupService raftClient,
+            ExecutorService executor,
+            IgniteSpinBusyLock busyLock,
+            int batchSize
+    ) {
+        this.taskId = taskId;
+        this.indexStorage = indexStorage;
+        this.partitionStorage = partitionStorage;
+        this.raftClient = raftClient;
+        this.executor = executor;
+        this.busyLock = busyLock;
+        this.batchSize = batchSize;
+    }
+
+    /**
+     * Starts building the index.
+     */
+    void start() {
+        if (!enterBusy()) {
+            taskFuture.complete(null);

Review Comment:
   why not `completeExceptionally(NodeStoppingException)`?



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuildTask.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.index;
+
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static java.util.concurrent.CompletableFuture.supplyAsync;
+import static java.util.stream.Collectors.toList;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.RowId;
+import org.apache.ignite.internal.storage.index.IndexStorage;
+import org.apache.ignite.internal.table.distributed.TableMessagesFactory;
+import org.apache.ignite.internal.table.distributed.command.BuildIndexCommand;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteStringFormatter;
+
+/**
+ * Task of building a table index.
+ */
+class IndexBuildTask {
+    private static final IgniteLogger LOG = Loggers.forClass(IndexBuildTask.class);
+
+    private static final TableMessagesFactory TABLE_MESSAGES_FACTORY = new TableMessagesFactory();
+
+    private final IndexBuildTaskId taskId;
+
+    private final IndexStorage indexStorage;
+
+    private final MvPartitionStorage partitionStorage;
+
+    private final RaftGroupService raftClient;
+
+    private final ExecutorService executor;
+
+    private final IgniteSpinBusyLock busyLock;
+
+    private final int batchSize;
+
+    private final IgniteSpinBusyLock taskBusyLock = new IgniteSpinBusyLock();
+
+    private final AtomicBoolean taskStopGuard = new AtomicBoolean();
+
+    private final CompletableFuture<Void> taskFuture = new CompletableFuture<>();
+
+    IndexBuildTask(
+            IndexBuildTaskId taskId,
+            IndexStorage indexStorage,
+            MvPartitionStorage partitionStorage,
+            RaftGroupService raftClient,
+            ExecutorService executor,
+            IgniteSpinBusyLock busyLock,
+            int batchSize
+    ) {
+        this.taskId = taskId;
+        this.indexStorage = indexStorage;
+        this.partitionStorage = partitionStorage;
+        this.raftClient = raftClient;
+        this.executor = executor;
+        this.busyLock = busyLock;
+        this.batchSize = batchSize;
+    }
+
+    /**
+     * Starts building the index.
+     */
+    void start() {
+        if (!enterBusy()) {
+            taskFuture.complete(null);
+
+            return;
+        }
+
+        LOG.info("Start building the index: [{}]", getCommonIndexInfo());

Review Comment:
   `getCommonIndexInfo` is not a free operation, you should wrap this logging with `isInfoEnabled`



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuildTask.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.index;
+
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static java.util.concurrent.CompletableFuture.supplyAsync;
+import static java.util.stream.Collectors.toList;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.RowId;
+import org.apache.ignite.internal.storage.index.IndexStorage;
+import org.apache.ignite.internal.table.distributed.TableMessagesFactory;
+import org.apache.ignite.internal.table.distributed.command.BuildIndexCommand;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteStringFormatter;
+
+/**
+ * Task of building a table index.
+ */
+class IndexBuildTask {
+    private static final IgniteLogger LOG = Loggers.forClass(IndexBuildTask.class);
+
+    private static final TableMessagesFactory TABLE_MESSAGES_FACTORY = new TableMessagesFactory();
+
+    private final IndexBuildTaskId taskId;
+
+    private final IndexStorage indexStorage;
+
+    private final MvPartitionStorage partitionStorage;
+
+    private final RaftGroupService raftClient;
+
+    private final ExecutorService executor;
+
+    private final IgniteSpinBusyLock busyLock;
+
+    private final int batchSize;
+
+    private final IgniteSpinBusyLock taskBusyLock = new IgniteSpinBusyLock();
+
+    private final AtomicBoolean taskStopGuard = new AtomicBoolean();
+
+    private final CompletableFuture<Void> taskFuture = new CompletableFuture<>();
+
+    IndexBuildTask(
+            IndexBuildTaskId taskId,
+            IndexStorage indexStorage,
+            MvPartitionStorage partitionStorage,
+            RaftGroupService raftClient,
+            ExecutorService executor,
+            IgniteSpinBusyLock busyLock,
+            int batchSize
+    ) {
+        this.taskId = taskId;
+        this.indexStorage = indexStorage;
+        this.partitionStorage = partitionStorage;
+        this.raftClient = raftClient;
+        this.executor = executor;
+        this.busyLock = busyLock;
+        this.batchSize = batchSize;
+    }
+
+    /**
+     * Starts building the index.
+     */
+    void start() {
+        if (!enterBusy()) {
+            taskFuture.complete(null);
+
+            return;
+        }
+
+        LOG.info("Start building the index: [{}]", getCommonIndexInfo());
+
+        try {
+            supplyAsync(this::handleNextBatch, executor)
+                    .thenCompose(Function.identity())
+                    .whenComplete((unused, throwable) -> {
+                        if (throwable != null) {
+                            LOG.error("Index build error: [{}]", throwable, getCommonIndexInfo());
+
+                            taskFuture.completeExceptionally(throwable);
+                        } else {
+                            taskFuture.complete(null);
+                        }
+                    });
+        } catch (Throwable t) {
+            taskFuture.completeExceptionally(t);
+
+            throw t;
+        } finally {
+            leaveBusy();
+        }
+    }
+
+    /**
+     * Stops index building.
+     */
+    void stop() {
+        if (!taskStopGuard.compareAndSet(false, true)) {
+            return;
+        }
+
+        taskBusyLock.block();
+    }
+
+    /**
+     * Returns the index build future.
+     */
+    CompletableFuture<Void> getTaskFuture() {
+        return taskFuture;
+    }
+
+    private CompletableFuture<Void> handleNextBatch() {
+        if (!enterBusy()) {
+            return completedFuture(null);
+        }
+
+        try {
+            List<RowId> batchRowIds = createBatchRowIds();
+
+            return raftClient.run(createBuildIndexCommand(batchRowIds))
+                    .thenComposeAsync(unused -> {
+                        if (indexStorage.getNextRowIdToBuild() == null) {
+                            // Index has been built.
+                            return completedFuture(null);
+                        }
+
+                        return handleNextBatch();
+                    }, executor);
+        } catch (Throwable t) {
+            return failedFuture(t);
+        } finally {
+            leaveBusy();
+        }
+    }
+
+    private List<RowId> createBatchRowIds() {
+        RowId nextRowIdToBuild = indexStorage.getNextRowIdToBuild();
+
+        List<RowId> batch = new ArrayList<>(batchSize);
+
+        for (int i = 0; i < batchSize && nextRowIdToBuild != null; i++) {
+            nextRowIdToBuild = partitionStorage.closestRowId(nextRowIdToBuild);
+
+            if (nextRowIdToBuild == null) {
+                break;
+            }
+
+            batch.add(nextRowIdToBuild);
+
+            nextRowIdToBuild = nextRowIdToBuild.increment();
+        }
+
+        return batch;
+    }
+
+    private BuildIndexCommand createBuildIndexCommand(List<RowId> rowIds) {
+        boolean finish = rowIds.size() < batchSize;
+
+        return TABLE_MESSAGES_FACTORY.buildIndexCommand()
+                .tablePartitionId(TABLE_MESSAGES_FACTORY.tablePartitionIdMessage()
+                        .tableId(taskId.getTableId())
+                        .partitionId(taskId.getPartitionId())
+                        .build()
+                )
+                .indexId(taskId.getIndexId())
+                .rowIds(rowIds.stream().map(RowId::uuid).collect(toList()))
+                .finish(finish)
+                .build();
+    }
+
+    private boolean enterBusy() {
+        if (!taskBusyLock.enterBusy()) {
+            return false;
+        }
+
+        if (!busyLock.enterBusy()) {

Review Comment:
   Wouldn't it be easier to check the `busyLock` first and then the `taskBusyLock`?



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuildTask.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.index;
+
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static java.util.concurrent.CompletableFuture.supplyAsync;
+import static java.util.stream.Collectors.toList;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.RowId;
+import org.apache.ignite.internal.storage.index.IndexStorage;
+import org.apache.ignite.internal.table.distributed.TableMessagesFactory;
+import org.apache.ignite.internal.table.distributed.command.BuildIndexCommand;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteStringFormatter;
+
+/**
+ * Task of building a table index.
+ */
+class IndexBuildTask {
+    private static final IgniteLogger LOG = Loggers.forClass(IndexBuildTask.class);
+
+    private static final TableMessagesFactory TABLE_MESSAGES_FACTORY = new TableMessagesFactory();
+
+    private final IndexBuildTaskId taskId;
+
+    private final IndexStorage indexStorage;
+
+    private final MvPartitionStorage partitionStorage;
+
+    private final RaftGroupService raftClient;
+
+    private final ExecutorService executor;
+
+    private final IgniteSpinBusyLock busyLock;
+
+    private final int batchSize;
+
+    private final IgniteSpinBusyLock taskBusyLock = new IgniteSpinBusyLock();
+
+    private final AtomicBoolean taskStopGuard = new AtomicBoolean();
+
+    private final CompletableFuture<Void> taskFuture = new CompletableFuture<>();
+
+    IndexBuildTask(
+            IndexBuildTaskId taskId,
+            IndexStorage indexStorage,
+            MvPartitionStorage partitionStorage,
+            RaftGroupService raftClient,
+            ExecutorService executor,
+            IgniteSpinBusyLock busyLock,
+            int batchSize
+    ) {
+        this.taskId = taskId;
+        this.indexStorage = indexStorage;
+        this.partitionStorage = partitionStorage;
+        this.raftClient = raftClient;
+        this.executor = executor;
+        this.busyLock = busyLock;
+        this.batchSize = batchSize;
+    }
+
+    /**
+     * Starts building the index.
+     */
+    void start() {
+        if (!enterBusy()) {
+            taskFuture.complete(null);
+
+            return;
+        }
+
+        LOG.info("Start building the index: [{}]", getCommonIndexInfo());
+
+        try {
+            supplyAsync(this::handleNextBatch, executor)

Review Comment:
   Why do you call `supplyAsync` here? So that `createBatchRowIds` are computed on the executor? Can we do that explicitly instead?



##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ClusterPerClassIntegrationTest.java:
##########
@@ -515,14 +513,20 @@ public static void waitForIndexBuild(String tableName, String indexName) throws
                     continue;
                 }
 
+                TablesView tablesView = getTablesConfiguration(clusterNode).value();
+
                 IndexStorage index = internalTable.storage().getOrCreateIndex(
                         partitionId,
-                        createIndexDescription(getTablesConfiguration(clusterNode).value(), indexName)
+                        createIndexDescriptor(tablesView, tablesView.indexes().get(indexName.toUpperCase()).id())

Review Comment:
   please extract `tablesView.indexes().get(indexName.toUpperCase()).id()` into a variable



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuildTask.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.index;
+
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static java.util.concurrent.CompletableFuture.supplyAsync;
+import static java.util.stream.Collectors.toList;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.RowId;
+import org.apache.ignite.internal.storage.index.IndexStorage;
+import org.apache.ignite.internal.table.distributed.TableMessagesFactory;
+import org.apache.ignite.internal.table.distributed.command.BuildIndexCommand;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteStringFormatter;
+
+/**
+ * Task of building a table index.
+ */
+class IndexBuildTask {
+    private static final IgniteLogger LOG = Loggers.forClass(IndexBuildTask.class);
+
+    private static final TableMessagesFactory TABLE_MESSAGES_FACTORY = new TableMessagesFactory();
+
+    private final IndexBuildTaskId taskId;
+
+    private final IndexStorage indexStorage;
+
+    private final MvPartitionStorage partitionStorage;
+
+    private final RaftGroupService raftClient;
+
+    private final ExecutorService executor;
+
+    private final IgniteSpinBusyLock busyLock;
+
+    private final int batchSize;
+
+    private final IgniteSpinBusyLock taskBusyLock = new IgniteSpinBusyLock();
+
+    private final AtomicBoolean taskStopGuard = new AtomicBoolean();
+
+    private final CompletableFuture<Void> taskFuture = new CompletableFuture<>();
+
+    IndexBuildTask(
+            IndexBuildTaskId taskId,
+            IndexStorage indexStorage,
+            MvPartitionStorage partitionStorage,
+            RaftGroupService raftClient,
+            ExecutorService executor,
+            IgniteSpinBusyLock busyLock,
+            int batchSize
+    ) {
+        this.taskId = taskId;
+        this.indexStorage = indexStorage;
+        this.partitionStorage = partitionStorage;
+        this.raftClient = raftClient;
+        this.executor = executor;
+        this.busyLock = busyLock;
+        this.batchSize = batchSize;
+    }
+
+    /**
+     * Starts building the index.
+     */
+    void start() {
+        if (!enterBusy()) {
+            taskFuture.complete(null);
+
+            return;
+        }
+
+        LOG.info("Start building the index: [{}]", getCommonIndexInfo());
+
+        try {
+            supplyAsync(this::handleNextBatch, executor)
+                    .thenCompose(Function.identity())
+                    .whenComplete((unused, throwable) -> {
+                        if (throwable != null) {
+                            LOG.error("Index build error: [{}]", throwable, getCommonIndexInfo());

Review Comment:
   Same here



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuildTaskId.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.index;
+
+import java.util.UUID;
+import org.apache.ignite.internal.tostring.S;
+
+/**
+ * {@link IndexBuildTask} ID.
+ */
+class IndexBuildTaskId {
+    private final UUID tableId;
+
+    private final int partitionId;
+
+    private final UUID indexId;
+
+    IndexBuildTaskId(UUID tableId, int partitionId, UUID indexId) {
+        this.tableId = tableId;
+        this.partitionId = partitionId;
+        this.indexId = indexId;
+    }
+
+    public UUID getTableId() {

Review Comment:
   we don't use `get` prefixes



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuildTask.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.index;
+
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static java.util.concurrent.CompletableFuture.supplyAsync;
+import static java.util.stream.Collectors.toList;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.RowId;
+import org.apache.ignite.internal.storage.index.IndexStorage;
+import org.apache.ignite.internal.table.distributed.TableMessagesFactory;
+import org.apache.ignite.internal.table.distributed.command.BuildIndexCommand;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteStringFormatter;
+
+/**
+ * Task of building a table index.
+ */
+class IndexBuildTask {
+    private static final IgniteLogger LOG = Loggers.forClass(IndexBuildTask.class);
+
+    private static final TableMessagesFactory TABLE_MESSAGES_FACTORY = new TableMessagesFactory();
+
+    private final IndexBuildTaskId taskId;
+
+    private final IndexStorage indexStorage;
+
+    private final MvPartitionStorage partitionStorage;
+
+    private final RaftGroupService raftClient;
+
+    private final ExecutorService executor;
+
+    private final IgniteSpinBusyLock busyLock;
+
+    private final int batchSize;
+
+    private final IgniteSpinBusyLock taskBusyLock = new IgniteSpinBusyLock();
+
+    private final AtomicBoolean taskStopGuard = new AtomicBoolean();
+
+    private final CompletableFuture<Void> taskFuture = new CompletableFuture<>();
+
+    IndexBuildTask(
+            IndexBuildTaskId taskId,
+            IndexStorage indexStorage,
+            MvPartitionStorage partitionStorage,
+            RaftGroupService raftClient,
+            ExecutorService executor,
+            IgniteSpinBusyLock busyLock,
+            int batchSize
+    ) {
+        this.taskId = taskId;
+        this.indexStorage = indexStorage;
+        this.partitionStorage = partitionStorage;
+        this.raftClient = raftClient;
+        this.executor = executor;
+        this.busyLock = busyLock;
+        this.batchSize = batchSize;
+    }
+
+    /**
+     * Starts building the index.
+     */
+    void start() {
+        if (!enterBusy()) {
+            taskFuture.complete(null);
+
+            return;
+        }
+
+        LOG.info("Start building the index: [{}]", getCommonIndexInfo());
+
+        try {
+            supplyAsync(this::handleNextBatch, executor)
+                    .thenCompose(Function.identity())
+                    .whenComplete((unused, throwable) -> {
+                        if (throwable != null) {
+                            LOG.error("Index build error: [{}]", throwable, getCommonIndexInfo());

Review Comment:
   Also, the format doesn't look correct



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sashapolo commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "sashapolo (via GitHub)" <gi...@apache.org>.
sashapolo commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1197678217


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuildTask.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.index;
+
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static java.util.concurrent.CompletableFuture.supplyAsync;
+import static java.util.stream.Collectors.toList;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.RowId;
+import org.apache.ignite.internal.storage.index.IndexStorage;
+import org.apache.ignite.internal.table.distributed.TableMessagesFactory;
+import org.apache.ignite.internal.table.distributed.command.BuildIndexCommand;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteStringFormatter;
+
+/**
+ * Task of building a table index.
+ */
+class IndexBuildTask {
+    private static final IgniteLogger LOG = Loggers.forClass(IndexBuildTask.class);
+
+    private static final TableMessagesFactory TABLE_MESSAGES_FACTORY = new TableMessagesFactory();
+
+    private final IndexBuildTaskId taskId;
+
+    private final IndexStorage indexStorage;
+
+    private final MvPartitionStorage partitionStorage;
+
+    private final RaftGroupService raftClient;
+
+    private final ExecutorService executor;
+
+    private final IgniteSpinBusyLock busyLock;
+
+    private final int batchSize;
+
+    private final IgniteSpinBusyLock taskBusyLock = new IgniteSpinBusyLock();
+
+    private final AtomicBoolean taskStopGuard = new AtomicBoolean();
+
+    private final CompletableFuture<Void> taskFuture = new CompletableFuture<>();
+
+    IndexBuildTask(
+            IndexBuildTaskId taskId,
+            IndexStorage indexStorage,
+            MvPartitionStorage partitionStorage,
+            RaftGroupService raftClient,
+            ExecutorService executor,
+            IgniteSpinBusyLock busyLock,
+            int batchSize
+    ) {
+        this.taskId = taskId;
+        this.indexStorage = indexStorage;
+        this.partitionStorage = partitionStorage;
+        this.raftClient = raftClient;
+        this.executor = executor;
+        this.busyLock = busyLock;
+        this.batchSize = batchSize;
+    }
+
+    /**
+     * Starts building the index.
+     */
+    void start() {
+        if (!enterBusy()) {
+            taskFuture.complete(null);
+
+            return;
+        }
+
+        LOG.info("Start building the index: [{}]", getCommonIndexInfo());
+
+        try {
+            supplyAsync(this::handleNextBatch, executor)
+                    .thenCompose(Function.identity())
+                    .whenComplete((unused, throwable) -> {
+                        if (throwable != null) {
+                            LOG.error("Index build error: [{}]", throwable, getCommonIndexInfo());

Review Comment:
   > The error level has no such check
   
   indeed, forgot about that
   
   > what's wrong with the format
   
   originally I thought you would need another `{}` for the throwable, but that's not true, so everything is correct 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sashapolo commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "sashapolo (via GitHub)" <gi...@apache.org>.
sashapolo commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1197681502


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -382,8 +382,17 @@ public boolean stopReplica(ReplicationGroupId replicaGrpId) throws NodeStoppingE
      * @param replicaGrpId Replication group id.
      * @return True if the replica is found and closed, false otherwise.
      */
+    // TODO: IGNITE-19494 We need to correctly stop the replica

Review Comment:
   To quote the ticket:
   
   ```
   I think that this is not right, and we should wait for it or something and only after that stop it.
   Maybe I don't understand something and then we'll just close the ticket.
   ```
   
   Please ask @sanpwc or someone from his team what is the correct course of action, I would like to see a more detailed description of the problem. We didn't wait for this future before and everything was ok.



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuilder.java:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.index;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.close.ManuallyCloseable;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.index.IndexStorage;
+import org.apache.ignite.internal.table.distributed.command.BuildIndexCommand;
+import org.apache.ignite.internal.thread.NamedThreadFactory;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.internal.util.IgniteUtils;
+
+/**
+ * Class for managing the building of table indexes.
+ */
+public class IndexBuilder implements ManuallyCloseable {
+    private static final IgniteLogger LOG = Loggers.forClass(IndexBuilder.class);
+
+    private static final int BATCH_SIZE = 100;
+
+    private final ExecutorService executor;
+
+    private final Map<IndexBuildTaskId, IndexBuildTask> indexBuildTaskById = new ConcurrentHashMap<>();
+
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    private final AtomicBoolean closeGuard = new AtomicBoolean();
+
+    /**
+     * Constructor.
+     *
+     * @param nodeName Node name.
+     * @param threadCount Number of threads to build indexes.
+     */
+    public IndexBuilder(String nodeName, int threadCount) {
+        executor = new ThreadPoolExecutor(
+                threadCount,
+                threadCount,
+                30,
+                TimeUnit.SECONDS,
+                new LinkedBlockingQueue<>(),
+                NamedThreadFactory.create(nodeName, "build-index", LOG)
+        );
+    }
+
+    /**
+     * Starts building the index if it is not already built or is not yet in progress.
+     *
+     * <p>Index is built in batches using {@link BuildIndexCommand} (via raft), batches are sent sequentially.
+     *
+     * <p>It is expected that the index building is triggered by the leader of the raft group.
+     *
+     * @param tableId Table ID.
+     * @param partitionId Partition ID.
+     * @param indexId Index ID.
+     * @param indexStorage Index storage to build.
+     * @param partitionStorage Multi-versioned partition storage.
+     * @param raftClient Raft client.
+     */
+    // TODO: IGNITE-19498 Perhaps we need to start building the index only once
+    public void startBuildIndex(
+            UUID tableId,
+            int partitionId,
+            UUID indexId,
+            IndexStorage indexStorage,
+            MvPartitionStorage partitionStorage,
+            RaftGroupService raftClient
+    ) {
+        inBusyLock(() -> {
+            if (indexStorage.getNextRowIdToBuild() == null) {
+                return;
+            }
+
+            IndexBuildTaskId taskId = new IndexBuildTaskId(tableId, partitionId, indexId);
+
+            IndexBuildTask newTask = new IndexBuildTask(taskId, indexStorage, partitionStorage, raftClient, executor, busyLock, BATCH_SIZE);
+
+            IndexBuildTask previousTask = indexBuildTaskById.putIfAbsent(taskId, newTask);
+
+            if (previousTask != null) {
+                // Index building is already in progress.
+                return;
+            }
+
+            newTask.start();
+
+            newTask.getTaskFuture().whenComplete((unused, throwable) -> indexBuildTaskById.remove(taskId));
+        });
+    }
+
+    /**
+     * Stops index building if it is in progress.
+     *
+     * @param tableId Table ID.
+     * @param partitionId Partition ID.
+     * @param indexId Index ID.
+     */
+    public void stopBuildIndex(UUID tableId, int partitionId, UUID indexId) {
+        inBusyLock(() -> {
+            IndexBuildTask removed = indexBuildTaskById.remove(new IndexBuildTaskId(tableId, partitionId, indexId));
+
+            if (removed != null) {
+                removed.stop();
+            }
+        });
+    }
+
+    /**
+     * Stops building all indexes (for a table partition) if they are in progress.
+     *
+     * @param tableId Table ID.
+     * @param partitionId Partition ID.
+     */
+    public void stopBuildIndexes(UUID tableId, int partitionId) {
+        for (Iterator<Entry<IndexBuildTaskId, IndexBuildTask>> it = indexBuildTaskById.entrySet().iterator(); it.hasNext(); ) {
+            if (!busyLock.enterBusy()) {
+                return;
+            }
+
+            try {
+                Entry<IndexBuildTaskId, IndexBuildTask> entry = it.next();
+
+                IndexBuildTaskId taskId = entry.getKey();
+
+                if (tableId.equals(taskId.getTableId()) && partitionId == taskId.getPartitionId()) {
+                    it.remove();
+
+                    entry.getValue().stop();
+                }
+            } finally {
+                busyLock.leaveBusy();
+            }
+        }
+    }
+
+    @Override
+    public void close() {
+        if (!closeGuard.compareAndSet(false, true)) {
+            return;
+        }
+
+        busyLock.block();

Review Comment:
   Do you mean that `stopBuildIndex` will be called externally before stopping the `IndexBuilder` itself? I would suggest to add an assertion then



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1197732152


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuilder.java:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.index;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.close.ManuallyCloseable;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.index.IndexStorage;
+import org.apache.ignite.internal.table.distributed.command.BuildIndexCommand;
+import org.apache.ignite.internal.thread.NamedThreadFactory;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.internal.util.IgniteUtils;
+
+/**
+ * Class for managing the building of table indexes.
+ */
+public class IndexBuilder implements ManuallyCloseable {
+    private static final IgniteLogger LOG = Loggers.forClass(IndexBuilder.class);
+
+    private static final int BATCH_SIZE = 100;
+
+    private final ExecutorService executor;
+
+    private final Map<IndexBuildTaskId, IndexBuildTask> indexBuildTaskById = new ConcurrentHashMap<>();
+
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    private final AtomicBoolean closeGuard = new AtomicBoolean();
+
+    /**
+     * Constructor.
+     *
+     * @param nodeName Node name.
+     * @param threadCount Number of threads to build indexes.
+     */
+    public IndexBuilder(String nodeName, int threadCount) {
+        executor = new ThreadPoolExecutor(
+                threadCount,
+                threadCount,
+                30,
+                TimeUnit.SECONDS,
+                new LinkedBlockingQueue<>(),
+                NamedThreadFactory.create(nodeName, "build-index", LOG)
+        );
+    }
+
+    /**
+     * Starts building the index if it is not already built or is not yet in progress.
+     *
+     * <p>Index is built in batches using {@link BuildIndexCommand} (via raft), batches are sent sequentially.
+     *
+     * <p>It is expected that the index building is triggered by the leader of the raft group.
+     *
+     * @param tableId Table ID.
+     * @param partitionId Partition ID.
+     * @param indexId Index ID.
+     * @param indexStorage Index storage to build.
+     * @param partitionStorage Multi-versioned partition storage.
+     * @param raftClient Raft client.
+     */
+    // TODO: IGNITE-19498 Perhaps we need to start building the index only once
+    public void startBuildIndex(
+            UUID tableId,
+            int partitionId,
+            UUID indexId,
+            IndexStorage indexStorage,
+            MvPartitionStorage partitionStorage,
+            RaftGroupService raftClient
+    ) {
+        inBusyLock(() -> {
+            if (indexStorage.getNextRowIdToBuild() == null) {
+                return;
+            }
+
+            IndexBuildTaskId taskId = new IndexBuildTaskId(tableId, partitionId, indexId);
+
+            IndexBuildTask newTask = new IndexBuildTask(taskId, indexStorage, partitionStorage, raftClient, executor, busyLock, BATCH_SIZE);
+
+            IndexBuildTask previousTask = indexBuildTaskById.putIfAbsent(taskId, newTask);
+
+            if (previousTask != null) {
+                // Index building is already in progress.
+                return;
+            }
+
+            newTask.start();
+
+            newTask.getTaskFuture().whenComplete((unused, throwable) -> indexBuildTaskById.remove(taskId));
+        });
+    }
+
+    /**
+     * Stops index building if it is in progress.
+     *
+     * @param tableId Table ID.
+     * @param partitionId Partition ID.
+     * @param indexId Index ID.
+     */
+    public void stopBuildIndex(UUID tableId, int partitionId, UUID indexId) {
+        inBusyLock(() -> {
+            IndexBuildTask removed = indexBuildTaskById.remove(new IndexBuildTaskId(tableId, partitionId, indexId));
+
+            if (removed != null) {
+                removed.stop();
+            }
+        });
+    }
+
+    /**
+     * Stops building all indexes (for a table partition) if they are in progress.
+     *
+     * @param tableId Table ID.
+     * @param partitionId Partition ID.
+     */
+    public void stopBuildIndexes(UUID tableId, int partitionId) {
+        for (Iterator<Entry<IndexBuildTaskId, IndexBuildTask>> it = indexBuildTaskById.entrySet().iterator(); it.hasNext(); ) {
+            if (!busyLock.enterBusy()) {
+                return;
+            }
+
+            try {
+                Entry<IndexBuildTaskId, IndexBuildTask> entry = it.next();
+
+                IndexBuildTaskId taskId = entry.getKey();
+
+                if (tableId.equals(taskId.getTableId()) && partitionId == taskId.getPartitionId()) {
+                    it.remove();
+
+                    entry.getValue().stop();
+                }
+            } finally {
+                busyLock.leaveBusy();
+            }
+        }
+    }
+
+    @Override
+    public void close() {
+        if (!closeGuard.compareAndSet(false, true)) {
+            return;
+        }
+
+        busyLock.block();

Review Comment:
   No, I meant that index building tasks will eventually be stopped because this busyLock is passed to all tasks.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1197590370


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ClusterPerClassIntegrationTest.java:
##########
@@ -515,14 +513,20 @@ public static void waitForIndexBuild(String tableName, String indexName) throws
                     continue;
                 }
 
+                TablesView tablesView = getTablesConfiguration(clusterNode).value();
+
                 IndexStorage index = internalTable.storage().getOrCreateIndex(
                         partitionId,
-                        createIndexDescription(getTablesConfiguration(clusterNode).value(), indexName)
+                        createIndexDescriptor(tablesView, tablesView.indexes().get(indexName.toUpperCase()).id())

Review Comment:
   fix it



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1197663991


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/StorageUpdateHandler.java:
##########
@@ -417,4 +417,12 @@ public void buildIndex(UUID indexId, List<UUID> rowUuids, boolean finish) {
     public PendingComparableValuesTracker<HybridTimestamp, Void> getSafeTimeTracker() {
         return safeTimeTracker;
     }
+
+    /**
+     * Waits for indexes to be created.
+     */
+    // TODO: IGNITE-19513 Fix it, we should have already waited for the indexes to be created

Review Comment:
   Try to fix it



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1197605338


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuildTask.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.index;
+
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static java.util.concurrent.CompletableFuture.supplyAsync;
+import static java.util.stream.Collectors.toList;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.RowId;
+import org.apache.ignite.internal.storage.index.IndexStorage;
+import org.apache.ignite.internal.table.distributed.TableMessagesFactory;
+import org.apache.ignite.internal.table.distributed.command.BuildIndexCommand;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteStringFormatter;
+
+/**
+ * Task of building a table index.
+ */
+class IndexBuildTask {
+    private static final IgniteLogger LOG = Loggers.forClass(IndexBuildTask.class);
+
+    private static final TableMessagesFactory TABLE_MESSAGES_FACTORY = new TableMessagesFactory();
+
+    private final IndexBuildTaskId taskId;
+
+    private final IndexStorage indexStorage;
+
+    private final MvPartitionStorage partitionStorage;
+
+    private final RaftGroupService raftClient;
+
+    private final ExecutorService executor;
+
+    private final IgniteSpinBusyLock busyLock;
+
+    private final int batchSize;
+
+    private final IgniteSpinBusyLock taskBusyLock = new IgniteSpinBusyLock();
+
+    private final AtomicBoolean taskStopGuard = new AtomicBoolean();
+
+    private final CompletableFuture<Void> taskFuture = new CompletableFuture<>();
+
+    IndexBuildTask(
+            IndexBuildTaskId taskId,
+            IndexStorage indexStorage,
+            MvPartitionStorage partitionStorage,
+            RaftGroupService raftClient,
+            ExecutorService executor,
+            IgniteSpinBusyLock busyLock,
+            int batchSize
+    ) {
+        this.taskId = taskId;
+        this.indexStorage = indexStorage;
+        this.partitionStorage = partitionStorage;
+        this.raftClient = raftClient;
+        this.executor = executor;
+        this.busyLock = busyLock;
+        this.batchSize = batchSize;
+    }
+
+    /**
+     * Starts building the index.
+     */
+    void start() {
+        if (!enterBusy()) {
+            taskFuture.complete(null);

Review Comment:
   I'm not sure what is appropriate, since we can stop building indexes both at node shutdown and, for example, after a change assignments.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1187279363


##########
modules/index/src/main/java/org/apache/ignite/internal/index/IndexBuilder.java:
##########
@@ -100,10 +111,14 @@ void stop() {
      */
     void startIndexBuild(TableIndexView tableIndexView, TableImpl table) {
         for (int partitionId = 0; partitionId < table.internalTable().partitions(); partitionId++) {
+            // Let's check if there is a node in the assignments for the partition.
+            if (!replicaManager.startedGroups().contains(new TablePartitionId(table.tableId(), partitionId))) {

Review Comment:
   That's definitely not what I've meant. What about having build index logic inside replica itself? The basic question here is whether we have any index related logic if there's no replica locally? 
   If not, we should neither check assignments nor replicas but rather perform index related logic inside replica. Of course it's still possible to used IndexBuilder as an abstraction that will cover corresponding index related logic.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1197608051


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuildTask.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.index;
+
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static java.util.concurrent.CompletableFuture.supplyAsync;
+import static java.util.stream.Collectors.toList;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.RowId;
+import org.apache.ignite.internal.storage.index.IndexStorage;
+import org.apache.ignite.internal.table.distributed.TableMessagesFactory;
+import org.apache.ignite.internal.table.distributed.command.BuildIndexCommand;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteStringFormatter;
+
+/**
+ * Task of building a table index.
+ */
+class IndexBuildTask {
+    private static final IgniteLogger LOG = Loggers.forClass(IndexBuildTask.class);
+
+    private static final TableMessagesFactory TABLE_MESSAGES_FACTORY = new TableMessagesFactory();
+
+    private final IndexBuildTaskId taskId;
+
+    private final IndexStorage indexStorage;
+
+    private final MvPartitionStorage partitionStorage;
+
+    private final RaftGroupService raftClient;
+
+    private final ExecutorService executor;
+
+    private final IgniteSpinBusyLock busyLock;
+
+    private final int batchSize;
+
+    private final IgniteSpinBusyLock taskBusyLock = new IgniteSpinBusyLock();
+
+    private final AtomicBoolean taskStopGuard = new AtomicBoolean();
+
+    private final CompletableFuture<Void> taskFuture = new CompletableFuture<>();
+
+    IndexBuildTask(
+            IndexBuildTaskId taskId,
+            IndexStorage indexStorage,
+            MvPartitionStorage partitionStorage,
+            RaftGroupService raftClient,
+            ExecutorService executor,
+            IgniteSpinBusyLock busyLock,
+            int batchSize
+    ) {
+        this.taskId = taskId;
+        this.indexStorage = indexStorage;
+        this.partitionStorage = partitionStorage;
+        this.raftClient = raftClient;
+        this.executor = executor;
+        this.busyLock = busyLock;
+        this.batchSize = batchSize;
+    }
+
+    /**
+     * Starts building the index.
+     */
+    void start() {
+        if (!enterBusy()) {
+            taskFuture.complete(null);
+
+            return;
+        }
+
+        LOG.info("Start building the index: [{}]", getCommonIndexInfo());
+
+        try {
+            supplyAsync(this::handleNextBatch, executor)
+                    .thenCompose(Function.identity())
+                    .whenComplete((unused, throwable) -> {
+                        if (throwable != null) {
+                            LOG.error("Index build error: [{}]", throwable, getCommonIndexInfo());

Review Comment:
   The error level has no such check, what's wrong with the format?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sashapolo commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "sashapolo (via GitHub)" <gi...@apache.org>.
sashapolo commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1197676831


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuildTask.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.index;
+
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static java.util.concurrent.CompletableFuture.supplyAsync;
+import static java.util.stream.Collectors.toList;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.RowId;
+import org.apache.ignite.internal.storage.index.IndexStorage;
+import org.apache.ignite.internal.table.distributed.TableMessagesFactory;
+import org.apache.ignite.internal.table.distributed.command.BuildIndexCommand;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteStringFormatter;
+
+/**
+ * Task of building a table index.
+ */
+class IndexBuildTask {
+    private static final IgniteLogger LOG = Loggers.forClass(IndexBuildTask.class);
+
+    private static final TableMessagesFactory TABLE_MESSAGES_FACTORY = new TableMessagesFactory();
+
+    private final IndexBuildTaskId taskId;
+
+    private final IndexStorage indexStorage;
+
+    private final MvPartitionStorage partitionStorage;
+
+    private final RaftGroupService raftClient;
+
+    private final ExecutorService executor;
+
+    private final IgniteSpinBusyLock busyLock;
+
+    private final int batchSize;
+
+    private final IgniteSpinBusyLock taskBusyLock = new IgniteSpinBusyLock();
+
+    private final AtomicBoolean taskStopGuard = new AtomicBoolean();
+
+    private final CompletableFuture<Void> taskFuture = new CompletableFuture<>();
+
+    IndexBuildTask(
+            IndexBuildTaskId taskId,
+            IndexStorage indexStorage,
+            MvPartitionStorage partitionStorage,
+            RaftGroupService raftClient,
+            ExecutorService executor,
+            IgniteSpinBusyLock busyLock,
+            int batchSize
+    ) {
+        this.taskId = taskId;
+        this.indexStorage = indexStorage;
+        this.partitionStorage = partitionStorage;
+        this.raftClient = raftClient;
+        this.executor = executor;
+        this.busyLock = busyLock;
+        this.batchSize = batchSize;
+    }
+
+    /**
+     * Starts building the index.
+     */
+    void start() {
+        if (!enterBusy()) {
+            taskFuture.complete(null);

Review Comment:
   your version is correct then



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1197660758


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -382,8 +382,17 @@ public boolean stopReplica(ReplicationGroupId replicaGrpId) throws NodeStoppingE
      * @param replicaGrpId Replication group id.
      * @return True if the replica is found and closed, false otherwise.
      */
+    // TODO: IGNITE-19494 We need to correctly stop the replica

Review Comment:
   I tried to paraphrase, but I don’t know what else to add, do you advise something?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sashapolo commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "sashapolo (via GitHub)" <gi...@apache.org>.
sashapolo commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1197679037


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuildTask.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.index;
+
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static java.util.concurrent.CompletableFuture.supplyAsync;
+import static java.util.stream.Collectors.toList;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.RowId;
+import org.apache.ignite.internal.storage.index.IndexStorage;
+import org.apache.ignite.internal.table.distributed.TableMessagesFactory;
+import org.apache.ignite.internal.table.distributed.command.BuildIndexCommand;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteStringFormatter;
+
+/**
+ * Task of building a table index.
+ */
+class IndexBuildTask {
+    private static final IgniteLogger LOG = Loggers.forClass(IndexBuildTask.class);
+
+    private static final TableMessagesFactory TABLE_MESSAGES_FACTORY = new TableMessagesFactory();
+
+    private final IndexBuildTaskId taskId;
+
+    private final IndexStorage indexStorage;
+
+    private final MvPartitionStorage partitionStorage;
+
+    private final RaftGroupService raftClient;
+
+    private final ExecutorService executor;
+
+    private final IgniteSpinBusyLock busyLock;
+
+    private final int batchSize;
+
+    private final IgniteSpinBusyLock taskBusyLock = new IgniteSpinBusyLock();
+
+    private final AtomicBoolean taskStopGuard = new AtomicBoolean();
+
+    private final CompletableFuture<Void> taskFuture = new CompletableFuture<>();
+
+    IndexBuildTask(
+            IndexBuildTaskId taskId,
+            IndexStorage indexStorage,
+            MvPartitionStorage partitionStorage,
+            RaftGroupService raftClient,
+            ExecutorService executor,
+            IgniteSpinBusyLock busyLock,
+            int batchSize
+    ) {
+        this.taskId = taskId;
+        this.indexStorage = indexStorage;
+        this.partitionStorage = partitionStorage;
+        this.raftClient = raftClient;
+        this.executor = executor;
+        this.busyLock = busyLock;
+        this.batchSize = batchSize;
+    }
+
+    /**
+     * Starts building the index.
+     */
+    void start() {
+        if (!enterBusy()) {
+            taskFuture.complete(null);
+
+            return;
+        }
+
+        LOG.info("Start building the index: [{}]", getCommonIndexInfo());
+
+        try {
+            supplyAsync(this::handleNextBatch, executor)

Review Comment:
   I agree that it is right, I'm proposing to do `runAsync(createBatchRowIds)` inside the `handleNextBatch` method, not here. This will also help to get rid of `Funciton.identity` below



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sashapolo commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "sashapolo (via GitHub)" <gi...@apache.org>.
sashapolo commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1197692958


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/StorageUpdateHandler.java:
##########
@@ -417,4 +417,12 @@ public void buildIndex(UUID indexId, List<UUID> rowUuids, boolean finish) {
     public PendingComparableValuesTracker<HybridTimestamp, Void> getSafeTimeTracker() {
         return safeTimeTracker;
     }
+
+    /**
+     * Waits for indexes to be created.
+     */
+    // TODO: IGNITE-19513 Fix it, we should have already waited for the indexes to be created

Review Comment:
   it's better now



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1197745697


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuildTask.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.index;
+
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static java.util.concurrent.CompletableFuture.supplyAsync;
+import static java.util.stream.Collectors.toList;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.raft.service.RaftGroupService;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.RowId;
+import org.apache.ignite.internal.storage.index.IndexStorage;
+import org.apache.ignite.internal.table.distributed.TableMessagesFactory;
+import org.apache.ignite.internal.table.distributed.command.BuildIndexCommand;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteStringFormatter;
+
+/**
+ * Task of building a table index.
+ */
+class IndexBuildTask {
+    private static final IgniteLogger LOG = Loggers.forClass(IndexBuildTask.class);
+
+    private static final TableMessagesFactory TABLE_MESSAGES_FACTORY = new TableMessagesFactory();
+
+    private final IndexBuildTaskId taskId;
+
+    private final IndexStorage indexStorage;
+
+    private final MvPartitionStorage partitionStorage;
+
+    private final RaftGroupService raftClient;
+
+    private final ExecutorService executor;
+
+    private final IgniteSpinBusyLock busyLock;
+
+    private final int batchSize;
+
+    private final IgniteSpinBusyLock taskBusyLock = new IgniteSpinBusyLock();
+
+    private final AtomicBoolean taskStopGuard = new AtomicBoolean();
+
+    private final CompletableFuture<Void> taskFuture = new CompletableFuture<>();
+
+    IndexBuildTask(
+            IndexBuildTaskId taskId,
+            IndexStorage indexStorage,
+            MvPartitionStorage partitionStorage,
+            RaftGroupService raftClient,
+            ExecutorService executor,
+            IgniteSpinBusyLock busyLock,
+            int batchSize
+    ) {
+        this.taskId = taskId;
+        this.indexStorage = indexStorage;
+        this.partitionStorage = partitionStorage;
+        this.raftClient = raftClient;
+        this.executor = executor;
+        this.busyLock = busyLock;
+        this.batchSize = batchSize;
+    }
+
+    /**
+     * Starts building the index.
+     */
+    void start() {
+        if (!enterBusy()) {
+            taskFuture.complete(null);
+
+            return;
+        }
+
+        LOG.info("Start building the index: [{}]", getCommonIndexInfo());
+
+        try {
+            supplyAsync(this::handleNextBatch, executor)

Review Comment:
   In personal correspondence, we decided to leave it as it is.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #2030: IGNITE-19177 Add assignments checks when building indexes

Posted by "sanpwc (via GitHub)" <gi...@apache.org>.
sanpwc commented on code in PR #2030:
URL: https://github.com/apache/ignite-3/pull/2030#discussion_r1199578952


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/listener/ReplicaListener.java:
##########
@@ -31,4 +33,20 @@ public interface ReplicaListener {
      * @return Listener response.
      */
     CompletableFuture<?> invoke(ReplicaRequest request);
+
+    /**
+     * Callback on leader election.
+     *
+     * @param clusterNode Leader node.
+     */
+    default void onLeaderElected(ClusterNode clusterNode) {

Review Comment:
   We may have leaderless replication protocols. I'd say that we should use onBecomePrimary instead.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org