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/06/26 05:38:07 UTC

[GitHub] [ignite-3] tkalkirill opened a new pull request, #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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

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


-- 
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 #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/index/src/main/java/org/apache/ignite/internal/index/IndexManager.java:
##########
@@ -96,414 +86,180 @@ public class IndexManager extends Producer<IndexEvent, IndexEventParameters> imp
     /** Table manager. */
     private final TableManager tableManager;
 
+    /** Catalog manager. */
+    private final CatalogManager catalogManager;
+
+    /** Meta storage manager. */
+    private final MetaStorageManager metaStorageManager;
+
     /** Busy lock to stop synchronously. */
     private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
 
     /** Prevents double stopping of the component. */
     private final AtomicBoolean stopGuard = new AtomicBoolean();
 
+    /** Versioned value used only at the start of the manager. */
+    private final IncrementalVersionedValue<Void> startVv;
+
     /**
      * Constructor.
      *
      * @param tablesCfg Tables and indexes configuration.
      * @param schemaManager Schema manager.
      * @param tableManager Table manager.
+     * @param catalogManager Catalog manager.
      */
     public IndexManager(
             TablesConfiguration tablesCfg,
             SchemaManager schemaManager,
-            TableManager tableManager
+            TableManager tableManager,
+            CatalogManager catalogManager,
+            MetaStorageManager metaStorageManager,
+            Consumer<LongFunction<CompletableFuture<?>>> registry
     ) {
         this.tablesCfg = Objects.requireNonNull(tablesCfg, "tablesCfg");
         this.schemaManager = Objects.requireNonNull(schemaManager, "schemaManager");
         this.tableManager = tableManager;
+        this.catalogManager = catalogManager;
+        this.metaStorageManager = metaStorageManager;
+
+        startVv = new IncrementalVersionedValue<>(registry);
     }
 
-    /** {@inheritDoc} */
     @Override
     public void start() {
-        LOG.debug("Index manager is about to start");
-
-        tablesCfg.indexes().listenElements(new ConfigurationListener());
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Index manager is about to start");
+        }
 
-        tableManager.listen(TableEvent.CREATE, (param, ex) -> {
-            if (ex != null) {
-                return completedFuture(false);
-            }
+        startIndexes();
 
-            // We can't return this future as the listener's result, because a deadlock can happen in the configuration component:
-            // this listener is called inside a configuration notification thread and all notifications are required to finish before
-            // new configuration modifications can occur (i.e. we are creating an index below). Therefore we create the index fully
-            // asynchronously and rely on the underlying components to handle PK index synchronisation.
-            tableManager.tableAsync(param.causalityToken(), param.tableId())
-                    .thenCompose(table -> {
-                        String[] pkColumns = Arrays.stream(table.schemaView().schema().keyColumns().columns())
-                                .map(Column::name)
-                                .toArray(String[]::new);
+        catalogManager.listen(INDEX_CREATE, (parameters, exception) -> {
+            assert exception == null : parameters;
 
-                        String pkName = table.name() + "_PK";
+            return onIndexCreate((CreateIndexEventParameters) parameters);
+        });
 
-                        return createIndexAsync("PUBLIC", pkName, table.name(), false,
-                                change -> change.changeUniq(true).convert(HashIndexChange.class)
-                                        .changeColumnNames(pkColumns)
-                        );
-                    })
-                    .whenComplete((v, e) -> {
-                        if (e != null) {
-                            LOG.error("Error when creating index: " + e);
-                        }
-                    });
+        catalogManager.listen(INDEX_DROP, (parameters, exception) -> {
+            assert exception == null;
 
-            return completedFuture(false);
+            return onIndexDrop((DropIndexEventParameters) parameters);
         });
 
-        LOG.info("Index manager started");
+        if (LOG.isInfoEnabled()) {
+            LOG.info("Index manager started");
+        }
     }
 
-    /** {@inheritDoc} */
     @Override
     public void stop() throws Exception {
-        LOG.debug("Index manager is about to stop");
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Index manager is about to stop");
+        }

Review Comment:
   Yes, you are right, but there is one caveat, `String` will be created in the heap that we will not write to the log.
   
   We discussed this issue within the team for a long time, and decided that it is better to use such a rule to check through a predicate.
   
   Maybe this should be moved to code style.



-- 
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 #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItCommonApiTest.java:
##########
@@ -123,37 +115,25 @@ public void testSessionExpiration() throws Exception {
     @Test
     public void checkTimestampOperations() {
         String kvTblName = "tbl_all_columns_sql";
-        String schemaName = "PUBLIC";
-        String keyCol = "key";
-        int maxTimePrecision = TemporalColumnType.MAX_TIME_PRECISION;
+        String keyCol = "KEY";
 
         Ignite node = CLUSTER_NODES.get(0);
 
         // TODO: https://issues.apache.org/jira/browse/IGNITE-19162 Trim all less than millisecond information from timestamp
         //String tsStr = "2023-03-29T08:22:33.005007Z";
-        String tsStr = "2023-03-29T08:22:33.005Z";
+        String tsStr = "2023-03-29T08:22:33Z";
 
         Instant ins = Instant.parse(tsStr);
 
         sql("CREATE TABLE timestamps(id INTEGER PRIMARY KEY, i TIMESTAMP(9))");
 
-        TableDefinition schTblAllSql = SchemaBuilders.tableBuilder(schemaName, kvTblName).columns(
-                SchemaBuilders.column(keyCol, ColumnType.INT64).build(),
-                SchemaBuilders.column("time", ColumnType.time(maxTimePrecision)).asNullable(true).build(),
-                SchemaBuilders.column("timestamp", ColumnType.timestamp(maxTimePrecision)).asNullable(true).build(),
-                SchemaBuilders.column("datetime", ColumnType.datetime(maxTimePrecision)).asNullable(true).build()
-        ).withPrimaryKey(keyCol).build();
-
-        await(((TableManager) node.tables()).createTableAsync(schTblAllSql.name(), DEFAULT_ZONE_NAME, tblCh ->
-                SchemaConfigurationConverter.convert(schTblAllSql, tblCh)
-        ));
+        sql(String.format("CREATE TABLE %s(\"%s\" INTEGER PRIMARY KEY, \"TIMESTAMP\" TIMESTAMP(9))", kvTblName, keyCol));

Review Comment:
   There is a nuance here, here I left `org.apache.ignite.sql.ColumnType#DATETIME` which in SQL has the `org.apache.calcite.sql.type.SqlTypeName#TIMESTAMP`, but I can’t use `org.apache.ignite.sql.ColumnType#TIMESTAMP` because in SQL it is type `org.apache.calcite.sql.type.SqlTypeName#TIMESTAMP_WITH_LOCAL_TIME_ZONE` which does not currently work here is the IGNITE-17376 and IGNITE-19274.
   
   I looked at the mapping in `org.apache.ignite.internal.sql.engine.util.TypeUtils#columnType`.



##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItCommonApiTest.java:
##########
@@ -123,37 +115,25 @@ public void testSessionExpiration() throws Exception {
     @Test
     public void checkTimestampOperations() {
         String kvTblName = "tbl_all_columns_sql";
-        String schemaName = "PUBLIC";
-        String keyCol = "key";
-        int maxTimePrecision = TemporalColumnType.MAX_TIME_PRECISION;
+        String keyCol = "KEY";
 
         Ignite node = CLUSTER_NODES.get(0);
 
         // TODO: https://issues.apache.org/jira/browse/IGNITE-19162 Trim all less than millisecond information from timestamp
         //String tsStr = "2023-03-29T08:22:33.005007Z";
-        String tsStr = "2023-03-29T08:22:33.005Z";
+        String tsStr = "2023-03-29T08:22:33Z";
 
         Instant ins = Instant.parse(tsStr);
 
         sql("CREATE TABLE timestamps(id INTEGER PRIMARY KEY, i TIMESTAMP(9))");
 
-        TableDefinition schTblAllSql = SchemaBuilders.tableBuilder(schemaName, kvTblName).columns(
-                SchemaBuilders.column(keyCol, ColumnType.INT64).build(),
-                SchemaBuilders.column("time", ColumnType.time(maxTimePrecision)).asNullable(true).build(),
-                SchemaBuilders.column("timestamp", ColumnType.timestamp(maxTimePrecision)).asNullable(true).build(),
-                SchemaBuilders.column("datetime", ColumnType.datetime(maxTimePrecision)).asNullable(true).build()
-        ).withPrimaryKey(keyCol).build();
-
-        await(((TableManager) node.tables()).createTableAsync(schTblAllSql.name(), DEFAULT_ZONE_NAME, tblCh ->
-                SchemaConfigurationConverter.convert(schTblAllSql, tblCh)
-        ));
+        sql(String.format("CREATE TABLE %s(\"%s\" INTEGER PRIMARY KEY, \"TIMESTAMP\" TIMESTAMP(9))", kvTblName, keyCol));

Review Comment:
   There is a nuance here, here I left `org.apache.ignite.sql.ColumnType#DATETIME` which in SQL has the `org.apache.calcite.sql.type.SqlTypeName#TIMESTAMP`, but I can’t use `org.apache.ignite.sql.ColumnType#TIMESTAMP` because in SQL it is type `org.apache.calcite.sql.type.SqlTypeName#TIMESTAMP_WITH_LOCAL_TIME_ZONE` which does not currently work here is the IGNITE-17376 and IGNITE-19274.
   
   I looked at the mapping in `org.apache.ignite.internal.sql.engine.util.TypeUtils#columnType`.



-- 
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] AMashenkov commented on a diff in pull request #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/StopCalciteModuleTest.java:
##########


Review Comment:
   I'd drop this test class, as we don't check real behaviour of SqlQueryProcessor.
   There are bunch of mocks, without making clear how they affect the SqlQueryProcessor.
   
   If we want to check all nodes stop their threads in various scenarious, then it would be more affective making this check at tearDown stage for every integration test.



-- 
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 #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItTablesApiTest.java:
##########
@@ -255,21 +256,44 @@ public void testGetTableFromLaggedNode() {
      */
     @Test
     public void testAddIndex() {
-        clusterNodes.forEach(ign -> assertNull(ign.tables().table(TABLE_NAME)));
+        Ignite ignite = clusterNodes.get(0);
 
-        Ignite ignite0 = clusterNodes.get(0);
+        createTable(ignite, TABLE_NAME);
 
-        createTable(ignite0, TABLE_NAME);
+        String indexName = "TEST_INDEX";

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] AMashenkov commented on a diff in pull request #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/index/src/main/java/org/apache/ignite/internal/index/IndexManager.java:
##########
@@ -108,226 +111,164 @@ public class IndexManager extends Producer<IndexEvent, IndexEventParameters> imp
      * @param tablesCfg Tables and indexes configuration.
      * @param schemaManager Schema manager.
      * @param tableManager Table manager.
+     * @param catalogManager Catalog manager.
      */
     public IndexManager(
             TablesConfiguration tablesCfg,
             SchemaManager schemaManager,
-            TableManager tableManager
+            TableManager tableManager,
+            CatalogManager catalogManager
     ) {
         this.tablesCfg = Objects.requireNonNull(tablesCfg, "tablesCfg");
         this.schemaManager = Objects.requireNonNull(schemaManager, "schemaManager");
         this.tableManager = tableManager;
+        this.catalogManager = catalogManager;
     }
 
-    /** {@inheritDoc} */
     @Override
     public void start() {
-        LOG.debug("Index manager is about to start");
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Index manager is about to start");
+        }
 
-        tablesCfg.indexes().listenElements(new ConfigurationListener());
+        catalogManager.listen(INDEX_CREATE, (parameters, exception) -> {
+            assert exception == null;
 
-        tableManager.listen(TableEvent.CREATE, (param, ex) -> {
-            if (ex != null) {
-                return completedFuture(false);
-            }
+            return onIndexCreate((CreateIndexEventParameters) parameters);
+        });
 
-            // We can't return this future as the listener's result, because a deadlock can happen in the configuration component:
-            // this listener is called inside a configuration notification thread and all notifications are required to finish before
-            // new configuration modifications can occur (i.e. we are creating an index below). Therefore we create the index fully
-            // asynchronously and rely on the underlying components to handle PK index synchronisation.
-            tableManager.tableAsync(param.causalityToken(), param.tableId())
-                    .thenCompose(table -> {
-                        String[] pkColumns = Arrays.stream(table.schemaView().schema().keyColumns().columns())
-                                .map(Column::name)
-                                .toArray(String[]::new);
-
-                        String pkName = table.name() + "_PK";
-
-                        return createIndexAsync("PUBLIC", pkName, table.name(), false,
-                                change -> change.changeUniq(true).convert(HashIndexChange.class)
-                                        .changeColumnNames(pkColumns)
-                        );
-                    })
-                    .whenComplete((v, e) -> {
-                        if (e != null) {
-                            LOG.error("Error when creating index: " + e);
-                        }
-                    });
+        catalogManager.listen(INDEX_DROP, (parameters, exception) -> {
+            assert exception == null;
 
-            return completedFuture(false);
+            return onIndexDrop((DropIndexEventParameters) parameters);
         });
 
-        LOG.info("Index manager started");
+        if (LOG.isInfoEnabled()) {
+            LOG.info("Index manager started");
+        }

Review Comment:
   Log level is checked in the call in depth.



-- 
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 #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItCommonApiTest.java:
##########
@@ -123,37 +115,25 @@ public void testSessionExpiration() throws Exception {
     @Test
     public void checkTimestampOperations() {
         String kvTblName = "tbl_all_columns_sql";
-        String schemaName = "PUBLIC";
-        String keyCol = "key";
-        int maxTimePrecision = TemporalColumnType.MAX_TIME_PRECISION;
+        String keyCol = "KEY";
 
         Ignite node = CLUSTER_NODES.get(0);
 
         // TODO: https://issues.apache.org/jira/browse/IGNITE-19162 Trim all less than millisecond information from timestamp
         //String tsStr = "2023-03-29T08:22:33.005007Z";
-        String tsStr = "2023-03-29T08:22:33.005Z";
+        String tsStr = "2023-03-29T08:22:33Z";

Review Comment:
   created a https://issues.apache.org/jira/browse/IGNITE-20105



##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItCommonApiTest.java:
##########
@@ -123,37 +115,25 @@ public void testSessionExpiration() throws Exception {
     @Test
     public void checkTimestampOperations() {
         String kvTblName = "tbl_all_columns_sql";
-        String schemaName = "PUBLIC";
-        String keyCol = "key";
-        int maxTimePrecision = TemporalColumnType.MAX_TIME_PRECISION;
+        String keyCol = "KEY";
 
         Ignite node = CLUSTER_NODES.get(0);
 
         // TODO: https://issues.apache.org/jira/browse/IGNITE-19162 Trim all less than millisecond information from timestamp
         //String tsStr = "2023-03-29T08:22:33.005007Z";
-        String tsStr = "2023-03-29T08:22:33.005Z";
+        String tsStr = "2023-03-29T08:22:33Z";

Review Comment:
   created https://issues.apache.org/jira/browse/IGNITE-20105



-- 
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] AMashenkov commented on a diff in pull request #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ddl/DdlCommandHandler.java:
##########
@@ -352,59 +343,42 @@ private CompletableFuture<Boolean> handleCreateIndex(CreateIndexCommand cmd) {
                             "Can't create index on duplicate columns: " + String.join(", ", cmd.columns()));
                 });
 
-        Consumer<TableIndexChange> indexChanger = tableIndexChange -> {
-            switch (cmd.type()) {
-                case SORTED:
-                    createSortedIndexInternal(cmd, tableIndexChange.convert(SortedIndexChange.class));
+        boolean failIfExists = !cmd.ifNotExists();
 
-                    break;
-                case HASH:
-                    createHashIndexInternal(cmd, tableIndexChange.convert(HashIndexChange.class));
+        return catalogCreateIndexAsync(cmd)
+                .handle((unused, throwable) -> {
+                    if (throwable != null) {
+                        throwable = ExceptionUtils.unwrapCause(throwable);

Review Comment:
   We already have postprocessing method `handleModificationResult`.
   Please, reuse 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] AMashenkov commented on a diff in pull request #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ddl/DdlCommandHandler.java:
##########
@@ -352,59 +343,42 @@ private CompletableFuture<Boolean> handleCreateIndex(CreateIndexCommand cmd) {
                             "Can't create index on duplicate columns: " + String.join(", ", cmd.columns()));
                 });
 
-        Consumer<TableIndexChange> indexChanger = tableIndexChange -> {
-            switch (cmd.type()) {
-                case SORTED:
-                    createSortedIndexInternal(cmd, tableIndexChange.convert(SortedIndexChange.class));
+        boolean failIfExists = !cmd.ifNotExists();
 
-                    break;
-                case HASH:
-                    createHashIndexInternal(cmd, tableIndexChange.convert(HashIndexChange.class));
+        return catalogCreateIndexAsync(cmd)
+                .handle((unused, throwable) -> {
+                    if (throwable != null) {
+                        throwable = ExceptionUtils.unwrapCause(throwable);

Review Comment:
   ```suggestion
                   .handle(handleModificationResult(!failIfExists, IndexAlreadyExistsException.class));
   ```



-- 
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] AMashenkov commented on a diff in pull request #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItTablesApiTest.java:
##########
@@ -582,29 +596,36 @@ protected void addColumnIfNotExists(Ignite node, String tableName) {
     }
 
     /**
-     * Adds a column.
+     * Tries to create the index.
      *
      * @param node Cluster node.
      * @param tableName Table name.
+     * @param indexName Index name.
+     * @param failIfNotExist Throw an exception if the index does not exist.
      */
-    protected void tryToCreateIndex(Ignite node, String tableName, boolean failIfNotExist) {
-        var tmpl  = "CREATE INDEX %s testHI ON %s (valInt, valStr)";
-        var sql = String.format(tmpl, failIfNotExist ? "" : "IF NOT EXISTS", tableName);
+    protected static void tryToCreateIndex(Ignite node, String tableName, String indexName, boolean failIfNotExist) {

Review Comment:
   We have no public methods on Table interface for creating tables\indices anymore.
   Test for creating a table make sense, to check if Table is (non-)available from any node right after creation (removal).
   But tests for creating/dropping indices looks uselsess here. They should be moved to another integration test and shoudl looks like `ItCreateTableDdlTest`



-- 
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 #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ddl/DdlCommandHandler.java:
##########
@@ -352,59 +343,42 @@ private CompletableFuture<Boolean> handleCreateIndex(CreateIndexCommand cmd) {
                             "Can't create index on duplicate columns: " + String.join(", ", cmd.columns()));
                 });
 
-        Consumer<TableIndexChange> indexChanger = tableIndexChange -> {
-            switch (cmd.type()) {
-                case SORTED:
-                    createSortedIndexInternal(cmd, tableIndexChange.convert(SortedIndexChange.class));
+        boolean failIfExists = !cmd.ifNotExists();
 
-                    break;
-                case HASH:
-                    createHashIndexInternal(cmd, tableIndexChange.convert(HashIndexChange.class));
+        return catalogCreateIndexAsync(cmd)
+                .handle((unused, throwable) -> {
+                    if (throwable != null) {
+                        throwable = ExceptionUtils.unwrapCause(throwable);

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 #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/index/src/main/java/org/apache/ignite/internal/index/IndexManager.java:
##########
@@ -96,414 +86,180 @@ public class IndexManager extends Producer<IndexEvent, IndexEventParameters> imp
     /** Table manager. */
     private final TableManager tableManager;
 
+    /** Catalog manager. */
+    private final CatalogManager catalogManager;
+
+    /** Meta storage manager. */
+    private final MetaStorageManager metaStorageManager;
+
     /** Busy lock to stop synchronously. */
     private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
 
     /** Prevents double stopping of the component. */
     private final AtomicBoolean stopGuard = new AtomicBoolean();
 
+    /** Versioned value used only at the start of the manager. */
+    private final IncrementalVersionedValue<Void> startVv;
+
     /**
      * Constructor.
      *
      * @param tablesCfg Tables and indexes configuration.
      * @param schemaManager Schema manager.
      * @param tableManager Table manager.
+     * @param catalogManager Catalog manager.
      */
     public IndexManager(
             TablesConfiguration tablesCfg,
             SchemaManager schemaManager,
-            TableManager tableManager
+            TableManager tableManager,
+            CatalogManager catalogManager,
+            MetaStorageManager metaStorageManager,
+            Consumer<LongFunction<CompletableFuture<?>>> registry
     ) {
         this.tablesCfg = Objects.requireNonNull(tablesCfg, "tablesCfg");
         this.schemaManager = Objects.requireNonNull(schemaManager, "schemaManager");
         this.tableManager = tableManager;
+        this.catalogManager = catalogManager;
+        this.metaStorageManager = metaStorageManager;
+
+        startVv = new IncrementalVersionedValue<>(registry);
     }
 
-    /** {@inheritDoc} */
     @Override
     public void start() {
-        LOG.debug("Index manager is about to start");
-
-        tablesCfg.indexes().listenElements(new ConfigurationListener());
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Index manager is about to start");
+        }
 
-        tableManager.listen(TableEvent.CREATE, (param, ex) -> {
-            if (ex != null) {
-                return completedFuture(false);
-            }
+        startIndexes();
 
-            // We can't return this future as the listener's result, because a deadlock can happen in the configuration component:
-            // this listener is called inside a configuration notification thread and all notifications are required to finish before
-            // new configuration modifications can occur (i.e. we are creating an index below). Therefore we create the index fully
-            // asynchronously and rely on the underlying components to handle PK index synchronisation.
-            tableManager.tableAsync(param.causalityToken(), param.tableId())
-                    .thenCompose(table -> {
-                        String[] pkColumns = Arrays.stream(table.schemaView().schema().keyColumns().columns())
-                                .map(Column::name)
-                                .toArray(String[]::new);
+        catalogManager.listen(INDEX_CREATE, (parameters, exception) -> {
+            assert exception == null : parameters;
 
-                        String pkName = table.name() + "_PK";
+            return onIndexCreate((CreateIndexEventParameters) parameters);
+        });
 
-                        return createIndexAsync("PUBLIC", pkName, table.name(), false,
-                                change -> change.changeUniq(true).convert(HashIndexChange.class)
-                                        .changeColumnNames(pkColumns)
-                        );
-                    })
-                    .whenComplete((v, e) -> {
-                        if (e != null) {
-                            LOG.error("Error when creating index: " + e);
-                        }
-                    });
+        catalogManager.listen(INDEX_DROP, (parameters, exception) -> {
+            assert exception == null;
 
-            return completedFuture(false);
+            return onIndexDrop((DropIndexEventParameters) parameters);
         });
 
-        LOG.info("Index manager started");
+        if (LOG.isInfoEnabled()) {
+            LOG.info("Index manager started");
+        }
     }
 
-    /** {@inheritDoc} */
     @Override
     public void stop() throws Exception {
-        LOG.debug("Index manager is about to stop");
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Index manager is about to stop");
+        }

Review Comment:
   Yes, you are right, but there is one caveat, a line will be created in the heap that we will not write to the log.
   
   We discussed this issue within the team for a long time, and decided that it is better to use such a rule to check through a predicate.
   
   Maybe this should be moved to code style.



-- 
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] AMashenkov commented on a diff in pull request #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/index/src/test/java/org/apache/ignite/internal/index/IndexManagerTest.java:
##########
@@ -104,74 +128,59 @@ public void setUp() {
 
         when(schManager.schemaRegistry(anyLong(), anyInt())).thenReturn(completedFuture(null));
 
-        indexManager = new IndexManager(tablesConfig, schManager, tableManagerMock);
-        indexManager.start();
+        String nodeName = "test";
 
-        assertThat(
-                tablesConfig.tables().get("tName")
-                        .change(tableChange -> ((ExtendedTableChange) tableChange).changeSchemaId(1)),
-                willCompleteSuccessfully()
-        );
-    }
+        vaultManager = new VaultManager(new InMemoryVaultService());
 
-    @AfterEach
-    void tearDown() throws Exception {
-        if (indexManager != null) {
-            indexManager.stop();
-        }
-    }
+        metaStorageManager = StandaloneMetaStorageManager.create(vaultManager, new SimpleInMemoryKeyValueStorage(nodeName));
 
-    @Test
-    void configurationChangedWhenCreateIsInvoked() {
-        String indexName = "idx";
+        clockWaiter = new ClockWaiter(nodeName, clock);
+
+        catalogManager = new CatalogManagerImpl(new UpdateLogImpl(metaStorageManager), clockWaiter);
 
-        assertThat(indexManager.createIndexAsync("sName", indexName, "tName", true, indexChange -> {
-            SortedIndexChange sortedIndexChange = indexChange.convert(SortedIndexChange.class);
-
-            sortedIndexChange.changeColumns(columns -> {
-                columns.create("c1", columnChange -> columnChange.changeAsc(true));
-                columns.create("c2", columnChange -> columnChange.changeAsc(false));
-            });
-
-            sortedIndexChange.changeTableId(tableId());
-        }), willCompleteSuccessfully());
-
-        var expected = List.of(
-                Map.of(
-                        "columns", List.of(
-                                Map.of(
-                                        "asc", true,
-                                        "name", "c1"
-                                ),
-                                Map.of(
-                                        "asc", false,
-                                        "name", "c2"
-                                )
-                        ),
-                        "name", indexName,
-                        "type", "SORTED",
-                        "uniq", false,
-                        "tableId", tableId(),
-                        "id", 1
-                )
+        indexManager = new IndexManager(
+                tablesConfig,
+                schManager,
+                tableManagerMock,
+                catalogManager,
+                metaStorageManager,
+                mock(Consumer.class)
         );
 
-        assertSameObjects(expected, toMap(tablesConfig.indexes().value()));
-    }
+        vaultManager.start();
+        metaStorageManager.start();
+        clockWaiter.start();
+        catalogManager.start();
+        indexManager.start();
+
+        assertThat(metaStorageManager.deployWatches(), willCompleteSuccessfully());
 
-    @Test
-    public void createIndexWithEmptyName() {
         assertThat(
-                indexManager.createIndexAsync("sName", "", "tName", true, indexChange -> {/* doesn't matter */}),
-                willThrowFast(IgniteInternalException.class, "Index name should be at least 1 character long")
+                catalogManager.createTable(
+                        CreateTableParams.builder()
+                                .schemaName(DEFAULT_SCHEMA_NAME)
+                                .zone(DEFAULT_ZONE_NAME)
+                                .tableName(TABLE_NAME)
+                                .columns(List.of(
+                                        ColumnParams.builder().name("c1").type(STRING).build(),
+                                        ColumnParams.builder().name("c2").type(STRING).build()
+                                ))
+                                .colocationColumns(List.of("c1"))
+                                .primaryKeyColumns(List.of("c1"))
+                                .build()
+                ),
+                willBe(nullValue())
         );
     }
 
-    @Test
-    public void dropNonExistingIndex() {
-        assertThat(
-                indexManager.dropIndexAsync("sName", "nonExisting", true),
-                willThrowFast(IndexNotFoundException.class, "Index does not exist [name=\"sName\".\"nonExisting\"]")
+    @AfterEach
+    void tearDown() throws Exception {
+        IgniteUtils.closeAll(
+                vaultManager == null ? null : vaultManager::stop,
+                metaStorageManager == null ? null : metaStorageManager::stop,
+                clockWaiter == null ? null : clockWaiter::stop,
+                catalogManager == null ? null : catalogManager::stop,
+                indexManager == null ? null : indexManager::stop

Review Comment:
   Let's add a Utility method to stop components, 
   ```
   IgniteUtils.stopComponents(IgniteComponent... comps) {
       for (IgniteComponent c : comps) {
           if (c != null) {
               try {
                   c.stop();
               } catch(Throwable th) {
                   th.printStacktrace(); // log properly
               }
           }
       }
   }
   ```
   to make code looks like
   ```suggestion
           IgniteUtils.stopComponents(
                   vaultManager,
                   metaStorageManager,
                   clockWaiter,
                   catalogManager,
                   indexManager,
   ```



-- 
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 #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/StopCalciteModuleTest.java:
##########


Review Comment:
   Removed this class.



-- 
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 #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItCommonApiTest.java:
##########
@@ -123,37 +115,25 @@ public void testSessionExpiration() throws Exception {
     @Test
     public void checkTimestampOperations() {
         String kvTblName = "tbl_all_columns_sql";
-        String schemaName = "PUBLIC";
-        String keyCol = "key";
-        int maxTimePrecision = TemporalColumnType.MAX_TIME_PRECISION;
+        String keyCol = "KEY";
 
         Ignite node = CLUSTER_NODES.get(0);
 
         // TODO: https://issues.apache.org/jira/browse/IGNITE-19162 Trim all less than millisecond information from timestamp
         //String tsStr = "2023-03-29T08:22:33.005007Z";
-        String tsStr = "2023-03-29T08:22:33.005Z";
+        String tsStr = "2023-03-29T08:22:33Z";
 
         Instant ins = Instant.parse(tsStr);
 
         sql("CREATE TABLE timestamps(id INTEGER PRIMARY KEY, i TIMESTAMP(9))");
 
-        TableDefinition schTblAllSql = SchemaBuilders.tableBuilder(schemaName, kvTblName).columns(
-                SchemaBuilders.column(keyCol, ColumnType.INT64).build(),
-                SchemaBuilders.column("time", ColumnType.time(maxTimePrecision)).asNullable(true).build(),
-                SchemaBuilders.column("timestamp", ColumnType.timestamp(maxTimePrecision)).asNullable(true).build(),
-                SchemaBuilders.column("datetime", ColumnType.datetime(maxTimePrecision)).asNullable(true).build()
-        ).withPrimaryKey(keyCol).build();
-
-        await(((TableManager) node.tables()).createTableAsync(schTblAllSql.name(), DEFAULT_ZONE_NAME, tblCh ->
-                SchemaConfigurationConverter.convert(schTblAllSql, tblCh)
-        ));
+        sql(String.format("CREATE TABLE %s(\"%s\" INTEGER PRIMARY KEY, \"TIMESTAMP\" TIMESTAMP(9))", kvTblName, keyCol));

Review Comment:
   Did 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] AMashenkov commented on a diff in pull request #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItTablesApiTest.java:
##########
@@ -582,29 +596,36 @@ protected void addColumnIfNotExists(Ignite node, String tableName) {
     }
 
     /**
-     * Adds a column.
+     * Tries to create the index.
      *
      * @param node Cluster node.
      * @param tableName Table name.
+     * @param indexName Index name.
+     * @param failIfNotExist Throw an exception if the index does not exist.
      */
-    protected void tryToCreateIndex(Ignite node, String tableName, boolean failIfNotExist) {
-        var tmpl  = "CREATE INDEX %s testHI ON %s (valInt, valStr)";
-        var sql = String.format(tmpl, failIfNotExist ? "" : "IF NOT EXISTS", tableName);
+    protected static void tryToCreateIndex(Ignite node, String tableName, String indexName, boolean failIfNotExist) {

Review Comment:
   We have no public methods on Table interface for creating tables\indices anymore.
   Test for creating a table make sense, just to check if Table is (non-)available from any node right after creation (removal).
   
   But tests for creating/dropping indices looks uselsess here, because index can't be reached via Tables API. The test should be moved to another integration test and shoudl looks like `ItCreateTableDdlTest`



-- 
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] AMashenkov commented on a diff in pull request #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItTablesApiTest.java:
##########
@@ -255,21 +256,44 @@ public void testGetTableFromLaggedNode() {
      */
     @Test
     public void testAddIndex() {
-        clusterNodes.forEach(ign -> assertNull(ign.tables().table(TABLE_NAME)));
+        Ignite ignite = clusterNodes.get(0);
 
-        Ignite ignite0 = clusterNodes.get(0);
+        createTable(ignite, TABLE_NAME);
 
-        createTable(ignite0, TABLE_NAME);
+        String indexName = "TEST_INDEX";

Review Comment:
   Let's make it constant as it done for "TABLE_NAME".



-- 
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] AMashenkov commented on a diff in pull request #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItTablesApiTest.java:
##########
@@ -582,29 +596,36 @@ protected void addColumnIfNotExists(Ignite node, String tableName) {
     }
 
     /**
-     * Adds a column.
+     * Tries to create the index.
      *
      * @param node Cluster node.
      * @param tableName Table name.
+     * @param indexName Index name.
+     * @param failIfNotExist Throw an exception if the index does not exist.
      */
-    protected void tryToCreateIndex(Ignite node, String tableName, boolean failIfNotExist) {
-        var tmpl  = "CREATE INDEX %s testHI ON %s (valInt, valStr)";
-        var sql = String.format(tmpl, failIfNotExist ? "" : "IF NOT EXISTS", tableName);
+    protected static void tryToCreateIndex(Ignite node, String tableName, String indexName, boolean failIfNotExist) {

Review Comment:
   We have no public methods on Table interface for creating tables\indices anymore.
   Test for creating a table make sense, just to check if Table is (non-)available from any node right after creation (removal).
   
   But tests for creating/dropping indices looks uselsess here, because index can't be reached via Tables API. The test should be moved to another integration test and shoudl looks like `ItCreateTableDdlTest`
   
   Let's just drop irrelevant tests.



-- 
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] AMashenkov commented on a diff in pull request #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItCommonApiTest.java:
##########
@@ -123,37 +115,25 @@ public void testSessionExpiration() throws Exception {
     @Test
     public void checkTimestampOperations() {
         String kvTblName = "tbl_all_columns_sql";
-        String schemaName = "PUBLIC";
-        String keyCol = "key";
-        int maxTimePrecision = TemporalColumnType.MAX_TIME_PRECISION;
+        String keyCol = "KEY";
 
         Ignite node = CLUSTER_NODES.get(0);
 
         // TODO: https://issues.apache.org/jira/browse/IGNITE-19162 Trim all less than millisecond information from timestamp
         //String tsStr = "2023-03-29T08:22:33.005007Z";
-        String tsStr = "2023-03-29T08:22:33.005Z";
+        String tsStr = "2023-03-29T08:22:33Z";
 
         Instant ins = Instant.parse(tsStr);
 
         sql("CREATE TABLE timestamps(id INTEGER PRIMARY KEY, i TIMESTAMP(9))");
 
-        TableDefinition schTblAllSql = SchemaBuilders.tableBuilder(schemaName, kvTblName).columns(
-                SchemaBuilders.column(keyCol, ColumnType.INT64).build(),
-                SchemaBuilders.column("time", ColumnType.time(maxTimePrecision)).asNullable(true).build(),
-                SchemaBuilders.column("timestamp", ColumnType.timestamp(maxTimePrecision)).asNullable(true).build(),
-                SchemaBuilders.column("datetime", ColumnType.datetime(maxTimePrecision)).asNullable(true).build()
-        ).withPrimaryKey(keyCol).build();
-
-        await(((TableManager) node.tables()).createTableAsync(schTblAllSql.name(), DEFAULT_ZONE_NAME, tblCh ->
-                SchemaConfigurationConverter.convert(schTblAllSql, tblCh)
-        ));
+        sql(String.format("CREATE TABLE %s(\"%s\" INTEGER PRIMARY KEY, \"TIMESTAMP\" TIMESTAMP(9))", kvTblName, keyCol));

Review Comment:
   Why do you drop DATETIME column?



-- 
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] AMashenkov commented on a diff in pull request #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItCommonApiTest.java:
##########
@@ -123,37 +115,25 @@ public void testSessionExpiration() throws Exception {
     @Test
     public void checkTimestampOperations() {
         String kvTblName = "tbl_all_columns_sql";
-        String schemaName = "PUBLIC";
-        String keyCol = "key";
-        int maxTimePrecision = TemporalColumnType.MAX_TIME_PRECISION;
+        String keyCol = "KEY";
 
         Ignite node = CLUSTER_NODES.get(0);
 
         // TODO: https://issues.apache.org/jira/browse/IGNITE-19162 Trim all less than millisecond information from timestamp
         //String tsStr = "2023-03-29T08:22:33.005007Z";
-        String tsStr = "2023-03-29T08:22:33.005Z";
+        String tsStr = "2023-03-29T08:22:33Z";

Review Comment:
   There is a bug in` CatalogUtils.fromParams(ColumnParams)` method.
   Precision, scale and length are set to `0` by default, but we expected using default values described in SQL standard.
   So now, table descriptor, which was built from Catalog, differs from descriptor, which was built from Configuration.
   
   To fix the issue, next default should be used.
   ```
   /**
        * Default TIMESTAMP type precision: microseconds.
        *
        * <p>SQL`16 part 2 section 6.1 syntax rule 36
        */
       public static final int DEFAULT_TIMESTAMP_PRECISION = 6;
       /**
        * Default TIME type precision: seconds.
        *
        * <p>SQL`16 part 2 section 6.1 syntax rule 36
        */
       public static final int DEFAULT_TIME_PRECISION = 0;
       /**
        * Default DECIMAL precision is implementation-defined.
        *
        * <p>SQL`16 part 2 section 6.1 syntax rule 25
        */
       public static final int DEFAULT_DECIMAL_PRECISION = 19;
       /**
        * Default DECIMAL scale is implementation-defined.
        *
        * <p>SQL`16 part 2 section 6.1 syntax rule 25
        */
       public static final int DEFAULT_DECIMAL_SCALE = 3;
   ```



-- 
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 #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


-- 
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] rpuch commented on a diff in pull request #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItIndexDdlTest.java:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.sql.engine;
+
+import static org.apache.ignite.internal.catalog.CatalogService.DEFAULT_SCHEMA_NAME;
+
+import org.apache.ignite.internal.testframework.IgniteTestUtils;
+import org.apache.ignite.lang.IndexAlreadyExistsException;
+import org.apache.ignite.lang.IndexNotFoundException;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Integration tests for DDL statements that affect indexes.
+ */
+public class ItIndexDdlTest extends ClusterPerClassIntegrationTest {
+    private static final String TABLE_NAME = "TEST_TABLE";
+
+    private static final String INDEX_NAME = "TEST_INDEX";
+
+    @BeforeEach
+    void setUp() {
+        sql(String.format("CREATE TABLE IF NOT EXISTS %s (key BIGINT PRIMARY KEY, valInt INT, valStr VARCHAR)", TABLE_NAME));
+    }
+
+    @AfterEach
+    void tearDown() {
+        sql(String.format("DROP TABLE IF EXISTS %S", TABLE_NAME));
+    }
+
+    @Test
+    public void testAddIndex() {
+        tryToCreateIndex(TABLE_NAME, INDEX_NAME, true);
+
+        IgniteTestUtils.assertThrows(
+                IndexAlreadyExistsException.class,
+                () -> tryToCreateIndex(TABLE_NAME, INDEX_NAME, true),
+                String.format("Index already exists [name=\"%s\".\"%s\"]", DEFAULT_SCHEMA_NAME, INDEX_NAME)
+        );
+
+        tryToCreateIndex(TABLE_NAME, INDEX_NAME, false);
+    }
+
+    @Test
+    void testDropIndex() {
+        tryToCreateIndex(TABLE_NAME, INDEX_NAME, true);
+
+        // Let's check the drop on an existing index.
+        tryToDropIndex(INDEX_NAME, true);
+
+        // Let's check the drop on a non-existent index.
+        IgniteTestUtils.assertThrows(
+                IndexNotFoundException.class,
+                () -> tryToDropIndex(INDEX_NAME, true),
+                String.format("Index does not exist [name=\"%s\".\"%s\"]", DEFAULT_SCHEMA_NAME, INDEX_NAME)
+        );
+
+        tryToCreateIndex(TABLE_NAME, INDEX_NAME, false);
+    }
+
+    /**
+     * Tries to create the index.
+     *
+     * @param tableName Table name.
+     * @param indexName Index name.
+     * @param failIfNotExist Throw an exception if the index does not exist.
+     */
+    private static void tryToCreateIndex(String tableName, String indexName, boolean failIfNotExist) {

Review Comment:
   `failIfNotExist` seems to have inverted meaning. If the parameter is true, we'll fail if the index EXISTS, not if it DOES NOT



-- 
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] rpuch commented on a diff in pull request #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -2718,4 +2752,39 @@ private PartitionUpdateHandlers createPartitionUpdateHandlers(
     private CatalogZoneDescriptor getZoneDescriptor(int id) {
         return toZoneDescriptor(getZoneById(zonesConfig, id).value());
     }
+
+    private static @Nullable TableImpl findTableImplByName(Collection<TableImpl> tables, String name) {
+        return tables.stream().filter(table -> table.name().equals(name)).findAny().orElse(null);
+    }
+
+    /**
+     * Fires table creation events so that indexes can be correctly created at IndexManager startup.
+     *
+     * <p>NOTE: This is a temporary solution that must be get rid/remake/change.
+     */
+    // TODO: IGNITE-19499 Need to get rid/remake/change
+    private void fireCreateTablesOnStartManager() {

Review Comment:
   ```suggestion
       private void fireCreateTablesOnManagerStart() {
   ```



##########
modules/index/src/main/java/org/apache/ignite/internal/index/IndexManager.java:
##########
@@ -96,414 +86,180 @@ public class IndexManager extends Producer<IndexEvent, IndexEventParameters> imp
     /** Table manager. */
     private final TableManager tableManager;
 
+    /** Catalog manager. */
+    private final CatalogManager catalogManager;
+
+    /** Meta storage manager. */
+    private final MetaStorageManager metaStorageManager;
+
     /** Busy lock to stop synchronously. */
     private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
 
     /** Prevents double stopping of the component. */
     private final AtomicBoolean stopGuard = new AtomicBoolean();
 
+    /** Versioned value used only at the start of the manager. */
+    private final IncrementalVersionedValue<Void> startVv;
+
     /**
      * Constructor.
      *
      * @param tablesCfg Tables and indexes configuration.
      * @param schemaManager Schema manager.
      * @param tableManager Table manager.
+     * @param catalogManager Catalog manager.
      */
     public IndexManager(
             TablesConfiguration tablesCfg,
             SchemaManager schemaManager,
-            TableManager tableManager
+            TableManager tableManager,
+            CatalogManager catalogManager,
+            MetaStorageManager metaStorageManager,
+            Consumer<LongFunction<CompletableFuture<?>>> registry
     ) {
         this.tablesCfg = Objects.requireNonNull(tablesCfg, "tablesCfg");
         this.schemaManager = Objects.requireNonNull(schemaManager, "schemaManager");
         this.tableManager = tableManager;
+        this.catalogManager = catalogManager;
+        this.metaStorageManager = metaStorageManager;
+
+        startVv = new IncrementalVersionedValue<>(registry);
     }
 
-    /** {@inheritDoc} */
     @Override
     public void start() {
-        LOG.debug("Index manager is about to start");
-
-        tablesCfg.indexes().listenElements(new ConfigurationListener());
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Index manager is about to start");
+        }
 
-        tableManager.listen(TableEvent.CREATE, (param, ex) -> {
-            if (ex != null) {
-                return completedFuture(false);
-            }
+        startIndexes();
 
-            // We can't return this future as the listener's result, because a deadlock can happen in the configuration component:
-            // this listener is called inside a configuration notification thread and all notifications are required to finish before
-            // new configuration modifications can occur (i.e. we are creating an index below). Therefore we create the index fully
-            // asynchronously and rely on the underlying components to handle PK index synchronisation.
-            tableManager.tableAsync(param.causalityToken(), param.tableId())
-                    .thenCompose(table -> {
-                        String[] pkColumns = Arrays.stream(table.schemaView().schema().keyColumns().columns())
-                                .map(Column::name)
-                                .toArray(String[]::new);
+        catalogManager.listen(INDEX_CREATE, (parameters, exception) -> {
+            assert exception == null : parameters;
 
-                        String pkName = table.name() + "_PK";
+            return onIndexCreate((CreateIndexEventParameters) parameters);
+        });
 
-                        return createIndexAsync("PUBLIC", pkName, table.name(), false,
-                                change -> change.changeUniq(true).convert(HashIndexChange.class)
-                                        .changeColumnNames(pkColumns)
-                        );
-                    })
-                    .whenComplete((v, e) -> {
-                        if (e != null) {
-                            LOG.error("Error when creating index: " + e);
-                        }
-                    });
+        catalogManager.listen(INDEX_DROP, (parameters, exception) -> {
+            assert exception == null;
 
-            return completedFuture(false);
+            return onIndexDrop((DropIndexEventParameters) parameters);
         });
 
-        LOG.info("Index manager started");
+        if (LOG.isInfoEnabled()) {
+            LOG.info("Index manager started");
+        }
     }
 
-    /** {@inheritDoc} */
     @Override
     public void stop() throws Exception {
-        LOG.debug("Index manager is about to stop");
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Index manager is about to stop");
+        }

Review Comment:
   This is a `stop()` method that is called once per JVM. It makes no sense to save one object here, but a line of code might make a difference. Same thing for `start()`: it's not on a hot path.



-- 
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 #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/index/src/test/java/org/apache/ignite/internal/index/IndexManagerTest.java:
##########
@@ -104,74 +128,59 @@ public void setUp() {
 
         when(schManager.schemaRegistry(anyLong(), anyInt())).thenReturn(completedFuture(null));
 
-        indexManager = new IndexManager(tablesConfig, schManager, tableManagerMock);
-        indexManager.start();
+        String nodeName = "test";
 
-        assertThat(
-                tablesConfig.tables().get("tName")
-                        .change(tableChange -> ((ExtendedTableChange) tableChange).changeSchemaId(1)),
-                willCompleteSuccessfully()
-        );
-    }
+        vaultManager = new VaultManager(new InMemoryVaultService());
 
-    @AfterEach
-    void tearDown() throws Exception {
-        if (indexManager != null) {
-            indexManager.stop();
-        }
-    }
+        metaStorageManager = StandaloneMetaStorageManager.create(vaultManager, new SimpleInMemoryKeyValueStorage(nodeName));
 
-    @Test
-    void configurationChangedWhenCreateIsInvoked() {
-        String indexName = "idx";
+        clockWaiter = new ClockWaiter(nodeName, clock);
+
+        catalogManager = new CatalogManagerImpl(new UpdateLogImpl(metaStorageManager), clockWaiter);
 
-        assertThat(indexManager.createIndexAsync("sName", indexName, "tName", true, indexChange -> {
-            SortedIndexChange sortedIndexChange = indexChange.convert(SortedIndexChange.class);
-
-            sortedIndexChange.changeColumns(columns -> {
-                columns.create("c1", columnChange -> columnChange.changeAsc(true));
-                columns.create("c2", columnChange -> columnChange.changeAsc(false));
-            });
-
-            sortedIndexChange.changeTableId(tableId());
-        }), willCompleteSuccessfully());
-
-        var expected = List.of(
-                Map.of(
-                        "columns", List.of(
-                                Map.of(
-                                        "asc", true,
-                                        "name", "c1"
-                                ),
-                                Map.of(
-                                        "asc", false,
-                                        "name", "c2"
-                                )
-                        ),
-                        "name", indexName,
-                        "type", "SORTED",
-                        "uniq", false,
-                        "tableId", tableId(),
-                        "id", 1
-                )
+        indexManager = new IndexManager(
+                tablesConfig,
+                schManager,
+                tableManagerMock,
+                catalogManager,
+                metaStorageManager,
+                mock(Consumer.class)
         );
 
-        assertSameObjects(expected, toMap(tablesConfig.indexes().value()));
-    }
+        vaultManager.start();
+        metaStorageManager.start();
+        clockWaiter.start();
+        catalogManager.start();
+        indexManager.start();
+
+        assertThat(metaStorageManager.deployWatches(), willCompleteSuccessfully());
 
-    @Test
-    public void createIndexWithEmptyName() {
         assertThat(
-                indexManager.createIndexAsync("sName", "", "tName", true, indexChange -> {/* doesn't matter */}),
-                willThrowFast(IgniteInternalException.class, "Index name should be at least 1 character long")
+                catalogManager.createTable(
+                        CreateTableParams.builder()
+                                .schemaName(DEFAULT_SCHEMA_NAME)
+                                .zone(DEFAULT_ZONE_NAME)
+                                .tableName(TABLE_NAME)
+                                .columns(List.of(
+                                        ColumnParams.builder().name("c1").type(STRING).build(),
+                                        ColumnParams.builder().name("c2").type(STRING).build()
+                                ))
+                                .colocationColumns(List.of("c1"))
+                                .primaryKeyColumns(List.of("c1"))
+                                .build()
+                ),
+                willBe(nullValue())
         );
     }
 
-    @Test
-    public void dropNonExistingIndex() {
-        assertThat(
-                indexManager.dropIndexAsync("sName", "nonExisting", true),
-                willThrowFast(IndexNotFoundException.class, "Index does not exist [name=\"sName\".\"nonExisting\"]")
+    @AfterEach
+    void tearDown() throws Exception {
+        IgniteUtils.closeAll(
+                vaultManager == null ? null : vaultManager::stop,
+                metaStorageManager == null ? null : metaStorageManager::stop,
+                clockWaiter == null ? null : clockWaiter::stop,
+                catalogManager == null ? null : catalogManager::stop,
+                indexManager == null ? null : indexManager::stop

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 #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -2718,4 +2752,39 @@ private PartitionUpdateHandlers createPartitionUpdateHandlers(
     private CatalogZoneDescriptor getZoneDescriptor(int id) {
         return toZoneDescriptor(getZoneById(zonesConfig, id).value());
     }
+
+    private static @Nullable TableImpl findTableImplByName(Collection<TableImpl> tables, String name) {
+        return tables.stream().filter(table -> table.name().equals(name)).findAny().orElse(null);
+    }
+
+    /**
+     * Fires table creation events so that indexes can be correctly created at IndexManager startup.
+     *
+     * <p>NOTE: This is a temporary solution that must be get rid/remake/change.
+     */
+    // TODO: IGNITE-19499 Need to get rid/remake/change
+    private void fireCreateTablesOnStartManager() {

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] AMashenkov commented on a diff in pull request #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/index/src/main/java/org/apache/ignite/internal/index/IndexManager.java:
##########
@@ -96,414 +86,180 @@ public class IndexManager extends Producer<IndexEvent, IndexEventParameters> imp
     /** Table manager. */
     private final TableManager tableManager;
 
+    /** Catalog manager. */
+    private final CatalogManager catalogManager;
+
+    /** Meta storage manager. */
+    private final MetaStorageManager metaStorageManager;
+
     /** Busy lock to stop synchronously. */
     private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
 
     /** Prevents double stopping of the component. */
     private final AtomicBoolean stopGuard = new AtomicBoolean();
 
+    /** Versioned value used only at the start of the manager. */
+    private final IncrementalVersionedValue<Void> startVv;
+
     /**
      * Constructor.
      *
      * @param tablesCfg Tables and indexes configuration.
      * @param schemaManager Schema manager.
      * @param tableManager Table manager.
+     * @param catalogManager Catalog manager.
      */
     public IndexManager(
             TablesConfiguration tablesCfg,
             SchemaManager schemaManager,
-            TableManager tableManager
+            TableManager tableManager,
+            CatalogManager catalogManager,
+            MetaStorageManager metaStorageManager,
+            Consumer<LongFunction<CompletableFuture<?>>> registry
     ) {
         this.tablesCfg = Objects.requireNonNull(tablesCfg, "tablesCfg");
         this.schemaManager = Objects.requireNonNull(schemaManager, "schemaManager");
         this.tableManager = tableManager;
+        this.catalogManager = catalogManager;
+        this.metaStorageManager = metaStorageManager;
+
+        startVv = new IncrementalVersionedValue<>(registry);
     }
 
-    /** {@inheritDoc} */
     @Override
     public void start() {
-        LOG.debug("Index manager is about to start");
-
-        tablesCfg.indexes().listenElements(new ConfigurationListener());
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Index manager is about to start");
+        }
 
-        tableManager.listen(TableEvent.CREATE, (param, ex) -> {
-            if (ex != null) {
-                return completedFuture(false);
-            }
+        startIndexes();
 
-            // We can't return this future as the listener's result, because a deadlock can happen in the configuration component:
-            // this listener is called inside a configuration notification thread and all notifications are required to finish before
-            // new configuration modifications can occur (i.e. we are creating an index below). Therefore we create the index fully
-            // asynchronously and rely on the underlying components to handle PK index synchronisation.
-            tableManager.tableAsync(param.causalityToken(), param.tableId())
-                    .thenCompose(table -> {
-                        String[] pkColumns = Arrays.stream(table.schemaView().schema().keyColumns().columns())
-                                .map(Column::name)
-                                .toArray(String[]::new);
+        catalogManager.listen(INDEX_CREATE, (parameters, exception) -> {
+            assert exception == null : parameters;
 
-                        String pkName = table.name() + "_PK";
+            return onIndexCreate((CreateIndexEventParameters) parameters);
+        });
 
-                        return createIndexAsync("PUBLIC", pkName, table.name(), false,
-                                change -> change.changeUniq(true).convert(HashIndexChange.class)
-                                        .changeColumnNames(pkColumns)
-                        );
-                    })
-                    .whenComplete((v, e) -> {
-                        if (e != null) {
-                            LOG.error("Error when creating index: " + e);
-                        }
-                    });
+        catalogManager.listen(INDEX_DROP, (parameters, exception) -> {
+            assert exception == null;
 
-            return completedFuture(false);
+            return onIndexDrop((DropIndexEventParameters) parameters);
         });
 
-        LOG.info("Index manager started");
+        if (LOG.isInfoEnabled()) {
+            LOG.info("Index manager started");
+        }
     }
 
-    /** {@inheritDoc} */
     @Override
     public void stop() throws Exception {
-        LOG.debug("Index manager is about to stop");
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Index manager is about to stop");
+        }

Review Comment:
   Why do we need an `if` statement here?  
   isDebugEnabled is checked by IgniteLogger.
   ```
    private void logInternal(Level level, String msg, Throwable th, Object... params) {
           Objects.requireNonNull(level);
   
           if (!delegate.isLoggable(level)) {
               return;
           }
   ```



-- 
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 #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/index/src/main/java/org/apache/ignite/internal/index/IndexManager.java:
##########
@@ -96,414 +86,180 @@ public class IndexManager extends Producer<IndexEvent, IndexEventParameters> imp
     /** Table manager. */
     private final TableManager tableManager;
 
+    /** Catalog manager. */
+    private final CatalogManager catalogManager;
+
+    /** Meta storage manager. */
+    private final MetaStorageManager metaStorageManager;
+
     /** Busy lock to stop synchronously. */
     private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
 
     /** Prevents double stopping of the component. */
     private final AtomicBoolean stopGuard = new AtomicBoolean();
 
+    /** Versioned value used only at the start of the manager. */
+    private final IncrementalVersionedValue<Void> startVv;
+
     /**
      * Constructor.
      *
      * @param tablesCfg Tables and indexes configuration.
      * @param schemaManager Schema manager.
      * @param tableManager Table manager.
+     * @param catalogManager Catalog manager.
      */
     public IndexManager(
             TablesConfiguration tablesCfg,
             SchemaManager schemaManager,
-            TableManager tableManager
+            TableManager tableManager,
+            CatalogManager catalogManager,
+            MetaStorageManager metaStorageManager,
+            Consumer<LongFunction<CompletableFuture<?>>> registry
     ) {
         this.tablesCfg = Objects.requireNonNull(tablesCfg, "tablesCfg");
         this.schemaManager = Objects.requireNonNull(schemaManager, "schemaManager");
         this.tableManager = tableManager;
+        this.catalogManager = catalogManager;
+        this.metaStorageManager = metaStorageManager;
+
+        startVv = new IncrementalVersionedValue<>(registry);
     }
 
-    /** {@inheritDoc} */
     @Override
     public void start() {
-        LOG.debug("Index manager is about to start");
-
-        tablesCfg.indexes().listenElements(new ConfigurationListener());
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Index manager is about to start");
+        }
 
-        tableManager.listen(TableEvent.CREATE, (param, ex) -> {
-            if (ex != null) {
-                return completedFuture(false);
-            }
+        startIndexes();
 
-            // We can't return this future as the listener's result, because a deadlock can happen in the configuration component:
-            // this listener is called inside a configuration notification thread and all notifications are required to finish before
-            // new configuration modifications can occur (i.e. we are creating an index below). Therefore we create the index fully
-            // asynchronously and rely on the underlying components to handle PK index synchronisation.
-            tableManager.tableAsync(param.causalityToken(), param.tableId())
-                    .thenCompose(table -> {
-                        String[] pkColumns = Arrays.stream(table.schemaView().schema().keyColumns().columns())
-                                .map(Column::name)
-                                .toArray(String[]::new);
+        catalogManager.listen(INDEX_CREATE, (parameters, exception) -> {
+            assert exception == null : parameters;
 
-                        String pkName = table.name() + "_PK";
+            return onIndexCreate((CreateIndexEventParameters) parameters);
+        });
 
-                        return createIndexAsync("PUBLIC", pkName, table.name(), false,
-                                change -> change.changeUniq(true).convert(HashIndexChange.class)
-                                        .changeColumnNames(pkColumns)
-                        );
-                    })
-                    .whenComplete((v, e) -> {
-                        if (e != null) {
-                            LOG.error("Error when creating index: " + e);
-                        }
-                    });
+        catalogManager.listen(INDEX_DROP, (parameters, exception) -> {
+            assert exception == null;
 
-            return completedFuture(false);
+            return onIndexDrop((DropIndexEventParameters) parameters);
         });
 
-        LOG.info("Index manager started");
+        if (LOG.isInfoEnabled()) {
+            LOG.info("Index manager started");
+        }
     }
 
-    /** {@inheritDoc} */
     @Override
     public void stop() throws Exception {
-        LOG.debug("Index manager is about to stop");
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Index manager is about to stop");
+        }

Review Comment:
   Okay, got rid of the checks



-- 
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] AMashenkov commented on a diff in pull request #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/catalog/src/test/java/org/apache/ignite/internal/catalog/CatalogServiceSelfTest.java:
##########
@@ -263,6 +268,13 @@ public void testCreateTable() {
         assertEquals(table.primaryKeyColumns(), pkIndex.columns());
         assertTrue(pkIndex.unique());
 
+        // Validate newly created pk index
+        assertEquals(3L, pkIndex.id());

Review Comment:
   We shouldn't rely on certain ID in tests.



-- 
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 #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItTablesApiTest.java:
##########
@@ -582,29 +596,36 @@ protected void addColumnIfNotExists(Ignite node, String tableName) {
     }
 
     /**
-     * Adds a column.
+     * Tries to create the index.
      *
      * @param node Cluster node.
      * @param tableName Table name.
+     * @param indexName Index name.
+     * @param failIfNotExist Throw an exception if the index does not exist.
      */
-    protected void tryToCreateIndex(Ignite node, String tableName, boolean failIfNotExist) {
-        var tmpl  = "CREATE INDEX %s testHI ON %s (valInt, valStr)";
-        var sql = String.format(tmpl, failIfNotExist ? "" : "IF NOT EXISTS", tableName);
+    protected static void tryToCreateIndex(Ignite node, String tableName, String indexName, boolean failIfNotExist) {

Review Comment:
   Moved tests to **org.apache.ignite.internal.sql.engine.ItIndexDdlTest**



-- 
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] AMashenkov commented on a diff in pull request #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/index/src/main/java/org/apache/ignite/internal/index/IndexManager.java:
##########
@@ -96,414 +86,180 @@ public class IndexManager extends Producer<IndexEvent, IndexEventParameters> imp
     /** Table manager. */
     private final TableManager tableManager;
 
+    /** Catalog manager. */
+    private final CatalogManager catalogManager;
+
+    /** Meta storage manager. */
+    private final MetaStorageManager metaStorageManager;
+
     /** Busy lock to stop synchronously. */
     private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
 
     /** Prevents double stopping of the component. */
     private final AtomicBoolean stopGuard = new AtomicBoolean();
 
+    /** Versioned value used only at the start of the manager. */
+    private final IncrementalVersionedValue<Void> startVv;
+
     /**
      * Constructor.
      *
      * @param tablesCfg Tables and indexes configuration.
      * @param schemaManager Schema manager.
      * @param tableManager Table manager.
+     * @param catalogManager Catalog manager.
      */
     public IndexManager(
             TablesConfiguration tablesCfg,
             SchemaManager schemaManager,
-            TableManager tableManager
+            TableManager tableManager,
+            CatalogManager catalogManager,
+            MetaStorageManager metaStorageManager,
+            Consumer<LongFunction<CompletableFuture<?>>> registry
     ) {
         this.tablesCfg = Objects.requireNonNull(tablesCfg, "tablesCfg");
         this.schemaManager = Objects.requireNonNull(schemaManager, "schemaManager");
         this.tableManager = tableManager;
+        this.catalogManager = catalogManager;
+        this.metaStorageManager = metaStorageManager;
+
+        startVv = new IncrementalVersionedValue<>(registry);
     }
 
-    /** {@inheritDoc} */
     @Override
     public void start() {
-        LOG.debug("Index manager is about to start");
-
-        tablesCfg.indexes().listenElements(new ConfigurationListener());
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Index manager is about to start");
+        }
 
-        tableManager.listen(TableEvent.CREATE, (param, ex) -> {
-            if (ex != null) {
-                return completedFuture(false);
-            }
+        startIndexes();
 
-            // We can't return this future as the listener's result, because a deadlock can happen in the configuration component:
-            // this listener is called inside a configuration notification thread and all notifications are required to finish before
-            // new configuration modifications can occur (i.e. we are creating an index below). Therefore we create the index fully
-            // asynchronously and rely on the underlying components to handle PK index synchronisation.
-            tableManager.tableAsync(param.causalityToken(), param.tableId())
-                    .thenCompose(table -> {
-                        String[] pkColumns = Arrays.stream(table.schemaView().schema().keyColumns().columns())
-                                .map(Column::name)
-                                .toArray(String[]::new);
+        catalogManager.listen(INDEX_CREATE, (parameters, exception) -> {
+            assert exception == null : parameters;
 
-                        String pkName = table.name() + "_PK";
+            return onIndexCreate((CreateIndexEventParameters) parameters);
+        });
 
-                        return createIndexAsync("PUBLIC", pkName, table.name(), false,
-                                change -> change.changeUniq(true).convert(HashIndexChange.class)
-                                        .changeColumnNames(pkColumns)
-                        );
-                    })
-                    .whenComplete((v, e) -> {
-                        if (e != null) {
-                            LOG.error("Error when creating index: " + e);
-                        }
-                    });
+        catalogManager.listen(INDEX_DROP, (parameters, exception) -> {
+            assert exception == null;
 
-            return completedFuture(false);
+            return onIndexDrop((DropIndexEventParameters) parameters);
         });
 
-        LOG.info("Index manager started");
+        if (LOG.isInfoEnabled()) {
+            LOG.info("Index manager started");
+        }
     }
 
-    /** {@inheritDoc} */
     @Override
     public void stop() throws Exception {
-        LOG.debug("Index manager is about to stop");
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Index manager is about to stop");
+        }

Review Comment:
   Why do we need an `if` statement here and below?  
   isDebugEnabled is checked by IgniteLogger.
   ```
    private void logInternal(Level level, String msg, Throwable th, Object... params) {
           Objects.requireNonNull(level);
   
           if (!delegate.isLoggable(level)) {
               return;
           }
   ```



-- 
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] AMashenkov commented on a diff in pull request #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItCommonApiTest.java:
##########
@@ -123,37 +115,25 @@ public void testSessionExpiration() throws Exception {
     @Test
     public void checkTimestampOperations() {
         String kvTblName = "tbl_all_columns_sql";
-        String schemaName = "PUBLIC";
-        String keyCol = "key";
-        int maxTimePrecision = TemporalColumnType.MAX_TIME_PRECISION;
+        String keyCol = "KEY";
 
         Ignite node = CLUSTER_NODES.get(0);
 
         // TODO: https://issues.apache.org/jira/browse/IGNITE-19162 Trim all less than millisecond information from timestamp
         //String tsStr = "2023-03-29T08:22:33.005007Z";
-        String tsStr = "2023-03-29T08:22:33.005Z";
+        String tsStr = "2023-03-29T08:22:33Z";
 
         Instant ins = Instant.parse(tsStr);
 
         sql("CREATE TABLE timestamps(id INTEGER PRIMARY KEY, i TIMESTAMP(9))");
 
-        TableDefinition schTblAllSql = SchemaBuilders.tableBuilder(schemaName, kvTblName).columns(
-                SchemaBuilders.column(keyCol, ColumnType.INT64).build(),
-                SchemaBuilders.column("time", ColumnType.time(maxTimePrecision)).asNullable(true).build(),
-                SchemaBuilders.column("timestamp", ColumnType.timestamp(maxTimePrecision)).asNullable(true).build(),
-                SchemaBuilders.column("datetime", ColumnType.datetime(maxTimePrecision)).asNullable(true).build()
-        ).withPrimaryKey(keyCol).build();
-
-        await(((TableManager) node.tables()).createTableAsync(schTblAllSql.name(), DEFAULT_ZONE_NAME, tblCh ->
-                SchemaConfigurationConverter.convert(schTblAllSql, tblCh)
-        ));
+        sql(String.format("CREATE TABLE %s(\"%s\" INTEGER PRIMARY KEY, \"TIMESTAMP\" TIMESTAMP(9))", kvTblName, keyCol));

Review Comment:
   Ok. Let's add these tickets to comment as well, or disable the test with these tickets numbers.
   



-- 
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 #2254: IGNITE-19500 IndexManager should listen CatalogService events instead of configuration

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


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItIndexDdlTest.java:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.sql.engine;
+
+import static org.apache.ignite.internal.catalog.CatalogService.DEFAULT_SCHEMA_NAME;
+
+import org.apache.ignite.internal.testframework.IgniteTestUtils;
+import org.apache.ignite.lang.IndexAlreadyExistsException;
+import org.apache.ignite.lang.IndexNotFoundException;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Integration tests for DDL statements that affect indexes.
+ */
+public class ItIndexDdlTest extends ClusterPerClassIntegrationTest {
+    private static final String TABLE_NAME = "TEST_TABLE";
+
+    private static final String INDEX_NAME = "TEST_INDEX";
+
+    @BeforeEach
+    void setUp() {
+        sql(String.format("CREATE TABLE IF NOT EXISTS %s (key BIGINT PRIMARY KEY, valInt INT, valStr VARCHAR)", TABLE_NAME));
+    }
+
+    @AfterEach
+    void tearDown() {
+        sql(String.format("DROP TABLE IF EXISTS %S", TABLE_NAME));
+    }
+
+    @Test
+    public void testAddIndex() {
+        tryToCreateIndex(TABLE_NAME, INDEX_NAME, true);
+
+        IgniteTestUtils.assertThrows(
+                IndexAlreadyExistsException.class,
+                () -> tryToCreateIndex(TABLE_NAME, INDEX_NAME, true),
+                String.format("Index already exists [name=\"%s\".\"%s\"]", DEFAULT_SCHEMA_NAME, INDEX_NAME)
+        );
+
+        tryToCreateIndex(TABLE_NAME, INDEX_NAME, false);
+    }
+
+    @Test
+    void testDropIndex() {
+        tryToCreateIndex(TABLE_NAME, INDEX_NAME, true);
+
+        // Let's check the drop on an existing index.
+        tryToDropIndex(INDEX_NAME, true);
+
+        // Let's check the drop on a non-existent index.
+        IgniteTestUtils.assertThrows(
+                IndexNotFoundException.class,
+                () -> tryToDropIndex(INDEX_NAME, true),
+                String.format("Index does not exist [name=\"%s\".\"%s\"]", DEFAULT_SCHEMA_NAME, INDEX_NAME)
+        );
+
+        tryToCreateIndex(TABLE_NAME, INDEX_NAME, false);
+    }
+
+    /**
+     * Tries to create the index.
+     *
+     * @param tableName Table name.
+     * @param indexName Index name.
+     * @param failIfNotExist Throw an exception if the index does not exist.
+     */
+    private static void tryToCreateIndex(String tableName, String indexName, boolean failIfNotExist) {

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