You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@stratos.apache.org by sa...@apache.org on 2013/12/03 11:21:31 UTC

git commit: Adding file listener to cartridge agent to periodically commits, fixing STRATOS-227

Updated Branches:
  refs/heads/master ceb407eb0 -> 762394b3f


Adding file listener to cartridge agent to periodically commits, fixing STRATOS-227


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

Branch: refs/heads/master
Commit: 762394b3f2898cf36dd90bf22c26e32b95260be6
Parents: ceb407e
Author: Sajith Kariyawasam <sa...@wso2.com>
Authored: Tue Dec 3 15:50:53 2013 +0530
Committer: Sajith Kariyawasam <sa...@wso2.com>
Committed: Tue Dec 3 15:50:53 2013 +0530

----------------------------------------------------------------------
 .../git/impl/GitBasedArtifactRepository.java    | 136 ++++++++++---------
 .../event/subscriber/ArtifactListener.java      |   6 +-
 .../cartridge/agent/event/subscriber/Main.java  |  13 +-
 .../subscriber/RepositoryFileListener.java      |  38 ++++++
 4 files changed, 125 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-stratos/blob/762394b3/products/cartridge-agent/modules/deployment-synchronizer/src/main/java/org/apache/stratos/deployment/synchronizer/git/impl/GitBasedArtifactRepository.java
----------------------------------------------------------------------
diff --git a/products/cartridge-agent/modules/deployment-synchronizer/src/main/java/org/apache/stratos/deployment/synchronizer/git/impl/GitBasedArtifactRepository.java b/products/cartridge-agent/modules/deployment-synchronizer/src/main/java/org/apache/stratos/deployment/synchronizer/git/impl/GitBasedArtifactRepository.java
index c3c9629..98f99ab 100644
--- a/products/cartridge-agent/modules/deployment-synchronizer/src/main/java/org/apache/stratos/deployment/synchronizer/git/impl/GitBasedArtifactRepository.java
+++ b/products/cartridge-agent/modules/deployment-synchronizer/src/main/java/org/apache/stratos/deployment/synchronizer/git/impl/GitBasedArtifactRepository.java
@@ -22,6 +22,8 @@ package org.apache.stratos.deployment.synchronizer.git.impl;
 import java.io.File;
 import java.io.IOException;
 import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 
@@ -41,6 +43,7 @@ import org.eclipse.jgit.api.PullCommand;
 import org.eclipse.jgit.api.PushCommand;
 import org.eclipse.jgit.api.RmCommand;
 import org.eclipse.jgit.api.Status;
+import org.eclipse.jgit.api.StatusCommand;
 import org.eclipse.jgit.api.errors.CheckoutConflictException;
 import org.eclipse.jgit.api.errors.GitAPIException;
 import org.eclipse.jgit.api.errors.InvalidConfigurationException;
@@ -53,19 +56,20 @@ import org.eclipse.jgit.transport.SshSessionFactory;
 import org.eclipse.jgit.transport.UsernamePasswordCredentialsProvider;
 
 /**
- * Git based artifact repository
+ * Git based artifact repository.
+ * 
+ * 
  */
-
-public class GitBasedArtifactRepository implements ArtifactRepository {
+public class GitBasedArtifactRepository /*implements ArtifactRepository*/ {
 
     private static final Log log = LogFactory.getLog(GitBasedArtifactRepository.class);
 
     //Map to keep track of git context per tenant (remote urls, jgit git objects, etc.)
-    private ConcurrentHashMap<Integer, RepositoryContext> tenantToRepoContextMap;
+    private static ConcurrentHashMap<Integer, RepositoryContext> 
+    					tenantToRepoContextMap = new ConcurrentHashMap<Integer, RepositoryContext>();
 
-    public GitBasedArtifactRepository () {
+    private GitBasedArtifactRepository () {
 
-        tenantToRepoContextMap = new ConcurrentHashMap<Integer, RepositoryContext>();        
     }
 
     /**
@@ -75,7 +79,7 @@ public class GitBasedArtifactRepository implements ArtifactRepository {
      *
      * @throws DeploymentSynchronizerException
      */
-    synchronized private void initGitContext (RepositoryInformation repositoryInformation)  {
+    private static void initGitContext (RepositoryInformation repositoryInformation)  {
 
      /*   if (tenantId == GitDeploymentSynchronizerConstants.SUPER_TENANT_ID)
             return;*/
@@ -134,7 +138,7 @@ public class GitBasedArtifactRepository implements ArtifactRepository {
      *
      * @throws DeploymentSynchronizerException
      */
-    private boolean isKeyBasedAuthentication(String url, int tenantId) throws Exception {
+    private static boolean isKeyBasedAuthentication(String url, int tenantId) throws Exception {
 
         if (url.startsWith(GitDeploymentSynchronizerConstants.GIT_HTTP_REPO_URL_PREFIX) ||
                 url.startsWith(GitDeploymentSynchronizerConstants.GIT_HTTPS_REPO_URL_PREFIX)) {//http or https url
@@ -162,7 +166,7 @@ public class GitBasedArtifactRepository implements ArtifactRepository {
     /**
      * Initializes SSH authentication
      */
-    private void initSSHAuthentication () {
+    private static void initSSHAuthentication () {
 
         SshSessionFactory.setInstance(new CustomJschConfigSessionFactory());
     }
@@ -173,7 +177,7 @@ public class GitBasedArtifactRepository implements ArtifactRepository {
      * @param tenantId tenant repository path
      * @param gitRepoCtx RepositoryContext instance for tenant
      */
-    private void cacheGitRepoContext(int tenantId, RepositoryContext gitRepoCtx) {
+    private static void cacheGitRepoContext(int tenantId, RepositoryContext gitRepoCtx) {
 
     	log.info("caching repo context....");
         tenantToRepoContextMap.put(tenantId, gitRepoCtx);
@@ -187,7 +191,7 @@ public class GitBasedArtifactRepository implements ArtifactRepository {
      * @return corresponding RepositoryContext instance for the
      * tenant's local repo if available, else null
      */
-    private RepositoryContext retrieveCachedGitContext (int tenantId) {
+    private static RepositoryContext retrieveCachedGitContext (int tenantId) {
 
         return tenantToRepoContextMap.get(tenantId);
     }
@@ -195,51 +199,60 @@ public class GitBasedArtifactRepository implements ArtifactRepository {
     /**
      * Commits any changes in the local repository to the relevant remote repository
      *
-     * @param localRepoPath tenant's local repository path
-     *
      * @return
-     *
-     * @throws DeploymentSynchronizerException
+     *      
      */
-    public boolean commit(String localRepoPath) throws Exception {
+    public static boolean commit() {
 
+    	// foreach tenant's local path check for commit
     	// TODO implement later, this is applicable for management node.
-    	return false;
+    	//log.info(" In commit.... map count : " + tenantToRepoContextMap.size());
     	
-        /*RepositoryContext gitRepoCtx = retrieveCachedGitContext(localRepoPath);
-        if (gitRepoCtx == null) {
-            if(log.isDebugEnabled())
-                log.debug("No git repository context information found for deployment synchronizer at " + localRepoPath);
-
-            return false;
-        }
-
-        Git git = gitRepoCtx.getGit();
-        StatusCommand statusCmd = git.status();
-        Status status = null;
-        try {
-            status = statusCmd.call();
-
-        } catch (GitAPIException e) {
-            log.error("Git status operation for tenant " + gitRepoCtx.getTenantId() + " failed, ", e);
-            e.printStackTrace();
-            return false;
-        }
-
-        if(status.isClean()) {//no changes, nothing to commit
-            if(log.isDebugEnabled())
-                log.debug("No changes detected in the local repository at " + localRepoPath);
-            return false;
-        }
+		for (Entry<Integer, RepositoryContext> tenantMap : tenantToRepoContextMap
+				.entrySet()) {
+
+			int tenantId = tenantMap.getKey();
+			//log.info("map count has values..tenant Id : " + tenantId);
+			
+			RepositoryContext gitRepoCtx = retrieveCachedGitContext(tenantId);
+			if (gitRepoCtx == null) {
+				
+					log.info("No git repository context information found for tenant "
+							+ tenantId);
+
+				return false;
+			}
 
-        addArtifacts(gitRepoCtx, getNewArtifacts(status));
-        addArtifacts(gitRepoCtx, getModifiedArtifacts(status));
-        removeArtifacts(gitRepoCtx, getRemovedArtifacts(status));
+			Git git = gitRepoCtx.getGit();
+			StatusCommand statusCmd = git.status();
+			Status status = null;
+			try {
+				status = statusCmd.call();
+
+			} catch (GitAPIException e) {
+				log.error(
+						"Git status operation for tenant "
+								+ gitRepoCtx.getTenantId() + " failed, ", e);
+				return false;
+			}
+			//log.info("status : " + status.toString());
+			if (status.isClean()) {// no changes, nothing to commit
+				
+					log.info("No changes detected in the local repository for tenant "
+							+ tenantId);
+				return false;
+			}
+			
+			addArtifacts(gitRepoCtx, getNewArtifacts(status));
+			addArtifacts(gitRepoCtx, getModifiedArtifacts(status));
+			removeArtifacts(gitRepoCtx, getRemovedArtifacts(status));
+			commitToLocalRepo(gitRepoCtx);
+			pushToRemoteRepo(gitRepoCtx);
 
-        commitToLocalRepo(gitRepoCtx);
-        pushToRemoteRepo(gitRepoCtx);
+			return false;
 
-        return false;*/ 
+		}
+		return false;
     }
 
     /**
@@ -249,7 +262,7 @@ public class GitBasedArtifactRepository implements ArtifactRepository {
      *
      * @return artifact names set
      */
-    private Set<String> getNewArtifacts (Status status) {
+    private static Set<String> getNewArtifacts (Status status) {
 
         return status.getUntracked();
     }
@@ -261,7 +274,7 @@ public class GitBasedArtifactRepository implements ArtifactRepository {
      *
      * @return artifact names set
      */
-    private Set<String> getRemovedArtifacts (Status status) {
+    private static Set<String> getRemovedArtifacts (Status status) {
 
         return status.getMissing();
     }
@@ -273,7 +286,7 @@ public class GitBasedArtifactRepository implements ArtifactRepository {
      *
      * @return artifact names set
      */
-    private Set<String> getModifiedArtifacts (Status status) {
+    private static Set<String> getModifiedArtifacts (Status status) {
 
         return status.getModified();
     }
@@ -284,7 +297,7 @@ public class GitBasedArtifactRepository implements ArtifactRepository {
      * @param gitRepoCtx RepositoryContext instance
      * @param artifacts set of artifacts
      */
-    private void addArtifacts (RepositoryContext gitRepoCtx, Set<String> artifacts) {
+    private static void addArtifacts (RepositoryContext gitRepoCtx, Set<String> artifacts) {
 
         if(artifacts.isEmpty())
             return;
@@ -309,7 +322,7 @@ public class GitBasedArtifactRepository implements ArtifactRepository {
      * @param gitRepoCtx RepositoryContext instance
      * @param artifacts Set of artifact names to remove
      */
-    private void removeArtifacts (RepositoryContext gitRepoCtx, Set<String> artifacts) {
+    private static void removeArtifacts (RepositoryContext gitRepoCtx, Set<String> artifacts) {
 
         if(artifacts.isEmpty())
             return;
@@ -334,7 +347,7 @@ public class GitBasedArtifactRepository implements ArtifactRepository {
      *
      * @param gitRepoCtx RepositoryContext instance for the tenant
      */
-    private void commitToLocalRepo (RepositoryContext gitRepoCtx) {
+    private static void commitToLocalRepo (RepositoryContext gitRepoCtx) {
 
         CommitCommand commitCmd = gitRepoCtx.getGit().commit();
         commitCmd.setMessage("tenant " + gitRepoCtx.getTenantId() + "'s artifacts committed to local repo at " +
@@ -354,7 +367,7 @@ public class GitBasedArtifactRepository implements ArtifactRepository {
      *
      * @param gitRepoCtx RepositoryContext instance for the tenant
      */
-    private void pushToRemoteRepo(RepositoryContext gitRepoCtx) {
+    private static void pushToRemoteRepo(RepositoryContext gitRepoCtx) {
 
         PushCommand pushCmd = gitRepoCtx.getGit().push();
         if(!gitRepoCtx.getKeyBasedAuthentication()) {
@@ -372,14 +385,13 @@ public class GitBasedArtifactRepository implements ArtifactRepository {
         }
     }
 
-   // This should be invoked when there is a git clone / pull notification
-    public boolean checkout(RepositoryInformation repositoryInformation) {
+    public static boolean checkout(RepositoryInformation repositoryInformation) {
 
     	log.info("In checkout............");
     	
     	int tenantId = Integer.parseInt(repositoryInformation.getTenantId());
     	
-    	// if tenant is not initialized, init tenant first
+    	// if context for tenant is not initialized
     	if(tenantToRepoContextMap.get(tenantId) == null)
 	    	initGitContext(repositoryInformation);
     	
@@ -408,7 +420,7 @@ public class GitBasedArtifactRepository implements ArtifactRepository {
      *
      * @return true if success, else false
      */
-    private boolean pullArtifacts (RepositoryContext gitRepoCtx) {
+    private static boolean pullArtifacts (RepositoryContext gitRepoCtx) {
 
     	log.info("Pullingg.....");
         PullCommand pullCmd = gitRepoCtx.getGit().pull();
@@ -502,7 +514,7 @@ public class GitBasedArtifactRepository implements ArtifactRepository {
      *
      * @param gitRepoCtx RepositoryContext for the tenant
      */
-    private void cloneRepository (RepositoryContext gitRepoCtx) { //should happen only at the beginning
+    private static void cloneRepository (RepositoryContext gitRepoCtx) { //should happen only at the beginning
 
         File gitRepoDir = new File(gitRepoCtx.getGitLocalRepoPath());
         if (gitRepoDir.exists()) {
@@ -552,7 +564,7 @@ public class GitBasedArtifactRepository implements ArtifactRepository {
      * @return UsernamePasswordCredentialsProvider instance or null if service invocation failed or
      * username/password is not valid
      */
-    private UsernamePasswordCredentialsProvider createCredentialsProvider (RepositoryContext gitRepoCtx) {
+    private static UsernamePasswordCredentialsProvider createCredentialsProvider (RepositoryContext gitRepoCtx) {
 
         //RepositoryCredentials repoCredentials = null;
         // TODO - set repo creds using the received message
@@ -589,7 +601,7 @@ public class GitBasedArtifactRepository implements ArtifactRepository {
      *
      * @return true if a valid git repo, else false
      */
-    private boolean isValidGitRepo (RepositoryContext gitRepoCtx) {
+    private static boolean isValidGitRepo (RepositoryContext gitRepoCtx) {
 
         for (Ref ref : gitRepoCtx.getLocalRepo().getAllRefs().values()) { //check if has been previously cloned successfully, not empty
             if (ref.getObjectId() == null)

http://git-wip-us.apache.org/repos/asf/incubator-stratos/blob/762394b3/products/cartridge-agent/modules/event-subscriber/src/main/java/org/apache/stratos/cartridge/agent/event/subscriber/ArtifactListener.java
----------------------------------------------------------------------
diff --git a/products/cartridge-agent/modules/event-subscriber/src/main/java/org/apache/stratos/cartridge/agent/event/subscriber/ArtifactListener.java b/products/cartridge-agent/modules/event-subscriber/src/main/java/org/apache/stratos/cartridge/agent/event/subscriber/ArtifactListener.java
index f86c5d8..5796726 100644
--- a/products/cartridge-agent/modules/event-subscriber/src/main/java/org/apache/stratos/cartridge/agent/event/subscriber/ArtifactListener.java
+++ b/products/cartridge-agent/modules/event-subscriber/src/main/java/org/apache/stratos/cartridge/agent/event/subscriber/ArtifactListener.java
@@ -68,16 +68,14 @@ public class ArtifactListener implements MessageListener{
 		log.info("cluster id in payload " + clusterIdInPayload);
 		log.info("cluster id in message " + clusterIdInMessage);
 		
-		if(clusterIdInPayload != null && clusterIdInPayload.equals(clusterIdInMessage)) {
-			
-	    	ArtifactRepository gitBasedArtifactRepository = new GitBasedArtifactRepository();
+		if(clusterIdInPayload != null && clusterIdInPayload.equals(clusterIdInMessage)) {			
 	    	RepositoryInformation repoInformation = new RepositoryInformation();
 	    	repoInformation.setRepoUsername(repoUsername);
 	    	repoInformation.setRepoPassword(repoPassword);
 	    	repoInformation.setRepoUrl(repoURL);
 	    	repoInformation.setRepoPath(localRepoPath);
 	    	repoInformation.setTenantId(tenantId);    	
-	    	gitBasedArtifactRepository.checkout(repoInformation);			
+	    	GitBasedArtifactRepository.checkout(repoInformation);			
 		}
 		
 	}

http://git-wip-us.apache.org/repos/asf/incubator-stratos/blob/762394b3/products/cartridge-agent/modules/event-subscriber/src/main/java/org/apache/stratos/cartridge/agent/event/subscriber/Main.java
----------------------------------------------------------------------
diff --git a/products/cartridge-agent/modules/event-subscriber/src/main/java/org/apache/stratos/cartridge/agent/event/subscriber/Main.java b/products/cartridge-agent/modules/event-subscriber/src/main/java/org/apache/stratos/cartridge/agent/event/subscriber/Main.java
index 5345e6b..2c4daa6 100644
--- a/products/cartridge-agent/modules/event-subscriber/src/main/java/org/apache/stratos/cartridge/agent/event/subscriber/Main.java
+++ b/products/cartridge-agent/modules/event-subscriber/src/main/java/org/apache/stratos/cartridge/agent/event/subscriber/Main.java
@@ -19,6 +19,10 @@
 
 package org.apache.stratos.cartridge.agent.event.subscriber;
 
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.stratos.messaging.broker.subscribe.TopicSubscriber;
@@ -37,13 +41,18 @@ public class Main {
     	
     	System.setProperty(CartridgeAgentConstants.JNDI_PROPERTIES_DIR, args[0]); 
     	System.setProperty(CartridgeAgentConstants.PARAM_FILE_PATH, args[1]);
-    	
     	    	
         //initialting the subscriber
         TopicSubscriber subscriber = new TopicSubscriber(Constants.ARTIFACT_SYNCHRONIZATION_TOPIC);
         subscriber.setMessageListener(new ArtifactListener());
         Thread tsubscriber = new Thread(subscriber);
-		tsubscriber.start();     
+		tsubscriber.start(); 
+		
+		// Start periodical file checker task
+		// TODO -- start this thread only if this node configured as a commit true node
+		ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(1);       
+        scheduler.scheduleWithFixedDelay(new RepositoryFileListener(), 0, 10, TimeUnit.SECONDS);
+		
     }
     
 }

http://git-wip-us.apache.org/repos/asf/incubator-stratos/blob/762394b3/products/cartridge-agent/modules/event-subscriber/src/main/java/org/apache/stratos/cartridge/agent/event/subscriber/RepositoryFileListener.java
----------------------------------------------------------------------
diff --git a/products/cartridge-agent/modules/event-subscriber/src/main/java/org/apache/stratos/cartridge/agent/event/subscriber/RepositoryFileListener.java b/products/cartridge-agent/modules/event-subscriber/src/main/java/org/apache/stratos/cartridge/agent/event/subscriber/RepositoryFileListener.java
new file mode 100644
index 0000000..6068630
--- /dev/null
+++ b/products/cartridge-agent/modules/event-subscriber/src/main/java/org/apache/stratos/cartridge/agent/event/subscriber/RepositoryFileListener.java
@@ -0,0 +1,38 @@
+/*
+ * 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.stratos.cartridge.agent.event.subscriber;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.stratos.deployment.synchronizer.git.impl.GitBasedArtifactRepository;
+
+public class RepositoryFileListener implements Runnable {
+
+	private static final Log log = LogFactory.getLog(RepositoryFileListener.class);
+	
+	@Override
+	public void run() {
+		
+		log.info(" Executing file listener ");		
+		boolean commitStatus = GitBasedArtifactRepository.commit();
+		log.info(" Commit status : " + commitStatus);
+	}
+
+}