You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@stratos.apache.org by ga...@apache.org on 2015/12/07 11:19:41 UTC

[12/50] [abbrv] stratos git commit: PCA - STRATOS-1634 and STRATOS-1635 - Log displays passworded git URL, PCA doesn't validate app_path

PCA  - STRATOS-1634 and STRATOS-1635 - Log displays passworded git URL, PCA doesn't validate app_path


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

Branch: refs/heads/release-4.1.5
Commit: a64d1ac48bbb6c53dcbd8964f71412b7869642c3
Parents: 46eee21
Author: Chamila de Alwis <ch...@apache.org>
Authored: Mon Nov 30 12:33:57 2015 +0530
Committer: gayangunarathne <ga...@wso2.com>
Committed: Mon Dec 7 10:16:26 2015 +0000

----------------------------------------------------------------------
 .../modules/artifactmgt/git/agentgithandler.py  |   7 +-
 .../modules/event/eventhandler.py               |  12 ++
 .../modules/util/cartridgeagentutils.py         |   4 +-
 .../plugins/DefaultArtifactCheckout.py          |   4 +-
 .../tests/ADCValidationTestCase.java            | 176 +++++++++++++++++++
 .../resources/ADCValidationTestCase/agent.conf  |  46 +++++
 .../resources/ADCValidationTestCase/logging.ini |  52 ++++++
 .../ADCValidationTestCase/payload/launch-params |   1 +
 8 files changed, 296 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/stratos/blob/a64d1ac4/components/org.apache.stratos.python.cartridge.agent/src/main/python/cartridge.agent/cartridge.agent/modules/artifactmgt/git/agentgithandler.py
----------------------------------------------------------------------
diff --git a/components/org.apache.stratos.python.cartridge.agent/src/main/python/cartridge.agent/cartridge.agent/modules/artifactmgt/git/agentgithandler.py b/components/org.apache.stratos.python.cartridge.agent/src/main/python/cartridge.agent/cartridge.agent/modules/artifactmgt/git/agentgithandler.py
index c283011..2170a33 100644
--- a/components/org.apache.stratos.python.cartridge.agent/src/main/python/cartridge.agent/cartridge.agent/modules/artifactmgt/git/agentgithandler.py
+++ b/components/org.apache.stratos.python.cartridge.agent/src/main/python/cartridge.agent/cartridge.agent/modules/artifactmgt/git/agentgithandler.py
@@ -152,7 +152,7 @@ class AgentGitHandler:
             # clone the repo to a temporary location first to avoid conflicts
             AgentGitHandler.log.debug(
                 "Cloning artifacts from URL: %s to temp location: %s" % (git_repo.repo_url, temp_repo_path))
-            Repo.clone_from(git_repo.repo_url, temp_repo_path)
+            Repo.clone_from(git_repo.auth_url, temp_repo_path)
 
             # move the cloned dir to application path
             copy_tree(temp_repo_path, git_repo.local_repo_path)
@@ -216,7 +216,8 @@ class AgentGitHandler:
         git_repo = GitRepository()
         git_repo.tenant_id = repo_info.tenant_id
         git_repo.local_repo_path = repo_info.repo_path
-        git_repo.repo_url = AgentGitHandler.create_auth_url(repo_info)
+        git_repo.repo_url = repo_info.repo_url
+        git_repo.auth_url = AgentGitHandler.create_auth_url(repo_info)
         git_repo.repo_username = repo_info.repo_username
         git_repo.repo_password = repo_info.repo_password
         git_repo.commit_enabled = repo_info.commit_enabled
@@ -378,6 +379,8 @@ class GitRepository:
     def __init__(self):
         self.repo_url = None
         """ :type : str  """
+        self.auth_url = None
+        """ :type : str """
         self.local_repo_path = None
         """ :type : str  """
         self.cloned = False

http://git-wip-us.apache.org/repos/asf/stratos/blob/a64d1ac4/components/org.apache.stratos.python.cartridge.agent/src/main/python/cartridge.agent/cartridge.agent/modules/event/eventhandler.py
----------------------------------------------------------------------
diff --git a/components/org.apache.stratos.python.cartridge.agent/src/main/python/cartridge.agent/cartridge.agent/modules/event/eventhandler.py b/components/org.apache.stratos.python.cartridge.agent/src/main/python/cartridge.agent/cartridge.agent/modules/event/eventhandler.py
index 85624ae..6e2aa4f 100644
--- a/components/org.apache.stratos.python.cartridge.agent/src/main/python/cartridge.agent/cartridge.agent/modules/event/eventhandler.py
+++ b/components/org.apache.stratos.python.cartridge.agent/src/main/python/cartridge.agent/cartridge.agent/modules/event/eventhandler.py
@@ -90,6 +90,11 @@ class EventHandler:
             self.__log.error("Repository path is empty. Failed to process artifact updated event.")
             return
 
+        if not EventHandler.validate_repo_path(Config.app_path):
+            self.__log.error(
+                "Repository path cannot be accessed, or is invalid. Failed to process artifact updated event.")
+            return
+
         repo_username = artifacts_updated_event.repo_username
         tenant_id = artifacts_updated_event.tenant_id
         is_multitenant = Config.is_multiTenant
@@ -633,6 +638,13 @@ class EventHandler:
 
         return tenant.tenant_domain
 
+    @staticmethod
+    def validate_repo_path(app_path):
+        # app path would be ex: /var/www, or /opt/server/data
+        return os.access(app_path, os.W_OK)
+
+
+
 
 class PluginExecutor(Thread):
     """ Executes a given plugin on a separate thread, passing the given dictionary of values to the plugin entry method

http://git-wip-us.apache.org/repos/asf/stratos/blob/a64d1ac4/components/org.apache.stratos.python.cartridge.agent/src/main/python/cartridge.agent/cartridge.agent/modules/util/cartridgeagentutils.py
----------------------------------------------------------------------
diff --git a/components/org.apache.stratos.python.cartridge.agent/src/main/python/cartridge.agent/cartridge.agent/modules/util/cartridgeagentutils.py b/components/org.apache.stratos.python.cartridge.agent/src/main/python/cartridge.agent/cartridge.agent/modules/util/cartridgeagentutils.py
index ebd6889..79bc6c5 100644
--- a/components/org.apache.stratos.python.cartridge.agent/src/main/python/cartridge.agent/cartridge.agent/modules/util/cartridgeagentutils.py
+++ b/components/org.apache.stratos.python.cartridge.agent/src/main/python/cartridge.agent/cartridge.agent/modules/util/cartridgeagentutils.py
@@ -59,8 +59,8 @@ def decrypt_password(pass_str, secret):
 
     # remove nonprintable characters that are padded in the decrypted password
     dec_pass = filter(lambda x: x in string.printable, dec_pass)
-    dec_pass_md5 = hashlib.md5(dec_pass.encode('utf-8')).hexdigest()
-    log.debug("Decrypted password md5sum: [%r]" % dec_pass_md5)
+    # dec_pass_md5 = hashlib.md5(dec_pass.encode('utf-8')).hexdigest()
+    # log.debug("Decrypted password md5sum: [%r]" % dec_pass_md5)
     return dec_pass
 
 

http://git-wip-us.apache.org/repos/asf/stratos/blob/a64d1ac4/components/org.apache.stratos.python.cartridge.agent/src/main/python/cartridge.agent/cartridge.agent/plugins/DefaultArtifactCheckout.py
----------------------------------------------------------------------
diff --git a/components/org.apache.stratos.python.cartridge.agent/src/main/python/cartridge.agent/cartridge.agent/plugins/DefaultArtifactCheckout.py b/components/org.apache.stratos.python.cartridge.agent/src/main/python/cartridge.agent/cartridge.agent/plugins/DefaultArtifactCheckout.py
index c25d0e8..27cf99c 100644
--- a/components/org.apache.stratos.python.cartridge.agent/src/main/python/cartridge.agent/cartridge.agent/plugins/DefaultArtifactCheckout.py
+++ b/components/org.apache.stratos.python.cartridge.agent/src/main/python/cartridge.agent/cartridge.agent/plugins/DefaultArtifactCheckout.py
@@ -72,8 +72,8 @@ class DefaultArtifactCheckout(IArtifactCheckoutPlugin):
                 self.log.debug("Executing git pull: [tenant-id] %s [repo-url] %s",
                                git_repo.tenant_id, git_repo.repo_url)
                 updated = AgentGitHandler.pull(git_repo)
-                self.log.debug("Git pull executed: [tenant-id] %s [repo-url] %s",
-                               git_repo.tenant_id, git_repo.repo_url)
+                self.log.debug("Git pull executed: [tenant-id] %s [repo-url] %s [SUCCESS] %s",
+                               git_repo.tenant_id, git_repo.repo_url, updated)
             else:
                 # not a valid repository, might've been corrupted. do a re-clone
                 self.log.debug("Local repository is not valid. Doing a re-clone to purify.")

http://git-wip-us.apache.org/repos/asf/stratos/blob/a64d1ac4/products/python-cartridge-agent/modules/integration/test-integration/src/test/java/org/apache/stratos/python/cartridge/agent/integration/tests/ADCValidationTestCase.java
----------------------------------------------------------------------
diff --git a/products/python-cartridge-agent/modules/integration/test-integration/src/test/java/org/apache/stratos/python/cartridge/agent/integration/tests/ADCValidationTestCase.java b/products/python-cartridge-agent/modules/integration/test-integration/src/test/java/org/apache/stratos/python/cartridge/agent/integration/tests/ADCValidationTestCase.java
new file mode 100644
index 0000000..5148992
--- /dev/null
+++ b/products/python-cartridge-agent/modules/integration/test-integration/src/test/java/org/apache/stratos/python/cartridge/agent/integration/tests/ADCValidationTestCase.java
@@ -0,0 +1,176 @@
+/*
+ * 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.python.cartridge.agent.integration.tests;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.stratos.common.domain.LoadBalancingIPType;
+import org.apache.stratos.messaging.domain.topology.*;
+import org.apache.stratos.messaging.event.instance.notifier.ArtifactUpdatedEvent;
+import org.apache.stratos.messaging.event.topology.CompleteTopologyEvent;
+import org.apache.stratos.messaging.event.topology.MemberInitializedEvent;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+
+/**
+ * Test validation for application path input on the PCA
+ */
+public class ADCValidationTestCase extends PythonAgentIntegrationTest {
+    private static final int ADC_TEST_TIMEOUT = 300000;
+    private final Log log = LogFactory.getLog(ADCValidationTestCase.class);
+//    private final String INVALID_APP_PATH = "ddd/ffs/ss";
+    private static final String CLUSTER_ID = "tomcat.domain";
+    private static final String DEPLOYMENT_POLICY_NAME = "deployment-policy-2";
+    private static final String AUTOSCALING_POLICY_NAME = "autoscaling-policy-2";
+    private static final String APP_ID = "application-2";
+    private static final String MEMBER_ID = "tomcat.member-1";
+    private static final String INSTANCE_ID = "instance-1";
+    private static final String CLUSTER_INSTANCE_ID = "cluster-1-instance-1";
+    private static final String NETWORK_PARTITION_ID = "network-partition-1";
+    private static final String PARTITION_ID = "partition-1";
+    private static final String TENANT_ID = "-1234";
+    private static final String SERVICE_NAME = "tomcat";
+
+    private boolean logDetected = false;
+
+    public ADCValidationTestCase() throws IOException {
+    }
+
+    @BeforeMethod(alwaysRun = true)
+    public void setUp() throws Exception {
+        log.info("Setting up ADCTestCase");
+        // Set jndi.properties.dir system property for initializing event publishers and receivers
+        System.setProperty("jndi.properties.dir", getCommonResourcesPath());
+
+        super.setup(ADC_TEST_TIMEOUT);
+        startServerSocket(8080);
+
+    }
+
+    @AfterMethod(alwaysRun = true)
+    public void tearDownADC(){
+        tearDown();
+    }
+
+    @Test(timeOut = ADC_TEST_TIMEOUT, groups = {"adddddd"})
+    public void testAppPathValidation(){
+        log.info("Testing app path validation for ADC");
+        startCommunicatorThread();
+
+        Thread startupTestThread = new Thread(new Runnable() {
+            @Override
+            public void run() {
+                while (!eventReceiverInitialized) {
+                    sleep(1000);
+                }
+                List<String> outputLines = new ArrayList<>();
+                while (!outputStream.isClosed()) {
+                    List<String> newLines = getNewLines(outputLines, outputStream.toString());
+                    if (newLines.size() > 0) {
+                        for (String line : newLines) {
+                            if (line.contains("Subscribed to 'topology/#'")) {
+                                sleep(2000);
+                                // Send complete topology event
+                                log.info("Publishing complete topology event...");
+                                Topology topology = createTestTopology();
+                                CompleteTopologyEvent completeTopologyEvent = new CompleteTopologyEvent(topology);
+                                publishEvent(completeTopologyEvent);
+                                log.info("Complete topology event published");
+
+                                // Publish member initialized event
+                                log.info("Publishing member initialized event...");
+                                MemberInitializedEvent memberInitializedEvent = new MemberInitializedEvent(SERVICE_NAME,
+                                        CLUSTER_ID, CLUSTER_INSTANCE_ID, MEMBER_ID, NETWORK_PARTITION_ID, PARTITION_ID,
+                                        INSTANCE_ID);
+                                publishEvent(memberInitializedEvent);
+                                log.info("Member initialized event published");
+                            }
+
+                            // Send artifact updated event to activate the instance first
+                            if (line.contains("Artifact repository found")) {
+                                publishEvent(getArtifactUpdatedEventForPrivateRepo());
+                                log.info("Artifact updated event published");
+                            }
+
+                            if (line.contains("Repository path cannot be accessed, or is invalid.")){
+                                logDetected = true;
+                                log.info("PCA Event handler failed validation for an invalid app path.");
+                            }
+                        }
+                    }
+                    sleep(1000);
+                }
+            }
+        });
+        startupTestThread.start();
+
+        while (!logDetected) {
+            sleep(1000);
+        }
+    }
+
+    public static ArtifactUpdatedEvent getArtifactUpdatedEventForPrivateRepo() {
+        ArtifactUpdatedEvent privateRepoEvent = createTestArtifactUpdatedEvent();
+        privateRepoEvent.setRepoURL("https://bitbucket.org/testapache2211/testrepo.git");
+        privateRepoEvent.setRepoUserName("testapache2211");
+        privateRepoEvent.setRepoPassword("+to2qVW16jzy+Xb/zuafQQ==");
+        return privateRepoEvent;
+    }
+
+    private static ArtifactUpdatedEvent createTestArtifactUpdatedEvent() {
+        ArtifactUpdatedEvent artifactUpdatedEvent = new ArtifactUpdatedEvent();
+        artifactUpdatedEvent.setClusterId(CLUSTER_ID);
+        artifactUpdatedEvent.setTenantId(TENANT_ID);
+        return artifactUpdatedEvent;
+    }
+
+    /**
+     * Create test topology
+     *
+     * @return Topology object with mock information
+     */
+    private Topology createTestTopology() {
+        Topology topology = new Topology();
+        Service service = new Service(SERVICE_NAME, ServiceType.SingleTenant);
+        topology.addService(service);
+
+        Cluster cluster = new Cluster(service.getServiceName(), CLUSTER_ID, DEPLOYMENT_POLICY_NAME,
+                AUTOSCALING_POLICY_NAME, APP_ID);
+        service.addCluster(cluster);
+
+        Member member = new Member(service.getServiceName(), cluster.getClusterId(), MEMBER_ID, CLUSTER_INSTANCE_ID,
+                NETWORK_PARTITION_ID, PARTITION_ID, LoadBalancingIPType.Private, System.currentTimeMillis());
+
+        member.setDefaultPrivateIP("10.0.0.1");
+        member.setDefaultPublicIP("20.0.0.1");
+        Properties properties = new Properties();
+        properties.setProperty("prop1", "value1");
+        member.setProperties(properties);
+        member.setStatus(MemberStatus.Created);
+        cluster.addMember(member);
+        return topology;
+    }
+}

http://git-wip-us.apache.org/repos/asf/stratos/blob/a64d1ac4/products/python-cartridge-agent/modules/integration/test-integration/src/test/resources/ADCValidationTestCase/agent.conf
----------------------------------------------------------------------
diff --git a/products/python-cartridge-agent/modules/integration/test-integration/src/test/resources/ADCValidationTestCase/agent.conf b/products/python-cartridge-agent/modules/integration/test-integration/src/test/resources/ADCValidationTestCase/agent.conf
new file mode 100755
index 0000000..d8363e7
--- /dev/null
+++ b/products/python-cartridge-agent/modules/integration/test-integration/src/test/resources/ADCValidationTestCase/agent.conf
@@ -0,0 +1,46 @@
+# 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.
+
+[agent]
+mb.urls                               =localhost:1885
+mb.username                           =system
+mb.password                           =manager
+mb.publisher.timeout                  =900
+listen.address                        =localhost
+thrift.receiver.urls                  =localhost:7712
+thrift.server.admin.username          =admin
+thrift.server.admin.password          =admin
+cep.stats.publisher.enabled           =true
+lb.private.ip                         =
+lb.public.ip                          =
+enable.artifact.update                =true
+auto.commit                           =true
+auto.checkout                         =true
+artifact.update.interval              =15
+artifact.clone.retries                =5
+artifact.clone.interval               =10
+port.check.timeout                    =600000
+enable.data.publisher                 =false
+monitoring.server.ip                  =localhost
+monitoring.server.port                =7612
+monitoring.server.secure.port         =7712
+monitoring.server.admin.username      =admin
+monitoring.server.admin.password      =admin
+log.file.paths                        =/tmp/agent.screen-adc-test.log
+metadata.service.url                  =https://localhost:9443
+super.tenant.repository.path          =/repository/deployment/server/
+tenant.repository.path                =/repository/tenants/

http://git-wip-us.apache.org/repos/asf/stratos/blob/a64d1ac4/products/python-cartridge-agent/modules/integration/test-integration/src/test/resources/ADCValidationTestCase/logging.ini
----------------------------------------------------------------------
diff --git a/products/python-cartridge-agent/modules/integration/test-integration/src/test/resources/ADCValidationTestCase/logging.ini b/products/python-cartridge-agent/modules/integration/test-integration/src/test/resources/ADCValidationTestCase/logging.ini
new file mode 100755
index 0000000..15cad9b
--- /dev/null
+++ b/products/python-cartridge-agent/modules/integration/test-integration/src/test/resources/ADCValidationTestCase/logging.ini
@@ -0,0 +1,52 @@
+# 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.
+
+
+[formatters]
+keys=default
+
+[formatter_default]
+format=[%(asctime)s] %(levelname)s {%(filename)s:%(funcName)s} - %(message)s
+class=logging.Formatter
+
+[handlers]
+keys=console, error_file, log_file
+
+[handler_console]
+class=logging.StreamHandler
+formatter=default
+args=tuple()
+
+[handler_log_file]
+class=logging.FileHandler
+level=DEBUG
+formatter=default
+args=("agent.log", "w")
+
+[handler_error_file]
+class=logging.FileHandler
+level=ERROR
+formatter=default
+args=("error.log", "w")
+
+[loggers]
+keys=root
+
+[logger_root]
+level=DEBUG
+formatter=default
+handlers=console,error_file,log_file
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/stratos/blob/a64d1ac4/products/python-cartridge-agent/modules/integration/test-integration/src/test/resources/ADCValidationTestCase/payload/launch-params
----------------------------------------------------------------------
diff --git a/products/python-cartridge-agent/modules/integration/test-integration/src/test/resources/ADCValidationTestCase/payload/launch-params b/products/python-cartridge-agent/modules/integration/test-integration/src/test/resources/ADCValidationTestCase/payload/launch-params
new file mode 100755
index 0000000..375ff6a
--- /dev/null
+++ b/products/python-cartridge-agent/modules/integration/test-integration/src/test/resources/ADCValidationTestCase/payload/launch-params
@@ -0,0 +1 @@
+APPLICATION_ID=application1,SERVICE_NAME=tomcat,HOST_NAME=tomcat.stratos.org,MULTITENANT=false,TENANT_ID=-1234,TENANT_RANGE=*,CARTRIDGE_ALIAS=tomcat,CLUSTER_ID=tomcat.domain,CLUSTER_INSTANCE_ID=cluster-1-instance-1,CARTRIDGE_KEY=PUjpXCLujDhYr5A6,DEPLOYMENT=default,REPO_URL=https://github.com/imesh/stratos-php-applications.git,PORTS=8080,PUPPET_IP=127.0.0.1,PUPPET_HOSTNAME=puppet.apache.stratos.org,PUPPET_ENV=false,MEMBER_ID=tomcat.member-1,LB_CLUSTER_ID=null,NETWORK_PARTITION_ID=network-p1,PARTITION_ID=p1,APPLICATION_PATH=ddd/ffs/ss,MIN_COUNT=1,INTERNAL=false,CLUSTERING_PRIMARY_KEY=A,LOG_FILE_PATHS=/tmp/temp.log,PERSISTENCE_MAPPING=null
\ No newline at end of file