sanpwc commented on code in PR #4769:
URL: https://github.com/apache/ignite-3/pull/4769#discussion_r1872759815


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/RebalanceMinimumRequiredTimeProviderImpl.java:
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.distributionzones.rebalance;
+
+import static java.lang.Math.min;
+import static java.util.Collections.emptyMap;
+import static java.util.stream.Collectors.toMap;
+import static 
org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.PENDING_ASSIGNMENTS_PREFIX_BYTES;
+import static 
org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.PENDING_CHANGE_TRIGGER_PREFIX_BYTES;
+import static 
org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.STABLE_ASSIGNMENTS_PREFIX_BYTES;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Set;
+import java.util.TreeMap;
+import org.apache.ignite.internal.catalog.Catalog;
+import org.apache.ignite.internal.catalog.CatalogService;
+import org.apache.ignite.internal.catalog.descriptors.CatalogTableDescriptor;
+import org.apache.ignite.internal.catalog.descriptors.CatalogZoneDescriptor;
+import org.apache.ignite.internal.lang.ByteArray;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.partitiondistribution.Assignments;
+import org.apache.ignite.internal.replicator.TablePartitionId;
+import org.apache.ignite.internal.util.ByteUtils;
+import org.apache.ignite.internal.util.Cursor;
+
+/**
+ * {@link RebalanceMinimumRequiredTimeProvider} implementation for the current 
implementation of assignments. Assumes that each table has
+ * its own assignments, but assignments within a zone are still somehow 
coordinated.
+ */
+public class RebalanceMinimumRequiredTimeProviderImpl implements 
RebalanceMinimumRequiredTimeProvider {
+    private final MetaStorageManager metaStorageManager;
+    private final CatalogService catalogService;
+
+    /**
+     * Constructor.
+     */
+    public RebalanceMinimumRequiredTimeProviderImpl(MetaStorageManager 
metaStorageManager, CatalogService catalogService) {
+        this.metaStorageManager = metaStorageManager;
+        this.catalogService = catalogService;
+    }
+
+    @Override
+    public long minimumRequiredTime() {
+        // Use the same revision to read all the data, in order to guarantee 
consistency of data.
+        long appliedRevision = metaStorageManager.appliedRevision();
+
+        // Ignore the real safe time, having a time associated with revision 
is enough. Also, acquiring real safe time would be
+        // unnecessarily more complicated due to handling of possible data 
races.
+        long metaStorageSafeTime = 
metaStorageManager.timestampByRevisionLocally(appliedRevision).longValue();
+
+        long minTimestamp = metaStorageSafeTime;
+
+        Map<Integer, Map<Integer, Assignments>> stableAssignments = 
readAssignments(STABLE_ASSIGNMENTS_PREFIX_BYTES, appliedRevision);
+        Map<Integer, Map<Integer, Assignments>> pendingAssignments = 
readAssignments(PENDING_ASSIGNMENTS_PREFIX_BYTES, appliedRevision);
+
+        Map<Integer, Long> pendingChangeTriggerRevisions = 
readPendingChangeTriggerRevisions(
+                PENDING_CHANGE_TRIGGER_PREFIX_BYTES,
+                appliedRevision
+        );
+
+        int earliestCatalogVersion = catalogService.earliestCatalogVersion();
+        int latestCatalogVersion = catalogService.latestCatalogVersion();
+
+        Map<Integer, Integer> tableIdToZoneIdMap = 
tableIdToZoneIdMap(earliestCatalogVersion, latestCatalogVersion);
+        Map<Integer, NavigableMap<Long, CatalogZoneDescriptor>> 
allZonesByTimestamp = allZonesByTimestamp(
+                earliestCatalogVersion,
+                latestCatalogVersion
+        );
+        Map<Integer, NavigableMap<Long, CatalogZoneDescriptor>> 
allZonesByRevision = allZonesByRevision(allZonesByTimestamp);
+        Map<Integer, Long> zoneDeletionTimestamps = 
zoneDeletionTimestamps(earliestCatalogVersion, latestCatalogVersion);
+
+        for (Map.Entry<Integer, Integer> entry : 
tableIdToZoneIdMap.entrySet()) {
+            Integer tableId = entry.getKey();
+            Integer zoneId = entry.getValue();
+
+            NavigableMap<Long, CatalogZoneDescriptor> zoneDescriptors = 
allZonesByTimestamp.get(zoneId);
+            int zonePartitions = 
zoneDescriptors.lastEntry().getValue().partitions();
+
+            Long pendingChangeTriggerKey = 
pendingChangeTriggerRevisions.get(tableId);

Review Comment:
   pendingChangeTriggerKey**Revision**?



##########
modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/rebalance/RebalanceMinimumRequiredTimeProviderImplTest.java:
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.distributionzones.rebalance;
+
+import static 
org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCondition;
+import static 
org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.lessThan;
+import static org.hamcrest.Matchers.lessThanOrEqualTo;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import org.apache.ignite.catalog.annotations.Table;
+import org.apache.ignite.internal.catalog.Catalog;
+import org.apache.ignite.internal.catalog.commands.ColumnParams;
+import org.apache.ignite.internal.catalog.commands.CreateTableCommand;
+import org.apache.ignite.internal.catalog.commands.DropTableCommand;
+import org.apache.ignite.internal.catalog.commands.TableHashPrimaryKey;
+import org.apache.ignite.internal.catalog.descriptors.CatalogTableDescriptor;
+import org.apache.ignite.internal.catalog.descriptors.CatalogZoneDescriptor;
+import 
org.apache.ignite.internal.distributionzones.BaseDistributionZoneManagerTest;
+import org.apache.ignite.internal.lang.ByteArray;
+import org.apache.ignite.internal.partitiondistribution.Assignment;
+import org.apache.ignite.internal.partitiondistribution.Assignments;
+import org.apache.ignite.internal.replicator.TablePartitionId;
+import org.apache.ignite.internal.util.ByteUtils;
+import org.apache.ignite.internal.util.CollectionUtils;
+import org.apache.ignite.sql.ColumnType;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Test for {@link RebalanceMinimumRequiredTimeProviderImpl}.
+ */
+class RebalanceMinimumRequiredTimeProviderImplTest extends 
BaseDistributionZoneManagerTest {
+    private static final String TABLE_NAME = "tableName";
+
+    private RebalanceMinimumRequiredTimeProvider minimumRequiredTimeProvider;
+
+    @BeforeEach
+    void initProvider() {
+        minimumRequiredTimeProvider = new 
RebalanceMinimumRequiredTimeProviderImpl(metaStorageManager, catalogManager);
+    }
+
+    private long getMinimumRequiredTime() {
+        long minimumRequiredTime = 
minimumRequiredTimeProvider.minimumRequiredTime();
+
+        assertThat(minimumRequiredTime, 
is(lessThanOrEqualTo(currentSafeTime())));
+
+        return minimumRequiredTime;
+    }
+
+    /**
+     * "No tables" scenario means that no tables exist in zone. In this case, 
we should return safe time of latest meta-storage revision.
+     */
+    @Test
+    void testNoTables() throws Exception {
+        startDistributionZoneManager();
+
+        // Wait for default zone update to be applied.
+        assertTrue(waitForCondition(() -> metaStorageManager.appliedRevision() 
> 0, 1000L));
+
+        long timeBefore = currentSafeTime();
+
+        long minimumRequiredTime = 
minimumRequiredTimeProvider.minimumRequiredTime();
+
+        long timeAfter = currentSafeTime();
+
+        // This check is fine, because there's no idle safe time propagation 
happening in these tests. Otherwise this check would fail
+        // randomly: provider's time corresponds to a revision, which might 
actually be earlier than the safe time we read.
+        assertThat(timeBefore, is(lessThanOrEqualTo(minimumRequiredTime)));
+
+        assertThat(minimumRequiredTime, is(lessThanOrEqualTo(timeAfter)));
+    }
+
+    /**
+     * Scenario where table exists, but assignments are not yet saved. In this 
case, we should return safe time that corresponds to earliest
+     * zone version.
+     */
+    @Test
+    void testNoAssignments() throws Exception {
+        startDistributionZoneManager();
+
+        String defaultZoneName = getDefaultZone().name();
+
+        createTable(defaultZoneName);
+
+        Catalog earliestCatalog = latestCatalogVersion();
+        alterZone(defaultZoneName, 0, 0, null);
+        Catalog latestCatalog = latestCatalogVersion();
+
+        long minimumRequiredTime = getMinimumRequiredTime();
+
+        assertThat(earliestCatalog.time(), 
is(lessThanOrEqualTo(minimumRequiredTime)));
+        assertThat(minimumRequiredTime, is(lessThan(latestCatalog.time())));
+    }
+
+    /**
+     * Scenario where table exists, its initial assignments are saved, and we 
updated the zone after that. In this case, we should return
+     * time, that corresponds to zone before its update.
+     */
+    @Test
+    void testOldStableAssignments() throws Exception {

Review Comment:
   I'd extend the test a bit by having several zone updates.



##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/RebalanceMinimumRequiredTimeProviderImpl.java:
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.distributionzones.rebalance;
+
+import static java.lang.Math.min;
+import static java.util.Collections.emptyMap;
+import static java.util.stream.Collectors.toMap;
+import static 
org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.PENDING_ASSIGNMENTS_PREFIX_BYTES;
+import static 
org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.PENDING_CHANGE_TRIGGER_PREFIX_BYTES;
+import static 
org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.STABLE_ASSIGNMENTS_PREFIX_BYTES;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Set;
+import java.util.TreeMap;
+import org.apache.ignite.internal.catalog.Catalog;
+import org.apache.ignite.internal.catalog.CatalogService;
+import org.apache.ignite.internal.catalog.descriptors.CatalogTableDescriptor;
+import org.apache.ignite.internal.catalog.descriptors.CatalogZoneDescriptor;
+import org.apache.ignite.internal.lang.ByteArray;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.partitiondistribution.Assignments;
+import org.apache.ignite.internal.replicator.TablePartitionId;
+import org.apache.ignite.internal.util.ByteUtils;
+import org.apache.ignite.internal.util.Cursor;
+
+/**
+ * {@link RebalanceMinimumRequiredTimeProvider} implementation for the current 
implementation of assignments. Assumes that each table has
+ * its own assignments, but assignments within a zone are still somehow 
coordinated.
+ */
+public class RebalanceMinimumRequiredTimeProviderImpl implements 
RebalanceMinimumRequiredTimeProvider {
+    private final MetaStorageManager metaStorageManager;
+    private final CatalogService catalogService;
+
+    /**
+     * Constructor.
+     */
+    public RebalanceMinimumRequiredTimeProviderImpl(MetaStorageManager 
metaStorageManager, CatalogService catalogService) {
+        this.metaStorageManager = metaStorageManager;
+        this.catalogService = catalogService;
+    }
+
+    @Override
+    public long minimumRequiredTime() {
+        // Use the same revision to read all the data, in order to guarantee 
consistency of data.
+        long appliedRevision = metaStorageManager.appliedRevision();
+
+        // Ignore the real safe time, having a time associated with revision 
is enough. Also, acquiring real safe time would be
+        // unnecessarily more complicated due to handling of possible data 
races.
+        long metaStorageSafeTime = 
metaStorageManager.timestampByRevisionLocally(appliedRevision).longValue();
+
+        long minTimestamp = metaStorageSafeTime;
+
+        Map<Integer, Map<Integer, Assignments>> stableAssignments = 
readAssignments(STABLE_ASSIGNMENTS_PREFIX_BYTES, appliedRevision);
+        Map<Integer, Map<Integer, Assignments>> pendingAssignments = 
readAssignments(PENDING_ASSIGNMENTS_PREFIX_BYTES, appliedRevision);
+
+        Map<Integer, Long> pendingChangeTriggerRevisions = 
readPendingChangeTriggerRevisions(
+                PENDING_CHANGE_TRIGGER_PREFIX_BYTES,
+                appliedRevision
+        );
+
+        int earliestCatalogVersion = catalogService.earliestCatalogVersion();
+        int latestCatalogVersion = catalogService.latestCatalogVersion();
+
+        Map<Integer, Integer> tableIdToZoneIdMap = 
tableIdToZoneIdMap(earliestCatalogVersion, latestCatalogVersion);
+        Map<Integer, NavigableMap<Long, CatalogZoneDescriptor>> 
allZonesByTimestamp = allZonesByTimestamp(
+                earliestCatalogVersion,
+                latestCatalogVersion
+        );
+        Map<Integer, NavigableMap<Long, CatalogZoneDescriptor>> 
allZonesByRevision = allZonesByRevision(allZonesByTimestamp);
+        Map<Integer, Long> zoneDeletionTimestamps = 
zoneDeletionTimestamps(earliestCatalogVersion, latestCatalogVersion);
+
+        for (Map.Entry<Integer, Integer> entry : 
tableIdToZoneIdMap.entrySet()) {
+            Integer tableId = entry.getKey();
+            Integer zoneId = entry.getValue();
+
+            NavigableMap<Long, CatalogZoneDescriptor> zoneDescriptors = 
allZonesByTimestamp.get(zoneId);
+            int zonePartitions = 
zoneDescriptors.lastEntry().getValue().partitions();
+
+            Long pendingChangeTriggerKey = 
pendingChangeTriggerRevisions.get(tableId);
+
+            // +-1 here ir required for 2 reasons:
+            // - we need timestamp right before deletion, if zone is deleted, 
thus we must subtract 1;
+            // - we need a "metaStorageSafeTime" if zone is not deleted, 
without any subtractions.
+            long latestTimestamp = zoneDeletionTimestamps.getOrDefault(zoneId, 
metaStorageSafeTime + 1) - 1;
+
+            long zoneRevision = pendingChangeTriggerKey == null

Review Comment:
   Let's say that zone was created on revision 10, and updated on revision 12. 
pendingChangeTriggerKey is null.
   What will be the value of 'zoneRevision'?



##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/RebalanceMinimumRequiredTimeProviderImpl.java:
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.distributionzones.rebalance;
+
+import static java.lang.Math.min;
+import static java.util.Collections.emptyMap;
+import static java.util.stream.Collectors.toMap;
+import static 
org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.PENDING_ASSIGNMENTS_PREFIX_BYTES;
+import static 
org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.PENDING_CHANGE_TRIGGER_PREFIX_BYTES;
+import static 
org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.STABLE_ASSIGNMENTS_PREFIX_BYTES;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Set;
+import java.util.TreeMap;
+import org.apache.ignite.internal.catalog.Catalog;
+import org.apache.ignite.internal.catalog.CatalogService;
+import org.apache.ignite.internal.catalog.descriptors.CatalogTableDescriptor;
+import org.apache.ignite.internal.catalog.descriptors.CatalogZoneDescriptor;
+import org.apache.ignite.internal.lang.ByteArray;
+import org.apache.ignite.internal.metastorage.Entry;
+import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.partitiondistribution.Assignments;
+import org.apache.ignite.internal.replicator.TablePartitionId;
+import org.apache.ignite.internal.util.ByteUtils;
+import org.apache.ignite.internal.util.Cursor;
+
+/**
+ * {@link RebalanceMinimumRequiredTimeProvider} implementation for the current 
implementation of assignments. Assumes that each table has
+ * its own assignments, but assignments within a zone are still somehow 
coordinated.
+ */
+public class RebalanceMinimumRequiredTimeProviderImpl implements 
RebalanceMinimumRequiredTimeProvider {
+    private final MetaStorageManager metaStorageManager;
+    private final CatalogService catalogService;
+
+    /**
+     * Constructor.
+     */
+    public RebalanceMinimumRequiredTimeProviderImpl(MetaStorageManager 
metaStorageManager, CatalogService catalogService) {
+        this.metaStorageManager = metaStorageManager;
+        this.catalogService = catalogService;
+    }
+
+    @Override
+    public long minimumRequiredTime() {
+        // Use the same revision to read all the data, in order to guarantee 
consistency of data.
+        long appliedRevision = metaStorageManager.appliedRevision();
+
+        // Ignore the real safe time, having a time associated with revision 
is enough. Also, acquiring real safe time would be
+        // unnecessarily more complicated due to handling of possible data 
races.
+        long metaStorageSafeTime = 
metaStorageManager.timestampByRevisionLocally(appliedRevision).longValue();
+
+        long minTimestamp = metaStorageSafeTime;
+
+        Map<Integer, Map<Integer, Assignments>> stableAssignments = 
readAssignments(STABLE_ASSIGNMENTS_PREFIX_BYTES, appliedRevision);
+        Map<Integer, Map<Integer, Assignments>> pendingAssignments = 
readAssignments(PENDING_ASSIGNMENTS_PREFIX_BYTES, appliedRevision);
+
+        Map<Integer, Long> pendingChangeTriggerRevisions = 
readPendingChangeTriggerRevisions(
+                PENDING_CHANGE_TRIGGER_PREFIX_BYTES,
+                appliedRevision
+        );
+
+        int earliestCatalogVersion = catalogService.earliestCatalogVersion();
+        int latestCatalogVersion = catalogService.latestCatalogVersion();
+
+        Map<Integer, Integer> tableIdToZoneIdMap = 
tableIdToZoneIdMap(earliestCatalogVersion, latestCatalogVersion);
+        Map<Integer, NavigableMap<Long, CatalogZoneDescriptor>> 
allZonesByTimestamp = allZonesByTimestamp(
+                earliestCatalogVersion,
+                latestCatalogVersion
+        );
+        Map<Integer, NavigableMap<Long, CatalogZoneDescriptor>> 
allZonesByRevision = allZonesByRevision(allZonesByTimestamp);
+        Map<Integer, Long> zoneDeletionTimestamps = 
zoneDeletionTimestamps(earliestCatalogVersion, latestCatalogVersion);
+
+        for (Map.Entry<Integer, Integer> entry : 
tableIdToZoneIdMap.entrySet()) {
+            Integer tableId = entry.getKey();
+            Integer zoneId = entry.getValue();
+
+            NavigableMap<Long, CatalogZoneDescriptor> zoneDescriptors = 
allZonesByTimestamp.get(zoneId);
+            int zonePartitions = 
zoneDescriptors.lastEntry().getValue().partitions();
+
+            Long pendingChangeTriggerKey = 
pendingChangeTriggerRevisions.get(tableId);
+
+            // +-1 here ir required for 2 reasons:
+            // - we need timestamp right before deletion, if zone is deleted, 
thus we must subtract 1;
+            // - we need a "metaStorageSafeTime" if zone is not deleted, 
without any subtractions.
+            long latestTimestamp = zoneDeletionTimestamps.getOrDefault(zoneId, 
metaStorageSafeTime + 1) - 1;
+
+            long zoneRevision = pendingChangeTriggerKey == null
+                    ? zoneDescriptors.firstEntry().getValue().updateToken()
+                    : pendingChangeTriggerKey;
+
+            NavigableMap<Long, CatalogZoneDescriptor> map = 
allZonesByRevision.get(zoneId);
+            Map.Entry<Long, CatalogZoneDescriptor> zone = 
map.floorEntry(zoneRevision);
+            long timestamp = 
metaStorageManager.timestampByRevisionLocally(zone.getValue().updateToken()).longValue();
+
+            timestamp = ceilTime(zoneDescriptors, timestamp, latestTimestamp);

Review Comment:
   Hmm, Let's say that we have
   
   - metaStorageSafeTime = t100.
   - table with stable assignments only and thus pendingChangeTriggerKey == 
null.
   - zone that was created at revision 10 (t10) and updated on revisions 
20(t20) and 30 (t30).
   - What will be the value of `minTimestamp = min(minTimestamp, timestamp);`?
   



-- 
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-unsubscr...@ignite.apache.org

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

Reply via email to