You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by am...@apache.org on 2022/12/21 10:47:50 UTC

[ignite-3] 05/06: wip. fix hanging on synchronous operation inside busylock in TableManager

This is an automated email from the ASF dual-hosted git repository.

amashenkov pushed a commit to branch ignite-18323
in repository https://gitbox.apache.org/repos/asf/ignite-3.git

commit 58d7d2ad4d7297f14cbadf5eda8416f5b79e3851
Author: amashenkov <an...@gmail.com>
AuthorDate: Tue Dec 20 15:31:20 2022 +0300

    wip. fix hanging on synchronous operation inside busylock in TableManager
---
 .../internal/table/distributed/TableManager.java   | 28 ++++++++++++++++------
 1 file changed, 21 insertions(+), 7 deletions(-)

diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
index ec47e486d5..946c5b5e39 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
@@ -634,17 +634,31 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
             return failedFuture(new NodeStoppingException());
         }
 
+        CompletableFuture<?> fut;
+
         try {
-            updateAssignmentInternal(assignmentsCtx);
+            //TODO: Fix synchronous Future.get under busy lock.
+            fut = updateAssignmentInternal(assignmentsCtx);
         } finally {
             busyLock.leaveBusy();
         }
 
-        for (var listener : assignmentsChangeListeners) {
-            listener.accept(this);
+        // TODO: Fix sync wait in FJP.
+        fut.join();
+
+        if (!busyLock.enterBusy()) {
+            return failedFuture(new NodeStoppingException());
         }
 
-        return completedFuture(null);
+        try {
+            for (var listener : assignmentsChangeListeners) {
+                listener.accept(this);
+            }
+        } finally {
+            busyLock.leaveBusy();
+        }
+
+        return fut;
     }
 
     /**
@@ -652,7 +666,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
      *
      * @param assignmentsCtx Change assignment event.
      */
-    private void updateAssignmentInternal(ConfigurationNotificationEvent<byte[]> assignmentsCtx) {
+    private CompletableFuture<?> updateAssignmentInternal(ConfigurationNotificationEvent<byte[]> assignmentsCtx) {
         ExtendedTableConfiguration tblCfg = assignmentsCtx.config(ExtendedTableConfiguration.class);
 
         UUID tblId = tblCfg.id().value();
@@ -677,7 +691,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
         String localMemberName = clusterService.topologyService().localMember().name();
 
         // Create new raft nodes according to new assignments.
-        tablesByIdVv.update(causalityToken, (tablesById, e) -> {
+        return tablesByIdVv.update(causalityToken, (tablesById, e) -> {
             if (e != null) {
                 return failedFuture(e);
             }
@@ -866,7 +880,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
             }
 
             return allOf(futures).thenApply(unused -> tablesById);
-        }).join();
+        });
     }
 
     private boolean isLocalPeer(Peer peer) {