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

[1/2] git commit: AS npe exception when getting partition context for a member

Updated Branches:
  refs/heads/master e94dcfad2 -> 52d9282ea


AS npe exception when getting partition context for a member


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

Branch: refs/heads/master
Commit: 6aac43bc405dd50887d2935f0c0bc6958b03ec61
Parents: ede69d5
Author: Udara Liyanage <ud...@wso2.com>
Authored: Thu Dec 19 14:44:03 2013 -0500
Committer: Udara Liyanage <ud...@wso2.com>
Committed: Thu Dec 19 14:44:03 2013 -0500

----------------------------------------------------------------------
 .../stratos/autoscaler/AutoscalerContext.java   |   3 +
 .../health/HealthEventMessageDelegator.java     |  30 ++--
 .../autoscaler/monitor/AbstractMonitor.java     | 157 +++++++++++++++++-
 .../autoscaler/monitor/ClusterMonitor.java      | 160 +------------------
 .../autoscaler/monitor/LbClusterMonitor.java    | 156 +-----------------
 .../topology/AutoscalerTopologyReceiver.java    |   4 +-
 6 files changed, 181 insertions(+), 329 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-stratos/blob/6aac43bc/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/AutoscalerContext.java
----------------------------------------------------------------------
diff --git a/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/AutoscalerContext.java b/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/AutoscalerContext.java
index 5f59dcc..e809db9 100644
--- a/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/AutoscalerContext.java
+++ b/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/AutoscalerContext.java
@@ -24,7 +24,10 @@ public class AutoscalerContext {
             log.error("Rule evaluateMinCheck error", e);
         }
     }
+    
+    // Map<ClusterId, ClusterMonitor>
     private Map<String, ClusterMonitor> monitors;
+    // Map<LBClusterId, LBClusterMonitor>
     private Map<String, LbClusterMonitor> lbMonitors;
 
     public static AutoscalerContext getInstance() {

http://git-wip-us.apache.org/repos/asf/incubator-stratos/blob/6aac43bc/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/message/receiver/health/HealthEventMessageDelegator.java
----------------------------------------------------------------------
diff --git a/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/message/receiver/health/HealthEventMessageDelegator.java b/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/message/receiver/health/HealthEventMessageDelegator.java
index 1c5eba5..d8171e7 100644
--- a/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/message/receiver/health/HealthEventMessageDelegator.java
+++ b/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/message/receiver/health/HealthEventMessageDelegator.java
@@ -37,6 +37,7 @@ import org.apache.stratos.cloud.controller.deployment.partition.Partition;
 import org.apache.stratos.messaging.domain.topology.Cluster;
 import org.apache.stratos.messaging.domain.topology.Member;
 import org.apache.stratos.messaging.domain.topology.Service;
+import org.apache.stratos.messaging.domain.topology.Topology;
 import org.apache.stratos.messaging.message.receiver.topology.TopologyManager;
 
 import javax.jms.TextMessage;
@@ -319,21 +320,29 @@ public class HealthEventMessageDelegator implements Runnable {
         	AutoscalerContext asCtx = AutoscalerContext.getInstance();
         	AbstractMonitor monitor = null;
         	
-        	if(asCtx.moniterExist(clusterId)){        		
-        		monitor = asCtx.getMonitor(clusterId);        		
-        	}else if(asCtx.lbMoniterExist(clusterId)){        		
-        		monitor = asCtx.getLBMonitor(clusterId);        		
+        	if(asCtx.moniterExist(clusterId)){
+        		monitor = asCtx.getMonitor(clusterId);
+        	}else if(asCtx.lbMoniterExist(clusterId)){
+        		monitor = asCtx.getLBMonitor(clusterId);
         	}else{
         		String errMsg = "A monitor is not found for this custer";
         		log.error(errMsg);
         		throw new RuntimeException(errMsg);
-        	}        
-            
-            if (!monitor.memberExist(memberId)) {
-                // member has already terminated. So no action required
-                return;
+        	}
+        	
+        	NetworkPartitionContext nwPartitionCtxt;
+            try{
+            	TopologyManager.acquireReadLock();
+            	Member member = monitor.getMember(memberId);
+	            if (null == member) {
+	                // member has already terminated. So no action required
+	                return;
+	            } else{
+	            	nwPartitionCtxt = monitor.getNetworkPartitionCtxt(member);
+	            }
+            }finally{
+            	TopologyManager.releaseReadLock();
             }
-
             // terminate the faulty member
             CloudControllerClient ccClient = CloudControllerClient.getInstance();
             ccClient.terminate(memberId);
@@ -341,7 +350,6 @@ public class HealthEventMessageDelegator implements Runnable {
             // start a new member in the same Partition
             String partitionId = monitor.getPartitionOfMember(memberId);
             Partition partition = monitor.getDeploymentPolicy().getPartitionById(partitionId);
-            NetworkPartitionContext nwPartitionCtxt = monitor.findNetworkPartition(memberId);
             PartitionContext partitionCtxt = nwPartitionCtxt.getPartitionCtxt(partitionId);
             
             String lbClusterId = AutoscalerRuleEvaluator.getLbClusterId(partitionCtxt, nwPartitionCtxt);

http://git-wip-us.apache.org/repos/asf/incubator-stratos/blob/6aac43bc/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/AbstractMonitor.java
----------------------------------------------------------------------
diff --git a/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/AbstractMonitor.java b/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/AbstractMonitor.java
index 740db07..aa2bb8c 100644
--- a/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/AbstractMonitor.java
+++ b/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/AbstractMonitor.java
@@ -18,8 +18,20 @@
  */
 package org.apache.stratos.autoscaler.monitor;
 
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.stratos.autoscaler.NetworkPartitionContext;
 import org.apache.stratos.autoscaler.deployment.policy.DeploymentPolicy;
+import org.apache.stratos.autoscaler.policy.model.AutoscalePolicy;
+import org.apache.stratos.autoscaler.rule.AutoscalerRuleEvaluator;
+import org.apache.stratos.messaging.domain.topology.Cluster;
+import org.apache.stratos.messaging.domain.topology.Member;
+import org.apache.stratos.messaging.domain.topology.Service;
+import org.apache.stratos.messaging.message.receiver.topology.TopologyManager;
+import org.drools.runtime.StatefulKnowledgeSession;
+import org.drools.runtime.rule.FactHandle;
 
 /**
  * Is responsible for monitoring a service cluster. This runs periodically
@@ -29,19 +41,152 @@ import org.apache.stratos.autoscaler.deployment.policy.DeploymentPolicy;
  */
    abstract public class AbstractMonitor implements Runnable{
 
+	private static final Log log = LogFactory.getLog(AbstractMonitor.class);
+	// Map<NetworkpartitionId, Network Partition Context>
+	protected Map<String, NetworkPartitionContext> networkPartitionCtxts;
+	protected DeploymentPolicy deploymentPolicy;
+	protected AutoscalePolicy autoscalePolicy;
+	
+
+	protected FactHandle minCheckFactHandle;
+	protected FactHandle scaleCheckFactHandle;
+	
+	protected StatefulKnowledgeSession minCheckKnowledgeSession;
+	protected StatefulKnowledgeSession scaleCheckKnowledgeSession;
+	protected boolean isDestroyed;
+	
+	protected String clusterId;
+	protected String serviceId;
+	
+	protected AutoscalerRuleEvaluator autoscalerRuleEvaluator;
+	
 	@Override
 	public void run() {
 		// TODO Auto-generated method stub
 		
 	}
 	
-	public abstract String getPartitionOfMember(String memberId);
-	public abstract DeploymentPolicy getDeploymentPolicy();
-	public abstract boolean memberExist(String memberId);
-	public abstract NetworkPartitionContext findNetworkPartition(String memberId);
+	    
+   	public NetworkPartitionContext getNetworkPartitionCtxt(Member member) {
+   		log.info("***** getNetworkPartitionCtxt " + member.getNetworkPartitionId());
+		String networkPartitionId = member.getNetworkPartitionId();
+    	if(networkPartitionCtxts.containsKey(networkPartitionId)) {
+    		log.info("returnnig network partition context " + networkPartitionCtxts.get(networkPartitionId));
+    		return networkPartitionCtxts.get(networkPartitionId);
+    	}
+    	log.info("returning null getNetworkPartitionCtxt");
+   	    return null;
+   	}
+   	
+    public String getPartitionOfMember(String memberId){
+        for(Service service: TopologyManager.getTopology().getServices()){
+            for(Cluster cluster: service.getClusters()){
+                if(cluster.memberExists(memberId)){
+                    return cluster.getMember(memberId).getPartitionId();
+                }
+            }
+        }
+        return null;
+   	}
+        
+    public Member getMember(String memberId){
+    	
+        for(Service service: TopologyManager.getTopology().getServices()){
+            for(Cluster cluster: service.getClusters()){
+                if(cluster.memberExists(memberId)){
+                    return cluster.getMember(memberId);
+                }
+            }
+        }
+        return null;
+   	}
+    
+    public void destroy() {
+        minCheckKnowledgeSession.dispose();
+        scaleCheckKnowledgeSession.dispose();
+        setDestroyed(true);
+        if(log.isDebugEnabled()) {
+            log.debug("Cluster Monitor Drools session has been disposed. "+this.toString());
+        }
+    }
+    
+    public boolean isDestroyed() {
+        return isDestroyed;
+    }
+
+    public void setDestroyed(boolean isDestroyed) {
+        this.isDestroyed = isDestroyed;
+    }
+
+    public String getServiceId() {
+        return serviceId;
+    }
+
+    public void setServiceId(String serviceId) {
+        this.serviceId = serviceId;
+    }
+
+    public DeploymentPolicy getDeploymentPolicy() {
+        return deploymentPolicy;
+    }
+
+    public void setDeploymentPolicy(DeploymentPolicy deploymentPolicy) {
+        this.deploymentPolicy = deploymentPolicy;
+    }
+
+    public AutoscalePolicy getAutoscalePolicy() {
+        return autoscalePolicy;
+    }
+
+    public void setAutoscalePolicy(AutoscalePolicy autoscalePolicy) {
+        this.autoscalePolicy = autoscalePolicy;
+    }    
+    
+    public String getClusterId() {
+        return clusterId;
+    }
+
+    public void setClusterId(String clusterId) {
+        this.clusterId = clusterId;
+    }
+
+    public Map<String, NetworkPartitionContext> getNetworkPartitionCtxts() {
+        return networkPartitionCtxts;
+    }
+
+    public NetworkPartitionContext getNetworkPartitionCtxt(String networkPartitionId) {
+        return networkPartitionCtxts.get(networkPartitionId);
+    }
+
+    public void setPartitionCtxt(Map<String, NetworkPartitionContext> partitionCtxt) {
+        this.networkPartitionCtxts = partitionCtxt;
+    }
+
+    public boolean partitionCtxtAvailable(String partitionId) {
+        return networkPartitionCtxts.containsKey(partitionId);
+    }
+
+    public void addNetworkPartitionCtxt(NetworkPartitionContext ctxt) {
+        this.networkPartitionCtxts.put(ctxt.getId(), ctxt);
+    }
+    
+    public NetworkPartitionContext getPartitionCtxt(String id) {
+        return this.networkPartitionCtxts.get(id);
+    }
+
+    public StatefulKnowledgeSession getMinCheckKnowledgeSession() {
+        return minCheckKnowledgeSession;
+    }
 
+    public void setMinCheckKnowledgeSession(StatefulKnowledgeSession minCheckKnowledgeSession) {
+        this.minCheckKnowledgeSession = minCheckKnowledgeSession;
+    }
 
-    public abstract NetworkPartitionContext getNetworkPartitionCtxt(String networkPartitionId);
+    public FactHandle getMinCheckFactHandle() {
+        return minCheckFactHandle;
+    }
 
-    public abstract void destroy();
+    public void setMinCheckFactHandle(FactHandle minCheckFactHandle) {
+        this.minCheckFactHandle = minCheckFactHandle;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-stratos/blob/6aac43bc/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/ClusterMonitor.java
----------------------------------------------------------------------
diff --git a/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/ClusterMonitor.java b/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/ClusterMonitor.java
index 21738a5..1d0bad4 100644
--- a/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/ClusterMonitor.java
+++ b/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/ClusterMonitor.java
@@ -18,6 +18,8 @@
  */
 package org.apache.stratos.autoscaler.monitor;
 
+import java.util.concurrent.ConcurrentHashMap;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.stratos.autoscaler.NetworkPartitionContext;
@@ -25,14 +27,6 @@ import org.apache.stratos.autoscaler.PartitionContext;
 import org.apache.stratos.autoscaler.deployment.policy.DeploymentPolicy;
 import org.apache.stratos.autoscaler.policy.model.AutoscalePolicy;
 import org.apache.stratos.autoscaler.rule.AutoscalerRuleEvaluator;
-import org.apache.stratos.messaging.domain.topology.Cluster;
-import org.apache.stratos.messaging.domain.topology.Service;
-import org.apache.stratos.messaging.message.receiver.topology.TopologyManager;
-import org.drools.runtime.StatefulKnowledgeSession;
-import org.drools.runtime.rule.FactHandle;
-
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
 
 /**
  * Is responsible for monitoring a service cluster. This runs periodically
@@ -42,30 +36,7 @@ import java.util.concurrent.ConcurrentHashMap;
  */
 public class ClusterMonitor extends AbstractMonitor{
 
-    private static final Log log = LogFactory.getLog(ClusterMonitor.class);
-    private String clusterId;
-
-    private String serviceId;
-
-    //key: network partition id, value: Network partition context
-    private Map<String, NetworkPartitionContext> networkPartitionCtxts;
-
-
-    private StatefulKnowledgeSession minCheckKnowledgeSession;
-    private StatefulKnowledgeSession scaleCheckKnowledgeSession;
-    private boolean isDestroyed;
-
-    private DeploymentPolicy deploymentPolicy;
-    private AutoscalePolicy autoscalePolicy;
-
-        // Key- MemberId Value- partitionId
-//    private Map<String, String> memberPartitionMap;
-
-    private FactHandle minCheckFactHandle;
-    private FactHandle scaleCheckFactHandle;
-
-    private AutoscalerRuleEvaluator autoscalerRuleEvaluator;
-    
+    private static final Log log = LogFactory.getLog(ClusterMonitor.class);    
     private String lbReferenceType;
 
     public ClusterMonitor(String clusterId, String serviceId, DeploymentPolicy deploymentPolicy,
@@ -82,53 +53,7 @@ public class ClusterMonitor extends AbstractMonitor{
         networkPartitionCtxts = new ConcurrentHashMap<String, NetworkPartitionContext>();
     }
 
-    public String getClusterId() {
-        return clusterId;
-    }
 
-    public void setClusterId(String clusterId) {
-        this.clusterId = clusterId;
-    }
-
-    public Map<String, NetworkPartitionContext> getNetworkPartitionCtxts() {
-        return networkPartitionCtxts;
-    }
-
-    public NetworkPartitionContext getNetworkPartitionCtxt(String networkPartitionId) {
-        return networkPartitionCtxts.get(networkPartitionId);
-    }
-
-    public void setPartitionCtxt(Map<String, NetworkPartitionContext> partitionCtxt) {
-        this.networkPartitionCtxts = partitionCtxt;
-    }
-
-    public boolean partitionCtxtAvailable(String partitionId) {
-        return networkPartitionCtxts.containsKey(partitionId);
-    }
-
-    public void addNetworkPartitionCtxt(NetworkPartitionContext ctxt) {
-        this.networkPartitionCtxts.put(ctxt.getId(), ctxt);
-    }
-    
-    public NetworkPartitionContext getPartitionCtxt(String id) {
-        return this.networkPartitionCtxts.get(id);
-    }
-
-    public StatefulKnowledgeSession getMinCheckKnowledgeSession() {
-        return minCheckKnowledgeSession;
-    }
-
-    public void setMinCheckKnowledgeSession(StatefulKnowledgeSession minCheckKnowledgeSession) {
-        this.minCheckKnowledgeSession = minCheckKnowledgeSession;
-    }
-
-    public FactHandle getMinCheckFactHandle() {
-        return minCheckFactHandle;
-    }
-
-    public void setMinCheckFactHandle(FactHandle minCheckFactHandle) {
-        this.minCheckFactHandle = minCheckFactHandle;
-    }
 
     @Override
     public void run() {
@@ -189,44 +114,6 @@ public class ClusterMonitor extends AbstractMonitor{
         }
     }
 
-    
-    public void destroy() {
-        minCheckKnowledgeSession.dispose();
-        scaleCheckKnowledgeSession.dispose();
-        setDestroyed(true);
-        if(log.isDebugEnabled()) {
-            log.debug("Cluster Monitor Drools session has been disposed. "+this.toString());
-        }
-    }
-
-    public boolean isDestroyed() {
-        return isDestroyed;
-    }
-
-    public void setDestroyed(boolean isDestroyed) {
-        this.isDestroyed = isDestroyed;
-    }
-
-    public String getServiceId() {
-        return serviceId;
-    }
-
-    public void setServiceId(String serviceId) {
-        this.serviceId = serviceId;
-    }
-
-    public DeploymentPolicy getDeploymentPolicy() {
-        return deploymentPolicy;
-    }
-
-    public void setDeploymentPolicy(DeploymentPolicy deploymentPolicy) {
-        this.deploymentPolicy = deploymentPolicy;
-    }
-
-    public AutoscalePolicy getAutoscalePolicy() {
-        return autoscalePolicy;
-    }
-
     @Override
     public String toString() {
         return "ClusterMonitor [clusterId=" + clusterId + ", serviceId=" + serviceId +
@@ -234,33 +121,6 @@ public class ClusterMonitor extends AbstractMonitor{
                ", lbReferenceType=" + lbReferenceType + "]";
     }
 
-    public void setAutoscalePolicy(AutoscalePolicy autoscalePolicy) {
-        this.autoscalePolicy = autoscalePolicy;
-    }
-
-    public String getPartitionOfMember(String memberId){
-        for(Service service: TopologyManager.getTopology().getServices()){
-            for(Cluster cluster: service.getClusters()){
-                if(cluster.memberExists(memberId)){
-                    cluster.getMember(memberId).getPartitionId();
-                }
-            }
-        }
-        return null;
-   	}
-
-   	@Override
-    public boolean memberExist(String memberId){
-        for(Service service: TopologyManager.getTopology().getServices()){
-            for(Cluster cluster: service.getClusters()){
-                if(cluster.memberExists(memberId)){
-                    return true;
-                }
-            }
-        }
-        return false;
-   	}
-
     public String getLbReferenceType() {
         return lbReferenceType;
     }
@@ -268,18 +128,4 @@ public class ClusterMonitor extends AbstractMonitor{
     public void setLbReferenceType(String lbReferenceType) {
         this.lbReferenceType = lbReferenceType;
     }
-
-	@Override
-	public NetworkPartitionContext findNetworkPartition(String memberId) {
-		 for(Service service: TopologyManager.getTopology().getServices()){
-	            for(Cluster cluster: service.getClusters()){
-
-                    String networkPartitionId = cluster.getMember(memberId).getNetworkPartitionId();
-	                if(networkPartitionCtxts.containsKey(networkPartitionId)) {
-                        networkPartitionCtxts.get(networkPartitionId);
-                    }
-	            }
-	      }
-	      return null;
-	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-stratos/blob/6aac43bc/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/LbClusterMonitor.java
----------------------------------------------------------------------
diff --git a/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/LbClusterMonitor.java b/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/LbClusterMonitor.java
index f180999..d5c1c92 100644
--- a/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/LbClusterMonitor.java
+++ b/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/monitor/LbClusterMonitor.java
@@ -18,6 +18,8 @@
  */
 package org.apache.stratos.autoscaler.monitor;
 
+import java.util.concurrent.ConcurrentHashMap;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.stratos.autoscaler.NetworkPartitionContext;
@@ -25,14 +27,6 @@ import org.apache.stratos.autoscaler.PartitionContext;
 import org.apache.stratos.autoscaler.deployment.policy.DeploymentPolicy;
 import org.apache.stratos.autoscaler.policy.model.AutoscalePolicy;
 import org.apache.stratos.autoscaler.rule.AutoscalerRuleEvaluator;
-import org.apache.stratos.messaging.domain.topology.Cluster;
-import org.apache.stratos.messaging.domain.topology.Service;
-import org.apache.stratos.messaging.message.receiver.topology.TopologyManager;
-import org.drools.runtime.StatefulKnowledgeSession;
-import org.drools.runtime.rule.FactHandle;
-
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
 
 /**
  * Is responsible for monitoring a service cluster. This runs periodically
@@ -43,25 +37,6 @@ import java.util.concurrent.ConcurrentHashMap;
 public class LbClusterMonitor extends AbstractMonitor{
 
     private static final Log log = LogFactory.getLog(LbClusterMonitor.class);
-    private String clusterId;
-    private String serviceId;
-
-    private Map<String, NetworkPartitionContext> networkPartitionCtxts;
-
-    private StatefulKnowledgeSession minCheckKnowledgeSession;
-    private StatefulKnowledgeSession scaleCheckKnowledgeSession;
-    private boolean isDestroyed;
-
-    private DeploymentPolicy deploymentPolicy;
-    private AutoscalePolicy autoscalePolicy;
-
-        // Key- MemberId Value- partitionId
-//    private Map<String, String> memberPartitionMap;
-
-    private FactHandle minCheckFactHandle;
-    private FactHandle scaleCheckFactHandle;
-
-    private AutoscalerRuleEvaluator autoscalerRuleEvaluator;
 
     public LbClusterMonitor(String clusterId, String serviceId, DeploymentPolicy deploymentPolicy,
                             AutoscalePolicy autoscalePolicy) {
@@ -77,54 +52,6 @@ public class LbClusterMonitor extends AbstractMonitor{
         networkPartitionCtxts = new ConcurrentHashMap<String, NetworkPartitionContext>();
     }
 
-    public String getClusterId() {
-        return clusterId;
-    }
-
-    public void setClusterId(String clusterId) {
-        this.clusterId = clusterId;
-    }
-
-    public Map<String, NetworkPartitionContext> getNetworkPartitionCtxts() {
-        return networkPartitionCtxts;
-    }
-
-    public NetworkPartitionContext getNetworkPartitionCtxt(String networkPartitionId) {
-        return networkPartitionCtxts.get(networkPartitionId);
-    }
-
-    public void setPartitionCtxt(Map<String, NetworkPartitionContext> partitionCtxt) {
-        this.networkPartitionCtxts = partitionCtxt;
-    }
-
-    public boolean partitionCtxtAvailable(String partitionId) {
-        return networkPartitionCtxts.containsKey(partitionId);
-    }
-
-    public void addNetworkPartitionCtxt(NetworkPartitionContext ctxt) {
-        this.networkPartitionCtxts.put(ctxt.getId(), ctxt);
-    }
-    
-    public NetworkPartitionContext getPartitionCtxt(String id) {
-        return this.networkPartitionCtxts.get(id);
-    }
-
-    public StatefulKnowledgeSession getMinCheckKnowledgeSession() {
-        return minCheckKnowledgeSession;
-    }
-
-    public void setMinCheckKnowledgeSession(StatefulKnowledgeSession minCheckKnowledgeSession) {
-        this.minCheckKnowledgeSession = minCheckKnowledgeSession;
-    }
-
-    public FactHandle getMinCheckFactHandle() {
-        return minCheckFactHandle;
-    }
-
-    public void setMinCheckFactHandle(FactHandle minCheckFactHandle) {
-        this.minCheckFactHandle = minCheckFactHandle;
-    }
-
     @Override
     public void run() {
 
@@ -172,89 +99,12 @@ public class LbClusterMonitor extends AbstractMonitor{
             }
 
         }
-    }
-
-    
-    public void destroy() {
-        minCheckKnowledgeSession.dispose();
-        scaleCheckKnowledgeSession.dispose();
-        setDestroyed(true);
-        if(log.isDebugEnabled()) {
-            log.debug("Cluster Monitor Drools session has been disposed. "+this.toString());
-        }
-    }
-
-    public boolean isDestroyed() {
-        return isDestroyed;
-    }
-
-    public void setDestroyed(boolean isDestroyed) {
-        this.isDestroyed = isDestroyed;
-    }
-
-    public String getServiceId() {
-        return serviceId;
-    }
-
-    public void setServiceId(String serviceId) {
-        this.serviceId = serviceId;
-    }
-
-    public DeploymentPolicy getDeploymentPolicy() {
-        return deploymentPolicy;
-    }
-
-    public void setDeploymentPolicy(DeploymentPolicy deploymentPolicy) {
-        this.deploymentPolicy = deploymentPolicy;
-    }
-
-    public AutoscalePolicy getAutoscalePolicy() {
-        return autoscalePolicy;
-    }
-
-    public void setAutoscalePolicy(AutoscalePolicy autoscalePolicy) {
-        this.autoscalePolicy = autoscalePolicy;
-    }
-
-    public String getPartitionOfMember(String memberId){
-        for(Service service: TopologyManager.getTopology().getServices()){
-            for(Cluster cluster: service.getClusters()){
-                if(cluster.memberExists(memberId)){
-                    return cluster.getMember(memberId).getPartitionId();
-                }
-            }
-        }
-        return null;
-   	}
-
-    @Override
-    public boolean memberExist(String memberId){
-        for(Service service: TopologyManager.getTopology().getServices()){
-            for(Cluster cluster: service.getClusters()){
-                if(cluster.memberExists(memberId)){
-                    return true;
-                }
-            }
-        }
-        return false;
-   	}
+    }       
 
     @Override
     public String toString() {
         return "LbClusterMonitor [clusterId=" + clusterId + ", serviceId=" + serviceId + "]";
     }
 
-    @Override
-   	public NetworkPartitionContext findNetworkPartition(String memberId) {
-   		 for(Service service: TopologyManager.getTopology().getServices()){
-   	            for(Cluster cluster: service.getClusters()){
 
-                       String networkPartitionId = cluster.getMember(memberId).getNetworkPartitionId();
-   	                if(networkPartitionCtxts.containsKey(networkPartitionId)) {
-                           return networkPartitionCtxts.get(networkPartitionId);
-                       }
-   	            }
-   	      }
-   	      return null;
-   	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-stratos/blob/6aac43bc/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/topology/AutoscalerTopologyReceiver.java
----------------------------------------------------------------------
diff --git a/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/topology/AutoscalerTopologyReceiver.java b/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/topology/AutoscalerTopologyReceiver.java
index cdf6dd5..5ecfad9 100644
--- a/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/topology/AutoscalerTopologyReceiver.java
+++ b/components/org.apache.stratos.autoscaler/src/main/java/org/apache/stratos/autoscaler/topology/AutoscalerTopologyReceiver.java
@@ -231,7 +231,7 @@ public class AutoscalerTopologyReceiver implements Runnable {
                     
 					if(AutoscalerContext.getInstance().moniterExist(clusterId)) {
                         monitor = AutoscalerContext.getInstance().getMonitor(clusterId);
-					    partitionContext = monitor.getNetworkPartitionCtxt(networkPartitionId).getPartitionCtxt(partitionId);
+					    partitionContext = monitor.getNetworkPartitionCtxt(networkPartitionId).getPartitionCtxt(partitionId);					    
 					} else {
 					    monitor = AutoscalerContext.getInstance().getLBMonitor(clusterId);
 					    partitionContext = monitor.getNetworkPartitionCtxt(networkPartitionId).getPartitionCtxt(partitionId);
@@ -239,7 +239,7 @@ public class AutoscalerTopologyReceiver implements Runnable {
 //					ClusterContext clusCtx = monitor.getClusterCtxt();
 //                    monitor.getNetworkPartitionCtxt(e.getId()).getPartitionCtxt(partitionId);
 //                            .addMemberStatsContext(new MemberStatsContext(e.getMemberId()));
-                    partitionContext.addMemberStatsContext(new MemberStatsContext(e.getMemberId()));
+                    partitionContext.addMemberStatsContext(new MemberStatsContext(memberId));
 //					PartitionContext partCtxt = monitor.getNetworkPartitionCtxt(e.getId())
 //                            .getPartitionCtxt(partitionId);
 					partitionContext.incrementCurrentMemberCount(1);


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

Posted by ud...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-stratos


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

Branch: refs/heads/master
Commit: 52d9282eaba12ca0c709b9b6eacded0e56128fca
Parents: 6aac43b e94dcfa
Author: Udara Liyanage <ud...@wso2.com>
Authored: Thu Dec 19 14:45:40 2013 -0500
Committer: Udara Liyanage <ud...@wso2.com>
Committed: Thu Dec 19 14:45:40 2013 -0500

----------------------------------------------------------------------
 .../adc/mgt/utils/PersistenceManager.java       |   5 +
 .../java/org/apache/stratos/cli/RestClient.java |  16 +-
 .../stratos/cli/RestCommandLineService.java     |  34 +--
 .../impl/CloudControllerServiceImpl.java        |   1 +
 components/org.apache.stratos.common/pom.xml    |  22 +-
 .../publisher/StatisticsPublisher.java          |  45 ++++
 .../publisher/WSO2CEPStatisticsPublisher.java   | 106 ++++++++++
 .../pom.xml                                     |  10 +-
 .../statistics/LoadBalancerStatsPublisher.java  |  45 ----
 .../WSO2CEPFaultyMemberPublisher.java           |  74 -------
 .../WSO2CEPInFlightRequestPublisher.java        |  76 -------
 .../statistics/WSO2CEPStatsPublisher.java       | 106 ----------
 .../publisher/WSO2CEPFaultyMemberPublisher.java |  75 +++++++
 .../WSO2CEPInFlightRequestPublisher.java        |  77 +++++++
 ...oadBalancerInFlightRequestCountNotifier.java |   4 +-
 .../WSO2CEPInFlightRequestCountObserver.java    |   2 +-
 .../messaging/message/filter/MessageFilter.java |   6 +-
 .../extension/FaultHandlingWindowProcessor.java |   2 +-
 .../cartridge-agent/ec2/php/cartridge-agent.sh  |  12 +-
 .../subscriber/CartridgeAgentConstants.java     |   5 +
 .../cartridge/agent/event/subscriber/Main.java  | 209 ++++++++++++++-----
 .../src/main/bin/health-publisher.sh            |   6 +-
 .../puppet/etc/puppet/files/cartridge-agent.sh  |  12 +-
 23 files changed, 551 insertions(+), 399 deletions(-)
----------------------------------------------------------------------