You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by "sergeyuttsel (via GitHub)" <gi...@apache.org> on 2023/04/20 20:43:02 UTC

[GitHub] [ignite-3] sergeyuttsel opened a new pull request, #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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

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


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

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

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


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -1343,72 +1355,132 @@ private Set<Assignment> calculateAssignments(TableConfiguration tableCfg, int pa
      *         </ul>
      * @see TableAlreadyExistsException
      */
-    public CompletableFuture<Table> createTableAsync(String name, Consumer<TableChange> tableInitChange) {
+    public CompletableFuture<Table> createTableAsync(String name, String zoneName, Consumer<TableChange> tableInitChange) {
         if (!busyLock.enterBusy()) {
             throw new IgniteException(new NodeStoppingException());
         }
         try {
-            return createTableAsyncInternal(name, tableInitChange);
+            return createTableAsyncInternal(name, zoneName, tableInitChange);
         } finally {
             busyLock.leaveBusy();
         }
     }
 
-    /** See {@link #createTableAsync(String, Consumer)} for details. */
-    private CompletableFuture<Table> createTableAsyncInternal(String name, Consumer<TableChange> tableInitChange) {
+    /** See {@link #createTableAsync(String, String, Consumer)} for details. */
+    private CompletableFuture<Table> createTableAsyncInternal(
+            String name,
+            String zoneName,
+            Consumer<TableChange> tableInitChange
+    ) {
         CompletableFuture<Table> tblFut = new CompletableFuture<>();
+        tableAsyncInternal(name)
+                .thenAccept(tbl -> {

Review Comment:
   Fixed



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

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

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


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -1345,72 +1357,158 @@ private Set<Assignment> calculateAssignments(TableConfiguration tableCfg, int pa
      *         </ul>
      * @see TableAlreadyExistsException
      */
-    public CompletableFuture<Table> createTableAsync(String name, Consumer<TableChange> tableInitChange) {
+    public CompletableFuture<Table> createTableAsync(String name, String zoneName, Consumer<TableChange> tableInitChange) {
         if (!busyLock.enterBusy()) {
             throw new IgniteException(new NodeStoppingException());
         }
         try {
-            return createTableAsyncInternal(name, tableInitChange);
+            return createTableAsyncInternal(name, zoneName, tableInitChange);
         } finally {
             busyLock.leaveBusy();
         }
     }
 
-    /** See {@link #createTableAsync(String, Consumer)} for details. */
-    private CompletableFuture<Table> createTableAsyncInternal(String name, Consumer<TableChange> tableInitChange) {
+    /** See {@link #createTableAsync(String, String, Consumer)} for details. */
+    private CompletableFuture<Table> createTableAsyncInternal(
+            String name,
+            String zoneName,
+            Consumer<TableChange> tableInitChange
+    ) {
         CompletableFuture<Table> tblFut = new CompletableFuture<>();
 
-        tableAsyncInternal(name).thenAccept(tbl -> {
-            if (tbl != null) {
-                tblFut.completeExceptionally(new TableAlreadyExistsException(DEFAULT_SCHEMA_NAME, name));
-            } else {
-                tablesCfg.change(tablesChange -> tablesChange.changeTables(tablesListChange -> {
-                    if (tablesListChange.get(name) != null) {
-                        throw new TableAlreadyExistsException(DEFAULT_SCHEMA_NAME, name);
+        tableAsyncInternal(name)
+                .thenAccept(tbl -> {
+                    if (tbl != null) {
+                        tblFut.completeExceptionally(new TableAlreadyExistsException(DEFAULT_SCHEMA_NAME, name));
+                    } else {
+                        if (!busyLock.enterBusy()) {
+                            throw new IgniteException(new NodeStoppingException());
+                        }
+
+                        try {
+                            distributionZoneManager.zoneIdAsyncInternal(zoneName).handle((zoneId, ex) -> {
+                                if (zoneId == null) {
+                                    tblFut.completeExceptionally(new DistributionZoneNotFoundException(zoneName));
+                                } else if (ex != null) {
+                                    tblFut.completeExceptionally(ex);
+                                } else {
+                                    if (!busyLock.enterBusy()) {
+                                        throw new IgniteException(new NodeStoppingException());
+                                    }
+
+                                    try {
+                                        cmgMgr.logicalTopology()
+                                                .handle((cmgTopology, e) -> {
+                                                    if (e == null) {
+                                                        if (!busyLock.enterBusy()) {

Review Comment:
   Here and there. Is it possible to use `IgniteUtils#inBusyLock()` instead?



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

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

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


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerWatchListenerTest.java:
##########
@@ -136,20 +141,20 @@ void testStaleVaultRevisionOnZoneManagerStart() throws InterruptedException {
 
         mockVaultZonesLogicalTopologyKey(nodes);
 
-        distributionZoneManager.start();
+        startDistributionZoneManager();
 
         verify(keyValueStorage, timeout(1000).times(2)).invoke(any());
 
         assertDataNodesForZone(DEFAULT_ZONE_ID, null, keyValueStorage);
     }
 
     @Test
-    void testDataNodesUpdatedOnZoneManagerStart() throws InterruptedException {
+    void testDataNodesUpdatedOnZoneManagerStart() throws Exception {
         Set<String> nodes = Set.of("node1", "node2");
 
         mockVaultZonesLogicalTopologyKey(nodes);
 
-        distributionZoneManager.start();
+        startDistributionZoneManager();

Review Comment:
   There's no need in such method 
   ```
       private void startDistributionZoneManager() throws Exception {
           deployWatchesAndUpdateMetaStorageRevision(metaStorageManager);
   
           distributionZoneManager.start();
       }
   ```
   because it's inappropriate encapsulation pattern: dzm start semantically doesn't include deploy watches.



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

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

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


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerWatchListenerTest.java:
##########
@@ -136,20 +141,20 @@ void testStaleVaultRevisionOnZoneManagerStart() throws InterruptedException {
 
         mockVaultZonesLogicalTopologyKey(nodes);
 
-        distributionZoneManager.start();
+        startDistributionZoneManager();
 
         verify(keyValueStorage, timeout(1000).times(2)).invoke(any());
 
         assertDataNodesForZone(DEFAULT_ZONE_ID, null, keyValueStorage);
     }
 
     @Test
-    void testDataNodesUpdatedOnZoneManagerStart() throws InterruptedException {
+    void testDataNodesUpdatedOnZoneManagerStart() throws Exception {
         Set<String> nodes = Set.of("node1", "node2");
 
         mockVaultZonesLogicalTopologyKey(nodes);
 
-        distributionZoneManager.start();
+        startDistributionZoneManager();

Review Comment:
   I will create another ticket to fix it properly. (upd: https://issues.apache.org/jira/browse/IGNITE-19403)



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

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

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


[GitHub] [ignite-3] sanpwc merged pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


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

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

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


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerConfigurationChangesTest.java:
##########
@@ -120,21 +125,28 @@ public void setUp() throws Exception {
 
         metaStorageManager.put(zonesLogicalTopologyVersionKey(), longToBytes(0));
 
+        ConfigurationRegistry clusterConfigRegistry = clusterCfgMgr.configurationRegistry();
+
+        Consumer<Function<Long, CompletableFuture<?>>> registry =
+                c -> clusterConfigRegistry.listenUpdateStorageRevision(c::apply);
+
         distributionZoneManager = new DistributionZoneManager(
                 zonesConfiguration,
                 tablesConfiguration,
                 metaStorageManager,
                 logicalTopologyService,
                 vaultMgr,
+                registry,
                 "test"
         );
 
         vaultMgr.start();
         clusterCfgMgr.start();
         metaStorageManager.start();
-        distributionZoneManager.start();
 
-        metaStorageManager.deployWatches();
+        deployWatchesAndUpdateMetaStorageRevision(metaStorageManager);

Review Comment:
   This approach was introduced by Aleksandr Polovtsev. I just reused it. I will create another ticket to fix it properly.



##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerScaleUpTest.java:
##########
@@ -1366,10 +1379,10 @@ private void assertNotEqualsDataNodesForZone(int zoneId, @Nullable Set<String> c
         ));
     }
 
-    private void startDistributionZoneManager() throws NodeStoppingException {
-        distributionZoneManager.start();
+    private void startDistributionZoneManager() throws Exception {
+        deployWatchesAndUpdateMetaStorageRevision(metaStorageManager);

Review Comment:
   I will create another ticket to fix it properly.



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

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

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


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/table/build.gradle:
##########
@@ -41,6 +41,7 @@ dependencies {
     implementation project(':ignite-configuration-api')
     implementation project(':ignite-distribution-zones')
     implementation project(':ignite-vault')
+    implementation project(':ignite-cluster-management')

Review Comment:
   Why do we need it? For which dependency?



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

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

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


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerTest.java:
##########
@@ -91,6 +91,7 @@ public void setUp() {
                 null,
                 null,
                 null,
+                x -> {},

Review Comment:
   Fixed.



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

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

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


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -161,6 +171,17 @@ public class DistributionZoneManager implements IgniteComponent {
     /** The logger. */
     private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneManager.class);
 
+    /**
+     * If this property is set to {@code true} then an attempt to get the configuration property directly from Meta storage will be skipped,
+     * and the local property will be returned.
+     * TODO: IGNITE-16774 This property and overall approach, access configuration directly through Meta storage,
+     * TODO: will be removed after fix of the issue.
+     */
+    private final boolean getMetadataLocallyOnly = IgniteSystemProperties.getBoolean("IGNITE_GET_METADATA_LOCALLY_ONLY");
+
+    /** Versioned store for zones id. */
+    private final IncrementalVersionedValue<Set<Integer>> zonesByIdVv;

Review Comment:
   I don't think that you need versioned value here because you only use it's side effects and nor it's core idea.
   E.g. it's possible to add cfg listener in order to await zone with needed id. I mean following:
   1. Read zone cfg through direct proxy and check whether zone with given name exists. Throw an exception if it doesn't.
   2. Register zones listener in order to await zone locally with given Id. 
   3. Check whether we already have such zone locally. Remove listener if we do have one.
   2' Within listener we should do some useful stuff and remove the listener itself. Please ask SE guys whether it's possible to unregister the listener from within listener.
   
   ```
               DistributionZoneConfiguration zoneCfg = directProxy(zonesConfiguration.distributionZones()).get(zoneName);
   
               // check that zone is present.
               
               zonesConfiguration.distributionZones().listenElements(new ConfigurationNamedListListener<DistributionZoneView>() {
                   @Override
                   public CompletableFuture<?> onCreate(ConfigurationNotificationEvent<DistributionZoneView> ctx) {
   
                       if (ctx.newValue().zoneId() == zoneCfg.zoneId().value()) {
                           // do something usefull
                           // unregister current listener. Please ask SE guys whether it's possible to unregister the listener from within listener.
                       }
                       return CompletableFuture.completedFuture(null);
                   }
               });
   ```
   I'm not sure whether solution above is best, however it seems better to versioned value one. We can think about other options too.



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

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

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


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -161,6 +171,17 @@ public class DistributionZoneManager implements IgniteComponent {
     /** The logger. */
     private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneManager.class);
 
+    /**
+     * If this property is set to {@code true} then an attempt to get the configuration property directly from Meta storage will be skipped,
+     * and the local property will be returned.
+     * TODO: IGNITE-16774 This property and overall approach, access configuration directly through Meta storage,
+     * TODO: will be removed after fix of the issue.
+     */
+    private final boolean getMetadataLocallyOnly = IgniteSystemProperties.getBoolean("IGNITE_GET_METADATA_LOCALLY_ONLY");
+
+    /** Versioned store for zones id. */
+    private final IncrementalVersionedValue<Set<Integer>> zonesByIdVv;

Review Comment:
   I don't think that you need versioned value here because you only use it's side effects and nor it's core idea.
   E.g. it's possible to add cfg listener in order to await zone with needed id. I mean following:
   1. Read zone cfg through direct proxy and check whether zone with given name exists. Throw an exception if it doesn't.
   2. Register zones listener in order to await zone locally with given Id. 
   3. Check whether we already have such zone locally. Remove listener if we do have one.
   2' Within listener we should do some useful stuff and remove the listener itself. Please ask SE guys whether it's possible to unregister the listener from within listener.
   
   ```
               DistributionZoneConfiguration zoneCfg = directProxy(zonesConfiguration.distributionZones()).get(zoneName);
   
               // check that zone is present.
               
               zonesConfiguration.distributionZones().listenElements(new ConfigurationNamedListListener<DistributionZoneView>() {
                   @Override
                   public CompletableFuture<?> onCreate(ConfigurationNotificationEvent<DistributionZoneView> ctx) {
   
                       if (ctx.newValue().zoneId() == zoneCfg.zoneId().value()) {
                           // do something usefull
                           // unregister current listener. Please ask SE guys whether it's possible to unregister the listener from within listener.
                       }
                       return CompletableFuture.completedFuture(null);
                   }
               });
   ```
   I'm not sure whether solution above is best, however it seems better in comparison to versioned value one. We can think about other options too.



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

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

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


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -1614,6 +1662,101 @@ CompletableFuture<Void> saveDataNodesToMetaStorageOnScaleDown(int zoneId, long r
         }
     }
 
+    /**
+     * Gets direct id of the distribution zone with {@code zoneName}.
+     *
+     * @param zoneName Name of the distribution zone.
+     * @return Direct id of the distribution zone, or {@code null} if the zone with the {@code zoneName} has not been found.
+     */
+    public CompletableFuture<Integer> zoneIdAsyncInternal(String zoneName) {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(new NodeStoppingException());
+        }
+        try {
+            if (DEFAULT_ZONE_NAME.equals(zoneName)) {
+                return completedFuture(DEFAULT_ZONE_ID);
+            }
+
+            // TODO: IGNITE-16288 directZoneId should use async configuration API
+            return supplyAsync(() -> inBusyLock(busyLock, () -> directZoneIdInternal(zoneName)), executor)
+                    .thenCompose(zoneId -> waitZoneIdLocally(zoneId).thenCompose(ignored -> completedFuture(zoneId)));
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    @Nullable
+    private Integer directZoneIdInternal(String zoneName) {
+        try {
+            DistributionZoneConfiguration zoneCfg = directProxy(zonesConfiguration.distributionZones()).get(zoneName);
+
+            if (zoneCfg == null) {
+                return null;
+            } else {
+                return zoneCfg.zoneId().value();
+            }
+        } catch (NoSuchElementException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Internal method for waiting that the zone is created locally.
+     *
+     * @param id Table id.
+     * @return Future representing pending completion of the operation.
+     */
+    private CompletableFuture<Void> waitZoneIdLocally(Integer id) {

Review Comment:
   Thanks. I've added null handling in zoneIdAsyncInternal.



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

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

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


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerConfigurationChangesTest.java:
##########
@@ -120,21 +125,28 @@ public void setUp() throws Exception {
 
         metaStorageManager.put(zonesLogicalTopologyVersionKey(), longToBytes(0));
 
+        ConfigurationRegistry clusterConfigRegistry = clusterCfgMgr.configurationRegistry();
+
+        Consumer<Function<Long, CompletableFuture<?>>> registry =
+                c -> clusterConfigRegistry.listenUpdateStorageRevision(c::apply);
+
         distributionZoneManager = new DistributionZoneManager(
                 zonesConfiguration,
                 tablesConfiguration,
                 metaStorageManager,
                 logicalTopologyService,
                 vaultMgr,
+                registry,
                 "test"
         );
 
         vaultMgr.start();
         clusterCfgMgr.start();
         metaStorageManager.start();
-        distributionZoneManager.start();
 
-        metaStorageManager.deployWatches();
+        deployWatchesAndUpdateMetaStorageRevision(metaStorageManager);

Review Comment:
   This approach was introduced by Aleksandr Polovtsev. I just reused it. I will create another ticket to fix it properly. (upd: I created TODO https://issues.apache.org/jira/browse/IGNITE-19403)



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

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

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


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java:
##########
@@ -980,11 +983,11 @@ private void assertTablePresent(TableManager tableManager, String tableName) {
     public void testOneNodeRestartWithGap() throws InterruptedException {
         IgniteImpl ignite = startNode(0);
 
-        List<IgniteComponent> components = startPartialNode(1, null);
+        startNode(1);

Review Comment:
   Why it's full node instead of partial?



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

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

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


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -1343,72 +1355,132 @@ private Set<Assignment> calculateAssignments(TableConfiguration tableCfg, int pa
      *         </ul>
      * @see TableAlreadyExistsException
      */
-    public CompletableFuture<Table> createTableAsync(String name, Consumer<TableChange> tableInitChange) {
+    public CompletableFuture<Table> createTableAsync(String name, String zoneName, Consumer<TableChange> tableInitChange) {
         if (!busyLock.enterBusy()) {
             throw new IgniteException(new NodeStoppingException());
         }
         try {
-            return createTableAsyncInternal(name, tableInitChange);
+            return createTableAsyncInternal(name, zoneName, tableInitChange);
         } finally {
             busyLock.leaveBusy();
         }
     }
 
-    /** See {@link #createTableAsync(String, Consumer)} for details. */
-    private CompletableFuture<Table> createTableAsyncInternal(String name, Consumer<TableChange> tableInitChange) {
+    /** See {@link #createTableAsync(String, String, Consumer)} for details. */
+    private CompletableFuture<Table> createTableAsyncInternal(
+            String name,
+            String zoneName,
+            Consumer<TableChange> tableInitChange
+    ) {
         CompletableFuture<Table> tblFut = new CompletableFuture<>();
+        tableAsyncInternal(name)
+                .thenAccept(tbl -> {

Review Comment:
   Seems that busy locks are missing, aren't they?



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

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

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


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -161,6 +171,17 @@ public class DistributionZoneManager implements IgniteComponent {
     /** The logger. */
     private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneManager.class);
 
+    /**
+     * If this property is set to {@code true} then an attempt to get the configuration property directly from Meta storage will be skipped,
+     * and the local property will be returned.
+     * TODO: IGNITE-16774 This property and overall approach, access configuration directly through Meta storage,
+     * TODO: will be removed after fix of the issue.
+     */
+    private final boolean getMetadataLocallyOnly = IgniteSystemProperties.getBoolean("IGNITE_GET_METADATA_LOCALLY_ONLY");
+
+    /** Versioned store for zones id. */
+    private final IncrementalVersionedValue<Set<Integer>> zonesByIdVv;

Review Comment:
   I don't think that you need versioned value here because you only use the it's side effects and nor it's core idea.
   E.g. it's possible to add cfg listener in order to await zone with needed id. I mean following:
   1. Read zone cfg through direct proxy and check whether zone with given name exists. Throw an exception if it doesn't.
   2. Register zones listener in order to await zone locally with given Id. 
   3. Check whether we already have such zone locally. Remove listener if we do have one.
   2' Within listener we should do some useful stuff and remove the listener itself. Please ask SE guys whether it's possible to unregister the listener from within listener.
   
   ```
               DistributionZoneConfiguration zoneCfg = directProxy(zonesConfiguration.distributionZones()).get(zoneName);
   
               // check that zone is present.
               
               zonesConfiguration.distributionZones().listenElements(new ConfigurationNamedListListener<DistributionZoneView>() {
                   @Override
                   public CompletableFuture<?> onCreate(ConfigurationNotificationEvent<DistributionZoneView> ctx) {
   
                       if (ctx.newValue().zoneId() == zoneCfg.zoneId().value()) {
                           // do something usefull
                           // unregister current listener. Please ask SE guys whether it's possible to unregister the listener from within listener.
                       }
                       return CompletableFuture.completedFuture(null);
                   }
               });
   ```
   I'm not sure whether solution above is best, however it seems better to versioned value one. We can think about other options too.



##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -859,7 +885,17 @@ public CompletableFuture<?> onCreate(ConfigurationNotificationEvent<Distribution
 
             saveDataNodesAndUpdateTriggerKeysInMetaStorage(zoneId, ctx.storageRevision(), logicalTopology);
 
-            return completedFuture(null);
+            return zonesByIdVv.update(ctx.storageRevision(), (zones, e) -> {
+                if (e != null) {

Review Comment:
   What about busy lock? Do we need it here?



##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -1614,6 +1662,101 @@ CompletableFuture<Void> saveDataNodesToMetaStorageOnScaleDown(int zoneId, long r
         }
     }
 
+    /**
+     * Gets direct id of the distribution zone with {@code zoneName}.
+     *
+     * @param zoneName Name of the distribution zone.
+     * @return Direct id of the distribution zone, or {@code null} if the zone with the {@code zoneName} has not been found.
+     */
+    public CompletableFuture<Integer> zoneIdAsyncInternal(String zoneName) {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(new NodeStoppingException());
+        }
+        try {
+            if (DEFAULT_ZONE_NAME.equals(zoneName)) {
+                return completedFuture(DEFAULT_ZONE_ID);
+            }
+
+            // TODO: IGNITE-16288 directZoneId should use async configuration API
+            return supplyAsync(() -> inBusyLock(busyLock, () -> directZoneIdInternal(zoneName)), executor)
+                    .thenCompose(zoneId -> waitZoneIdLocally(zoneId).thenCompose(ignored -> completedFuture(zoneId)));
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    @Nullable
+    private Integer directZoneIdInternal(String zoneName) {
+        try {
+            DistributionZoneConfiguration zoneCfg = directProxy(zonesConfiguration.distributionZones()).get(zoneName);
+
+            if (zoneCfg == null) {
+                return null;
+            } else {
+                return zoneCfg.zoneId().value();
+            }
+        } catch (NoSuchElementException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Internal method for waiting that the zone is created locally.
+     *
+     * @param id Table id.
+     * @return Future representing pending completion of the operation.
+     */
+    private CompletableFuture<Void> waitZoneIdLocally(Integer id) {

Review Comment:
   I'm not sure that I get how you handle null as an id. I mean that directZoneIdInternal may return null if there's no such zone in direct cfg. 



##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -1614,6 +1662,101 @@ CompletableFuture<Void> saveDataNodesToMetaStorageOnScaleDown(int zoneId, long r
         }
     }
 
+    /**
+     * Gets direct id of the distribution zone with {@code zoneName}.
+     *
+     * @param zoneName Name of the distribution zone.
+     * @return Direct id of the distribution zone, or {@code null} if the zone with the {@code zoneName} has not been found.
+     */
+    public CompletableFuture<Integer> zoneIdAsyncInternal(String zoneName) {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(new NodeStoppingException());
+        }
+        try {
+            if (DEFAULT_ZONE_NAME.equals(zoneName)) {
+                return completedFuture(DEFAULT_ZONE_ID);
+            }
+
+            // TODO: IGNITE-16288 directZoneId should use async configuration API
+            return supplyAsync(() -> inBusyLock(busyLock, () -> directZoneIdInternal(zoneName)), executor)
+                    .thenCompose(zoneId -> waitZoneIdLocally(zoneId).thenCompose(ignored -> completedFuture(zoneId)));
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    @Nullable
+    private Integer directZoneIdInternal(String zoneName) {
+        try {
+            DistributionZoneConfiguration zoneCfg = directProxy(zonesConfiguration.distributionZones()).get(zoneName);
+
+            if (zoneCfg == null) {
+                return null;
+            } else {
+                return zoneCfg.zoneId().value();
+            }
+        } catch (NoSuchElementException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Internal method for waiting that the zone is created locally.
+     *
+     * @param id Table id.

Review Comment:
   Seems that it's not table but zone id.



##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -859,7 +885,17 @@ public CompletableFuture<?> onCreate(ConfigurationNotificationEvent<Distribution
 
             saveDataNodesAndUpdateTriggerKeysInMetaStorage(zoneId, ctx.storageRevision(), logicalTopology);
 
-            return completedFuture(null);
+            return zonesByIdVv.update(ctx.storageRevision(), (zones, e) -> {
+                if (e != null) {
+                    return failedFuture(e);
+                }
+
+                HashSet<Integer> newZones = new HashSet<>(zones);

Review Comment:
   Why do we need to create a new set on every addition/removal?



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

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

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


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java:
##########
@@ -980,11 +983,11 @@ private void assertTablePresent(TableManager tableManager, String tableName) {
     public void testOneNodeRestartWithGap() throws InterruptedException {
         IgniteImpl ignite = startNode(0);
 
-        List<IgniteComponent> components = startPartialNode(1, null);
+        startNode(1);

Review Comment:
   Added TODO https://issues.apache.org/jira/browse/IGNITE-19408



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

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

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


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -859,7 +885,17 @@ public CompletableFuture<?> onCreate(ConfigurationNotificationEvent<Distribution
 
             saveDataNodesAndUpdateTriggerKeysInMetaStorage(zoneId, ctx.storageRevision(), logicalTopology);
 
-            return completedFuture(null);
+            return zonesByIdVv.update(ctx.storageRevision(), (zones, e) -> {
+                if (e != null) {

Review Comment:
   Fixed. I removed this code.



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

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

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


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java:
##########
@@ -980,11 +983,11 @@ private void assertTablePresent(TableManager tableManager, String tableName) {
     public void testOneNodeRestartWithGap() throws InterruptedException {
         IgniteImpl ignite = startNode(0);
 
-        List<IgniteComponent> components = startPartialNode(1, null);
+        startNode(1);

Review Comment:
   Please address given question. You may create separate ticket if needed. 



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

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

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


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -1343,72 +1355,132 @@ private Set<Assignment> calculateAssignments(TableConfiguration tableCfg, int pa
      *         </ul>
      * @see TableAlreadyExistsException
      */
-    public CompletableFuture<Table> createTableAsync(String name, Consumer<TableChange> tableInitChange) {
+    public CompletableFuture<Table> createTableAsync(String name, String zoneName, Consumer<TableChange> tableInitChange) {
         if (!busyLock.enterBusy()) {
             throw new IgniteException(new NodeStoppingException());
         }
         try {
-            return createTableAsyncInternal(name, tableInitChange);
+            return createTableAsyncInternal(name, zoneName, tableInitChange);
         } finally {
             busyLock.leaveBusy();
         }
     }
 
-    /** See {@link #createTableAsync(String, Consumer)} for details. */
-    private CompletableFuture<Table> createTableAsyncInternal(String name, Consumer<TableChange> tableInitChange) {
+    /** See {@link #createTableAsync(String, String, Consumer)} for details. */
+    private CompletableFuture<Table> createTableAsyncInternal(
+            String name,
+            String zoneName,
+            Consumer<TableChange> tableInitChange
+    ) {
         CompletableFuture<Table> tblFut = new CompletableFuture<>();
+        tableAsyncInternal(name)
+                .thenAccept(tbl -> {
+                    if (tbl != null) {
+                        tblFut.completeExceptionally(new TableAlreadyExistsException(DEFAULT_SCHEMA_NAME, name));
+                    } else {
+                        distributionZoneManager.zoneIdAsyncInternal(zoneName).handle((zoneId, ex) -> {

Review Comment:
   Which of the aforementioned tests do check what will we await dataNodes evaluation for immediate timers?
   Meaning that there should be at least following tests:
   1. The one that check that we await **topology version** within DZM.
   2. The one that check that we await immediate timers but skip non-immediate ones.



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

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

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


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -1345,72 +1357,158 @@ private Set<Assignment> calculateAssignments(TableConfiguration tableCfg, int pa
      *         </ul>
      * @see TableAlreadyExistsException
      */
-    public CompletableFuture<Table> createTableAsync(String name, Consumer<TableChange> tableInitChange) {
+    public CompletableFuture<Table> createTableAsync(String name, String zoneName, Consumer<TableChange> tableInitChange) {
         if (!busyLock.enterBusy()) {
             throw new IgniteException(new NodeStoppingException());
         }
         try {
-            return createTableAsyncInternal(name, tableInitChange);
+            return createTableAsyncInternal(name, zoneName, tableInitChange);
         } finally {
             busyLock.leaveBusy();
         }
     }
 
-    /** See {@link #createTableAsync(String, Consumer)} for details. */
-    private CompletableFuture<Table> createTableAsyncInternal(String name, Consumer<TableChange> tableInitChange) {
+    /** See {@link #createTableAsync(String, String, Consumer)} for details. */
+    private CompletableFuture<Table> createTableAsyncInternal(
+            String name,
+            String zoneName,
+            Consumer<TableChange> tableInitChange
+    ) {
         CompletableFuture<Table> tblFut = new CompletableFuture<>();
 
-        tableAsyncInternal(name).thenAccept(tbl -> {
-            if (tbl != null) {
-                tblFut.completeExceptionally(new TableAlreadyExistsException(DEFAULT_SCHEMA_NAME, name));
-            } else {
-                tablesCfg.change(tablesChange -> tablesChange.changeTables(tablesListChange -> {
-                    if (tablesListChange.get(name) != null) {
-                        throw new TableAlreadyExistsException(DEFAULT_SCHEMA_NAME, name);
+        tableAsyncInternal(name)
+                .thenAccept(tbl -> {
+                    if (tbl != null) {
+                        tblFut.completeExceptionally(new TableAlreadyExistsException(DEFAULT_SCHEMA_NAME, name));
+                    } else {
+                        if (!busyLock.enterBusy()) {
+                            throw new IgniteException(new NodeStoppingException());
+                        }
+
+                        try {
+                            distributionZoneManager.zoneIdAsyncInternal(zoneName).handle((zoneId, ex) -> {
+                                if (zoneId == null) {
+                                    tblFut.completeExceptionally(new DistributionZoneNotFoundException(zoneName));
+                                } else if (ex != null) {
+                                    tblFut.completeExceptionally(ex);
+                                } else {
+                                    if (!busyLock.enterBusy()) {
+                                        throw new IgniteException(new NodeStoppingException());
+                                    }
+
+                                    try {
+                                        cmgMgr.logicalTopology()
+                                                .handle((cmgTopology, e) -> {
+                                                    if (e == null) {
+                                                        if (!busyLock.enterBusy()) {

Review Comment:
   I cannot use inBusyLock because I need not only throw NodeStoppingException but also complete tblFut with NodeStoppingException.



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

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

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


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java:
##########
@@ -994,9 +997,9 @@ public void testOneNodeRestartWithGap() throws InterruptedException {
 
         createTableWithoutData(ignite, TABLE_NAME_2, 1, 1);
 
-        components = startPartialNode(1, null);
+        IgniteImpl ignite1 = startNode(1);

Review Comment:
   Same as above.



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

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

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


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/configuration/DistributionZoneConfigurationSchema.java:
##########
@@ -68,12 +69,12 @@ public class DistributionZoneConfigurationSchema {
     /** Timeout in seconds between node added topology event itself and data nodes switch. */
     @Range(min = 0)
     @Value(hasDefault = true)
-    public int dataNodesAutoAdjustScaleUp = INFINITE_TIMER_VALUE;
+    public int dataNodesAutoAdjustScaleUp = IMMEDIATE_TIMER_VALUE;
 
     /** Timeout in seconds between node left topology event itself and data nodes switch. */
     @Range(min = 0)
     @Value(hasDefault = true)
-    public int dataNodesAutoAdjustScaleDown = INFINITE_TIMER_VALUE;
+    public int dataNodesAutoAdjustScaleDown = IMMEDIATE_TIMER_VALUE;

Review Comment:
   Scale down should still be infinite by default.



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

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

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


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/configuration/DistributionZoneConfigurationSchema.java:
##########
@@ -68,12 +69,12 @@ public class DistributionZoneConfigurationSchema {
     /** Timeout in seconds between node added topology event itself and data nodes switch. */
     @Range(min = 0)
     @Value(hasDefault = true)
-    public int dataNodesAutoAdjustScaleUp = INFINITE_TIMER_VALUE;
+    public int dataNodesAutoAdjustScaleUp = IMMEDIATE_TIMER_VALUE;
 
     /** Timeout in seconds between node left topology event itself and data nodes switch. */
     @Range(min = 0)
     @Value(hasDefault = true)
-    public int dataNodesAutoAdjustScaleDown = INFINITE_TIMER_VALUE;
+    public int dataNodesAutoAdjustScaleDown = IMMEDIATE_TIMER_VALUE;

Review Comment:
   Fixed.



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

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

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


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerWatchListenerTest.java:
##########
@@ -136,20 +141,20 @@ void testStaleVaultRevisionOnZoneManagerStart() throws InterruptedException {
 
         mockVaultZonesLogicalTopologyKey(nodes);
 
-        distributionZoneManager.start();
+        startDistributionZoneManager();
 
         verify(keyValueStorage, timeout(1000).times(2)).invoke(any());
 
         assertDataNodesForZone(DEFAULT_ZONE_ID, null, keyValueStorage);
     }
 
     @Test
-    void testDataNodesUpdatedOnZoneManagerStart() throws InterruptedException {
+    void testDataNodesUpdatedOnZoneManagerStart() throws Exception {
         Set<String> nodes = Set.of("node1", "node2");
 
         mockVaultZonesLogicalTopologyKey(nodes);
 
-        distributionZoneManager.start();
+        startDistributionZoneManager();

Review Comment:
   I will create another ticket to fix it properly.



##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/MockedStructuresTest.java:
##########
@@ -162,13 +177,15 @@ public class MockedStructuresTest extends IgniteAbstractTest {
     @InjectConfiguration
     private TablesConfiguration tblsCfg;
 
-    @InjectConfiguration("mock.distributionZones.zone123{dataStorage.name = " + ENGINE_NAME + ", zoneId = 1}")
+    @InjectConfiguration("mock.distributionZones." + ZONE_NAME + "{dataStorage.name = " + ENGINE_NAME + ", zoneId = 1}")

Review Comment:
   Fixed.



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

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

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


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -1343,72 +1355,132 @@ private Set<Assignment> calculateAssignments(TableConfiguration tableCfg, int pa
      *         </ul>
      * @see TableAlreadyExistsException
      */
-    public CompletableFuture<Table> createTableAsync(String name, Consumer<TableChange> tableInitChange) {
+    public CompletableFuture<Table> createTableAsync(String name, String zoneName, Consumer<TableChange> tableInitChange) {
         if (!busyLock.enterBusy()) {
             throw new IgniteException(new NodeStoppingException());
         }
         try {
-            return createTableAsyncInternal(name, tableInitChange);
+            return createTableAsyncInternal(name, zoneName, tableInitChange);
         } finally {
             busyLock.leaveBusy();
         }
     }
 
-    /** See {@link #createTableAsync(String, Consumer)} for details. */
-    private CompletableFuture<Table> createTableAsyncInternal(String name, Consumer<TableChange> tableInitChange) {
+    /** See {@link #createTableAsync(String, String, Consumer)} for details. */
+    private CompletableFuture<Table> createTableAsyncInternal(
+            String name,
+            String zoneName,
+            Consumer<TableChange> tableInitChange
+    ) {
         CompletableFuture<Table> tblFut = new CompletableFuture<>();
+        tableAsyncInternal(name)
+                .thenAccept(tbl -> {
+                    if (tbl != null) {
+                        tblFut.completeExceptionally(new TableAlreadyExistsException(DEFAULT_SCHEMA_NAME, name));
+                    } else {
+                        distributionZoneManager.zoneIdAsyncInternal(zoneName).handle((zoneId, ex) -> {

Review Comment:
   1. DistributionZoneAwaitDataNodesTest - tests for topologyVersionedDataNodes.
   2. MockedStructuresTest#testCreateTableWithDistributionZone - I added additional asserts for exception handling.
   3. DistributionZoneMockTest#getNonExistingZoneFromDirectProxy - a test for null handling.
   4. ItRebalanceDistributedTest#testOnLeaderElectedRebalanceRestart - tests that the distribution zone created on node0 is available on node1.
   5. All other tests for a table creation.



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

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

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


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -161,6 +171,17 @@ public class DistributionZoneManager implements IgniteComponent {
     /** The logger. */
     private static final IgniteLogger LOG = Loggers.forClass(DistributionZoneManager.class);
 
+    /**
+     * If this property is set to {@code true} then an attempt to get the configuration property directly from Meta storage will be skipped,
+     * and the local property will be returned.
+     * TODO: IGNITE-16774 This property and overall approach, access configuration directly through Meta storage,
+     * TODO: will be removed after fix of the issue.
+     */
+    private final boolean getMetadataLocallyOnly = IgniteSystemProperties.getBoolean("IGNITE_GET_METADATA_LOCALLY_ONLY");
+
+    /** Versioned store for zones id. */
+    private final IncrementalVersionedValue<Set<Integer>> zonesByIdVv;

Review Comment:
   Thanks. I've removed VersiondValue and used ConfigurationNamedListListener to await the zone is created locally. I discussed this approach with Ivan Bessonov and he confirmed that it is possible
   to invoke NamedConfigurationTree#stopListenElements in the listener in this case.



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

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

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


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -1343,72 +1355,132 @@ private Set<Assignment> calculateAssignments(TableConfiguration tableCfg, int pa
      *         </ul>
      * @see TableAlreadyExistsException
      */
-    public CompletableFuture<Table> createTableAsync(String name, Consumer<TableChange> tableInitChange) {
+    public CompletableFuture<Table> createTableAsync(String name, String zoneName, Consumer<TableChange> tableInitChange) {
         if (!busyLock.enterBusy()) {
             throw new IgniteException(new NodeStoppingException());
         }
         try {
-            return createTableAsyncInternal(name, tableInitChange);
+            return createTableAsyncInternal(name, zoneName, tableInitChange);
         } finally {
             busyLock.leaveBusy();
         }
     }
 
-    /** See {@link #createTableAsync(String, Consumer)} for details. */
-    private CompletableFuture<Table> createTableAsyncInternal(String name, Consumer<TableChange> tableInitChange) {
+    /** See {@link #createTableAsync(String, String, Consumer)} for details. */
+    private CompletableFuture<Table> createTableAsyncInternal(
+            String name,
+            String zoneName,
+            Consumer<TableChange> tableInitChange
+    ) {
         CompletableFuture<Table> tblFut = new CompletableFuture<>();
+        tableAsyncInternal(name)
+                .thenAccept(tbl -> {

Review Comment:
   Seems that busy locks are missing here and there, aren't they?



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

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

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


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerScaleUpTest.java:
##########
@@ -1366,10 +1379,10 @@ private void assertNotEqualsDataNodesForZone(int zoneId, @Nullable Set<String> c
         ));
     }
 
-    private void startDistributionZoneManager() throws NodeStoppingException {
-        distributionZoneManager.start();
+    private void startDistributionZoneManager() throws Exception {
+        deployWatchesAndUpdateMetaStorageRevision(metaStorageManager);

Review Comment:
   And again, the order of deploying watches an starting dzm is probably incorrect.



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

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

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


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerTest.java:
##########
@@ -91,6 +91,7 @@ public void setUp() {
                 null,
                 null,
                 null,
+                x -> {},

Review Comment:
   We won't need this if we'll use listener instead of vv))



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

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

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


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -1343,72 +1355,132 @@ private Set<Assignment> calculateAssignments(TableConfiguration tableCfg, int pa
      *         </ul>
      * @see TableAlreadyExistsException
      */
-    public CompletableFuture<Table> createTableAsync(String name, Consumer<TableChange> tableInitChange) {
+    public CompletableFuture<Table> createTableAsync(String name, String zoneName, Consumer<TableChange> tableInitChange) {
         if (!busyLock.enterBusy()) {
             throw new IgniteException(new NodeStoppingException());
         }
         try {
-            return createTableAsyncInternal(name, tableInitChange);
+            return createTableAsyncInternal(name, zoneName, tableInitChange);
         } finally {
             busyLock.leaveBusy();
         }
     }
 
-    /** See {@link #createTableAsync(String, Consumer)} for details. */
-    private CompletableFuture<Table> createTableAsyncInternal(String name, Consumer<TableChange> tableInitChange) {
+    /** See {@link #createTableAsync(String, String, Consumer)} for details. */
+    private CompletableFuture<Table> createTableAsyncInternal(
+            String name,
+            String zoneName,
+            Consumer<TableChange> tableInitChange
+    ) {
         CompletableFuture<Table> tblFut = new CompletableFuture<>();
+        tableAsyncInternal(name)
+                .thenAccept(tbl -> {
+                    if (tbl != null) {
+                        tblFut.completeExceptionally(new TableAlreadyExistsException(DEFAULT_SCHEMA_NAME, name));
+                    } else {
+                        distributionZoneManager.zoneIdAsyncInternal(zoneName).handle((zoneId, ex) -> {

Review Comment:
   Tests in DistributionZoneAwaitDataNodesTest check it:
   testWithOutAwaiting - check that even when ScaleUp and ScaleDown equal to INFINITE_TIMER_VALUE we wait topology version.
   testAwaitingScaleUpOnly and testAwaitingScaleDownOnly - check that we await immediate timers but skip non-immediate ones.



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

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

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


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -1614,6 +1662,101 @@ CompletableFuture<Void> saveDataNodesToMetaStorageOnScaleDown(int zoneId, long r
         }
     }
 
+    /**
+     * Gets direct id of the distribution zone with {@code zoneName}.
+     *
+     * @param zoneName Name of the distribution zone.
+     * @return Direct id of the distribution zone, or {@code null} if the zone with the {@code zoneName} has not been found.
+     */
+    public CompletableFuture<Integer> zoneIdAsyncInternal(String zoneName) {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(new NodeStoppingException());
+        }
+        try {
+            if (DEFAULT_ZONE_NAME.equals(zoneName)) {
+                return completedFuture(DEFAULT_ZONE_ID);
+            }
+
+            // TODO: IGNITE-16288 directZoneId should use async configuration API
+            return supplyAsync(() -> inBusyLock(busyLock, () -> directZoneIdInternal(zoneName)), executor)
+                    .thenCompose(zoneId -> waitZoneIdLocally(zoneId).thenCompose(ignored -> completedFuture(zoneId)));
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    @Nullable
+    private Integer directZoneIdInternal(String zoneName) {
+        try {
+            DistributionZoneConfiguration zoneCfg = directProxy(zonesConfiguration.distributionZones()).get(zoneName);
+
+            if (zoneCfg == null) {
+                return null;
+            } else {
+                return zoneCfg.zoneId().value();
+            }
+        } catch (NoSuchElementException e) {
+            return null;
+        }
+    }
+
+    /**
+     * Internal method for waiting that the zone is created locally.
+     *
+     * @param id Table id.

Review Comment:
   Fixed.



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

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

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


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/configuration/DistributionZoneConfigurationSchema.java:
##########
@@ -68,12 +69,12 @@ public class DistributionZoneConfigurationSchema {
     /** Timeout in seconds between node added topology event itself and data nodes switch. */
     @Range(min = 0)
     @Value(hasDefault = true)
-    public int dataNodesAutoAdjustScaleUp = INFINITE_TIMER_VALUE;
+    public int dataNodesAutoAdjustScaleUp = IMMEDIATE_TIMER_VALUE;
 
     /** Timeout in seconds between node left topology event itself and data nodes switch. */
     @Range(min = 0)
     @Value(hasDefault = true)
-    public int dataNodesAutoAdjustScaleDown = INFINITE_TIMER_VALUE;
+    public int dataNodesAutoAdjustScaleDown = IMMEDIATE_TIMER_VALUE;

Review Comment:
   Scale down should still be infinite by default. And we should adjust this value in test where needed.



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

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

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


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/MockedStructuresTest.java:
##########
@@ -162,13 +177,15 @@ public class MockedStructuresTest extends IgniteAbstractTest {
     @InjectConfiguration
     private TablesConfiguration tblsCfg;
 
-    @InjectConfiguration("mock.distributionZones.zone123{dataStorage.name = " + ENGINE_NAME + ", zoneId = 1}")
+    @InjectConfiguration("mock.distributionZones." + ZONE_NAME + "{dataStorage.name = " + ENGINE_NAME + ", zoneId = 1}")

Review Comment:
   Should we also use ZONE_ID here?



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

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

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


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneManagerConfigurationChangesTest.java:
##########
@@ -120,21 +125,28 @@ public void setUp() throws Exception {
 
         metaStorageManager.put(zonesLogicalTopologyVersionKey(), longToBytes(0));
 
+        ConfigurationRegistry clusterConfigRegistry = clusterCfgMgr.configurationRegistry();
+
+        Consumer<Function<Long, CompletableFuture<?>>> registry =
+                c -> clusterConfigRegistry.listenUpdateStorageRevision(c::apply);
+
         distributionZoneManager = new DistributionZoneManager(
                 zonesConfiguration,
                 tablesConfiguration,
                 metaStorageManager,
                 logicalTopologyService,
                 vaultMgr,
+                registry,
                 "test"
         );
 
         vaultMgr.start();
         clusterCfgMgr.start();
         metaStorageManager.start();
-        distributionZoneManager.start();
 
-        metaStorageManager.deployWatches();
+        deployWatchesAndUpdateMetaStorageRevision(metaStorageManager);

Review Comment:
   Why do we deploy watches before starting the distributionZoneManager? It seems incorrect, because common practice is to deploy watches after all components start up.



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

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

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


[GitHub] [ignite-3] sanpwc commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -1343,72 +1355,132 @@ private Set<Assignment> calculateAssignments(TableConfiguration tableCfg, int pa
      *         </ul>
      * @see TableAlreadyExistsException
      */
-    public CompletableFuture<Table> createTableAsync(String name, Consumer<TableChange> tableInitChange) {
+    public CompletableFuture<Table> createTableAsync(String name, String zoneName, Consumer<TableChange> tableInitChange) {
         if (!busyLock.enterBusy()) {
             throw new IgniteException(new NodeStoppingException());
         }
         try {
-            return createTableAsyncInternal(name, tableInitChange);
+            return createTableAsyncInternal(name, zoneName, tableInitChange);
         } finally {
             busyLock.leaveBusy();
         }
     }
 
-    /** See {@link #createTableAsync(String, Consumer)} for details. */
-    private CompletableFuture<Table> createTableAsyncInternal(String name, Consumer<TableChange> tableInitChange) {
+    /** See {@link #createTableAsync(String, String, Consumer)} for details. */
+    private CompletableFuture<Table> createTableAsyncInternal(
+            String name,
+            String zoneName,
+            Consumer<TableChange> tableInitChange
+    ) {
         CompletableFuture<Table> tblFut = new CompletableFuture<>();
+        tableAsyncInternal(name)
+                .thenAccept(tbl -> {
+                    if (tbl != null) {
+                        tblFut.completeExceptionally(new TableAlreadyExistsException(DEFAULT_SCHEMA_NAME, name));
+                    } else {
+                        distributionZoneManager.zoneIdAsyncInternal(zoneName).handle((zoneId, ex) -> {

Review Comment:
   Could you please name the tests that check table await logic?



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

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

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


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java:
##########
@@ -994,9 +997,9 @@ public void testOneNodeRestartWithGap() throws InterruptedException {
 
         createTableWithoutData(ignite, TABLE_NAME_2, 1, 1);
 
-        components = startPartialNode(1, null);
+        IgniteImpl ignite1 = startNode(1);

Review Comment:
   Added TODO https://issues.apache.org/jira/browse/IGNITE-19408



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

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

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


[GitHub] [ignite-3] sergeyuttsel commented on a diff in pull request #1968: IGNITE-18624 Data nodes from DistributionZoneManager instead of BaselineManager#nodes on table creation.

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


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java:
##########
@@ -859,7 +885,17 @@ public CompletableFuture<?> onCreate(ConfigurationNotificationEvent<Distribution
 
             saveDataNodesAndUpdateTriggerKeysInMetaStorage(zoneId, ctx.storageRevision(), logicalTopology);
 
-            return completedFuture(null);
+            return zonesByIdVv.update(ctx.storageRevision(), (zones, e) -> {
+                if (e != null) {
+                    return failedFuture(e);
+                }
+
+                HashSet<Integer> newZones = new HashSet<>(zones);

Review Comment:
   I've removed VV and this code.



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