rpuch commented on code in PR #3424:
URL: https://github.com/apache/ignite-3/pull/3424#discussion_r1530540385


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ddl/DdlCommandHandler.java:
##########
@@ -169,12 +200,86 @@ private static BiFunction<Object, Throwable, Boolean> 
handleModificationResult(b
     /** Handles create index command. */
     private CompletableFuture<Boolean> handleCreateIndex(CreateIndexCommand 
cmd) {
         return 
catalogManager.execute(DdlToCatalogCommandConverter.convert(cmd))
+                .thenCompose(catalogVersion -> inBusyLock(busyLock, () -> 
waitTillIndexBecomesAvailableOrRemoved(cmd, catalogVersion)))
                 .handle(handleModificationResult(cmd.ifNotExists(), 
IndexExistsValidationException.class));
     }
 
+    private CompletionStage<Void> 
waitTillIndexBecomesAvailableOrRemoved(CreateIndexCommand cmd, Integer 
creationCatalogVersion) {
+        CompletableFuture<Void> future = inFlightFutures.registerFuture(new 
CompletableFuture<>());
+
+        Catalog catalog = catalogManager.catalog(creationCatalogVersion);
+        assert catalog != null : creationCatalogVersion;
+
+        CatalogSchemaDescriptor schema = catalog.schema(cmd.schemaName());
+        assert schema != null : "Did not find schema " + cmd.schemaName() + " 
in version " + creationCatalogVersion;
+
+        CatalogIndexDescriptor index = schema.aliveIndex(cmd.indexName());
+        assert index != null
+                : "Did not find index " + cmd.indexName() + " in schema " + 
cmd.schemaName() + " in version " + creationCatalogVersion;
+
+        EventListener<CatalogEventParameters> availabilityListener = 
EventListener.fromConsumer(event -> {
+            if (((MakeIndexAvailableEventParameters) event).indexId() == 
index.id()) {
+                completeFutureWhenEventVersionActivates(future, event);
+            }
+        });
+        catalogManager.listen(CatalogEvent.INDEX_AVAILABLE, 
availabilityListener);
+
+        EventListener<CatalogEventParameters> removalListener = 
EventListener.fromConsumer(event -> {
+            if (((RemoveIndexEventParameters) event).indexId() == index.id()) {
+                future.complete(null);
+            }
+        });
+        catalogManager.listen(CatalogEvent.INDEX_REMOVED, removalListener);
+
+        // We added listeners, but the index could switch to a state of 
interest before we added them, so check
+        // explicitly.
+        int latestVersion = catalogManager.latestCatalogVersion();
+        for (int version = creationCatalogVersion + 1; version <= 
latestVersion; version++) {

Review Comment:
   If we missed a few catalog updates, this allows us to find the earliest of 
them in which the index has become AVAILABLE, so the user will need to wait a 
little less (because we don't just complete the future as soon as we see an 
AVAILABLE version in the Catalog, we take its activation ts and wait till it 
comes). Also, it's unlikely that a thousand catalog versions will appear for 
scanning while we are adding the listeners :)
   
   To sum up: the scanning might yield an earlier completion moment, but if we 
take the latest version, it won't be too different, so I'll change this if you 
insist.



-- 
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