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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to