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

[GitHub] [ignite-3] vldpyatkov commented on a diff in pull request #2070: IGNITE-19405 Remove assignments from table configuration

vldpyatkov commented on code in PR #2070:
URL: https://github.com/apache/ignite-3/pull/2070#discussion_r1210783894


##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/DistributionZoneRebalanceEngine.java:
##########
@@ -187,39 +187,47 @@ public CompletableFuture<Void> onUpdate(WatchEvent evt) {
                         if (zoneId == tableZoneId) {
                             TableConfiguration tableCfg = tables.get(tableView.name());
 
-                            byte[] assignmentsBytes = ((ExtendedTableConfiguration) tableCfg).assignments().value();
+                            int tableId = tableCfg.id().value();
 
-                            List<Set<Assignment>> tableAssignments = assignmentsBytes == null
-                                    ? Collections.emptyList()
-                                    : ByteUtils.fromBytes(assignmentsBytes);
+                            CompletableFuture<List<Set<Assignment>>> tableAssignmentsFut = tableAssignments(
+                                    metaStorageManager,
+                                    tableId, distributionZoneConfiguration.partitions().value());
 
-                            for (int part = 0; part < distributionZoneConfiguration.partitions().value(); part++) {
-                                int tableId = tableCfg.id().value();
+                            tableAssignmentsFut.thenAccept(tbls -> {

Review Comment:
   Why do you call assignments tbls?



##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/DistributionZoneRebalanceEngine.java:
##########
@@ -187,39 +187,47 @@ public CompletableFuture<Void> onUpdate(WatchEvent evt) {
                         if (zoneId == tableZoneId) {
                             TableConfiguration tableCfg = tables.get(tableView.name());
 
-                            byte[] assignmentsBytes = ((ExtendedTableConfiguration) tableCfg).assignments().value();
+                            int tableId = tableCfg.id().value();
 
-                            List<Set<Assignment>> tableAssignments = assignmentsBytes == null
-                                    ? Collections.emptyList()
-                                    : ByteUtils.fromBytes(assignmentsBytes);
+                            CompletableFuture<List<Set<Assignment>>> tableAssignmentsFut = tableAssignments(
+                                    metaStorageManager,
+                                    tableId, distributionZoneConfiguration.partitions().value());
 
-                            for (int part = 0; part < distributionZoneConfiguration.partitions().value(); part++) {
-                                int tableId = tableCfg.id().value();
+                            tableAssignmentsFut.thenAccept(tbls -> {
 
-                                TablePartitionId replicaGrpId = new TablePartitionId(tableId, part);
+                                for (int part = 0; part < distributionZoneConfiguration.partitions().value(); part++) {
 
-                                int replicas = distributionZoneConfiguration.replicas().value();
+                                    TablePartitionId replicaGrpId = new TablePartitionId(tableId, part);
 
-                                int partId = part;
+                                    int replicas = distributionZoneConfiguration.replicas().value();
 
-                                updatePendingAssignmentsKeys(
-                                        tableView.name(),
-                                        replicaGrpId,
-                                        filteredDataNodes,
-                                        replicas,
-                                        evt.entryEvent().newEntry().revision(),
-                                        metaStorageManager,
-                                        part,
-                                        tableAssignments.isEmpty() ? Collections.emptySet() : tableAssignments.get(part)
-                                ).exceptionally(e -> {
-                                    LOG.error(
-                                            "Exception on updating assignments for [table={}, partition={}]", e, tableView.name(),
-                                            partId
-                                    );
-
-                                    return null;
-                                });
-                            }
+                                    int partId = part;
+
+                                    updatePendingAssignmentsKeys(
+                                            tableView.name(),
+                                            replicaGrpId,
+                                            filteredDataNodes,
+                                            replicas,
+                                            evt.entryEvent().newEntry().revision(),
+                                            metaStorageManager,
+                                            partId,
+                                            tbls.isEmpty() ? Collections.emptySet() : tbls.get(partId)

Review Comment:
   This is a suspicious condition.
   What happens if one assignment is not created in MS?



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -574,63 +623,38 @@ private CompletableFuture<?> onTableDelete(ConfigurationNotificationEvent<TableV
         }
 
         try {
+            int partitions =
+                    getZoneById(distributionZonesConfiguration, ctx.oldValue().zoneId()).value().partitions();
+
             dropTableLocally(
                     ctx.storageRevision(),
                     ctx.oldValue().name(),
                     ctx.oldValue().id(),
-                    ByteUtils.fromBytes(ctx.oldValue(ExtendedTableView.class).assignments())
-            );
+                    partitions);
         } finally {
             busyLock.leaveBusy();
         }
 
         return completedFuture(null);
     }
 
-    /**
-     * Listener of assignment configuration changes.
-     *
-     * @param assignmentsCtx Assignment configuration context.
-     * @return A future.
-     */
-    private CompletableFuture<?> onUpdateAssignments(ConfigurationNotificationEvent<byte[]> assignmentsCtx) {
-        if (!busyLock.enterBusy()) {
-            return failedFuture(new NodeStoppingException());
-        }
-
-        try {
-            return updateAssignmentInternal(assignmentsCtx)
-                    .whenComplete((v, e) -> {
-                        if (e == null) {
-                            for (var listener : assignmentsChangeListeners) {
-                                listener.accept(this);
-                            }
-                        }
-                    });
-        } finally {
-            busyLock.leaveBusy();
-        }
-    }
-
     /**
      * Updates or creates partition raft groups.
      *
-     * @param assignmentsCtx Change assignment event.
+     * @param evt Change assignment event.

Review Comment:
   Please, describe all method parameters.



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