You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@stratos.apache.org by re...@apache.org on 2014/10/15 12:03:10 UTC

[1/2] git commit: removing the local references to Topology from Monitors

Repository: stratos
Updated Branches:
  refs/heads/4.0.0-grouping fde396a03 -> d4a9a3b88


removing the local references to Topology from Monitors


Project: http://git-wip-us.apache.org/repos/asf/stratos/repo
Commit: http://git-wip-us.apache.org/repos/asf/stratos/commit/014e1562
Tree: http://git-wip-us.apache.org/repos/asf/stratos/tree/014e1562
Diff: http://git-wip-us.apache.org/repos/asf/stratos/diff/014e1562

Branch: refs/heads/4.0.0-grouping
Commit: 014e15627e4f9c6bdbb3bab9c4f0b7d634e4be3f
Parents: 961e584
Author: reka <rt...@gmail.com>
Authored: Wed Oct 15 15:29:41 2014 +0530
Committer: reka <rt...@gmail.com>
Committed: Wed Oct 15 15:29:41 2014 +0530

----------------------------------------------------------------------
 .../AutoscalerTopologyEventReceiver.java        | 24 +++----
 .../stratos/autoscaler/monitor/Monitor.java     | 69 ++++++++++++--------
 .../monitor/application/ApplicationMonitor.java |  4 +-
 .../autoscaler/monitor/group/GroupMonitor.java  | 15 +++--
 .../stratos/autoscaler/util/AutoscalerUtil.java | 39 +++++++++--
 5 files changed, 100 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/stratos/blob/014e1562/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/message/receiver/topology/AutoscalerTopologyEventReceiver.java
----------------------------------------------------------------------
diff --git a/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/message/receiver/topology/AutoscalerTopologyEventReceiver.java b/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/message/receiver/topology/AutoscalerTopologyEventReceiver.java
index a7b6eda..801af0e 100644
--- a/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/message/receiver/topology/AutoscalerTopologyEventReceiver.java
+++ b/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/message/receiver/topology/AutoscalerTopologyEventReceiver.java
@@ -97,7 +97,7 @@ public class AutoscalerTopologyEventReceiver implements Runnable {
                     TopologyManager.acquireReadLock();
                     try {
                         for (Application application : TopologyManager.getTopology().getApplications()) {
-                            startApplicationMonitor(application);
+                            startApplicationMonitor(application.getUniqueIdentifier());
                         }
 
                         topologyInitialized = true;
@@ -123,7 +123,7 @@ public class AutoscalerTopologyEventReceiver implements Runnable {
                         TopologyManager.acquireReadLockForApplication(
                                 applicationCreatedEvent.getApplication().getUniqueIdentifier());
                         //start the application monitor
-                        startApplicationMonitor(applicationCreatedEvent.getApplication());
+                        startApplicationMonitor(applicationCreatedEvent.getApplication().getUniqueIdentifier());
                     } catch (Exception e) {
                         String msg = "Error processing event " + e.getLocalizedMessage();
                         log.error(msg, e);
@@ -577,12 +577,12 @@ public class AutoscalerTopologyEventReceiver implements Runnable {
         terminated = true;
     }
 
-    protected synchronized void startApplicationMonitor(Application application) {
+    protected synchronized void startApplicationMonitor(String applicationId) {
         Thread th = null;
         if (!AutoscalerContext.getInstance()
-                .appMonitorExist(application.getUniqueIdentifier())) {
+                .appMonitorExist(applicationId)) {
             th = new Thread(
-                    new ApplicationMonitorAdder(application));
+                    new ApplicationMonitorAdder(applicationId));
         }
       //  if (th != null) {
             th.start();
@@ -594,16 +594,16 @@ public class AutoscalerTopologyEventReceiver implements Runnable {
             if (log.isDebugEnabled()) {
                 log.debug(String
                         .format("Application monitor thread has been started successfully: " +
-                                        "[application] %s ", application.getUniqueIdentifier()));
+                                        "[application] %s ", applicationId));
             }
       //  }
     }
 
     private class ApplicationMonitorAdder implements Runnable {
-        private Application application;
+        private String  appId;
 
-        public ApplicationMonitorAdder(Application application) {
-            this.application = application;
+        public ApplicationMonitorAdder(String appId) {
+            this.appId = appId;
         }
 
         public void run() {
@@ -619,9 +619,9 @@ public class AutoscalerTopologyEventReceiver implements Runnable {
                     long start = System.currentTimeMillis();
                     if (log.isDebugEnabled()) {
                         log.debug("application monitor is going to be started for [application] " +
-                                application.getUniqueIdentifier());
+                                appId);
                     }
-                    applicationMonitor = AutoscalerUtil.getApplicationMonitor(application);
+                    applicationMonitor = AutoscalerUtil.getApplicationMonitor(appId);
 
                     long end = System.currentTimeMillis();
                     log.info("Time taken to start app monitor: " + (end - start) / 1000);
@@ -639,7 +639,7 @@ public class AutoscalerTopologyEventReceiver implements Runnable {
 
             if (applicationMonitor == null) {
                 String msg = "Application monitor creation failed, even after retrying for 5 times, "
-                        + "for Application: " + application.getUniqueIdentifier();
+                        + "for Application: " + appId;
                 log.error(msg);
                 throw new RuntimeException(msg);
             }

http://git-wip-us.apache.org/repos/asf/stratos/blob/014e1562/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/Monitor.java
----------------------------------------------------------------------
diff --git a/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/Monitor.java b/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/Monitor.java
index cc30f7b..4f501d2 100644
--- a/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/Monitor.java
+++ b/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/Monitor.java
@@ -57,8 +57,6 @@ public abstract class Monitor implements EventHandler {
     protected Map<String, AbstractClusterMonitor> clusterIdToClusterMonitorsMap;
     //The monitors dependency tree with all the startable/killable dependencies
     protected DependencyTree dependencyTree;
-    //application/group reference from the Topology
-    protected ParentComponent component;
     //status of the monitor whether it is running/in_maintainable/terminated
     protected Status status;
     //Application id of this particular monitor
@@ -67,7 +65,8 @@ public abstract class Monitor implements EventHandler {
     public Monitor(ParentComponent component) throws DependencyBuilderException {
         aliasToGroupMonitorsMap = new HashMap<String, GroupMonitor>();
         clusterIdToClusterMonitorsMap = new HashMap<String, AbstractClusterMonitor>();
-        this.component = component;
+        this.id = component.getUniqueIdentifier();
+        this.status = component.getStatus();
         //Building the dependency for this monitor within the immediate children
         dependencyTree = DependencyBuilder.getInstance().buildDependency(component);
     }
@@ -120,15 +119,18 @@ public abstract class Monitor implements EventHandler {
                 log.debug("Dependency check for the Group " + context.getId() + " started");
             }
             if (context instanceof GroupContext) {
-                startGroupMonitor(this, context.getId(), component);
+                startGroupMonitor(this, context.getId());
             } else if (context instanceof ClusterContext) {
                 String clusterId = context.getId();
                 String serviceName = null;
-                for(ClusterDataHolder dataHolder : component.getClusterDataMap().values()) {
+                Group group = getGroupFromTopology(this.id);
+
+                for(ClusterDataHolder dataHolder : group.getClusterDataMap().values()) {
                     if(dataHolder.getClusterId().equals(clusterId)) {
                         serviceName = dataHolder.getServiceType();
                     }
                 }
+
                 Cluster cluster;
                 //acquire read lock for the service and cluster
                 TopologyManager.acquireReadLockForCluster(serviceName, clusterId);
@@ -162,6 +164,22 @@ public abstract class Monitor implements EventHandler {
 
     }
 
+    private Group getGroupFromTopology(String groupId) throws TopologyInConsistentException {
+        Application application = TopologyManager.getTopology().getApplication(this.appId);
+        if(application != null) {
+            Group group = application.getGroupRecursively(groupId);
+            if(group != null) {
+                return group;
+            } else {
+                String msg = "[Group] " + groupId + " cannot be found in the Topology";
+                throw new TopologyInConsistentException(msg);
+            }
+        } else {
+            String msg = "[Application] " + this.appId + " cannot be found in the Topology";
+            throw new TopologyInConsistentException(msg);
+        }
+    }
+
     protected synchronized void startClusterMonitor(Monitor parent, Cluster cluster) {
         Thread th = null;
         if (cluster.isLbCluster()
@@ -190,16 +208,17 @@ public abstract class Monitor implements EventHandler {
         }
     }
 
-    protected synchronized void startGroupMonitor(Monitor parent, String dependency, ParentComponent component) {
+    protected synchronized void startGroupMonitor(Monitor parent, String groupId) {
         Thread th = null;
-        if (!this.aliasToGroupMonitorsMap.containsKey(dependency)) {
+        //String groupId = group.getUniqueIdentifier();
+        if (!this.aliasToGroupMonitorsMap.containsKey(groupId)) {
             if (log.isDebugEnabled()) {
                 log.debug(String
                         .format("Group monitor Adder has been added: [group] %s ",
-                                dependency));
+                                groupId));
             }
             th = new Thread(
-                    new GroupMonitorAdder(parent, dependency, component));
+                    new GroupMonitorAdder(parent, groupId, this.appId));
         }
 
         if (th != null) {
@@ -211,7 +230,7 @@ public abstract class Monitor implements EventHandler {
 
             log.info(String
                     .format("Group monitor thread has been started successfully: [group] %s ",
-                            dependency));
+                            groupId));
         }
     }
 
@@ -330,14 +349,14 @@ public abstract class Monitor implements EventHandler {
     }
 
     private class GroupMonitorAdder implements Runnable {
-        private String dependency;
         private Monitor parent;
-        private ParentComponent component;
+        private String groupId;
+        private String appId;
 
-        public GroupMonitorAdder(Monitor parent, String dependency, ParentComponent group) {
-            this.dependency = dependency;
+        public GroupMonitorAdder(Monitor parent, String groupId, String appId) {
             this.parent = parent;
-            this.component = group;
+            this.groupId = groupId;
+            this.appId = appId;
         }
 
         public void run() {
@@ -353,25 +372,21 @@ public abstract class Monitor implements EventHandler {
                 try {
                     if (log.isDebugEnabled()) {
                         log.debug("Group monitor is going to be started for [group] "
-                                + dependency);
+                                + groupId);
                     }
-                    Group group = component.getGroup(dependency);
-                    monitor = AutoscalerUtil.getGroupMonitor(group);
+                    monitor = AutoscalerUtil.getGroupMonitor(groupId, appId);
                     monitor.setParent(parent);
                     //setting the status of cluster monitor w.r.t Topology cluster
                     //if(group.getStatus() != Status.Created &&
-                    if(group.getStatus() != monitor.getStatus()) {
-                        //updating the status, so that it will notify the parent
-                        monitor.setStatus(group.getStatus());
-                    }
+
                     //monitor.addObserver(parent);
                     success = true;
                 } catch (DependencyBuilderException e) {
-                    String msg = "Group monitor creation failed for group: " + dependency;
+                    String msg = "Group monitor creation failed for group: " + groupId;
                     log.warn(msg, e);
                     retries--;
                 } catch (TopologyInConsistentException e) {
-                    String msg = "Group monitor creation failed for group: " + dependency;
+                    String msg = "Group monitor creation failed for group: " + groupId;
                     log.warn(msg, e);
                     retries--;
                 }
@@ -379,19 +394,19 @@ public abstract class Monitor implements EventHandler {
 
             if (monitor == null) {
                 String msg = "Group monitor creation failed, even after retrying for 5 times, "
-                        + "for group: " + dependency;
+                        + "for group: " + groupId;
                 log.error(msg);
                 //TODO parent.notify(); as it got to failed
 
                 throw new RuntimeException(msg);
             }
 
-            aliasToGroupMonitorsMap.put(dependency, monitor);
+            aliasToGroupMonitorsMap.put(groupId, monitor);
             //parent.addObserver(monitor);
 
             if (log.isInfoEnabled()) {
                 log.info(String.format("Group monitor has been added successfully: [group] %s",
-                        dependency));
+                        groupId));
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/stratos/blob/014e1562/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/application/ApplicationMonitor.java
----------------------------------------------------------------------
diff --git a/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/application/ApplicationMonitor.java b/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/application/ApplicationMonitor.java
index 7a73e2b..5b6598a 100644
--- a/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/application/ApplicationMonitor.java
+++ b/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/application/ApplicationMonitor.java
@@ -29,6 +29,7 @@ import org.apache.stratos.autoscaler.monitor.events.MonitorStatusEvent;
 import org.apache.stratos.autoscaler.monitor.group.GroupMonitor;
 import org.apache.stratos.autoscaler.status.checker.StatusChecker;
 import org.apache.stratos.messaging.domain.topology.Application;
+import org.apache.stratos.messaging.domain.topology.ParentComponent;
 import org.apache.stratos.messaging.domain.topology.Status;
 
 import java.util.ArrayList;
@@ -44,9 +45,10 @@ public class ApplicationMonitor extends Monitor {
     public ApplicationMonitor(Application application) throws DependencyBuilderException,
                                                         TopologyInConsistentException {
         super(application);
+        this.appId = application.getUniqueIdentifier();
         //starting the first set of dependencies from its children
-        this.id = application.getUniqueIdentifier();
         startDependency();
+
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/stratos/blob/014e1562/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/group/GroupMonitor.java
----------------------------------------------------------------------
diff --git a/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/group/GroupMonitor.java b/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/group/GroupMonitor.java
index 5803822..0bdad16 100644
--- a/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/group/GroupMonitor.java
+++ b/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/group/GroupMonitor.java
@@ -43,6 +43,9 @@ public class GroupMonitor extends Monitor implements EventHandler {
 
     //Parent monitor of this monitor
     private Monitor parent;
+    //Application id of this particular monitor
+    protected String appId;
+
 
     /**
      * Constructor of GroupMonitor
@@ -53,10 +56,7 @@ public class GroupMonitor extends Monitor implements EventHandler {
     public GroupMonitor(Group group) throws DependencyBuilderException,
                                             TopologyInConsistentException {
         super(group);
-        this.id = group.getAlias();
         startDependency();
-        this.status = Status.Created;
-
     }
 
     /**
@@ -113,7 +113,14 @@ public class GroupMonitor extends Monitor implements EventHandler {
 
     public void setParent(Monitor parent) {
         this.parent = parent;
-        this.appId = parent.getAppId();
+    }
+
+    public String getAppId() {
+        return appId;
+    }
+
+    public void setAppId(String appId) {
+        this.appId = appId;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/stratos/blob/014e1562/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/util/AutoscalerUtil.java
----------------------------------------------------------------------
diff --git a/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/util/AutoscalerUtil.java b/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/util/AutoscalerUtil.java
index f574367..cb03158 100644
--- a/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/util/AutoscalerUtil.java
+++ b/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/util/AutoscalerUtil.java
@@ -45,6 +45,7 @@ import org.apache.stratos.cloud.controller.stub.pojo.MemberContext;
 import org.apache.stratos.cloud.controller.stub.pojo.Property;
 import org.apache.stratos.cloud.controller.stub.pojo.Properties;
 import org.apache.stratos.messaging.domain.topology.*;
+import org.apache.stratos.messaging.message.receiver.topology.TopologyManager;
 import org.apache.stratos.messaging.util.Constants;
 
 import javax.xml.namespace.QName;
@@ -323,20 +324,44 @@ public class AutoscalerUtil {
     }
 
     //TODO moving it into factory class
-    public static GroupMonitor getGroupMonitor(Group group) throws DependencyBuilderException,
+    public static GroupMonitor getGroupMonitor(String groupId, String appId) throws DependencyBuilderException,
                                                             TopologyInConsistentException {
-        GroupMonitor groupMonitor =
-                new GroupMonitor(group);
+        GroupMonitor groupMonitor;
+        TopologyManager.acquireReadLockForApplication(appId);
+
+        try {
+            Group group = TopologyManager.getTopology().getApplication(appId).getGroupRecursively(groupId);
+            groupMonitor = new GroupMonitor(group);
+            groupMonitor.setAppId(appId);
+            if(group.getStatus() != groupMonitor.getStatus()) {
+                //updating the status, so that it will notify the parent
+                groupMonitor.setStatus(group.getStatus());
+            }
+        } finally {
+            TopologyManager.releaseReadLockForApplication(appId);
+
+        }
         return groupMonitor;
 
     }
 
-    public static ApplicationMonitor getApplicationMonitor(Application application)
+    public static ApplicationMonitor getApplicationMonitor(String appId)
                                             throws DependencyBuilderException,
                                             TopologyInConsistentException {
-        ApplicationMonitor applicationMonitor =
-                new ApplicationMonitor(application);
-        applicationMonitor.setAppId(application.getUniqueIdentifier());
+        ApplicationMonitor applicationMonitor;
+        TopologyManager.acquireReadLockForApplication(appId);
+        try {
+            Application application = TopologyManager.getTopology().getApplication(appId);
+            if(application != null) {
+                applicationMonitor = new ApplicationMonitor(application);
+            } else {
+                String msg = "[Application] " + appId + " cannot be found in the Topology";
+                throw new TopologyInConsistentException(msg);
+            }
+        } finally {
+            TopologyManager.releaseReadLockForApplication(appId);
+        }
+
         return applicationMonitor;
 
     }


[2/2] git commit: Merge branch '4.0.0-grouping' of https://git-wip-us.apache.org/repos/asf/stratos into 4.0.0-grouping

Posted by re...@apache.org.
Merge branch '4.0.0-grouping' of https://git-wip-us.apache.org/repos/asf/stratos into 4.0.0-grouping


Project: http://git-wip-us.apache.org/repos/asf/stratos/repo
Commit: http://git-wip-us.apache.org/repos/asf/stratos/commit/d4a9a3b8
Tree: http://git-wip-us.apache.org/repos/asf/stratos/tree/d4a9a3b8
Diff: http://git-wip-us.apache.org/repos/asf/stratos/diff/d4a9a3b8

Branch: refs/heads/4.0.0-grouping
Commit: d4a9a3b883bd91594da3eb98905de126a2ebcca1
Parents: 014e156 fde396a
Author: reka <rt...@gmail.com>
Authored: Wed Oct 15 15:32:21 2014 +0530
Committer: reka <rt...@gmail.com>
Committed: Wed Oct 15 15:32:21 2014 +0530

----------------------------------------------------------------------
 .../org.apache.stratos.metadata.client/pom.xml  | 51 ++++++-----
 .../client/DefaultMetaDataServiceClient.java    | 39 +++-----
 .../metadata/client/MetaDataServiceClient.java  | 25 +++---
 .../metadata/client/beans/PropertyBean.java     | 23 ++---
 .../client/config/MetaDataClientConfig.java     | 74 +++++----------
 .../data/extractor/MetaDataExtractor.java       | 36 --------
 .../exception/MetaDataExtractorException.java   | 47 ----------
 .../factory/MetaDataExtractorFactory.java       | 51 -----------
 .../metadata/client/pojo/DataContext.java       | 88 ------------------
 .../metadata/client/rest/DefaultRestClient.java | 59 ++++++------
 .../client/rest/HTTPConnectionManager.java      | 47 ++++++++++
 .../metadata/client/rest/RestClient.java        | 26 +++++-
 .../client/sample/DefaultMetaDataExtractor.java | 45 ----------
 .../sample/MetaDataServiceClientSample.java     | 92 -------------------
 .../client/util/MetaDataClientConstants.java    |  2 -
 tools/puppet3/modules/mysql/manifests/init.pp   | 10 +++
 .../mysql/templates/instance-activated.sh.erb   | 28 ++++++
 .../mysql/templates/instance-started.sh.erb     | 43 +++++++++
 .../templates/mysql_root_password_setter.sh.erb | 16 ++++
 .../mysql/templates/publish-metadata.py.erb     | 95 ++++++++++++++++++++
 20 files changed, 374 insertions(+), 523 deletions(-)
----------------------------------------------------------------------