Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 438F6200BA6 for ; Tue, 18 Oct 2016 20:02:13 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 4221C160AE5; Tue, 18 Oct 2016 18:02:13 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 10DD1160ACE for ; Tue, 18 Oct 2016 20:02:11 +0200 (CEST) Received: (qmail 7542 invoked by uid 500); 18 Oct 2016 18:02:11 -0000 Mailing-List: contact commits-help@asterixdb.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@asterixdb.apache.org Delivered-To: mailing list commits@asterixdb.apache.org Received: (qmail 7528 invoked by uid 99); 18 Oct 2016 18:02:11 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 18 Oct 2016 18:02:11 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 1919FE3813; Tue, 18 Oct 2016 18:02:11 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: mblow@apache.org To: commits@asterixdb.apache.org Message-Id: <3b1928ee89a54734bc9ccf947637ea36@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: asterixdb git commit: Deps++, Refactor ClusterManager Date: Tue, 18 Oct 2016 18:02:11 +0000 (UTC) archived-at: Tue, 18 Oct 2016 18:02:13 -0000 Repository: asterixdb Updated Branches: refs/heads/master e17454aed -> 02291e4b0 Deps++, Refactor ClusterManager - Exclude runtime-scoped junit from appassembler-booter dep Change-Id: Ic574f51133ed32f3b850640260f7faf598b12219 Reviewed-on: https://asterix-gerrit.ics.uci.edu/1298 Reviewed-by: Till Westmann Tested-by: Jenkins Project: http://git-wip-us.apache.org/repos/asf/asterixdb/repo Commit: http://git-wip-us.apache.org/repos/asf/asterixdb/commit/02291e4b Tree: http://git-wip-us.apache.org/repos/asf/asterixdb/tree/02291e4b Diff: http://git-wip-us.apache.org/repos/asf/asterixdb/diff/02291e4b Branch: refs/heads/master Commit: 02291e4b0cdb2b905dc5616c6f9ddb87fd24cac7 Parents: e17454a Author: Michael Blow Authored: Tue Oct 18 13:22:04 2016 -0400 Committer: Michael Blow Committed: Tue Oct 18 11:01:35 2016 -0700 ---------------------------------------------------------------------- .../bootstrap/CCApplicationEntryPoint.java | 12 +-- .../bootstrap/ClusterLifecycleListener.java | 10 ++- .../hyracks/bootstrap/ClusterWorkExecutor.java | 4 +- asterixdb/asterix-client-helper/pom.xml | 6 ++ .../asterix/metadata/api/IClusterManager.java | 1 + .../metadata/cluster/ClusterManager.java | 70 ++++++++------- .../cluster/ClusterManagerProvider.java | 89 ++++++++++++++++++++ asterixdb/asterix-server/pom.xml | 6 ++ 8 files changed, 147 insertions(+), 51 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/asterixdb/blob/02291e4b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java index 764b559..919af33 100644 --- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java +++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java @@ -62,7 +62,7 @@ import org.apache.asterix.messaging.CCMessageBroker; import org.apache.asterix.metadata.MetadataManager; import org.apache.asterix.metadata.api.IAsterixStateProxy; import org.apache.asterix.metadata.bootstrap.AsterixStateProxy; -import org.apache.asterix.metadata.cluster.ClusterManager; +import org.apache.asterix.metadata.cluster.ClusterManagerProvider; import org.apache.asterix.runtime.util.AsterixAppContextInfo; import org.apache.hyracks.api.application.ICCApplicationContext; import org.apache.hyracks.api.application.ICCApplicationEntryPoint; @@ -128,7 +128,7 @@ public class CCApplicationEntryPoint implements ICCApplicationEntryPoint { server.start(); } - ClusterManager.INSTANCE.registerSubscriber(GlobalRecoveryManager.instance()); + ClusterManagerProvider.getClusterManager().registerSubscriber(GlobalRecoveryManager.instance()); ccAppCtx.addClusterLifecycleListener(ClusterLifecycleListener.INSTANCE); ccAppCtx.setMessageBroker(messageBroker); @@ -323,13 +323,7 @@ public class CCApplicationEntryPoint implements ICCApplicationEntryPoint { @Override public void startupCompleted() 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(ClusterProperties.INSTANCE.getCluster().getInstanceName(), - ClusterState.ACTIVE); - } + ClusterManagerProvider.getClusterManager().notifyStartupCompleted(); } public static synchronized void setAsterixStateProxy(IAsterixStateProxy proxy) { http://git-wip-us.apache.org/repos/asf/asterixdb/blob/02291e4b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterLifecycleListener.java ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterLifecycleListener.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterLifecycleListener.java index 75cbe44..7a4ff13 100644 --- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterLifecycleListener.java +++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterLifecycleListener.java @@ -37,7 +37,7 @@ import org.apache.asterix.event.schema.cluster.Node; import org.apache.asterix.metadata.MetadataManager; import org.apache.asterix.metadata.cluster.AddNodeWork; import org.apache.asterix.metadata.cluster.AddNodeWorkResponse; -import org.apache.asterix.metadata.cluster.ClusterManager; +import org.apache.asterix.metadata.cluster.ClusterManagerProvider; import org.apache.asterix.metadata.cluster.RemoveNodeWork; import org.apache.asterix.metadata.cluster.RemoveNodeWorkResponse; import org.apache.asterix.runtime.util.ClusterStateManager; @@ -79,7 +79,8 @@ public class ClusterLifecycleListener implements IClusterLifecycleListener { Set nodeAddition = new HashSet(); nodeAddition.add(nodeId); updateProgress(ClusterEventType.NODE_JOIN, nodeAddition); - Set subscribers = ClusterManager.INSTANCE.getRegisteredClusterEventSubscribers(); + Set subscribers = + ClusterManagerProvider.getClusterManager().getRegisteredClusterEventSubscribers(); Set work = new HashSet(); for (IClusterEventsSubscriber sub : subscribers) { Set workRequest = sub.notifyNodeJoin(nodeId); @@ -105,7 +106,8 @@ public class ClusterLifecycleListener implements IClusterLifecycleListener { } } updateProgress(ClusterEventType.NODE_FAILURE, deadNodeIds); - Set subscribers = ClusterManager.INSTANCE.getRegisteredClusterEventSubscribers(); + Set subscribers = + ClusterManagerProvider.getClusterManager().getRegisteredClusterEventSubscribers(); Set work = new HashSet(); for (IClusterEventsSubscriber sub : subscribers) { Set workRequest = sub.notifyNodeFailure(deadNodeIds); @@ -172,7 +174,7 @@ public class ClusterLifecycleListener implements IClusterLifecycleListener { Node node = ClusterStateManager.INSTANCE.getAvailableSubstitutionNode(); if (node != null) { try { - ClusterManager.INSTANCE.addNode(node); + ClusterManagerProvider.getClusterManager().addNode(node); addedNodes.add(asterixInstanceName + "_" + node.getId()); if (LOGGER.isLoggable(Level.INFO)) { LOGGER.info("Added NC at:" + node.getId()); http://git-wip-us.apache.org/repos/asf/asterixdb/blob/02291e4b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterWorkExecutor.java ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterWorkExecutor.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterWorkExecutor.java index f41c4f6..b1d8dd3 100644 --- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterWorkExecutor.java +++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterWorkExecutor.java @@ -28,7 +28,7 @@ import org.apache.asterix.common.api.IClusterManagementWork; import org.apache.asterix.common.exceptions.AsterixException; import org.apache.asterix.event.schema.cluster.Node; import org.apache.asterix.metadata.cluster.AddNodeWork; -import org.apache.asterix.metadata.cluster.ClusterManager; +import org.apache.asterix.metadata.cluster.ClusterManagerProvider; import org.apache.asterix.metadata.cluster.RemoveNodeWork; import org.apache.asterix.runtime.util.ClusterStateManager; @@ -71,7 +71,7 @@ public class ClusterWorkExecutor implements Runnable { Node node = ClusterStateManager.INSTANCE.getAvailableSubstitutionNode(); if (node != null) { try { - ClusterManager.INSTANCE.addNode(node); + ClusterManagerProvider.getClusterManager().addNode(node); addedNodes.add(node); if (LOGGER.isLoggable(Level.INFO)) { LOGGER.info("Added NC at:" + node.getId()); http://git-wip-us.apache.org/repos/asf/asterixdb/blob/02291e4b/asterixdb/asterix-client-helper/pom.xml ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-client-helper/pom.xml b/asterixdb/asterix-client-helper/pom.xml index a078d16..3270fbb 100644 --- a/asterixdb/asterix-client-helper/pom.xml +++ b/asterixdb/asterix-client-helper/pom.xml @@ -96,6 +96,12 @@ org.codehaus.mojo.appassembler appassembler-booter 1.10 + + + junit + junit + + commons-io http://git-wip-us.apache.org/repos/asf/asterixdb/blob/02291e4b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IClusterManager.java ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IClusterManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IClusterManager.java index 0131731..9ead1ee 100644 --- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IClusterManager.java +++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IClusterManager.java @@ -54,4 +54,5 @@ public interface IClusterManager { */ public Set getRegisteredClusterEventSubscribers(); + void notifyStartupCompleted() throws Exception; } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/02291e4b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/cluster/ClusterManager.java ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/cluster/ClusterManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/cluster/ClusterManager.java index 6371f3e..f473584 100644 --- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/cluster/ClusterManager.java +++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/cluster/ClusterManager.java @@ -30,6 +30,7 @@ import javax.xml.bind.JAXBContext; import javax.xml.bind.Unmarshaller; import org.apache.asterix.common.api.IClusterEventsSubscriber; +import org.apache.asterix.common.api.IClusterManagementWork; import org.apache.asterix.common.config.AsterixExternalProperties; import org.apache.asterix.common.config.ClusterProperties; import org.apache.asterix.common.exceptions.AsterixException; @@ -50,48 +51,42 @@ import org.apache.asterix.runtime.util.AsterixAppContextInfo; public class ClusterManager implements IClusterManager { - private static final Logger LOGGER = Logger.getLogger(AsterixEventServiceClient.class.getName()); + private static final Logger LOGGER = Logger.getLogger(ClusterManager.class.getName()); - public static ClusterManager INSTANCE = new ClusterManager(); + public static final IClusterManager INSTANCE = ClusterManagerProvider.getClusterManager(); - private static AsterixEventServiceClient client; + private final AsterixEventServiceClient client; - private static ILookupService lookupService; + private final ILookupService lookupService; - private static final Set eventSubscribers = new HashSet(); + private final Set eventSubscribers = new HashSet<>(); - private ClusterManager() { - Cluster asterixCluster = ClusterProperties.INSTANCE.getCluster(); - String eventHome = asterixCluster == null ? null - : asterixCluster.getWorkingDir() == null ? null : asterixCluster.getWorkingDir().getDir(); + ClusterManager(String eventHome) { + String asterixDir = System.getProperty("user.dir") + File.separator + "asterix"; + File configFile = new File(System.getProperty("user.dir") + File.separator + "configuration.xml"); + Configuration configuration = null; - if (eventHome != null) { - String asterixDir = System.getProperty("user.dir") + File.separator + "asterix"; - File configFile = new File(System.getProperty("user.dir") + File.separator + "configuration.xml"); - Configuration configuration = null; - - try { - JAXBContext configCtx = JAXBContext.newInstance(Configuration.class); - Unmarshaller unmarshaller = configCtx.createUnmarshaller(); - configuration = (Configuration) unmarshaller.unmarshal(configFile); - AsterixEventService.initialize(configuration, asterixDir, eventHome); - client = AsterixEventService.getAsterixEventServiceClient(ClusterProperties.INSTANCE.getCluster()); - - lookupService = ServiceProvider.INSTANCE.getLookupService(); - if (!lookupService.isRunning(configuration)) { - if (LOGGER.isLoggable(Level.INFO)) { - LOGGER.info("Lookup service not running. Starting lookup service ..."); - } - lookupService.startService(configuration); - } else { - if (LOGGER.isLoggable(Level.INFO)) { - LOGGER.info("Lookup service running"); - } + try { + JAXBContext configCtx = JAXBContext.newInstance(Configuration.class); + Unmarshaller unmarshaller = configCtx.createUnmarshaller(); + configuration = (Configuration) unmarshaller.unmarshal(configFile); + AsterixEventService.initialize(configuration, asterixDir, eventHome); + client = AsterixEventService.getAsterixEventServiceClient(ClusterProperties.INSTANCE.getCluster()); + + lookupService = ServiceProvider.INSTANCE.getLookupService(); + if (!lookupService.isRunning(configuration)) { + if (LOGGER.isLoggable(Level.INFO)) { + LOGGER.info("Lookup service not running. Starting lookup service ..."); + } + lookupService.startService(configuration); + } else { + if (LOGGER.isLoggable(Level.INFO)) { + LOGGER.info("Lookup service running"); } - - } catch (Exception e) { - throw new IllegalStateException("Unable to initialize cluster manager" + e); } + + } catch (Exception e) { + throw new IllegalStateException("Unable to initialize cluster manager" + e); } } @@ -170,7 +165,10 @@ public class ClusterManager implements IClusterManager { return eventSubscribers; } - public static ILookupService getLookupService() { - return lookupService; + @Override + public void notifyStartupCompleted() throws Exception { + // Notify Zookeeper that the startup is complete + lookupService.reportClusterState(ClusterProperties.INSTANCE.getCluster().getInstanceName(), + IClusterManagementWork.ClusterState.ACTIVE); } } http://git-wip-us.apache.org/repos/asf/asterixdb/blob/02291e4b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/cluster/ClusterManagerProvider.java ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/cluster/ClusterManagerProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/cluster/ClusterManagerProvider.java new file mode 100644 index 0000000..cbb3229 --- /dev/null +++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/cluster/ClusterManagerProvider.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.asterix.metadata.cluster; + +import java.util.Collections; +import java.util.Set; + +import org.apache.asterix.common.api.IClusterEventsSubscriber; +import org.apache.asterix.common.config.ClusterProperties; +import org.apache.asterix.common.exceptions.AsterixException; +import org.apache.asterix.event.schema.cluster.Cluster; +import org.apache.asterix.event.schema.cluster.Node; +import org.apache.asterix.metadata.api.IClusterManager; + +public class ClusterManagerProvider { + + private ClusterManagerProvider() { + } + + public static IClusterManager getClusterManager() { + return Holder.INSTANCE; + } + + private static final class Holder { + static final IClusterManager INSTANCE; + + static { + Cluster asterixCluster = ClusterProperties.INSTANCE.getCluster(); + String eventHome = asterixCluster == null ? null + : asterixCluster.getWorkingDir() == null ? null : asterixCluster.getWorkingDir().getDir(); + + if (eventHome != null) { + INSTANCE = new ClusterManager(eventHome); + } else { + INSTANCE = new NoopClusterManager(); + } + } + + private Holder() { + } + } + private static class NoopClusterManager implements IClusterManager { + @Override + public void addNode(Node node) throws AsterixException { + // no-op + } + + @Override + public void removeNode(Node node) throws AsterixException { + // no-op + } + + @Override + public void registerSubscriber(IClusterEventsSubscriber subscriber) { + // no-op + } + + @Override + public boolean deregisterSubscriber(IClusterEventsSubscriber sunscriber) { + return true; + } + + @Override + public Set getRegisteredClusterEventSubscribers() { + return Collections.emptySet(); + } + + @Override + public void notifyStartupCompleted() throws Exception { + // no-op + } + } +} http://git-wip-us.apache.org/repos/asf/asterixdb/blob/02291e4b/asterixdb/asterix-server/pom.xml ---------------------------------------------------------------------- diff --git a/asterixdb/asterix-server/pom.xml b/asterixdb/asterix-server/pom.xml index 919cb16..50db2fd 100644 --- a/asterixdb/asterix-server/pom.xml +++ b/asterixdb/asterix-server/pom.xml @@ -236,6 +236,12 @@ org.codehaus.mojo.appassembler appassembler-booter 1.10 + + + junit + junit + + org.apache.asterix