You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by dd...@apache.org on 2011/09/28 02:17:20 UTC

svn commit: r1176669 - in /incubator/ambari/trunk/controller/src/main/java/org/apache/ambari: components/ components/impl/ controller/ resource/statemachine/

Author: ddas
Date: Wed Sep 28 00:17:19 2011
New Revision: 1176669

URL: http://svn.apache.org/viewvc?rev=1176669&view=rev
Log:
AMBARI-10. Heartbeat iteration

Added:
    incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/components/impl/
    incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/components/impl/ClusterContextImpl.java
    incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/components/impl/HDFSPluginImpl.java
Modified:
    incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/components/ClusterContext.java
    incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/controller/HeartbeatHandler.java
    incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/Cluster.java
    incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/ClusterImpl.java
    incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/RoleImpl.java
    incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/Service.java
    incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/ServiceImpl.java
    incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/StateMachineInvoker.java

Modified: incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/components/ClusterContext.java
URL: http://svn.apache.org/viewvc/incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/components/ClusterContext.java?rev=1176669&r1=1176668&r2=1176669&view=diff
==============================================================================
--- incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/components/ClusterContext.java (original)
+++ incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/components/ClusterContext.java Wed Sep 28 00:17:19 2011
@@ -37,6 +37,12 @@ public interface ClusterContext {
   String[] getAllRoles();
   
   /**
+   * Get all of the components that are currently active in the cluster
+   * @return a list of all of the services for this node
+   */
+  String[] getClusterComponents();
+  
+  /**
    * Get the directory name for the directory that should contain the software.
    * @return the full pathname of the directory
    */

Added: incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/components/impl/ClusterContextImpl.java
URL: http://svn.apache.org/viewvc/incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/components/impl/ClusterContextImpl.java?rev=1176669&view=auto
==============================================================================
--- incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/components/impl/ClusterContextImpl.java (added)
+++ incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/components/impl/ClusterContextImpl.java Wed Sep 28 00:17:19 2011
@@ -0,0 +1,62 @@
+package org.apache.ambari.components.impl;
+
+import java.util.List;
+
+import org.apache.ambari.common.rest.entities.Blueprint;
+import org.apache.ambari.common.rest.entities.Cluster;
+import org.apache.ambari.common.rest.entities.ClusterDefinition;
+import org.apache.ambari.common.rest.entities.Node;
+import org.apache.ambari.components.ClusterContext;
+
+public class ClusterContextImpl implements ClusterContext {
+
+  Cluster cluster;
+  Node node;
+  
+  public ClusterContextImpl(Cluster cluster, Node node) {
+    this.cluster = cluster;
+    this.node = node;
+  }
+  
+  @Override
+  public String getClusterName() {
+    return cluster.getClusterDefinition().getName();
+  }
+
+  @Override
+  public String[] getAllRoles() {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public String getInstallDirectory() {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public String getConfigDirectory() {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public ClusterDefinition getClusterDefinition() {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public Blueprint getBlueprint() {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public String[] getClusterComponents() {
+    List<String> roles = cluster.getClusterDefinition().getActiveServices();
+    return roles.toArray(new String[1]);
+  }
+
+}

Added: incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/components/impl/HDFSPluginImpl.java
URL: http://svn.apache.org/viewvc/incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/components/impl/HDFSPluginImpl.java?rev=1176669&view=auto
==============================================================================
--- incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/components/impl/HDFSPluginImpl.java (added)
+++ incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/components/impl/HDFSPluginImpl.java Wed Sep 28 00:17:19 2011
@@ -0,0 +1,64 @@
+package org.apache.ambari.components.impl;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.ambari.common.rest.entities.agent.Action;
+import org.apache.ambari.common.rest.entities.agent.Command;
+import org.apache.ambari.components.ClusterContext;
+import org.apache.ambari.components.ComponentPlugin;
+
+public class HDFSPluginImpl extends ComponentPlugin {
+
+  @Override
+  public String[] getRoles() throws IOException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public String[] getRequiredComponents() throws IOException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public boolean isService() throws IOException {
+    // TODO Auto-generated method stub
+    return false;
+  }
+
+  @Override
+  public List<Action> writeConfiguration(ClusterContext cluster)
+      throws IOException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public List<Action> install(ClusterContext cluster) throws IOException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public List<Action> uninstall(ClusterContext cluster) throws IOException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public List<Action> startRoleServer(ClusterContext cluster, String role)
+      throws IOException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public List<Action> stopRoleServer(ClusterContext cluster, String role)
+      throws IOException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+}

Modified: incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/controller/HeartbeatHandler.java
URL: http://svn.apache.org/viewvc/incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/controller/HeartbeatHandler.java?rev=1176669&r1=1176668&r2=1176669&view=diff
==============================================================================
--- incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/controller/HeartbeatHandler.java (original)
+++ incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/controller/HeartbeatHandler.java Wed Sep 28 00:17:19 2011
@@ -1,20 +1,3 @@
-/*
- * 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.ambari.controller;
 
 import java.util.ArrayList;
@@ -24,12 +7,12 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
-
 import java.util.concurrent.LinkedBlockingQueue;
 
 import javax.xml.datatype.DatatypeConfigurationException;
 import javax.xml.datatype.DatatypeFactory;
 
+import org.apache.ambari.common.rest.entities.Cluster;
 import org.apache.ambari.common.rest.entities.Node;
 import org.apache.ambari.common.rest.entities.NodeState;
 import org.apache.ambari.controller.Clusters;
@@ -37,6 +20,10 @@ import org.apache.ambari.controller.Node
 import org.apache.ambari.common.rest.entities.agent.Action;
 import org.apache.ambari.common.rest.entities.agent.ControllerResponse;
 import org.apache.ambari.common.rest.entities.agent.HeartBeat;
+import org.apache.ambari.common.rest.entities.agent.ServerStatus;
+import org.apache.ambari.components.ClusterContext;
+import org.apache.ambari.components.impl.ClusterContextImpl;
+import org.apache.ambari.resource.statemachine.StateMachineInvoker;
 
 public class HeartbeatHandler {
   
@@ -74,16 +61,17 @@ public class HeartbeatHandler {
     c.setTime(new Date());
     state.setLastHeartbeatTime(
         DatatypeFactory.newInstance().newXMLGregorianCalendar(c));
+    
+    Cluster cluster = Clusters.getInstance().getClusterByName(state.getClusterName());
+    ClusterContext clusterContext = new ClusterContextImpl(cluster, node);
 
-    //queue the heartbeat for later processing
-    heartbeatQueue.add(heartbeat);
+    //get what is currently running on the node
+    List<ServerStatus> servers = heartbeat.getServersStatus();    
     
-    //get the current response for the node/role
-    List<String> roles = Clusters.getInstance().getAssociatedRoleNames(heartbeat.getHostname());
+    //get the state machine reference to the cluster
+    org.apache.ambari.resource.statemachine.Cluster stateMachineCluster = 
+        StateMachineInvoker.getStateMachineClusterInstance(state.getClusterName());
     
-    if (roles != null && roles.size() != 0) {
-      
-    }
     
     List<Action> actions = new ArrayList<Action>();
     synchronized (this) {

Modified: incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/Cluster.java
URL: http://svn.apache.org/viewvc/incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/Cluster.java?rev=1176669&r1=1176668&r2=1176669&view=diff
==============================================================================
--- incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/Cluster.java (original)
+++ incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/Cluster.java Wed Sep 28 00:17:19 2011
@@ -20,12 +20,13 @@ package org.apache.ambari.resource.state
 import java.util.List;
 import java.util.Map;
 
+import org.apache.ambari.components.ClusterContext;
+
 public interface Cluster extends LifeCycle {
-  public String getClusterName();
   public List<Service> getServices();
   public ClusterState getClusterState();
   public Map<String, String> getServiceStates();
-  
+  public String getClusterName();
   public void addServices(List<Service> services);
   public void terminate();
 }

Modified: incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/ClusterImpl.java
URL: http://svn.apache.org/viewvc/incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/ClusterImpl.java?rev=1176669&r1=1176668&r2=1176669&view=diff
==============================================================================
--- incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/ClusterImpl.java (original)
+++ incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/ClusterImpl.java Wed Sep 28 00:17:19 2011
@@ -32,6 +32,7 @@ import org.apache.ambari.common.state.Mu
 import org.apache.ambari.common.state.SingleArcTransition;
 import org.apache.ambari.common.state.StateMachine;
 import org.apache.ambari.common.state.StateMachineFactory;
+import org.apache.ambari.components.ClusterContext;
 import org.apache.ambari.event.EventHandler;
 
 public class ClusterImpl implements Cluster, EventHandler<ClusterEvent> {
@@ -81,12 +82,12 @@ public class ClusterImpl implements Clus
   private Map<String, Set<String>> roleToNodes;
   private StateMachine<ClusterState, ClusterEventType, ClusterEvent> 
           stateMachine;
-  private String clusterName;
   private int numServicesStarted;
   private int totalEnabledServices;
   private Lock readLock;
   private Lock writeLock;
   private short roleCount; 
+  private String clusterName;
     
   public ClusterImpl(String name) {
     this.clusterName = name;
@@ -97,11 +98,6 @@ public class ClusterImpl implements Clus
   }
 
   @Override
-  public String getClusterName() {
-    return clusterName;
-  }
-
-  @Override
   public ClusterState getClusterState() {
     return stateMachine.getCurrentState();
   }
@@ -122,6 +118,7 @@ public class ClusterImpl implements Clus
   
   @Override  
   public void addServices(List<Service> services) {
+    //The services start in the order they appear in the list
     this.services.addAll(services);
   }
   
@@ -212,4 +209,9 @@ public class ClusterImpl implements Clus
         new ClusterEvent(ClusterEventType.S_RELEASE_NODES, this));    
   }
 
+  @Override
+  public String getClusterName() {
+    return clusterName;
+  }
+
 }

Modified: incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/RoleImpl.java
URL: http://svn.apache.org/viewvc/incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/RoleImpl.java?rev=1176669&r1=1176668&r2=1176669&view=diff
==============================================================================
--- incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/RoleImpl.java (original)
+++ incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/RoleImpl.java Wed Sep 28 00:17:19 2011
@@ -59,7 +59,7 @@ public class RoleImpl implements Role, E
   private final StateMachine<RoleState, RoleEventType, RoleEvent>
       stateMachine;
   
-  public RoleImpl(Service service, String roleName, Set<String> hosts) {
+  public RoleImpl(Service service, String roleName) {
     this.roleName = roleName;
     this.service = service;
     this.myState = RoleState.INACTIVE;

Modified: incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/Service.java
URL: http://svn.apache.org/viewvc/incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/Service.java?rev=1176669&r1=1176668&r2=1176669&view=diff
==============================================================================
--- incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/Service.java (original)
+++ incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/Service.java Wed Sep 28 00:17:19 2011
@@ -17,8 +17,6 @@
 */
 package org.apache.ambari.resource.statemachine;
 
-import java.util.List;
-
 public interface Service extends LifeCycle {
   
   public ServiceState getServiceState();

Modified: incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/ServiceImpl.java
URL: http://svn.apache.org/viewvc/incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/ServiceImpl.java?rev=1176669&r1=1176668&r2=1176669&view=diff
==============================================================================
--- incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/ServiceImpl.java (original)
+++ incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/ServiceImpl.java Wed Sep 28 00:17:19 2011
@@ -17,6 +17,7 @@
 */
 package org.apache.ambari.resource.statemachine;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.List;
@@ -27,12 +28,15 @@ import org.apache.ambari.common.state.Mu
 import org.apache.ambari.common.state.SingleArcTransition;
 import org.apache.ambari.common.state.StateMachine;
 import org.apache.ambari.common.state.StateMachineFactory;
+import org.apache.ambari.components.ComponentPlugin;
+import org.apache.ambari.components.impl.HDFSPluginImpl;
 import org.apache.ambari.event.EventHandler;
 
 public class ServiceImpl implements Service, EventHandler<ServiceEvent> {
 
   private ServiceState myState;
   private Cluster cluster;
+  private ComponentPlugin plugin;
   
   /* The state machine for the service looks like:
    * INACTIVE --S_START--> STARTING --S_START_SUCCESS--> ACTIVE
@@ -65,12 +69,18 @@ public class ServiceImpl implements Serv
   private final String serviceName;
   private short roleCount;
   
-  public ServiceImpl(Cluster cluster, String serviceName) {
+  public ServiceImpl(Cluster cluster, String serviceName) throws IOException {
     this.cluster = cluster;
     this.serviceName = serviceName;
     this.myState = ServiceState.INACTIVE;
-    stateMachine = stateMachineFactory.make(this);
     //load plugin and get the roles and create them
+    this.plugin = new HDFSPluginImpl();
+    String[] roles = this.plugin.getRoles();
+    for (String role : roles) {
+      RoleImpl roleImpl = new RoleImpl(this, role);
+      serviceRoles.add(roleImpl);
+    }
+    stateMachine = stateMachineFactory.make(this);
   }
     
   public StateMachine getStateMachine() {

Modified: incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/StateMachineInvoker.java
URL: http://svn.apache.org/viewvc/incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/StateMachineInvoker.java?rev=1176669&r1=1176668&r2=1176669&view=diff
==============================================================================
--- incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/StateMachineInvoker.java (original)
+++ incubator/ambari/trunk/controller/src/main/java/org/apache/ambari/resource/statemachine/StateMachineInvoker.java Wed Sep 28 00:17:19 2011
@@ -17,10 +17,14 @@
 */
 package org.apache.ambari.resource.statemachine;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 
+import org.apache.ambari.components.ClusterContext;
 import org.apache.ambari.event.AsyncDispatcher;
 import org.apache.ambari.event.Dispatcher;
 import org.apache.ambari.event.EventHandler;
@@ -59,11 +63,20 @@ public class StateMachineInvoker {
     }
   }
   
-  public static Cluster createCluster(String clusterName) {
-    return new ClusterImpl(clusterName);
+  private static ConcurrentMap<String, Cluster> clusters = 
+      new ConcurrentHashMap<String, Cluster>();
+  public static Cluster createCluster(String clusterId) {
+    ClusterImpl cluster = new ClusterImpl(clusterId);
+    clusters.put(clusterId, cluster);
+    return cluster;
   }
   
-  public static Service addServiceInCluster(Cluster cluster, String serviceName) {
+  public static Cluster getStateMachineClusterInstance(String clusterId) {
+    return clusters.get(clusterId);
+  }
+  
+  public static Service addServiceInCluster(Cluster cluster, String serviceName) 
+      throws IOException {
     Service service = new ServiceImpl(cluster, serviceName);
     return addServiceInCluster(cluster, service);
   }