You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "Michael Blow (Code Review)" <de...@asterixdb.apache.org> on 2019/05/13 22:52:55 UTC

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Michael Blow has uploaded this change for review. ( https://asterix-gerrit.ics.uci.edu/3394


Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................

[NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Allow extensions to mandate that a rebalance is required in order for
the cluster to go active

Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
---
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterManagementWork.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
3 files changed, 55 insertions(+), 22 deletions(-)



  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/94/3394/1

diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterManagementWork.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterManagementWork.java
index c2d3303..e13756c 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterManagementWork.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterManagementWork.java
@@ -30,8 +30,8 @@
         PENDING, // the metadata node has not yet joined & initialized
         RECOVERING, // global recovery has not yet completed
         ACTIVE, // cluster is ACTIVE and ready for requests
-        REBALANCING, // replication is processing failbacks
-        SHUTTING_DOWN // a shutdown request has been received, and is underway
+        SHUTTING_DOWN, // a shutdown request has been received, and is underway
+        REBALANCE_REQUIRED // one or more datasets require rebalance before the cluster is usable
     }
 
     WorkType getClusterManagementWorkType();
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java
index 0e62851..6c39372 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java
@@ -21,6 +21,7 @@
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import java.util.function.Predicate;
 
 import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
@@ -107,6 +108,14 @@
      * @return true if the desired state was reached before timeout occurred
      */
     boolean waitForState(ClusterState waitForState, long timeout, TimeUnit unit)
+            throws HyracksDataException, InterruptedException;
+
+    /**
+     * Blocks until the cluster state matches supplied predicate, or timeout is exhausted.
+     *
+     * @return the cluster state matching the predicate if it was satisfied before timeout occurred, otherwise null
+     */
+    ClusterState waitForState(Predicate<ClusterState> condition, long timeout, TimeUnit unit)
             throws HyracksDataException, InterruptedException;
 
     /**
@@ -250,4 +259,10 @@
      * @return The metadata cluster partitions
      */
     ClusterPartition getMetadataPartition();
+
+    /**
+     * Indicate whether one or more datasets must be rebalanced before the cluster becomes ACTIVE
+     * @param rebalanceRequired
+     */
+    void setRebalanceRequired(boolean rebalanceRequired) throws HyracksDataException;
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
index 16a479e..7933cd2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
@@ -28,6 +28,7 @@
 import java.util.SortedMap;
 import java.util.TreeSet;
 import java.util.concurrent.TimeUnit;
+import java.util.function.Predicate;
 
 import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
 import org.apache.asterix.common.cluster.ClusterPartition;
@@ -75,6 +76,7 @@
     private INcLifecycleCoordinator lifecycleCoordinator;
     private ICcApplicationContext appCtx;
     private ClusterPartition metadataPartition;
+    private boolean rebalanceRequired;
 
     @Override
     public void setCcAppCtx(ICcApplicationContext appCtx) {
@@ -186,45 +188,55 @@
             return;
         }
         // the metadata bootstrap & global recovery must be complete before the cluster can be active
-        if (metadataNodeActive) {
-            if (state != ClusterState.ACTIVE && state != ClusterState.RECOVERING) {
-                setState(ClusterState.PENDING);
-            }
-            appCtx.getMetadataBootstrap().init();
-
-            if (appCtx.getGlobalRecoveryManager().isRecoveryCompleted()) {
-                setState(ClusterState.ACTIVE);
-            } else {
-                // start global recovery
-                setState(ClusterState.RECOVERING);
-                appCtx.getGlobalRecoveryManager().startGlobalRecovery(appCtx);
-            }
-        } else {
+        if (!metadataNodeActive) {
+            setState(ClusterState.PENDING);
+            return;
+        }
+        if (state != ClusterState.ACTIVE && state != ClusterState.RECOVERING) {
             setState(ClusterState.PENDING);
         }
+        appCtx.getMetadataBootstrap().init();
+
+        if (!appCtx.getGlobalRecoveryManager().isRecoveryCompleted()) {
+            // start global recovery
+            setState(ClusterState.RECOVERING);
+            appCtx.getGlobalRecoveryManager().startGlobalRecovery(appCtx);
+            return;
+        }
+        if (rebalanceRequired) {
+            setState(ClusterState.REBALANCE_REQUIRED);
+            return;
+        }
+        // finally- life is good, set the state to ACTIVE
+        setState(ClusterState.ACTIVE);
     }
 
     @Override
-    public synchronized void waitForState(ClusterState waitForState) throws HyracksDataException, InterruptedException {
+    public synchronized void waitForState(ClusterState waitForState) throws InterruptedException {
         while (state != waitForState) {
             wait();
         }
     }
 
     @Override
-    public synchronized boolean waitForState(ClusterState waitForState, long timeout, TimeUnit unit)
-            throws HyracksDataException, InterruptedException {
+    public boolean waitForState(ClusterState waitForState, long timeout, TimeUnit unit) throws InterruptedException {
+        return waitForState(waitForState::equals, timeout, unit) != null;
+    }
+
+    @Override
+    public synchronized ClusterState waitForState(Predicate<ClusterState> predicate, long timeout, TimeUnit unit)
+            throws InterruptedException {
         final long startMillis = System.currentTimeMillis();
         final long endMillis = startMillis + unit.toMillis(timeout);
-        while (state != waitForState) {
+        while (!predicate.test(state)) {
             long millisToSleep = endMillis - System.currentTimeMillis();
             if (millisToSleep > 0) {
                 wait(millisToSleep);
             } else {
-                return false;
+                return null;
             }
         }
-        return true;
+        return state;
     }
 
     @Override
@@ -458,6 +470,12 @@
         return metadataPartition;
     }
 
+    @Override
+    public synchronized void setRebalanceRequired(boolean rebalanceRequired) throws HyracksDataException {
+        this.rebalanceRequired = rebalanceRequired;
+        refreshState();
+    }
+
     private void updateClusterCounters(String nodeId, NcLocalCounters localCounters) {
         final IResourceIdManager resourceIdManager = appCtx.getResourceIdManager();
         resourceIdManager.report(nodeId, localCounters.getMaxResourceId());

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: newchange
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Michael Blow (Code Review)" <de...@asterixdb.apache.org>.
Michael Blow has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................


Patch Set 1:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/3394/1/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
File asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java:

https://asterix-gerrit.ics.uci.edu/#/c/3394/1/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java@195
PS1, Line 195: if (state != ClusterState.ACTIVE && state != ClusterState.RECOVERING) {
             :             setState(ClusterState.PENDING);
             :         }
> I doubt this is of any use since any subscribers won't be able to get the c
we explicitly pass the new state to the, there's no need to get the lock in that case- but we could clean this up later...



-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Wed, 15 May 2019 20:25:16 +0000
Gerrit-HasComments: Yes

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................


Patch Set 1: Contrib+1

BAD Compatibility Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/4403/ : SUCCESS


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 13 May 2019 23:13:02 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/5768/ (15/16)


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 13 May 2019 22:53:07 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Michael Blow (Code Review)" <de...@asterixdb.apache.org>.
Michael Blow has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................


Patch Set 1:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/3394/1/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
File asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java:

https://asterix-gerrit.ics.uci.edu/#/c/3394/1/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java@195
PS1, Line 195: if (state != ClusterState.ACTIVE && state != ClusterState.RECOVERING) {
             :             setState(ClusterState.PENDING);
             :         }
> is this needed? looks like some unnecessary transient state that will be ch
This ensures that we always transition from UNUSABLE -> PENDING -> RECOVERING -> [REBALANCE_REQUIRED ->] ACTIVE in case of normal startup- this might be helpful for cluster event subscribers.



-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Wed, 15 May 2019 11:43:54 +0000
Gerrit-HasComments: Yes

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/1045/ (8/16)


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 13 May 2019 22:53:05 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/5937/ (13/16)


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 13 May 2019 22:53:06 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/5684/ (1/16)


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 13 May 2019 22:53:03 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/9727/ (9/16)


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 13 May 2019 22:53:05 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/3780/ (10/16)


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 13 May 2019 22:53:05 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Murtadha Hubail (Code Review)" <de...@asterixdb.apache.org>.
Murtadha Hubail has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................


Patch Set 1: Code-Review+2


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Wed, 15 May 2019 19:23:17 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/11260/ (4/16)


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 13 May 2019 22:53:04 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ssl-compression/550/ (3/16)


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 13 May 2019 22:53:03 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................


Patch Set 1:

BAD Compatibility Tests Started https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/4403/


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 13 May 2019 22:56:27 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Murtadha Hubail (Code Review)" <de...@asterixdb.apache.org>.
Murtadha Hubail has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................


Patch Set 1:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/3394/1/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
File asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java:

https://asterix-gerrit.ics.uci.edu/#/c/3394/1/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java@195
PS1, Line 195: if (state != ClusterState.ACTIVE && state != ClusterState.RECOVERING) {
             :             setState(ClusterState.PENDING);
             :         }
is this needed? looks like some unnecessary transient state that will be changed no matter what in the checks to follow



-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Wed, 15 May 2019 01:12:55 +0000
Gerrit-HasComments: Yes

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Anon. E. Moose (Code Review)" <de...@asterixdb.apache.org>.
Anon. E. Moose (1000171) has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................


Patch Set 1: Contrib-2

Analytics Compatibility Tests Failed
https://cbjenkins.page.link/A4UryXcQMzcAuJeL7 : UNSTABLE


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 14 May 2019 03:11:18 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................


Patch Set 1: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/8548/ : SUCCESS


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 14 May 2019 00:09:38 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/5991/ (6/16)


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 13 May 2019 22:53:04 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-openjdk11/1137/ (11/16)


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 13 May 2019 22:53:06 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/5748/ (5/16)


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 13 May 2019 22:53:04 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Anon. E. Moose (Code Review)" <de...@asterixdb.apache.org>.
Anon. E. Moose (1000171) has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................


Patch Set 1:

Analytics Compatibility Compilation Successful
https://cbjenkins.page.link/v8DmyWJgpuy56KVk7 : SUCCESS


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 13 May 2019 23:07:46 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-txnlog/947/ (12/16)


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 13 May 2019 22:53:06 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/5786/ (16/16)


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Tue, 14 May 2019 01:26:22 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Michael Blow (Code Review)" <de...@asterixdb.apache.org>.
Michael Blow has submitted this change and it was merged. ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................

[NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Allow extensions to mandate that a rebalance is required in order for
the cluster to go active

Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Reviewed-on: https://asterix-gerrit.ics.uci.edu/3394
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Murtadha Hubail <mh...@apache.org>
---
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterManagementWork.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
3 files changed, 55 insertions(+), 22 deletions(-)

Approvals:
  Jenkins: Verified; ; Verified
  Murtadha Hubail: Looks good to me, approved

Objections:
  Jenkins: Violations found



diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterManagementWork.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterManagementWork.java
index c2d3303..e13756c 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterManagementWork.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterManagementWork.java
@@ -30,8 +30,8 @@
         PENDING, // the metadata node has not yet joined & initialized
         RECOVERING, // global recovery has not yet completed
         ACTIVE, // cluster is ACTIVE and ready for requests
-        REBALANCING, // replication is processing failbacks
-        SHUTTING_DOWN // a shutdown request has been received, and is underway
+        SHUTTING_DOWN, // a shutdown request has been received, and is underway
+        REBALANCE_REQUIRED // one or more datasets require rebalance before the cluster is usable
     }
 
     WorkType getClusterManagementWorkType();
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java
index 0e62851..6c39372 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/cluster/IClusterStateManager.java
@@ -21,6 +21,7 @@
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import java.util.function.Predicate;
 
 import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
@@ -107,6 +108,14 @@
      * @return true if the desired state was reached before timeout occurred
      */
     boolean waitForState(ClusterState waitForState, long timeout, TimeUnit unit)
+            throws HyracksDataException, InterruptedException;
+
+    /**
+     * Blocks until the cluster state matches supplied predicate, or timeout is exhausted.
+     *
+     * @return the cluster state matching the predicate if it was satisfied before timeout occurred, otherwise null
+     */
+    ClusterState waitForState(Predicate<ClusterState> condition, long timeout, TimeUnit unit)
             throws HyracksDataException, InterruptedException;
 
     /**
@@ -250,4 +259,10 @@
      * @return The metadata cluster partitions
      */
     ClusterPartition getMetadataPartition();
+
+    /**
+     * Indicate whether one or more datasets must be rebalanced before the cluster becomes ACTIVE
+     * @param rebalanceRequired
+     */
+    void setRebalanceRequired(boolean rebalanceRequired) throws HyracksDataException;
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
index 16a479e..7933cd2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
@@ -28,6 +28,7 @@
 import java.util.SortedMap;
 import java.util.TreeSet;
 import java.util.concurrent.TimeUnit;
+import java.util.function.Predicate;
 
 import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
 import org.apache.asterix.common.cluster.ClusterPartition;
@@ -75,6 +76,7 @@
     private INcLifecycleCoordinator lifecycleCoordinator;
     private ICcApplicationContext appCtx;
     private ClusterPartition metadataPartition;
+    private boolean rebalanceRequired;
 
     @Override
     public void setCcAppCtx(ICcApplicationContext appCtx) {
@@ -186,45 +188,55 @@
             return;
         }
         // the metadata bootstrap & global recovery must be complete before the cluster can be active
-        if (metadataNodeActive) {
-            if (state != ClusterState.ACTIVE && state != ClusterState.RECOVERING) {
-                setState(ClusterState.PENDING);
-            }
-            appCtx.getMetadataBootstrap().init();
-
-            if (appCtx.getGlobalRecoveryManager().isRecoveryCompleted()) {
-                setState(ClusterState.ACTIVE);
-            } else {
-                // start global recovery
-                setState(ClusterState.RECOVERING);
-                appCtx.getGlobalRecoveryManager().startGlobalRecovery(appCtx);
-            }
-        } else {
+        if (!metadataNodeActive) {
+            setState(ClusterState.PENDING);
+            return;
+        }
+        if (state != ClusterState.ACTIVE && state != ClusterState.RECOVERING) {
             setState(ClusterState.PENDING);
         }
+        appCtx.getMetadataBootstrap().init();
+
+        if (!appCtx.getGlobalRecoveryManager().isRecoveryCompleted()) {
+            // start global recovery
+            setState(ClusterState.RECOVERING);
+            appCtx.getGlobalRecoveryManager().startGlobalRecovery(appCtx);
+            return;
+        }
+        if (rebalanceRequired) {
+            setState(ClusterState.REBALANCE_REQUIRED);
+            return;
+        }
+        // finally- life is good, set the state to ACTIVE
+        setState(ClusterState.ACTIVE);
     }
 
     @Override
-    public synchronized void waitForState(ClusterState waitForState) throws HyracksDataException, InterruptedException {
+    public synchronized void waitForState(ClusterState waitForState) throws InterruptedException {
         while (state != waitForState) {
             wait();
         }
     }
 
     @Override
-    public synchronized boolean waitForState(ClusterState waitForState, long timeout, TimeUnit unit)
-            throws HyracksDataException, InterruptedException {
+    public boolean waitForState(ClusterState waitForState, long timeout, TimeUnit unit) throws InterruptedException {
+        return waitForState(waitForState::equals, timeout, unit) != null;
+    }
+
+    @Override
+    public synchronized ClusterState waitForState(Predicate<ClusterState> predicate, long timeout, TimeUnit unit)
+            throws InterruptedException {
         final long startMillis = System.currentTimeMillis();
         final long endMillis = startMillis + unit.toMillis(timeout);
-        while (state != waitForState) {
+        while (!predicate.test(state)) {
             long millisToSleep = endMillis - System.currentTimeMillis();
             if (millisToSleep > 0) {
                 wait(millisToSleep);
             } else {
-                return false;
+                return null;
             }
         }
-        return true;
+        return state;
     }
 
     @Override
@@ -458,6 +470,12 @@
         return metadataPartition;
     }
 
+    @Override
+    public synchronized void setRebalanceRequired(boolean rebalanceRequired) throws HyracksDataException {
+        this.rebalanceRequired = rebalanceRequired;
+        refreshState();
+    }
+
     private void updateClusterCounters(String nodeId, NcLocalCounters localCounters) {
         final IResourceIdManager resourceIdManager = appCtx.getResourceIdManager();
         resourceIdManager.report(nodeId, localCounters.getMaxResourceId());

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: merged
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 2
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................


Patch Set 1:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/8548/


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 13 May 2019 22:56:35 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/5784/ (2/16)


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 13 May 2019 22:53:03 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Michael Blow (Code Review)" <de...@asterixdb.apache.org>.
Michael Blow has removed Anon. E. Moose (1000171) from this change.  ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................


Removed reviewer null with the following votes:

* Contrib-2 by Anon. E. Moose (1000171)
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: deleteReviewer
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-spidersilk-tests/709/ (16/16)


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 13 May 2019 22:53:09 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/6351/ (7/16)


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 13 May 2019 22:53:05 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/6141/ (14/16)


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 13 May 2019 22:53:06 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state

Posted by "Murtadha Hubail (Code Review)" <de...@asterixdb.apache.org>.
Murtadha Hubail has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3394 )

Change subject: [NO ISSUE][*DB][CLUS] Add support for a rebalance required cluster state
......................................................................


Patch Set 1:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/3394/1/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java
File asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java:

https://asterix-gerrit.ics.uci.edu/#/c/3394/1/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ClusterStateManager.java@195
PS1, Line 195: if (state != ClusterState.ACTIVE && state != ClusterState.RECOVERING) {
             :             setState(ClusterState.PENDING);
             :         }
> This ensures that we always transition from UNUSABLE -> PENDING -> RECOVERI
I doubt this is of any use since any subscribers won't be able to get the current state until this synchronized block is complete. We can revise this on another change though.



-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3394
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I863f8bf3fe1ce8d59522c9a28a1283006ffa414c
Gerrit-Change-Number: 3394
Gerrit-PatchSet: 1
Gerrit-Owner: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Wed, 15 May 2019 19:03:03 +0000
Gerrit-HasComments: Yes