abdullah alamoudi has uploaded a new change for review.

  https://asterix-gerrit.ics.uci.edu/366

Change subject: This change fixes the sporadic connection refused errors after 
managix starts up an Asterix instance. This can mostly be seen during 
asterix-installer tests. The fix basically lets managix wait and listens 
through zookeeper for a signal from the cluster c
......................................................................

This change fixes the sporadic connection refused errors
after managix starts up an Asterix instance.
This can mostly be seen during asterix-installer tests.
The fix basically lets managix wait and listens
through zookeeper for a signal from the cluster controller.
Once the cluster controller sends the signal, Managix can proceed.

The following commits from your working branch will be included:

commit 71709d4ae1d14696efda4beaa9441b2c9d309440
Author: Abdullah Alamoudi <[email protected]>
Date:   Thu Aug 27 15:52:28 2015 +0300

    Removed unneeded files

commit bc73e456a875c2894335c16ec9324a958bf431fb
Author: Abdullah Alamoudi <[email protected]>
Date:   Thu Aug 27 15:51:48 2015 +0300

    Made Managix wait for CC to signal for startup complete

Change-Id: Ib730f50ab2fb492f3cf973d1cf2f03b34e24e5b3
---
M 
asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
M 
asterix-common/src/main/java/org/apache/asterix/common/api/IClusterManagementWork.java
M 
asterix-events/src/main/java/org/apache/asterix/event/model/AsterixInstance.java
M 
asterix-events/src/main/java/org/apache/asterix/event/service/AsterixEventServiceUtil.java
A 
asterix-events/src/main/java/org/apache/asterix/event/service/ClusterStateWatcher.java
M 
asterix-events/src/main/java/org/apache/asterix/event/service/ILookupService.java
M 
asterix-events/src/main/java/org/apache/asterix/event/service/ZooKeeperService.java
M asterix-events/src/main/resources/events/cc_start/cc_start.sh
M 
asterix-installer/src/main/java/org/apache/asterix/installer/command/CreateCommand.java
M 
asterix-installer/src/main/java/org/apache/asterix/installer/command/StartCommand.java
M 
asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixInstallerIntegrationUtil.java
M 
asterix-metadata/src/main/java/org/apache/asterix/metadata/cluster/ClusterManager.java
12 files changed, 219 insertions(+), 36 deletions(-)


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

diff --git 
a/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
 
b/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
index 2e86a33..dc3e619 100644
--- 
a/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
+++ 
b/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
@@ -20,8 +20,6 @@
 import org.eclipse.jetty.server.Server;
 import org.eclipse.jetty.servlet.ServletContextHandler;
 import org.eclipse.jetty.servlet.ServletHolder;
-import org.eclipse.jetty.util.component.AbstractLifeCycle;
-
 import org.apache.asterix.api.http.servlet.APIServlet;
 import org.apache.asterix.api.http.servlet.AQLAPIServlet;
 import org.apache.asterix.api.http.servlet.ConnectorAPIServlet;
@@ -33,9 +31,11 @@
 import org.apache.asterix.api.http.servlet.ShutdownAPIServlet;
 import org.apache.asterix.api.http.servlet.UpdateAPIServlet;
 import org.apache.asterix.common.api.AsterixThreadFactory;
+import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
 import org.apache.asterix.common.config.AsterixExternalProperties;
 import org.apache.asterix.common.config.AsterixMetadataProperties;
 import org.apache.asterix.common.feeds.api.ICentralFeedManager;
+import org.apache.asterix.event.service.ILookupService;
 import org.apache.asterix.feeds.CentralFeedManager;
 import org.apache.asterix.feeds.FeedLifecycleListener;
 import org.apache.asterix.metadata.MetadataManager;
@@ -43,6 +43,7 @@
 import org.apache.asterix.metadata.bootstrap.AsterixStateProxy;
 import org.apache.asterix.metadata.cluster.ClusterManager;
 import org.apache.asterix.om.util.AsterixAppContextInfo;
+import org.apache.asterix.om.util.AsterixClusterProperties;
 import org.apache.hyracks.api.application.ICCApplicationContext;
 import org.apache.hyracks.api.application.ICCApplicationEntryPoint;
 import org.apache.hyracks.api.client.HyracksConnection;
@@ -80,7 +81,7 @@
         MetadataManager.INSTANCE = new MetadataManager(proxy, 
metadataProperties);
 
         AsterixAppContextInfo.getInstance().getCCApplicationContext()
-        .addJobLifecycleListener(FeedLifecycleListener.INSTANCE);
+                .addJobLifecycleListener(FeedLifecycleListener.INSTANCE);
 
         AsterixExternalProperties externalProperties = 
AsterixAppContextInfo.getInstance().getExternalProperties();
         setupWebServer(externalProperties);
@@ -92,12 +93,8 @@
 
         setupFeedServer(externalProperties);
         feedServer.start();
-        centralFeedManager = CentralFeedManager.getInstance(); 
+        centralFeedManager = CentralFeedManager.getInstance();
         centralFeedManager.start();
-
-        waitUntilServerStart(webServer);
-        waitUntilServerStart(jsonAPIServer);
-        waitUntilServerStart(feedServer);
 
         AsterixGlobalRecoveryManager.INSTANCE = new 
AsterixGlobalRecoveryManager(
                 (HyracksConnection) getNewHyracksClientConnection());
@@ -106,25 +103,20 @@
         
ccAppCtx.addClusterLifecycleListener(ClusterLifecycleListener.INSTANCE);
     }
 
-    private void waitUntilServerStart(AbstractLifeCycle webServer) throws 
Exception {
-        while (!webServer.isStarted()) {
-            if (webServer.isFailed()) {
-                throw new Exception("Server failed to start");
-            }
-            wait(1000);
-        }
-    }
-
     @Override
     public void stop() throws Exception {
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Stopping Asterix cluster controller");
         }
         AsterixStateProxy.unregisterRemoteObject();
-
+        // Stop servers
         webServer.stop();
         jsonAPIServer.stop();
         feedServer.stop();
+        // Make sure servers are stopped before proceeding
+        webServer.join();
+        jsonAPIServer.join();
+        feedServer.join();
     }
 
     private IHyracksClientConnection getNewHyracksClientConnection() throws 
Exception {
@@ -178,7 +170,18 @@
 
         feedServer.setHandler(context);
         context.addServlet(new ServletHolder(new FeedServlet()), "/");
-   
+
         // add paths here
     }
+
+    @Override
+    public void startupCompleted(String driverIp) throws Exception {
+        // Notify Zookeeper that the startup is complete
+        ILookupService zookeeperService = ClusterManager.getLookupService();
+        if (zookeeperService != null) {
+            // Our asterix app runtimes tests don't use zookeeper
+            
zookeeperService.reportClusterState(AsterixClusterProperties.INSTANCE.getCluster().getInstanceName(),
+                    ClusterState.ACTIVE);
+        }
+    }
 }
\ No newline at end of file
diff --git 
a/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterManagementWork.java
 
b/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterManagementWork.java
index 283c419..62b4cb7 100644
--- 
a/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterManagementWork.java
+++ 
b/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterManagementWork.java
@@ -22,6 +22,7 @@
     }
 
     public enum ClusterState {
+        STARTING,
         ACTIVE,
         UNUSABLE
     }
diff --git 
a/asterix-events/src/main/java/org/apache/asterix/event/model/AsterixInstance.java
 
b/asterix-events/src/main/java/org/apache/asterix/event/model/AsterixInstance.java
index fc4a877..038305a 100644
--- 
a/asterix-events/src/main/java/org/apache/asterix/event/model/AsterixInstance.java
+++ 
b/asterix-events/src/main/java/org/apache/asterix/event/model/AsterixInstance.java
@@ -19,7 +19,6 @@
 import java.util.Date;
 import java.util.List;
 
-import org.apache.asterix.common.config.AsterixExternalProperties;
 import org.apache.asterix.common.configuration.AsterixConfiguration;
 import org.apache.asterix.common.configuration.Property;
 import org.apache.asterix.event.schema.cluster.Cluster;
diff --git 
a/asterix-events/src/main/java/org/apache/asterix/event/service/AsterixEventServiceUtil.java
 
b/asterix-events/src/main/java/org/apache/asterix/event/service/AsterixEventServiceUtil.java
index 2114388..2881439 100644
--- 
a/asterix-events/src/main/java/org/apache/asterix/event/service/AsterixEventServiceUtil.java
+++ 
b/asterix-events/src/main/java/org/apache/asterix/event/service/AsterixEventServiceUtil.java
@@ -28,6 +28,7 @@
 import java.net.InetAddress;
 import java.net.URL;
 import java.net.URLClassLoader;
+import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Enumeration;
 import java.util.List;
@@ -46,7 +47,6 @@
 import javax.xml.bind.Marshaller;
 
 import org.apache.commons.io.IOUtils;
-
 import org.apache.asterix.common.configuration.AsterixConfiguration;
 import org.apache.asterix.common.configuration.Coredump;
 import org.apache.asterix.common.configuration.Store;
@@ -147,6 +147,13 @@
         clusterProperties.add(new Property("CLIENT_NET_PORT", "" + 
clientNetPort));
         clusterProperties.add(new Property("CLUSTER_NET_PORT", "" + 
clusterNetPort));
         clusterProperties.add(new Property("HTTP_PORT", "" + httpPort));
+        // New one added for the confirmation
+        try {
+            clusterProperties.add(new Property("MANAGIX_IP", 
InetAddress.getLocalHost().getHostAddress()));
+        } catch (UnknownHostException e) {
+            e.printStackTrace();
+            clusterProperties.add(new Property("MANAGIX_IP", 
InetAddress.getLoopbackAddress().getHostAddress()));
+        }
 
         cluster.setEnv(new Env(clusterProperties));
     }
@@ -175,6 +182,7 @@
         new File(asterixInstanceDir + File.separator + 
ASTERIX_CONFIGURATION_FILE).delete();
     }
 
+    // This method is never used locally. Should we delete it?
     private static void injectAsterixLogPropertyFile(String 
asterixInstanceDir, AsterixInstance asterixInstance)
             throws IOException, EventException {
         final String asterixJarPath = asterixJarPath(asterixInstance, 
asterixInstanceDir);
diff --git 
a/asterix-events/src/main/java/org/apache/asterix/event/service/ClusterStateWatcher.java
 
b/asterix-events/src/main/java/org/apache/asterix/event/service/ClusterStateWatcher.java
new file mode 100644
index 0000000..59da5b1
--- /dev/null
+++ 
b/asterix-events/src/main/java/org/apache/asterix/event/service/ClusterStateWatcher.java
@@ -0,0 +1,108 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed 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 from
+ *
+ *     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.asterix.event.service;
+
+import java.io.File;
+import java.util.List;
+
+import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+
+//A zookeeper watcher that watches the change in the state of the cluster
+public class ClusterStateWatcher implements Watcher {
+    private static Integer mutex;
+    private static ZooKeeper zk;
+    private String clusterStatePath;
+    private boolean done = false;
+    private ClusterState clusterState = ClusterState.STARTING;
+    private boolean failed = false;
+    private Exception failureCause = null;
+    private static Logger LOGGER = 
Logger.getLogger(ClusterStateWatcher.class.getName());
+
+    public ClusterStateWatcher(ZooKeeper zk, String clusterName) {
+        if (mutex == null) {
+            mutex = new Integer(-1);
+        }
+        this.clusterStatePath = ZooKeeperService.ASTERIX_INSTANCE_BASE_PATH + 
File.separator + clusterName
+                + File.separator + "state";
+        ClusterStateWatcher.zk = zk;
+    }
+
+    public ClusterState getClusterState() throws Exception {
+        while (true) {
+            synchronized (mutex) {
+                if (done) {
+                    if (failed) {
+                        LOGGER.error("An error took place in the startup 
sequence. Check the CC logs.");
+                        throw failureCause;
+                    } else {
+                        return clusterState;
+                    }
+                } else {
+                    mutex.wait();
+                }
+            }
+        }
+    }
+
+    public void monitorStateChange() {
+        try {
+            while (true) {
+                synchronized (mutex) {
+                    // Get the cluster state 
+                    List<String> list = zk.getChildren(clusterStatePath, this);
+                    if (list.size() == 0) {
+                        // Cluster state not found, wait to be awaken by 
Zookeeper
+                        mutex.wait();
+                    } else {
+                        // Cluster state found
+                        byte[] b = zk.getData(clusterStatePath + 
"/clusterState", false, null);
+                        zk.delete(clusterStatePath + "/clusterState", 0);
+                        clusterState = 
ClusterState.values()[(Integer.parseInt(Byte.toString(b[0])))];
+                        done = true;
+                        mutex.notifyAll();
+                        return;
+                    }
+                }
+            }
+        } catch (Exception e) {
+            // Exception was thrown, let Managix know that a failure took place
+            failed = true;
+            done = true;
+            failureCause = e;
+        }
+    }
+
+    public void startMonitoringThread() {
+        Runnable monitoringThread = new Runnable() {
+            @Override
+            public void run() {
+                monitorStateChange();
+            }
+        };
+        // Start the monitoring thread
+        (new Thread(monitoringThread)).start();
+    }
+
+    @Override
+    synchronized public void process(WatchedEvent event) {
+        synchronized (mutex) {
+            mutex.notifyAll();
+        }
+    }
+}
\ No newline at end of file
diff --git 
a/asterix-events/src/main/java/org/apache/asterix/event/service/ILookupService.java
 
b/asterix-events/src/main/java/org/apache/asterix/event/service/ILookupService.java
index 043fa75..d083c0e 100644
--- 
a/asterix-events/src/main/java/org/apache/asterix/event/service/ILookupService.java
+++ 
b/asterix-events/src/main/java/org/apache/asterix/event/service/ILookupService.java
@@ -16,6 +16,7 @@
 
 import java.util.List;
 
+import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
 import org.apache.asterix.event.model.AsterixInstance;
 import org.apache.asterix.installer.schema.conf.Configuration;
 
@@ -38,4 +39,8 @@
     public List<AsterixInstance> getAsterixInstances() throws Exception;
 
     public void updateAsterixInstance(AsterixInstance updatedInstance) throws 
Exception;
+
+    public void reportClusterState(String instanceName, ClusterState active) 
throws Exception;
+
+    public ClusterStateWatcher startWatchingClusterState(String 
asterixInstanceName);
 }
diff --git 
a/asterix-events/src/main/java/org/apache/asterix/event/service/ZooKeeperService.java
 
b/asterix-events/src/main/java/org/apache/asterix/event/service/ZooKeeperService.java
index 0a0277d..0944482 100644
--- 
a/asterix-events/src/main/java/org/apache/asterix/event/service/ZooKeeperService.java
+++ 
b/asterix-events/src/main/java/org/apache/asterix/event/service/ZooKeeperService.java
@@ -33,7 +33,7 @@
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.data.Stat;
-
+import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
 import org.apache.asterix.event.error.EventException;
 import org.apache.asterix.event.model.AsterixInstance;
 import org.apache.asterix.installer.schema.conf.Configuration;
@@ -51,8 +51,8 @@
     private boolean isRunning = false;
     private ZooKeeper zk;
     private String zkConnectionString;
-    private static final String ASTERIX_INSTANCE_BASE_PATH = "/Asterix";
-    private static final int DEFAULT_NODE_VERSION = -1;
+    public static final String ASTERIX_INSTANCE_BASE_PATH = "/Asterix";
+    public static final int DEFAULT_NODE_VERSION = -1;
     private LinkedBlockingQueue<String> msgQ = new 
LinkedBlockingQueue<String>();
     private ZooKeeperWatcher watcher = new ZooKeeperWatcher(msgQ);
 
@@ -140,6 +140,9 @@
         ObjectOutputStream o = new ObjectOutputStream(b);
         o.writeObject(asterixInstance);
         zk.create(instanceBasePath, b.toByteArray(), Ids.OPEN_ACL_UNSAFE, 
CreateMode.PERSISTENT);
+        // Create a place to put the state of the cluster in
+        String instanceStatePath = instanceBasePath + File.separator + "state";
+        zk.create(instanceStatePath, new byte[0], Ids.OPEN_ACL_UNSAFE, 
CreateMode.PERSISTENT);
     }
 
     private void createRootIfNotExist() throws Exception {
@@ -149,6 +152,9 @@
                 zk.create(ASTERIX_INSTANCE_BASE_PATH, "root".getBytes(), 
Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
             }
         } catch (Exception e) {
+            // Is this the right way to handle the exception (try again? 
forever?)
+            LOGGER.error("An error took place when creating the root in 
Zookeeper");
+            e.printStackTrace();
             createRootIfNotExist();
         }
     }
@@ -163,13 +169,21 @@
         return readAsterixInstanceObject(asterixInstanceBytes);
     }
 
-    public boolean exists(String asterixInstanceName) throws Exception {
-        return zk.exists(ASTERIX_INSTANCE_BASE_PATH + File.separator + 
asterixInstanceName, false) != null;
+    public boolean exists(String path) throws Exception {
+        return zk.exists(ASTERIX_INSTANCE_BASE_PATH + File.separator + path, 
false) != null;
     }
 
     public void removeAsterixInstance(String name) throws Exception {
         if (!exists(name)) {
             throw new EventException("Asterix instance by name " + name + " 
does not exists.");
+        }
+        if (exists(name + File.separator + "state")) {
+            if (exists(name + File.separator + "state" + File.separator + 
"clusterState")) {
+                zk.delete(ASTERIX_INSTANCE_BASE_PATH + File.separator + name + 
File.separator + "state"
+                        + File.separator + "clusterState", 
DEFAULT_NODE_VERSION);
+            }
+            zk.delete(ASTERIX_INSTANCE_BASE_PATH + File.separator + name + 
File.separator + "state",
+                    DEFAULT_NODE_VERSION);
         }
         zk.delete(ASTERIX_INSTANCE_BASE_PATH + File.separator + name, 
DEFAULT_NODE_VERSION);
     }
@@ -198,6 +212,24 @@
         writeAsterixInstance(updatedInstance);
     }
 
+    @Override
+    public ClusterStateWatcher startWatchingClusterState(String instanceName) {
+        ClusterStateWatcher watcher = new ClusterStateWatcher(zk, 
instanceName);
+        watcher.startMonitoringThread();
+        return watcher;
+    }
+
+    @Override
+    public void reportClusterState(String instanceName, ClusterState state) 
throws Exception {
+        String clusterStatePath = ZooKeeperService.ASTERIX_INSTANCE_BASE_PATH 
+ File.separator + instanceName
+                + File.separator + "state";
+        Integer value = state.ordinal();
+        byte[] stateValue = new byte[] { value.byteValue() };
+        // Create a place to put the state of the cluster in
+        zk.create(clusterStatePath + "/clusterState", stateValue, 
Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        return;
+    }
+
 }
 
 class ZooKeeperWatcher implements Watcher {
@@ -209,6 +241,7 @@
         this.msgQ = msgQ;
     }
 
+    @SuppressWarnings("incomplete-switch")
     public void process(WatchedEvent wEvent) {
         switch (wEvent.getState()) {
             case SyncConnected:
diff --git a/asterix-events/src/main/resources/events/cc_start/cc_start.sh 
b/asterix-events/src/main/resources/events/cc_start/cc_start.sh
index a5fb853..f69b738 100644
--- a/asterix-events/src/main/resources/events/cc_start/cc_start.sh
+++ b/asterix-events/src/main/resources/events/cc_start/cc_start.sh
@@ -19,4 +19,4 @@
   mkdir -p $LOG_DIR
 fi
 cd $WORKING_DIR
-$ASTERIX_HOME/bin/asterixcc -client-net-ip-address $CLIENT_NET_IP 
-client-net-port $CLIENT_NET_PORT -cluster-net-ip-address $CLUSTER_NET_IP 
-cluster-net-port $CLUSTER_NET_PORT -http-port $HTTP_PORT  &> $LOG_DIR/cc.log
+$ASTERIX_HOME/bin/asterixcc -client-net-ip-address $CLIENT_NET_IP 
-client-net-port $CLIENT_NET_PORT -cluster-net-ip-address $CLUSTER_NET_IP 
-cluster-net-port $CLUSTER_NET_PORT -http-port $HTTP_PORT -managix-ip 
$MANAGIX_IP &> $LOG_DIR/cc.log
diff --git 
a/asterix-installer/src/main/java/org/apache/asterix/installer/command/CreateCommand.java
 
b/asterix-installer/src/main/java/org/apache/asterix/installer/command/CreateCommand.java
index 2a83dee..e474efe 100644
--- 
a/asterix-installer/src/main/java/org/apache/asterix/installer/command/CreateCommand.java
+++ 
b/asterix-installer/src/main/java/org/apache/asterix/installer/command/CreateCommand.java
@@ -17,7 +17,7 @@
 import java.io.File;
 
 import org.kohsuke.args4j.Option;
-
+import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
 import org.apache.asterix.common.configuration.AsterixConfiguration;
 import org.apache.asterix.event.error.VerificationUtil;
 import org.apache.asterix.event.management.AsterixEventServiceClient;
@@ -28,6 +28,7 @@
 import org.apache.asterix.event.schema.pattern.Patterns;
 import org.apache.asterix.event.service.AsterixEventService;
 import org.apache.asterix.event.service.AsterixEventServiceUtil;
+import org.apache.asterix.event.service.ClusterStateWatcher;
 import org.apache.asterix.event.service.ServiceProvider;
 import org.apache.asterix.event.util.PatternCreator;
 import org.apache.asterix.installer.driver.InstallerDriver;
@@ -60,17 +61,25 @@
         AsterixEventServiceUtil.createClusterProperties(cluster, 
asterixConfiguration);
         AsterixEventServiceClient eventrixClient = 
AsterixEventService.getAsterixEventServiceClient(cluster, true,
                 false);
-
+        // Store the cluster initially in Zookeeper and start watching
+        
ServiceProvider.INSTANCE.getLookupService().writeAsterixInstance(asterixInstance);
+        ClusterStateWatcher stateWatcher = 
ServiceProvider.INSTANCE.getLookupService().startWatchingClusterState(
+                asterixInstanceName);
         Patterns asterixBinarytrasnferPattern = 
PatternCreator.INSTANCE.getAsterixBinaryTransferPattern(
                 asterixInstanceName, cluster);
         eventrixClient.submit(asterixBinarytrasnferPattern);
-
         Patterns patterns = 
PatternCreator.INSTANCE.getStartAsterixPattern(asterixInstanceName, cluster, 
true);
         eventrixClient.submit(patterns);
 
+        // Check the cluster state
+        ClusterState clusterState = stateWatcher.getClusterState();
+        if (clusterState != ClusterState.ACTIVE) {
+            throw new Exception("CC failed to start");
+        }
+
         AsterixRuntimeState runtimeState = 
VerificationUtil.getAsterixRuntimeState(asterixInstance);
         VerificationUtil.updateInstanceWithRuntimeDescription(asterixInstance, 
runtimeState, true);
-        
ServiceProvider.INSTANCE.getLookupService().writeAsterixInstance(asterixInstance);
+        
ServiceProvider.INSTANCE.getLookupService().updateAsterixInstance(asterixInstance);
         
AsterixEventServiceUtil.deleteDirectory(InstallerDriver.getManagixHome() + 
File.separator
                 + InstallerDriver.ASTERIX_DIR + File.separator + 
asterixInstanceName);
         LOGGER.info(asterixInstance.getDescription(false));
diff --git 
a/asterix-installer/src/main/java/org/apache/asterix/installer/command/StartCommand.java
 
b/asterix-installer/src/main/java/org/apache/asterix/installer/command/StartCommand.java
index 944f68d..e55aba6 100644
--- 
a/asterix-installer/src/main/java/org/apache/asterix/installer/command/StartCommand.java
+++ 
b/asterix-installer/src/main/java/org/apache/asterix/installer/command/StartCommand.java
@@ -17,7 +17,7 @@
 import java.io.File;
 
 import org.kohsuke.args4j.Option;
-
+import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
 import org.apache.asterix.event.error.VerificationUtil;
 import org.apache.asterix.event.management.AsterixEventServiceClient;
 import org.apache.asterix.event.model.AsterixInstance;
@@ -26,6 +26,7 @@
 import org.apache.asterix.event.schema.pattern.Patterns;
 import org.apache.asterix.event.service.AsterixEventService;
 import org.apache.asterix.event.service.AsterixEventServiceUtil;
+import org.apache.asterix.event.service.ClusterStateWatcher;
 import org.apache.asterix.event.service.ServiceProvider;
 import org.apache.asterix.event.util.PatternCreator;
 import org.apache.asterix.installer.driver.InstallerDriver;
@@ -43,9 +44,18 @@
         Patterns asterixBinaryTransferPattern = 
PatternCreator.INSTANCE.getAsterixBinaryTransferPattern(
                 asterixInstanceName, instance.getCluster());
         client.submit(asterixBinaryTransferPattern);
+        // Start the watcher
+        ClusterStateWatcher stateWatcher = 
ServiceProvider.INSTANCE.getLookupService().startWatchingClusterState(
+                asterixInstanceName);
         AsterixEventServiceUtil.createClusterProperties(instance.getCluster(), 
instance.getAsterixConfiguration());
-        Patterns patterns = 
PatternCreator.INSTANCE.getStartAsterixPattern(asterixInstanceName, 
instance.getCluster(), false);
+        Patterns patterns = 
PatternCreator.INSTANCE.getStartAsterixPattern(asterixInstanceName, 
instance.getCluster(),
+                false);
         client.submit(patterns);
+        // Check the cluster state
+        ClusterState clusterState = stateWatcher.getClusterState();
+        if (clusterState != ClusterState.ACTIVE) {
+            throw new Exception("CC failed to start");
+        }
         
AsterixEventServiceUtil.deleteDirectory(InstallerDriver.getManagixHome() + 
File.separator
                 + InstallerDriver.ASTERIX_DIR + File.separator + 
asterixInstanceName);
         AsterixRuntimeState runtimeState = 
VerificationUtil.getAsterixRuntimeState(instance);
diff --git 
a/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixInstallerIntegrationUtil.java
 
b/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixInstallerIntegrationUtil.java
index ad64319..308aae9 100644
--- 
a/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixInstallerIntegrationUtil.java
+++ 
b/asterix-installer/src/test/java/org/apache/asterix/installer/test/AsterixInstallerIntegrationUtil.java
@@ -165,6 +165,10 @@
             case INACTIVE:
                 command = "start -n " + ASTERIX_INSTANCE_NAME;
                 break;
+            case UNUSABLE:
+                command = "delete -n " + ASTERIX_INSTANCE_NAME;
+                cmdHandler.processCommand(command.split(" "));
+                throw new Exception("Cluster state was Unusable");
         }
         cmdHandler.processCommand(command.split(" "));
     }
diff --git 
a/asterix-metadata/src/main/java/org/apache/asterix/metadata/cluster/ClusterManager.java
 
b/asterix-metadata/src/main/java/org/apache/asterix/metadata/cluster/ClusterManager.java
index 1c69c56..c5bb9ab 100644
--- 
a/asterix-metadata/src/main/java/org/apache/asterix/metadata/cluster/ClusterManager.java
+++ 
b/asterix-metadata/src/main/java/org/apache/asterix/metadata/cluster/ClusterManager.java
@@ -50,8 +50,6 @@
 
     public static ClusterManager INSTANCE = new ClusterManager();
 
-    private static String eventsDir = System.getenv("user.dir") + 
File.separator + "eventrix";
-
     private static AsterixEventServiceClient client;
 
     private static ILookupService lookupService;
@@ -113,7 +111,8 @@
             String hostId = node.getId();
             String nodeControllerId = asterixInstanceName + "_" + node.getId();
             String iodevices = node.getIodevices() == null ? 
cluster.getIodevices() : node.getIodevices();
-            Pattern startNC = 
PatternCreator.INSTANCE.createNCStartPattern(ccHost, hostId, nodeControllerId, 
iodevices, false);
+            Pattern startNC = 
PatternCreator.INSTANCE.createNCStartPattern(ccHost, hostId, nodeControllerId, 
iodevices,
+                    false);
             pattern.add(startNC);
             Patterns startNCPattern = new Patterns(pattern);
             client.submit(startNCPattern);
@@ -166,4 +165,8 @@
     public Set<IClusterEventsSubscriber> 
getRegisteredClusterEventSubscribers() {
         return eventSubscribers;
     }
+
+    public static ILookupService getLookupService() {
+        return lookupService;
+    }
 }

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

Gerrit-MessageType: newchange
Gerrit-Change-Id: Ib730f50ab2fb492f3cf973d1cf2f03b34e24e5b3
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <[email protected]>

Reply via email to