You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airavata.apache.org by la...@apache.org on 2016/08/26 15:42:56 UTC

[01/50] [abbrv] airavata git commit: Refactored messaging module to remove duplicate code and support multiple publishers [Forced Update!]

Repository: airavata
Updated Branches:
  refs/heads/lahiru/AIRAVATA-2057 d18a53f8e -> 89bb0d48d (forced update)


Refactored messaging module to remove duplicate code and support multiple publishers


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: e4cc54d923d406542ed5d007aa1fb48a8094d29f
Parents: 2df4bb2
Author: Shameera Rathnayaka <sh...@gmail.com>
Authored: Wed Aug 10 15:18:53 2016 -0400
Committer: Shameera Rathnayaka <sh...@gmail.com>
Committed: Wed Aug 10 15:18:53 2016 -0400

----------------------------------------------------------------------
 .../org/apache/airavata/gfac/impl/Factory.java  |   9 +-
 .../airavata/gfac/server/GfacServerHandler.java |   6 +-
 .../messaging/client/RabbitMQListener.java      |   3 +-
 .../messaging/core/MessagingFactory.java        |  86 +++-
 .../airavata/messaging/core/Publisher.java      |   4 +-
 .../messaging/core/RabbitMQProperties.java      | 125 ++++++
 .../airavata/messaging/core/Subscriber.java     |   5 -
 .../messaging/core/SubscriberProperties.java    | 125 ------
 .../airavata/messaging/core/TestClient.java     |   2 +-
 .../apache/airavata/messaging/core/Type.java    |  27 ++
 .../impl/RabbitMQProcessLaunchPublisher.java    | 156 +++----
 .../messaging/core/impl/RabbitMQProducer.java   | 440 +++++++++----------
 .../messaging/core/impl/RabbitMQPublisher.java  | 118 +++++
 .../core/impl/RabbitMQStatusPublisher.java      | 212 ++++-----
 .../messaging/core/impl/RabbitMQSubscriber.java |   7 +-
 .../server/OrchestratorServerHandler.java       |   3 +-
 .../ExperimentExecution.java                    |   3 +-
 .../workflow/core/WorkflowEnactmentService.java |   7 +-
 .../workflow/core/WorkflowInterpreter.java      |   8 +-
 19 files changed, 777 insertions(+), 569 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/e4cc54d9/modules/gfac/gfac-impl/src/main/java/org/apache/airavata/gfac/impl/Factory.java
----------------------------------------------------------------------
diff --git a/modules/gfac/gfac-impl/src/main/java/org/apache/airavata/gfac/impl/Factory.java b/modules/gfac/gfac-impl/src/main/java/org/apache/airavata/gfac/impl/Factory.java
index d105c18..673f37b 100644
--- a/modules/gfac/gfac-impl/src/main/java/org/apache/airavata/gfac/impl/Factory.java
+++ b/modules/gfac/gfac-impl/src/main/java/org/apache/airavata/gfac/impl/Factory.java
@@ -66,7 +66,8 @@ import org.apache.airavata.gfac.monitor.email.EmailBasedMonitor;
 import org.apache.airavata.messaging.core.MessagingFactory;
 import org.apache.airavata.messaging.core.Publisher;
 import org.apache.airavata.messaging.core.Subscriber;
-import org.apache.airavata.messaging.core.impl.RabbitMQStatusPublisher;
+import org.apache.airavata.messaging.core.Type;
+import org.apache.airavata.messaging.core.impl.RabbitMQPublisher;
 import org.apache.airavata.model.appcatalog.computeresource.JobSubmissionProtocol;
 import org.apache.airavata.model.appcatalog.computeresource.MonitorMode;
 import org.apache.airavata.model.appcatalog.computeresource.ResourceJobManager;
@@ -145,9 +146,9 @@ public abstract class Factory {
 
 	public static Publisher getStatusPublisher() throws AiravataException {
 		if (statusPublisher == null) {
-			synchronized (RabbitMQStatusPublisher.class) {
+			synchronized (RabbitMQPublisher.class) {
 				if (statusPublisher == null) {
-					statusPublisher = new RabbitMQStatusPublisher();
+					statusPublisher = MessagingFactory.getPublisher(Type.STATUS);
 				}
 			}
 		}
@@ -169,7 +170,7 @@ public abstract class Factory {
 
 	public static synchronized  Subscriber getProcessLaunchSubscriber() throws AiravataException {
 		if (processLaunchSubscriber == null) {
-			processLaunchSubscriber = MessagingFactory.getSubscriber(message -> {}, new ArrayList<>(), Subscriber.Type.PROCESS_LAUNCH);
+			processLaunchSubscriber = MessagingFactory.getSubscriber(message -> {}, new ArrayList<>(), Type.PROCESS_LAUNCH);
 		}
 		return processLaunchSubscriber;
 	}

http://git-wip-us.apache.org/repos/asf/airavata/blob/e4cc54d9/modules/gfac/gfac-service/src/main/java/org/apache/airavata/gfac/server/GfacServerHandler.java
----------------------------------------------------------------------
diff --git a/modules/gfac/gfac-service/src/main/java/org/apache/airavata/gfac/server/GfacServerHandler.java b/modules/gfac/gfac-service/src/main/java/org/apache/airavata/gfac/server/GfacServerHandler.java
index a490d91..44073dc 100644
--- a/modules/gfac/gfac-service/src/main/java/org/apache/airavata/gfac/server/GfacServerHandler.java
+++ b/modules/gfac/gfac-service/src/main/java/org/apache/airavata/gfac/server/GfacServerHandler.java
@@ -39,7 +39,7 @@ import org.apache.airavata.messaging.core.MessageHandler;
 import org.apache.airavata.messaging.core.MessagingFactory;
 import org.apache.airavata.messaging.core.Publisher;
 import org.apache.airavata.messaging.core.Subscriber;
-import org.apache.airavata.messaging.core.impl.RabbitMQStatusPublisher;
+import org.apache.airavata.messaging.core.Type;
 import org.apache.airavata.model.messaging.event.MessageType;
 import org.apache.airavata.model.messaging.event.ProcessIdentifier;
 import org.apache.airavata.model.messaging.event.ProcessStatusChangeEvent;
@@ -98,9 +98,9 @@ public class GfacServerHandler implements GfacService.Iface {
 	    // init process consumer
         List<String> routingKeys = new ArrayList<>();
         routingKeys.add(ServerSettings.getRabbitmqProcessLaunchQueueName());
-        processLaunchSubscriber = MessagingFactory.getSubscriber(new ProcessLaunchMessageHandler(),routingKeys, Subscriber.Type.PROCESS_LAUNCH);
+        processLaunchSubscriber = MessagingFactory.getSubscriber(new ProcessLaunchMessageHandler(),routingKeys, Type.PROCESS_LAUNCH);
         // init status publisher
-	    statusPublisher = new RabbitMQStatusPublisher();
+	    statusPublisher = Factory.getStatusPublisher();
     }
 
     private void startCuratorClient() throws ApplicationSettingsException {

http://git-wip-us.apache.org/repos/asf/airavata/blob/e4cc54d9/modules/messaging/client/src/main/java/org/apache/airavata/messaging/client/RabbitMQListener.java
----------------------------------------------------------------------
diff --git a/modules/messaging/client/src/main/java/org/apache/airavata/messaging/client/RabbitMQListener.java b/modules/messaging/client/src/main/java/org/apache/airavata/messaging/client/RabbitMQListener.java
index 984aa59..9003897 100644
--- a/modules/messaging/client/src/main/java/org/apache/airavata/messaging/client/RabbitMQListener.java
+++ b/modules/messaging/client/src/main/java/org/apache/airavata/messaging/client/RabbitMQListener.java
@@ -27,6 +27,7 @@ import org.apache.airavata.common.utils.ThriftUtils;
 import org.apache.airavata.messaging.core.MessageHandler;
 import org.apache.airavata.messaging.core.MessagingFactory;
 import org.apache.airavata.messaging.core.Subscriber;
+import org.apache.airavata.messaging.core.Type;
 import org.apache.airavata.model.messaging.event.ExperimentStatusChangeEvent;
 import org.apache.airavata.model.messaging.event.JobStatusChangeEvent;
 import org.apache.airavata.model.messaging.event.MessageType;
@@ -69,7 +70,7 @@ public class RabbitMQListener {
             System.out.println("broker url " + brokerUrl);
             final String exchangeName = ServerSettings.getSetting(RABBITMQ_EXCHANGE_NAME);
             List<String> routingKeys = getRoutingKeys(level);
-            Subscriber subscriber = MessagingFactory.getSubscriber(null, routingKeys, Subscriber.Type.STATUS);
+            Subscriber subscriber = MessagingFactory.getSubscriber(message -> {}, routingKeys, Type.STATUS);
         } catch (ApplicationSettingsException e) {
             logger.error("Error reading airavata server properties", e);
         } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/airavata/blob/e4cc54d9/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessagingFactory.java
----------------------------------------------------------------------
diff --git a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessagingFactory.java b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessagingFactory.java
index ee68d0c..99c11b8 100644
--- a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessagingFactory.java
+++ b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessagingFactory.java
@@ -23,29 +23,37 @@ package org.apache.airavata.messaging.core;
 import org.apache.airavata.common.exception.AiravataException;
 import org.apache.airavata.common.utils.ServerSettings;
 import org.apache.airavata.messaging.core.impl.ProcessConsumer;
+import org.apache.airavata.messaging.core.impl.RabbitMQPublisher;
 import org.apache.airavata.messaging.core.impl.RabbitMQSubscriber;
 import org.apache.airavata.messaging.core.impl.StatusConsumer;
+import org.apache.airavata.model.messaging.event.ExperimentStatusChangeEvent;
+import org.apache.airavata.model.messaging.event.JobIdentifier;
+import org.apache.airavata.model.messaging.event.JobStatusChangeEvent;
+import org.apache.airavata.model.messaging.event.MessageType;
+import org.apache.airavata.model.messaging.event.ProcessIdentifier;
+import org.apache.airavata.model.messaging.event.ProcessStatusChangeEvent;
+import org.apache.airavata.model.messaging.event.TaskOutputChangeEvent;
+import org.apache.airavata.model.messaging.event.TaskStatusChangeEvent;
 
-import java.util.ArrayList;
 import java.util.List;
 
 public class MessagingFactory {
 
-    public static Subscriber getSubscriber(final MessageHandler messageHandler,List<String> routingKeys, Subscriber.Type type) throws AiravataException {
+    public static Subscriber getSubscriber(final MessageHandler messageHandler,List<String> routingKeys, Type type) throws AiravataException {
         Subscriber subscriber = null;
-        SubscriberProperties sp = getSubscriberProperties();
+        RabbitMQProperties rProperties = getProperties();
 
         switch (type) {
             case EXPERIMENT_LAUNCH:
                 break;
             case PROCESS_LAUNCH:
-                subscriber = getProcessSubscriber(sp);
+                subscriber = getProcessSubscriber(rProperties);
                 subscriber.listen((connection ,channel) -> new ProcessConsumer(messageHandler, connection, channel),
                         null,
                         routingKeys);
                 break;
             case STATUS:
-                subscriber = getStatusSubscriber(sp);
+                subscriber = getStatusSubscriber(rProperties);
                 subscriber.listen((connection, channel) -> new StatusConsumer(messageHandler, connection, channel),
                         null,
                         routingKeys);
@@ -57,24 +65,53 @@ public class MessagingFactory {
         return subscriber;
     }
 
-    private static SubscriberProperties getSubscriberProperties() {
-        return new SubscriberProperties()
+    public static Publisher getPublisher(Type type) throws AiravataException {
+        RabbitMQProperties rProperties = getProperties();
+        Publisher publiser = null;
+        switch (type) {
+            case EXPERIMENT_LAUNCH:
+                break;
+            case PROCESS_LAUNCH:
+                publiser = gerProcessPublisher(rProperties);
+                break;
+            case STATUS:
+                publiser = getStatusPublisher(rProperties);
+                break;
+            default:
+                throw new IllegalArgumentException("Publisher " + type + " is not handled");
+        }
+
+        return publiser;
+    }
+
+    private static Publisher getStatusPublisher(RabbitMQProperties rProperties) throws AiravataException {
+        rProperties.setExchangeName(ServerSettings.getRabbitmqStatusExchangeName());
+        return new RabbitMQPublisher(rProperties, MessagingFactory::statusRoutingkey);
+    }
+
+    private static Publisher gerProcessPublisher(RabbitMQProperties rProperties) throws AiravataException {
+        rProperties.setExchangeName(ServerSettings.getRabbitmqProcessExchangeName());
+        return new RabbitMQPublisher(rProperties, messageContext -> rProperties.getExchangeName());
+    }
+
+    private static RabbitMQProperties getProperties() {
+        return new RabbitMQProperties()
                 .setBrokerUrl(ServerSettings.RABBITMQ_BROKER_URL)
                 .setDurable(ServerSettings.getRabbitmqDurableQueue())
                 .setPrefetchCount(ServerSettings.getRabbitmqPrefetchCount())
                 .setAutoRecoveryEnable(true)
                 .setConsumerTag("default")
-                .setExchangeType(SubscriberProperties.EXCHANGE_TYPE.TOPIC);
+                .setExchangeType(RabbitMQProperties.EXCHANGE_TYPE.TOPIC);
     }
 
-    private static RabbitMQSubscriber getStatusSubscriber(SubscriberProperties sp) throws AiravataException {
+    private static RabbitMQSubscriber getStatusSubscriber(RabbitMQProperties sp) throws AiravataException {
         sp.setExchangeName(ServerSettings.getRabbitmqStatusExchangeName())
                 .setAutoAck(true);
         return new RabbitMQSubscriber(sp);
     }
 
 
-    private static RabbitMQSubscriber getProcessSubscriber(SubscriberProperties sp) throws AiravataException {
+    private static RabbitMQSubscriber getProcessSubscriber(RabbitMQProperties sp) throws AiravataException {
         sp.setExchangeName(ServerSettings.getRabbitmqProcessExchangeName())
                 .setQueueName(ServerSettings.getRabbitmqProcessLaunchQueueName())
                 .setAutoAck(false);
@@ -82,7 +119,34 @@ public class MessagingFactory {
     }
 
 
-
+    private static String statusRoutingkey(MessageContext msgCtx) {
+        String gatewayId = msgCtx.getGatewayId();
+        String routingKey = null;
+        if (msgCtx.getType() == MessageType.EXPERIMENT) {
+            ExperimentStatusChangeEvent event = (ExperimentStatusChangeEvent) msgCtx.getEvent();
+            routingKey = gatewayId + "." + event.getExperimentId();
+        } else if (msgCtx.getType() == MessageType.TASK) {
+            TaskStatusChangeEvent event = (TaskStatusChangeEvent) msgCtx.getEvent();
+            routingKey = gatewayId + "." + event.getTaskIdentity().getExperimentId() + "." +
+                    event.getTaskIdentity().getProcessId() + "." + event.getTaskIdentity().getTaskId();
+        } else if (msgCtx.getType() == MessageType.PROCESSOUTPUT) {
+            TaskOutputChangeEvent event = (TaskOutputChangeEvent) msgCtx.getEvent();
+            routingKey = gatewayId + "." + event.getTaskIdentity().getExperimentId() + "." +
+                    event.getTaskIdentity().getProcessId() + "." + event.getTaskIdentity().getTaskId();
+        } else if (msgCtx.getType() == MessageType.PROCESS) {
+            ProcessStatusChangeEvent event = (ProcessStatusChangeEvent) msgCtx.getEvent();
+            ProcessIdentifier processIdentifier = event.getProcessIdentity();
+            routingKey = gatewayId + "." + processIdentifier.getExperimentId() + "." + processIdentifier.getProcessId();
+        } else if (msgCtx.getType() == MessageType.JOB) {
+            JobStatusChangeEvent event = (JobStatusChangeEvent) msgCtx.getEvent();
+            JobIdentifier identity = event.getJobIdentity();
+            routingKey = gatewayId + "." + identity.getExperimentId() + "." +
+                    identity.getProcessId() + "." +
+                    identity.getTaskId() + "." +
+                    identity.getJobId();
+        }
+        return routingKey;
+    }
 
 
 }

http://git-wip-us.apache.org/repos/asf/airavata/blob/e4cc54d9/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/Publisher.java
----------------------------------------------------------------------
diff --git a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/Publisher.java b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/Publisher.java
index b8b586c..28a9a06 100644
--- a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/Publisher.java
+++ b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/Publisher.java
@@ -31,8 +31,8 @@ public interface Publisher {
 
     /**
      *
-     * @param message object of message context which will include actual event and other information
+     * @param messageContext object of message context which will include actual event and other information
      * @throws AiravataException
      */
-    public void publish(MessageContext message) throws AiravataException;
+    public void publish(MessageContext messageContext) throws AiravataException;
 }

http://git-wip-us.apache.org/repos/asf/airavata/blob/e4cc54d9/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/RabbitMQProperties.java
----------------------------------------------------------------------
diff --git a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/RabbitMQProperties.java b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/RabbitMQProperties.java
new file mode 100644
index 0000000..5bb1929
--- /dev/null
+++ b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/RabbitMQProperties.java
@@ -0,0 +1,125 @@
+/*
+ *
+ * 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.airavata.messaging.core;
+
+public class RabbitMQProperties {
+    private String brokerUrl;
+    private EXCHANGE_TYPE exchangeType;
+    private String exchangeName;
+    private int prefetchCount;
+    private boolean durable;
+    private String queueName;
+    private String consumerTag = "default";
+    private boolean autoRecoveryEnable;
+    private boolean autoAck;
+
+    public String getBrokerUrl() {
+        return brokerUrl;
+    }
+
+    public RabbitMQProperties setBrokerUrl(String brokerUrl) {
+        this.brokerUrl = brokerUrl;
+        return this;
+    }
+
+    public boolean isDurable() {
+        return durable;
+    }
+
+    public RabbitMQProperties setDurable(boolean durable) {
+        this.durable = durable;
+        return this;
+    }
+
+    public String getExchangeName() {
+        return exchangeName;
+    }
+
+    public RabbitMQProperties setExchangeName(String exchangeName) {
+        this.exchangeName = exchangeName;
+        return this;
+    }
+
+    public int getPrefetchCount() {
+        return prefetchCount;
+    }
+
+    public RabbitMQProperties setPrefetchCount(int prefetchCount) {
+        this.prefetchCount = prefetchCount;
+        return this;
+    }
+
+    public String getQueueName() {
+        return queueName;
+    }
+
+    public RabbitMQProperties setQueueName(String queueName) {
+        this.queueName = queueName;
+        return this;
+    }
+
+    public String getConsumerTag() {
+        return consumerTag;
+    }
+
+    public RabbitMQProperties setConsumerTag(String consumerTag) {
+        this.consumerTag = consumerTag;
+        return this;
+    }
+
+    public boolean isAutoRecoveryEnable() {
+        return autoRecoveryEnable;
+    }
+
+    public RabbitMQProperties setAutoRecoveryEnable(boolean autoRecoveryEnable) {
+        this.autoRecoveryEnable = autoRecoveryEnable;
+        return this;
+    }
+
+    public String getExchangeType() {
+        return exchangeType.type;
+    }
+
+    public RabbitMQProperties setExchangeType(EXCHANGE_TYPE exchangeType) {
+        this.exchangeType = exchangeType;
+        return this;
+    }
+
+    public boolean isAutoAck() {
+        return autoAck;
+    }
+
+    public RabbitMQProperties setAutoAck(boolean autoAck) {
+        this.autoAck = autoAck;
+        return this;
+    }
+
+    public enum EXCHANGE_TYPE{
+        TOPIC("topic"),
+        FANOUT("fanout");
+
+        private String type;
+
+        EXCHANGE_TYPE(String type) {
+            this.type = type;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/airavata/blob/e4cc54d9/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/Subscriber.java
----------------------------------------------------------------------
diff --git a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/Subscriber.java b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/Subscriber.java
index 7952cb3..cc357a0 100644
--- a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/Subscriber.java
+++ b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/Subscriber.java
@@ -50,9 +50,4 @@ public interface Subscriber {
 
     void sendAck(long deliveryTag);
 
-    enum Type {
-        EXPERIMENT_LAUNCH,
-        PROCESS_LAUNCH,
-        STATUS
-    }
 }

http://git-wip-us.apache.org/repos/asf/airavata/blob/e4cc54d9/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/SubscriberProperties.java
----------------------------------------------------------------------
diff --git a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/SubscriberProperties.java b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/SubscriberProperties.java
deleted file mode 100644
index 025e93b..0000000
--- a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/SubscriberProperties.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
- *
- * 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.airavata.messaging.core;
-
-public class SubscriberProperties {
-    private String brokerUrl;
-    private EXCHANGE_TYPE exchangeType;
-    private String exchangeName;
-    private int prefetchCount;
-    private boolean durable;
-    private String queueName;
-    private String consumerTag = "default";
-    private boolean autoRecoveryEnable;
-    private boolean autoAck;
-
-    public String getBrokerUrl() {
-        return brokerUrl;
-    }
-
-    public SubscriberProperties setBrokerUrl(String brokerUrl) {
-        this.brokerUrl = brokerUrl;
-        return this;
-    }
-
-    public boolean isDurable() {
-        return durable;
-    }
-
-    public SubscriberProperties setDurable(boolean durable) {
-        this.durable = durable;
-        return this;
-    }
-
-    public String getExchangeName() {
-        return exchangeName;
-    }
-
-    public SubscriberProperties setExchangeName(String exchangeName) {
-        this.exchangeName = exchangeName;
-        return this;
-    }
-
-    public int getPrefetchCount() {
-        return prefetchCount;
-    }
-
-    public SubscriberProperties setPrefetchCount(int prefetchCount) {
-        this.prefetchCount = prefetchCount;
-        return this;
-    }
-
-    public String getQueueName() {
-        return queueName;
-    }
-
-    public SubscriberProperties setQueueName(String queueName) {
-        this.queueName = queueName;
-        return this;
-    }
-
-    public String getConsumerTag() {
-        return consumerTag;
-    }
-
-    public SubscriberProperties setConsumerTag(String consumerTag) {
-        this.consumerTag = consumerTag;
-        return this;
-    }
-
-    public boolean isAutoRecoveryEnable() {
-        return autoRecoveryEnable;
-    }
-
-    public SubscriberProperties setAutoRecoveryEnable(boolean autoRecoveryEnable) {
-        this.autoRecoveryEnable = autoRecoveryEnable;
-        return this;
-    }
-
-    public String getExchangeType() {
-        return exchangeType.type;
-    }
-
-    public SubscriberProperties setExchangeType(EXCHANGE_TYPE exchangeType) {
-        this.exchangeType = exchangeType;
-        return this;
-    }
-
-    public boolean isAutoAck() {
-        return autoAck;
-    }
-
-    public SubscriberProperties setAutoAck(boolean autoAck) {
-        this.autoAck = autoAck;
-        return this;
-    }
-
-    public enum EXCHANGE_TYPE{
-        TOPIC("topic"),
-        FANOUT("fanout");
-
-        private String type;
-
-        EXCHANGE_TYPE(String type) {
-            this.type = type;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/airavata/blob/e4cc54d9/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/TestClient.java
----------------------------------------------------------------------
diff --git a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/TestClient.java b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/TestClient.java
index daa9886..65781fe 100644
--- a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/TestClient.java
+++ b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/TestClient.java
@@ -46,7 +46,7 @@ public class TestClient {
             List<String> routingKeys = new ArrayList<>();
             routingKeys.add(experimentId);
             routingKeys.add(experimentId + ".*");
-            MessagingFactory.getSubscriber(getMessageHandler(),routingKeys,  Subscriber.Type.STATUS);
+            MessagingFactory.getSubscriber(getMessageHandler(),routingKeys,  Type.STATUS);
         } catch (ApplicationSettingsException e) {
             logger.error("Error reading airavata server properties", e);
         }catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/airavata/blob/e4cc54d9/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/Type.java
----------------------------------------------------------------------
diff --git a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/Type.java b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/Type.java
new file mode 100644
index 0000000..6980e1c
--- /dev/null
+++ b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/Type.java
@@ -0,0 +1,27 @@
+/*
+ *
+ * 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.airavata.messaging.core;
+
+public enum Type {
+    EXPERIMENT_LAUNCH,
+    PROCESS_LAUNCH,
+    STATUS
+}

http://git-wip-us.apache.org/repos/asf/airavata/blob/e4cc54d9/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQProcessLaunchPublisher.java
----------------------------------------------------------------------
diff --git a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQProcessLaunchPublisher.java b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQProcessLaunchPublisher.java
index 5cf960e..ba4b6f2 100644
--- a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQProcessLaunchPublisher.java
+++ b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQProcessLaunchPublisher.java
@@ -1,78 +1,78 @@
-/*
- *
- * 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.airavata.messaging.core.impl;
-
-import org.apache.airavata.common.exception.AiravataException;
-import org.apache.airavata.common.exception.ApplicationSettingsException;
-import org.apache.airavata.common.utils.ServerSettings;
-import org.apache.airavata.common.utils.ThriftUtils;
-import org.apache.airavata.messaging.core.MessageContext;
-import org.apache.airavata.messaging.core.MessagingConstants;
-import org.apache.airavata.messaging.core.Publisher;
-import org.apache.airavata.model.messaging.event.*;
-import org.apache.thrift.TException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class RabbitMQProcessLaunchPublisher implements Publisher{
-    private final static Logger log = LoggerFactory.getLogger(RabbitMQProcessLaunchPublisher.class);
-    private  String launchTask;
-    
-    private RabbitMQProducer rabbitMQProducer;
-
-    public RabbitMQProcessLaunchPublisher() throws Exception {
-        String brokerUrl;
-        try {
-            brokerUrl = ServerSettings.getSetting(MessagingConstants.RABBITMQ_BROKER_URL);
-            launchTask = ServerSettings.getRabbitmqProcessLaunchQueueName();
-        } catch (ApplicationSettingsException e) {
-            String message = "Failed to get read the required properties from airavata to initialize rabbitmq";
-            log.error(message, e);
-            throw new AiravataException(message, e);
-        }
-        rabbitMQProducer = new RabbitMQProducer(brokerUrl, null,null);
-        rabbitMQProducer.open();
-    }
-
-    public void publish(MessageContext msgCtx) throws AiravataException {
-        try {
-            log.info("Publishing to launch queue ...");
-            byte[] body = ThriftUtils.serializeThriftObject(msgCtx.getEvent());
-            Message message = new Message();
-            message.setEvent(body);
-            message.setMessageId(msgCtx.getMessageId());
-            message.setMessageType(msgCtx.getType());
-            message.setUpdatedTime(msgCtx.getUpdatedTime().getTime());
-            String routingKey = launchTask;
-            byte[] messageBody = ThriftUtils.serializeThriftObject(message);
-            rabbitMQProducer.sendToWorkerQueue(messageBody, routingKey);
-            log.info("Successfully published to launch queue ...");
-        } catch (TException e) {
-            String msg = "Error while deserializing the object";
-            log.error(msg, e);
-            throw new AiravataException(msg, e);
-        } catch (Exception e) {
-            String msg = "Error while sending to rabbitmq";
-            log.error(msg, e);
-            throw new AiravataException(msg, e);
-        }
-    }
-}
+///*
+// *
+// * 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.airavata.messaging.core.impl;
+//
+//import org.apache.airavata.common.exception.AiravataException;
+//import org.apache.airavata.common.exception.ApplicationSettingsException;
+//import org.apache.airavata.common.utils.ServerSettings;
+//import org.apache.airavata.common.utils.ThriftUtils;
+//import org.apache.airavata.messaging.core.MessageContext;
+//import org.apache.airavata.messaging.core.MessagingConstants;
+//import org.apache.airavata.messaging.core.Publisher;
+//import org.apache.airavata.model.messaging.event.*;
+//import org.apache.thrift.TException;
+//import org.slf4j.Logger;
+//import org.slf4j.LoggerFactory;
+//
+//public class RabbitMQProcessLaunchPublisher implements Publisher{
+//    private final static Logger log = LoggerFactory.getLogger(RabbitMQProcessLaunchPublisher.class);
+//    private  String launchTask;
+//
+//    private RabbitMQProducer rabbitMQProducer;
+//
+//    public RabbitMQProcessLaunchPublisher() throws Exception {
+//        String brokerUrl;
+//        try {
+//            brokerUrl = ServerSettings.getSetting(MessagingConstants.RABBITMQ_BROKER_URL);
+//            launchTask = ServerSettings.getRabbitmqProcessLaunchQueueName();
+//        } catch (ApplicationSettingsException e) {
+//            String message = "Failed to get read the required properties from airavata to initialize rabbitmq";
+//            log.error(message, e);
+//            throw new AiravataException(message, e);
+//        }
+//        rabbitMQProducer = new RabbitMQProducer(brokerUrl, null,null);
+//        rabbitMQProducer.open();
+//    }
+//
+//    public void publish(MessageContext msgCtx) throws AiravataException {
+//        try {
+//            log.info("Publishing to launch queue ...");
+//            byte[] body = ThriftUtils.serializeThriftObject(msgCtx.getEvent());
+//            Message message = new Message();
+//            message.setEvent(body);
+//            message.setMessageId(msgCtx.getMessageId());
+//            message.setMessageType(msgCtx.getType());
+//            message.setUpdatedTime(msgCtx.getUpdatedTime().getTime());
+//            String routingKey = launchTask;
+//            byte[] messageBody = ThriftUtils.serializeThriftObject(message);
+//            rabbitMQProducer.sendToWorkerQueue(messageBody, routingKey);
+//            log.info("Successfully published to launch queue ...");
+//        } catch (TException e) {
+//            String msg = "Error while deserializing the object";
+//            log.error(msg, e);
+//            throw new AiravataException(msg, e);
+//        } catch (Exception e) {
+//            String msg = "Error while sending to rabbitmq";
+//            log.error(msg, e);
+//            throw new AiravataException(msg, e);
+//        }
+//    }
+//}

http://git-wip-us.apache.org/repos/asf/airavata/blob/e4cc54d9/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQProducer.java
----------------------------------------------------------------------
diff --git a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQProducer.java b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQProducer.java
index fc494d4..73138c7 100644
--- a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQProducer.java
+++ b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQProducer.java
@@ -1,220 +1,220 @@
-/*
- *
- * 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.airavata.messaging.core.impl;
-
-import com.rabbitmq.client.*;
-import org.apache.airavata.common.exception.AiravataException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-
-public class RabbitMQProducer {
-    public static final int DEFAULT_PRE_FETCH = 64;
-
-    private static Logger log = LoggerFactory.getLogger(RabbitMQProducer.class);
-
-    private Connection connection;
-
-    private Channel channel;
-
-    private QueueingConsumer consumer;
-
-    private String consumerTag;
-
-    private String exchangeName;
-
-    private int prefetchCount = DEFAULT_PRE_FETCH;
-
-    private boolean isReQueueOnFail = false;
-
-    private String url;
-
-    private String getExchangeType = "topic";
-
-
-    public RabbitMQProducer(String url, String exchangeName,String getExchangeType) {
-        this.exchangeName = exchangeName;
-        this.url = url;
-        this.getExchangeType = getExchangeType;
-    }
-
-    public RabbitMQProducer(String url, String exchangeName) {
-        this.exchangeName = exchangeName;
-        this.url = url;
-    }
-
-    public void setPrefetchCount(int prefetchCount) {
-        this.prefetchCount = prefetchCount;
-    }
-
-    public void setReQueueOnFail(boolean isReQueueOnFail) {
-        this.isReQueueOnFail = isReQueueOnFail;
-    }
-
-    private void reset() {
-        consumerTag = null;
-    }
-
-    private void reInitIfNecessary() throws Exception {
-        if (consumerTag == null || consumer == null) {
-            close();
-            open();
-        }
-    }
-
-    public void close() {
-        log.info("Closing channel to exchange {}", exchangeName);
-        try {
-            if (channel != null && channel.isOpen()) {
-                if (consumerTag != null) {
-                    channel.basicCancel(consumerTag);
-                }
-                channel.close();
-            }
-        } catch (Exception e) {
-            log.debug("error closing channel and/or cancelling consumer", e);
-        }
-        try {
-            log.info("closing connection to rabbitmq: " + connection);
-            connection.close();
-        } catch (Exception e) {
-            log.debug("error closing connection", e);
-        }
-        consumer = null;
-        consumerTag = null;
-        channel = null;
-        connection = null;
-    }
-
-    public void open() throws AiravataException {
-        try {
-            connection = createConnection();
-            channel = connection.createChannel();
-            if (prefetchCount > 0) {
-                log.info("setting basic.qos / prefetch count to " + prefetchCount + " for " + exchangeName);
-                channel.basicQos(prefetchCount);
-            }
-            if(exchangeName!=null) {
-                channel.exchangeDeclare(exchangeName, getExchangeType, false);
-            }
-            } catch (Exception e) {
-            reset();
-            String msg = "could not open channel for exchange " + exchangeName;
-            log.error(msg);
-            throw new AiravataException(msg, e);
-        }
-    }
-
-    public void send(byte []message, String routingKey) throws Exception {
-        try {
-            channel.basicPublish(exchangeName, routingKey, null, message);
-        } catch (IOException e) {
-            String msg = "Failed to publish message to exchange: " + exchangeName;
-            log.error(msg, e);
-            throw new Exception(msg, e);
-        }
-    }
-
-    public void sendToWorkerQueue(byte []message, String routingKey) throws Exception {
-        try {
-            channel.basicPublish( "", routingKey,
-                    MessageProperties.PERSISTENT_TEXT_PLAIN,
-                    message);
-        } catch (IOException e) {
-            String msg = "Failed to publish message to exchange: " + exchangeName;
-            log.error(msg, e);
-            throw new Exception(msg, e);
-        }
-    }
-
-    private Connection createConnection() throws IOException {
-        try {
-            ConnectionFactory connectionFactory = new ConnectionFactory();
-            connectionFactory.setUri(url);
-            connectionFactory.setAutomaticRecoveryEnabled(true);
-            Connection connection = connectionFactory.newConnection();
-            connection.addShutdownListener(new ShutdownListener() {
-                public void shutdownCompleted(ShutdownSignalException cause) {
-                }
-            });
-            log.info("connected to rabbitmq: " + connection + " for " + exchangeName);
-            return connection;
-        } catch (Exception e) {
-            log.info("connection failed to rabbitmq: " + connection + " for " + exchangeName);
-            return null;
-        }
-    }
-
-    public void ackMessage(Long msgId) throws Exception {
-        try {
-            channel.basicAck(msgId, false);
-        } catch (ShutdownSignalException sse) {
-            reset();
-            String msg = "shutdown signal received while attempting to ack message";
-            log.error(msg, sse);
-            throw new Exception(msg, sse);
-        } catch (Exception e) {
-            String s = "could not ack for msgId: " + msgId;
-            log.error(s, e);
-            throw new Exception(s, e);
-        }
-    }
-
-    public void failMessage(Long msgId) throws Exception {
-        if (isReQueueOnFail) {
-            failWithRedelivery(msgId);
-        } else {
-            deadLetter(msgId);
-        }
-    }
-
-    public void failWithRedelivery(Long msgId) throws Exception {
-        try {
-            channel.basicReject(msgId, true);
-        } catch (ShutdownSignalException sse) {
-            reset();
-            String msg = "shutdown signal received while attempting to fail with redelivery";
-            log.error(msg, sse);
-            throw new Exception(msg, sse);
-        } catch (Exception e) {
-            String msg = "could not fail with redelivery for msgId: " + msgId;
-            log.error(msg, e);
-            throw new Exception(msg, e);
-        }
-    }
-
-    public void deadLetter(Long msgId) throws Exception {
-        try {
-            channel.basicReject(msgId, false);
-        } catch (ShutdownSignalException sse) {
-            reset();
-            String msg = "shutdown signal received while attempting to fail with no redelivery";
-            log.error(msg, sse);
-            throw new Exception(msg, sse);
-        } catch (Exception e) {
-            String msg = "could not fail with dead-lettering (when configured) for msgId: " + msgId;
-            log.error(msg, e);
-            throw new Exception(msg, e);
-        }
-    }
-}
+///*
+// *
+// * 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.airavata.messaging.core.impl;
+//
+//import com.rabbitmq.client.*;
+//import org.apache.airavata.common.exception.AiravataException;
+//import org.slf4j.Logger;
+//import org.slf4j.LoggerFactory;
+//
+//import java.io.IOException;
+//
+//public class RabbitMQProducer {
+//    public static final int DEFAULT_PRE_FETCH = 64;
+//
+//    private static Logger log = LoggerFactory.getLogger(RabbitMQProducer.class);
+//
+//    private Connection connection;
+//
+//    private Channel channel;
+//
+//    private QueueingConsumer consumer;
+//
+//    private String consumerTag;
+//
+//    private String exchangeName;
+//
+//    private int prefetchCount = DEFAULT_PRE_FETCH;
+//
+//    private boolean isReQueueOnFail = false;
+//
+//    private String url;
+//
+//    private String getExchangeType = "topic";
+//
+//
+//    public RabbitMQProducer(String url, String exchangeName,String getExchangeType) {
+//        this.exchangeName = exchangeName;
+//        this.url = url;
+//        this.getExchangeType = getExchangeType;
+//    }
+//
+//    public RabbitMQProducer(String url, String exchangeName) {
+//        this.exchangeName = exchangeName;
+//        this.url = url;
+//    }
+//
+//    public void setPrefetchCount(int prefetchCount) {
+//        this.prefetchCount = prefetchCount;
+//    }
+//
+//    public void setReQueueOnFail(boolean isReQueueOnFail) {
+//        this.isReQueueOnFail = isReQueueOnFail;
+//    }
+//
+//    private void reset() {
+//        consumerTag = null;
+//    }
+//
+//    private void reInitIfNecessary() throws Exception {
+//        if (consumerTag == null || consumer == null) {
+//            close();
+//            open();
+//        }
+//    }
+//
+//    public void close() {
+//        log.info("Closing channel to exchange {}", exchangeName);
+//        try {
+//            if (channel != null && channel.isOpen()) {
+//                if (consumerTag != null) {
+//                    channel.basicCancel(consumerTag);
+//                }
+//                channel.close();
+//            }
+//        } catch (Exception e) {
+//            log.debug("error closing channel and/or cancelling consumer", e);
+//        }
+//        try {
+//            log.info("closing connection to rabbitmq: " + connection);
+//            connection.close();
+//        } catch (Exception e) {
+//            log.debug("error closing connection", e);
+//        }
+//        consumer = null;
+//        consumerTag = null;
+//        channel = null;
+//        connection = null;
+//    }
+//
+//    public void open() throws AiravataException {
+//        try {
+//            connection = createConnection();
+//            channel = connection.createChannel();
+//            if (prefetchCount > 0) {
+//                log.info("setting basic.qos / prefetch count to " + prefetchCount + " for " + exchangeName);
+//                channel.basicQos(prefetchCount);
+//            }
+//            if(exchangeName!=null) {
+//                channel.exchangeDeclare(exchangeName, getExchangeType, false);
+//            }
+//            } catch (Exception e) {
+//            reset();
+//            String msg = "could not open channel for exchange " + exchangeName;
+//            log.error(msg);
+//            throw new AiravataException(msg, e);
+//        }
+//    }
+//
+//    public void send(byte []message, String routingKey) throws Exception {
+//        try {
+//            channel.basicPublish(exchangeName, routingKey, null, message);
+//        } catch (IOException e) {
+//            String msg = "Failed to publish message to exchange: " + exchangeName;
+//            log.error(msg, e);
+//            throw new Exception(msg, e);
+//        }
+//    }
+//
+//    public void sendToWorkerQueue(byte []message, String routingKey) throws Exception {
+//        try {
+//            channel.basicPublish( "", routingKey,
+//                    MessageProperties.PERSISTENT_TEXT_PLAIN,
+//                    message);
+//        } catch (IOException e) {
+//            String msg = "Failed to publish message to exchange: " + exchangeName;
+//            log.error(msg, e);
+//            throw new Exception(msg, e);
+//        }
+//    }
+//
+//    private Connection createConnection() throws IOException {
+//        try {
+//            ConnectionFactory connectionFactory = new ConnectionFactory();
+//            connectionFactory.setUri(url);
+//            connectionFactory.setAutomaticRecoveryEnabled(true);
+//            Connection connection = connectionFactory.newConnection();
+//            connection.addShutdownListener(new ShutdownListener() {
+//                public void shutdownCompleted(ShutdownSignalException cause) {
+//                }
+//            });
+//            log.info("connected to rabbitmq: " + connection + " for " + exchangeName);
+//            return connection;
+//        } catch (Exception e) {
+//            log.info("connection failed to rabbitmq: " + connection + " for " + exchangeName);
+//            return null;
+//        }
+//    }
+//
+//    public void ackMessage(Long msgId) throws Exception {
+//        try {
+//            channel.basicAck(msgId, false);
+//        } catch (ShutdownSignalException sse) {
+//            reset();
+//            String msg = "shutdown signal received while attempting to ack message";
+//            log.error(msg, sse);
+//            throw new Exception(msg, sse);
+//        } catch (Exception e) {
+//            String s = "could not ack for msgId: " + msgId;
+//            log.error(s, e);
+//            throw new Exception(s, e);
+//        }
+//    }
+//
+//    public void failMessage(Long msgId) throws Exception {
+//        if (isReQueueOnFail) {
+//            failWithRedelivery(msgId);
+//        } else {
+//            deadLetter(msgId);
+//        }
+//    }
+//
+//    public void failWithRedelivery(Long msgId) throws Exception {
+//        try {
+//            channel.basicReject(msgId, true);
+//        } catch (ShutdownSignalException sse) {
+//            reset();
+//            String msg = "shutdown signal received while attempting to fail with redelivery";
+//            log.error(msg, sse);
+//            throw new Exception(msg, sse);
+//        } catch (Exception e) {
+//            String msg = "could not fail with redelivery for msgId: " + msgId;
+//            log.error(msg, e);
+//            throw new Exception(msg, e);
+//        }
+//    }
+//
+//    public void deadLetter(Long msgId) throws Exception {
+//        try {
+//            channel.basicReject(msgId, false);
+//        } catch (ShutdownSignalException sse) {
+//            reset();
+//            String msg = "shutdown signal received while attempting to fail with no redelivery";
+//            log.error(msg, sse);
+//            throw new Exception(msg, sse);
+//        } catch (Exception e) {
+//            String msg = "could not fail with dead-lettering (when configured) for msgId: " + msgId;
+//            log.error(msg, e);
+//            throw new Exception(msg, e);
+//        }
+//    }
+//}

http://git-wip-us.apache.org/repos/asf/airavata/blob/e4cc54d9/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQPublisher.java
----------------------------------------------------------------------
diff --git a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQPublisher.java b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQPublisher.java
new file mode 100644
index 0000000..3fdb3a1
--- /dev/null
+++ b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQPublisher.java
@@ -0,0 +1,118 @@
+/*
+ *
+ * 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.airavata.messaging.core.impl;
+
+import com.rabbitmq.client.Channel;
+import com.rabbitmq.client.Connection;
+import com.rabbitmq.client.ConnectionFactory;
+import com.rabbitmq.client.MessageProperties;
+import com.rabbitmq.client.ShutdownListener;
+import com.rabbitmq.client.ShutdownSignalException;
+import org.apache.airavata.common.exception.AiravataException;
+import org.apache.airavata.common.utils.ThriftUtils;
+import org.apache.airavata.messaging.core.MessageContext;
+import org.apache.airavata.messaging.core.Publisher;
+import org.apache.airavata.messaging.core.RabbitMQProperties;
+import org.apache.airavata.model.messaging.event.Message;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.function.Function;
+
+public class RabbitMQPublisher implements Publisher {
+    private static final Logger log = LoggerFactory.getLogger(RabbitMQPublisher.class);
+    private final RabbitMQProperties properties;
+    private final Function<MessageContext, String> routingKeySupplier;
+    private Connection connection;
+    private Channel channel;
+
+    public RabbitMQPublisher(RabbitMQProperties properties, Function<MessageContext, String> routingKeySupplier) throws AiravataException {
+        this.properties = properties;
+        this.routingKeySupplier = routingKeySupplier;
+        connect();
+    }
+
+    private void connect() throws AiravataException {
+        try {
+            ConnectionFactory connectionFactory = new ConnectionFactory();
+            connectionFactory.setUri(properties.getBrokerUrl());
+            connectionFactory.setAutomaticRecoveryEnabled(properties.isAutoRecoveryEnable());
+            connection = connectionFactory.newConnection();
+            connection.addShutdownListener(new ShutdownListener() {
+                public void shutdownCompleted(ShutdownSignalException cause) {
+                }
+            });
+            log.info("connected to rabbitmq: " + connection + " for " + properties.getExchangeName());
+            channel = connection.createChannel();
+            if (properties.getPrefetchCount() > 0) {
+                channel.basicQos(properties.getPrefetchCount());
+            }
+
+            if (properties.getExchangeName() != null) {
+                channel.exchangeDeclare(properties.getExchangeName(),
+                                        properties.getExchangeType(),
+                                        false);
+            }
+        } catch (Exception e) {
+            String msg = "RabbitMQ connection issue for exchange : " + properties.getExchangeName();
+            log.error(msg);
+            throw new AiravataException(msg, e);
+        }
+
+
+    }
+
+
+    @Override
+    public void publish(MessageContext messageContext) throws AiravataException {
+        try {
+            byte[] body = ThriftUtils.serializeThriftObject(messageContext.getEvent());
+            Message message = new Message();
+            message.setEvent(body);
+            message.setMessageId(messageContext.getMessageId());
+            message.setMessageType(messageContext.getType());
+            message.setUpdatedTime(messageContext.getUpdatedTime().getTime());
+            String routingKey = routingKeySupplier.apply(messageContext);
+            byte[] messageBody = ThriftUtils.serializeThriftObject(message);
+            send(messageBody, routingKey);
+        } catch (TException e) {
+            String msg = "Error while deserializing the object";
+            log.error(msg, e);
+            throw new AiravataException(msg, e);
+        } catch (Exception e) {
+            String msg = "Error while sending to rabbitmq";
+            log.error(msg, e);
+            throw new AiravataException(msg, e);
+        }
+    }
+
+    public void send(byte []message, String routingKey) throws Exception {
+        try {
+            channel.basicPublish(properties.getExchangeName(), routingKey, MessageProperties.PERSISTENT_TEXT_PLAIN, message);
+        } catch (IOException e) {
+            String msg = "Failed to publish message to exchange: " + properties.getExchangeName();
+            log.error(msg, e);
+            throw new Exception(msg, e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/airavata/blob/e4cc54d9/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQStatusPublisher.java
----------------------------------------------------------------------
diff --git a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQStatusPublisher.java b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQStatusPublisher.java
index 75077e9..6f728fe 100644
--- a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQStatusPublisher.java
+++ b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQStatusPublisher.java
@@ -1,106 +1,106 @@
-/*
- *
- * 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.airavata.messaging.core.impl;
-
-import org.apache.airavata.common.exception.AiravataException;
-import org.apache.airavata.common.exception.ApplicationSettingsException;
-import org.apache.airavata.common.utils.ServerSettings;
-import org.apache.airavata.common.utils.ThriftUtils;
-import org.apache.airavata.messaging.core.MessageContext;
-import org.apache.airavata.messaging.core.MessagingConstants;
-import org.apache.airavata.messaging.core.Publisher;
-import org.apache.airavata.model.messaging.event.*;
-import org.apache.thrift.TException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class RabbitMQStatusPublisher implements Publisher {
-
-    private static Logger log = LoggerFactory.getLogger(RabbitMQStatusPublisher.class);
-
-    private RabbitMQProducer rabbitMQProducer;
-
-//    StatCounter statCounter = StatCounter.getInstance();
-
-    public RabbitMQStatusPublisher() throws AiravataException {
-        String brokerUrl;
-        String exchangeName;
-        try {
-            brokerUrl = ServerSettings.getSetting(MessagingConstants.RABBITMQ_BROKER_URL);
-            exchangeName = ServerSettings.getSetting(MessagingConstants.RABBITMQ_STATUS_EXCHANGE_NAME);
-        } catch (ApplicationSettingsException e) {
-            String message = "Failed to get read the required properties from airavata to initialize rabbitmq";
-            log.error(message, e);
-            throw new AiravataException(message, e);
-        }
-        rabbitMQProducer = new RabbitMQProducer(brokerUrl, exchangeName);
-        rabbitMQProducer.open();
-    }
-
-    public void publish(MessageContext msgCtx) throws AiravataException {
-        try {
-            log.info("Publishing status to rabbitmq...");
-            byte[] body = ThriftUtils.serializeThriftObject(msgCtx.getEvent());
-            Message message = new Message();
-            message.setEvent(body);
-            message.setMessageId(msgCtx.getMessageId());
-            message.setMessageType(msgCtx.getType());
-            message.setUpdatedTime(msgCtx.getUpdatedTime().getTime());
-            String gatewayId = msgCtx.getGatewayId();
-            String routingKey = null;
-            if (msgCtx.getType() == MessageType.EXPERIMENT) {
-                ExperimentStatusChangeEvent event = (ExperimentStatusChangeEvent) msgCtx.getEvent();
-                routingKey = gatewayId + "." + event.getExperimentId();
-            } else if (msgCtx.getType() == MessageType.TASK) {
-                TaskStatusChangeEvent event = (TaskStatusChangeEvent) msgCtx.getEvent();
-                routingKey = gatewayId + "." + event.getTaskIdentity().getExperimentId() + "." +
-                        event.getTaskIdentity().getProcessId() + "." + event.getTaskIdentity().getTaskId();
-            } else if (msgCtx.getType() == MessageType.PROCESSOUTPUT) {
-                TaskOutputChangeEvent event = (TaskOutputChangeEvent) msgCtx.getEvent();
-                routingKey = gatewayId + "." + event.getTaskIdentity().getExperimentId() + "." +
-                        event.getTaskIdentity().getProcessId() + "." + event.getTaskIdentity().getTaskId();
-            } else if (msgCtx.getType() == MessageType.PROCESS) {
-                ProcessStatusChangeEvent event = (ProcessStatusChangeEvent) msgCtx.getEvent();
-                ProcessIdentifier processIdentifier = event.getProcessIdentity();
-                routingKey = gatewayId + "." + processIdentifier.getExperimentId() + "." + processIdentifier.getProcessId();
-            } else if (msgCtx.getType() == MessageType.JOB) {
-                JobStatusChangeEvent event = (JobStatusChangeEvent) msgCtx.getEvent();
-                JobIdentifier identity = event.getJobIdentity();
-                routingKey = gatewayId + "." + identity.getExperimentId() + "." +
-                        identity.getProcessId() + "." +
-                        identity.getTaskId() + "." +
-                        identity.getJobId();
-            }
-            byte[] messageBody = ThriftUtils.serializeThriftObject(message);
-            rabbitMQProducer.send(messageBody, routingKey);
-//            statCounter.add(message);
-        } catch (TException e) {
-            String msg = "Error while deserializing the object";
-            log.error(msg, e);
-            throw new AiravataException(msg, e);
-        } catch (Exception e) {
-            String msg = "Error while sending to rabbitmq";
-            log.error(msg, e);
-            throw new AiravataException(msg, e);
-        }
-    }
-}
+///*
+// *
+// * 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.airavata.messaging.core.impl;
+//
+//import org.apache.airavata.common.exception.AiravataException;
+//import org.apache.airavata.common.exception.ApplicationSettingsException;
+//import org.apache.airavata.common.utils.ServerSettings;
+//import org.apache.airavata.common.utils.ThriftUtils;
+//import org.apache.airavata.messaging.core.MessageContext;
+//import org.apache.airavata.messaging.core.MessagingConstants;
+//import org.apache.airavata.messaging.core.Publisher;
+//import org.apache.airavata.model.messaging.event.*;
+//import org.apache.thrift.TException;
+//import org.slf4j.Logger;
+//import org.slf4j.LoggerFactory;
+//
+//public class RabbitMQStatusPublisher implements Publisher {
+//
+//    private static Logger log = LoggerFactory.getLogger(RabbitMQStatusPublisher.class);
+//
+//    private RabbitMQProducer rabbitMQProducer;
+//
+////    StatCounter statCounter = StatCounter.getInstance();
+//
+//    public RabbitMQStatusPublisher() throws AiravataException {
+//        String brokerUrl;
+//        String exchangeName;
+//        try {
+//            brokerUrl = ServerSettings.getSetting(MessagingConstants.RABBITMQ_BROKER_URL);
+//            exchangeName = ServerSettings.getSetting(MessagingConstants.RABBITMQ_STATUS_EXCHANGE_NAME);
+//        } catch (ApplicationSettingsException e) {
+//            String message = "Failed to get read the required properties from airavata to initialize rabbitmq";
+//            log.error(message, e);
+//            throw new AiravataException(message, e);
+//        }
+//        rabbitMQProducer = new RabbitMQProducer(brokerUrl, exchangeName);
+//        rabbitMQProducer.open();
+//    }
+//
+//    public void publish(MessageContext msgCtx) throws AiravataException {
+//        try {
+//            log.info("Publishing status to rabbitmq...");
+//            byte[] body = ThriftUtils.serializeThriftObject(msgCtx.getEvent());
+//            Message message = new Message();
+//            message.setEvent(body);
+//            message.setMessageId(msgCtx.getMessageId());
+//            message.setMessageType(msgCtx.getType());
+//            message.setUpdatedTime(msgCtx.getUpdatedTime().getTime());
+//            String gatewayId = msgCtx.getGatewayId();
+//            String routingKey = null;
+//            if (msgCtx.getType() == MessageType.EXPERIMENT) {
+//                ExperimentStatusChangeEvent event = (ExperimentStatusChangeEvent) msgCtx.getEvent();
+//                routingKey = gatewayId + "." + event.getExperimentId();
+//            } else if (msgCtx.getType() == MessageType.TASK) {
+//                TaskStatusChangeEvent event = (TaskStatusChangeEvent) msgCtx.getEvent();
+//                routingKey = gatewayId + "." + event.getTaskIdentity().getExperimentId() + "." +
+//                        event.getTaskIdentity().getProcessId() + "." + event.getTaskIdentity().getTaskId();
+//            } else if (msgCtx.getType() == MessageType.PROCESSOUTPUT) {
+//                TaskOutputChangeEvent event = (TaskOutputChangeEvent) msgCtx.getEvent();
+//                routingKey = gatewayId + "." + event.getTaskIdentity().getExperimentId() + "." +
+//                        event.getTaskIdentity().getProcessId() + "." + event.getTaskIdentity().getTaskId();
+//            } else if (msgCtx.getType() == MessageType.PROCESS) {
+//                ProcessStatusChangeEvent event = (ProcessStatusChangeEvent) msgCtx.getEvent();
+//                ProcessIdentifier processIdentifier = event.getProcessIdentity();
+//                routingKey = gatewayId + "." + processIdentifier.getExperimentId() + "." + processIdentifier.getProcessId();
+//            } else if (msgCtx.getType() == MessageType.JOB) {
+//                JobStatusChangeEvent event = (JobStatusChangeEvent) msgCtx.getEvent();
+//                JobIdentifier identity = event.getJobIdentity();
+//                routingKey = gatewayId + "." + identity.getExperimentId() + "." +
+//                        identity.getProcessId() + "." +
+//                        identity.getTaskId() + "." +
+//                        identity.getJobId();
+//            }
+//            byte[] messageBody = ThriftUtils.serializeThriftObject(message);
+//            rabbitMQProducer.send(messageBody, routingKey);
+////            statCounter.add(message);
+//        } catch (TException e) {
+//            String msg = "Error while deserializing the object";
+//            log.error(msg, e);
+//            throw new AiravataException(msg, e);
+//        } catch (Exception e) {
+//            String msg = "Error while sending to rabbitmq";
+//            log.error(msg, e);
+//            throw new AiravataException(msg, e);
+//        }
+//    }
+//}

http://git-wip-us.apache.org/repos/asf/airavata/blob/e4cc54d9/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQSubscriber.java
----------------------------------------------------------------------
diff --git a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQSubscriber.java b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQSubscriber.java
index 188847f..441281d 100644
--- a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQSubscriber.java
+++ b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQSubscriber.java
@@ -27,9 +27,8 @@ import com.rabbitmq.client.Consumer;
 import com.rabbitmq.client.ShutdownListener;
 import com.rabbitmq.client.ShutdownSignalException;
 import org.apache.airavata.common.exception.AiravataException;
-import org.apache.airavata.common.utils.ServerSettings;
 import org.apache.airavata.messaging.core.Subscriber;
-import org.apache.airavata.messaging.core.SubscriberProperties;
+import org.apache.airavata.messaging.core.RabbitMQProperties;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -46,9 +45,9 @@ public class RabbitMQSubscriber implements Subscriber {
     private Connection connection;
     private Channel channel;
     private Map<String, QueueDetail> queueDetailMap = new HashMap<>();
-    private SubscriberProperties properties;
+    private RabbitMQProperties properties;
 
-    public RabbitMQSubscriber(SubscriberProperties properties) throws AiravataException {
+    public RabbitMQSubscriber(RabbitMQProperties properties) throws AiravataException {
         this.properties = properties;
         createConnection();
     }

http://git-wip-us.apache.org/repos/asf/airavata/blob/e4cc54d9/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java
----------------------------------------------------------------------
diff --git a/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java b/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java
index f5c4d2a..5d02100 100644
--- a/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java
+++ b/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java
@@ -35,6 +35,7 @@ import org.apache.airavata.messaging.core.MessagingFactory;
 import org.apache.airavata.messaging.core.Publisher;
 import org.apache.airavata.messaging.core.PublisherFactory;
 import org.apache.airavata.messaging.core.Subscriber;
+import org.apache.airavata.messaging.core.Type;
 import org.apache.airavata.model.appcatalog.appdeployment.ApplicationDeploymentDescription;
 import org.apache.airavata.model.appcatalog.appinterface.ApplicationInterfaceDescription;
 import org.apache.airavata.model.appcatalog.computeresource.ComputeResourceDescription;
@@ -130,7 +131,7 @@ public class OrchestratorServerHandler implements OrchestratorService.Iface {
 //			routingKeys.add("*"); // listen for gateway level messages
 //			routingKeys.add("*.*"); // listen for gateway/experiment level messages
 			routingKeys.add("*.*.*"); // listen for gateway/experiment/process level messages
-			statusSubscribe = MessagingFactory.getSubscriber(new ProcessStatusHandler(),routingKeys, Subscriber.Type.STATUS);
+			statusSubscribe = MessagingFactory.getSubscriber(new ProcessStatusHandler(),routingKeys, Type.STATUS);
 			startCurator();
 		} catch (OrchestratorException | RegistryException | AppCatalogException | AiravataException e) {
 			log.error(e.getMessage(), e);

http://git-wip-us.apache.org/repos/asf/airavata/blob/e4cc54d9/modules/test-suite/multi-tenanted-airavata/src/main/java/org/apache/airavata/testsuite/multitenantedairavata/ExperimentExecution.java
----------------------------------------------------------------------
diff --git a/modules/test-suite/multi-tenanted-airavata/src/main/java/org/apache/airavata/testsuite/multitenantedairavata/ExperimentExecution.java b/modules/test-suite/multi-tenanted-airavata/src/main/java/org/apache/airavata/testsuite/multitenantedairavata/ExperimentExecution.java
index fa4c3de..1e6edd8 100644
--- a/modules/test-suite/multi-tenanted-airavata/src/main/java/org/apache/airavata/testsuite/multitenantedairavata/ExperimentExecution.java
+++ b/modules/test-suite/multi-tenanted-airavata/src/main/java/org/apache/airavata/testsuite/multitenantedairavata/ExperimentExecution.java
@@ -26,6 +26,7 @@ import org.apache.airavata.common.utils.ThriftUtils;
 import org.apache.airavata.messaging.core.MessageContext;
 import org.apache.airavata.messaging.core.MessagingFactory;
 import org.apache.airavata.messaging.core.Subscriber;
+import org.apache.airavata.messaging.core.Type;
 import org.apache.airavata.model.application.io.InputDataObjectType;
 import org.apache.airavata.model.application.io.OutputDataObjectType;
 import org.apache.airavata.model.commons.ErrorModel;
@@ -187,7 +188,7 @@ public class ExperimentExecution {
         String brokerUrl = propertyReader.readProperty(TestFrameworkConstants.AiravataClientConstants.RABBIT_BROKER_URL, PropertyFileType.AIRAVATA_CLIENT);
         System.out.println("broker url " + brokerUrl);
         final String exchangeName = propertyReader.readProperty(TestFrameworkConstants.AiravataClientConstants.RABBIT_EXCHANGE_NAME, PropertyFileType.AIRAVATA_CLIENT);
-        Subscriber statusSubscriber = MessagingFactory.getSubscriber(this::processMessage, null, Subscriber.Type.STATUS);
+        Subscriber statusSubscriber = MessagingFactory.getSubscriber(this::processMessage, null, Type.STATUS);
     }
 
     private List<String> getRoutingKeys() {

http://git-wip-us.apache.org/repos/asf/airavata/blob/e4cc54d9/modules/workflow/workflow-core/src/main/java/org/apache/airavata/workflow/core/WorkflowEnactmentService.java
----------------------------------------------------------------------
diff --git a/modules/workflow/workflow-core/src/main/java/org/apache/airavata/workflow/core/WorkflowEnactmentService.java b/modules/workflow/workflow-core/src/main/java/org/apache/airavata/workflow/core/WorkflowEnactmentService.java
index a492ef2..33f11c8 100644
--- a/modules/workflow/workflow-core/src/main/java/org/apache/airavata/workflow/core/WorkflowEnactmentService.java
+++ b/modules/workflow/workflow-core/src/main/java/org/apache/airavata/workflow/core/WorkflowEnactmentService.java
@@ -25,8 +25,9 @@ import org.apache.airavata.common.exception.AiravataException;
 import org.apache.airavata.common.utils.ServerSettings;
 import org.apache.airavata.messaging.core.MessageContext;
 import org.apache.airavata.messaging.core.MessagingFactory;
+import org.apache.airavata.messaging.core.Publisher;
 import org.apache.airavata.messaging.core.Subscriber;
-import org.apache.airavata.messaging.core.impl.RabbitMQProcessLaunchPublisher;
+import org.apache.airavata.messaging.core.Type;
 import org.apache.airavata.model.messaging.event.MessageType;
 import org.apache.airavata.model.messaging.event.ProcessIdentifier;
 import org.apache.airavata.model.messaging.event.ProcessStatusChangeEvent;
@@ -55,7 +56,7 @@ public class WorkflowEnactmentService {
         workflowMap = new ConcurrentHashMap<>();
         statusSubscriber = MessagingFactory.getSubscriber((message -> executor.execute(new StatusHandler(message))),
                                                            getRoutingKeys(),
-                                                           Subscriber.Type.STATUS);
+                                                           Type.STATUS);
         // register the shutdown hook to un-bind status consumer.
         Runtime.getRuntime().addShutdownHook(new EnactmentShutDownHook());
     }
@@ -74,7 +75,7 @@ public class WorkflowEnactmentService {
     public void submitWorkflow(String experimentId,
                                   String credentialToken,
                                   String gatewayName,
-                                  RabbitMQProcessLaunchPublisher publisher) throws Exception {
+                                  Publisher publisher) throws Exception {
 
         WorkflowInterpreter workflowInterpreter = new WorkflowInterpreter(
                 experimentId, credentialToken,gatewayName, publisher);

http://git-wip-us.apache.org/repos/asf/airavata/blob/e4cc54d9/modules/workflow/workflow-core/src/main/java/org/apache/airavata/workflow/core/WorkflowInterpreter.java
----------------------------------------------------------------------
diff --git a/modules/workflow/workflow-core/src/main/java/org/apache/airavata/workflow/core/WorkflowInterpreter.java b/modules/workflow/workflow-core/src/main/java/org/apache/airavata/workflow/core/WorkflowInterpreter.java
index ecfdeea..e637c29 100644
--- a/modules/workflow/workflow-core/src/main/java/org/apache/airavata/workflow/core/WorkflowInterpreter.java
+++ b/modules/workflow/workflow-core/src/main/java/org/apache/airavata/workflow/core/WorkflowInterpreter.java
@@ -22,7 +22,7 @@
 package org.apache.airavata.workflow.core;
 
 import org.apache.airavata.common.exception.AiravataException;
-import org.apache.airavata.messaging.core.impl.RabbitMQProcessLaunchPublisher;
+import org.apache.airavata.messaging.core.Publisher;
 import org.apache.airavata.model.ComponentState;
 import org.apache.airavata.model.ComponentStatus;
 import org.apache.airavata.model.application.io.OutputDataObjectType;
@@ -75,18 +75,18 @@ class WorkflowInterpreter {
     private Map<String, WorkflowNode> completeList = new HashMap<>();
     private Registry registry;
     private List<OutputNode> completeWorkflowOutputs = new ArrayList<>();
-    private RabbitMQProcessLaunchPublisher publisher;
+    private Publisher publisher;
     private String consumerId;
     private boolean continueWorkflow = true;
 
-    public WorkflowInterpreter(String experimentId, String credentialToken, String gatewayName, RabbitMQProcessLaunchPublisher publisher) throws RegistryException {
+    public WorkflowInterpreter(String experimentId, String credentialToken, String gatewayName, Publisher publisher) throws RegistryException {
         this.gatewayName = gatewayName;
         setExperiment(experimentId);
         this.credentialToken = credentialToken;
         this.publisher = publisher;
     }
 
-    public WorkflowInterpreter(ExperimentModel experiment, String credentialStoreToken, String gatewayName, RabbitMQProcessLaunchPublisher publisher) {
+    public WorkflowInterpreter(ExperimentModel experiment, String credentialStoreToken, String gatewayName, Publisher publisher) {
         this.gatewayName = gatewayName;
         this.experiment = experiment;
         this.credentialToken = credentialStoreToken;


[21/50] [abbrv] airavata git commit: Set update time for experiment launch and cancel events

Posted by la...@apache.org.
Set update time for experiment launch and cancel events


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: 3fcde5295b75e58ff39f3442ed060d969443594c
Parents: f4aeee5
Author: Shameera Rathnayaka <sh...@gmail.com>
Authored: Mon Aug 15 13:12:34 2016 -0400
Committer: Shameera Rathnayaka <sh...@gmail.com>
Committed: Mon Aug 15 13:12:34 2016 -0400

----------------------------------------------------------------------
 .../apache/airavata/api/server/handler/AiravataServerHandler.java  | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/3fcde529/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java b/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
index 5d8a8f9..8ce1c65 100644
--- a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
+++ b/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
@@ -3651,12 +3651,14 @@ public class AiravataServerHandler implements Airavata.Iface {
     private void submitExperiment(String gatewayId,String experimentId) throws AiravataException {
         ExperimentSubmitEvent event = new ExperimentSubmitEvent(experimentId, gatewayId);
         MessageContext messageContext = new MessageContext(event, MessageType.EXPERIMENT, "LAUNCH.EXP-" + UUID.randomUUID().toString(), gatewayId);
+        messageContext.setUpdatedTime(AiravataUtils.getCurrentTimestamp());
         experimentPublisher.publish(messageContext);
     }
 
     private void submitCancelExperiment(String gatewayId, String experimentId) throws AiravataException {
         ExperimentSubmitEvent event = new ExperimentSubmitEvent(experimentId, gatewayId);
         MessageContext messageContext = new MessageContext(event, MessageType.EXPERIMENT_CANCEL, "CANCEL.EXP-" + UUID.randomUUID().toString(), gatewayId);
+        messageContext.setUpdatedTime(AiravataUtils.getCurrentTimestamp());
         experimentPublisher.publish(messageContext);
     }
 


[45/50] [abbrv] airavata git commit: fixing the database creation failure

Posted by la...@apache.org.
fixing the database creation failure


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: bfff64c58c58b62116f64d7c81b6fbc4d23d7d52
Parents: 415e9b7
Author: scnakandala <su...@gmail.com>
Authored: Fri Aug 26 02:06:02 2016 -0400
Committer: scnakandala <su...@gmail.com>
Committed: Fri Aug 26 02:06:02 2016 -0400

----------------------------------------------------------------------
 modules/registry/registry-core/pom.xml                            | 2 +-
 .../registry-core/src/main/resources/expcatalog-derby.sql         | 3 ++-
 .../registry-core/src/main/resources/expcatalog-mysql.sql         | 3 ++-
 3 files changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/bfff64c5/modules/registry/registry-core/pom.xml
----------------------------------------------------------------------
diff --git a/modules/registry/registry-core/pom.xml b/modules/registry/registry-core/pom.xml
index 85986ae..aa9a61b 100644
--- a/modules/registry/registry-core/pom.xml
+++ b/modules/registry/registry-core/pom.xml
@@ -162,7 +162,7 @@
                 <inherited>true</inherited>
                 <configuration>
                     <failIfNoTests>false</failIfNoTests>
-                    <skipTests>true</skipTests>
+                    <skipTests>${skipTests}</skipTests>
                     <workingDirectory>${project.build.testOutputDirectory}</workingDirectory>
                     <!-- making sure that the sure-fire plugin doesn't run the integration tests-->
                     <!-- Integration tests are run using the fail-safe plugin in the module pom-->

http://git-wip-us.apache.org/repos/asf/airavata/blob/bfff64c5/modules/registry/registry-core/src/main/resources/expcatalog-derby.sql
----------------------------------------------------------------------
diff --git a/modules/registry/registry-core/src/main/resources/expcatalog-derby.sql b/modules/registry/registry-core/src/main/resources/expcatalog-derby.sql
index 275c772..3388dea 100644
--- a/modules/registry/registry-core/src/main/resources/expcatalog-derby.sql
+++ b/modules/registry/registry-core/src/main/resources/expcatalog-derby.sql
@@ -37,7 +37,8 @@ CREATE TABLE GATEWAY
         DECLINED_REASON varchar(255),
         OAUTH_CLIENT_SECRET varchar(255),
         OAUTH_CLIENT_ID varchar(255),
-        REQUEST_CREATION_TIME datetime DEFAULT CURRENT_TIMESTAMP,
+        REQUEST_CREATION_TIME TIMESTAMP DEFAULT CURRENT_TIMESTAMP,
+        REQUESTER_USERNAME VARCHAR(255),
         PRIMARY KEY (GATEWAY_ID)
 );
 

http://git-wip-us.apache.org/repos/asf/airavata/blob/bfff64c5/modules/registry/registry-core/src/main/resources/expcatalog-mysql.sql
----------------------------------------------------------------------
diff --git a/modules/registry/registry-core/src/main/resources/expcatalog-mysql.sql b/modules/registry/registry-core/src/main/resources/expcatalog-mysql.sql
index e5d6b09..2c776af 100644
--- a/modules/registry/registry-core/src/main/resources/expcatalog-mysql.sql
+++ b/modules/registry/registry-core/src/main/resources/expcatalog-mysql.sql
@@ -37,7 +37,8 @@ CREATE TABLE GATEWAY
         DECLINED_REASON varchar(255),
         OAUTH_CLIENT_SECRET varchar(255),
         OAUTH_CLIENT_ID varchar(255),
-        REQUEST_CREATION_TIME datetime DEFAULT CURRENT_TIMESTAMP,
+        REQUEST_CREATION_TIME TIMESTAMP DEFAULT CURRENT_TIMESTAMP,
+        REQUESTER_USERNAME VARCHAR(255),
         PRIMARY KEY (GATEWAY_ID)
 );
 


[12/50] [abbrv] airavata git commit: Fixed merge confilcts

Posted by la...@apache.org.
Fixed merge confilcts


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: 01716e5360417926a32cf1e4e1aef441c4b939d7
Parents: 54f5c34 b9b2480
Author: Shameera Rathnayaka <sh...@gmail.com>
Authored: Thu Aug 11 15:06:52 2016 -0400
Committer: Shameera Rathnayaka <sh...@gmail.com>
Committed: Thu Aug 11 15:06:52 2016 -0400

----------------------------------------------------------------------
 airavata-api/airavata-api-server/pom.xml        |   7 +-
 .../airavata/api/server/AiravataAPIServer.java  |  16 +-
 .../server/handler/AiravataServerHandler.java   |  40 +-
 .../DefaultAiravataSecurityManager.java         |  29 +-
 .../util/AiravataServerThreadPoolExecutor.java  |  55 ---
 .../api/server/util/AppCatalogInitUtil.java     | 168 --------
 .../api/server/util/ConnectionPool.java         | 383 -------------------
 .../server/util/CredentialStoreInitUtil.java    | 156 --------
 .../api/server/util/DataModelUtils.java         |  56 ---
 .../api/server/util/DatabaseCreator.java        | 353 -----------------
 .../server/util/ExperimentCatalogInitUtil.java  | 203 ----------
 .../airavata/api/server/util/JdbcStorage.java   | 176 ---------
 .../api/server/util/OrchestratorInitUtil.java   |  25 --
 .../api/server/util/ReplicaCatalogInitUtil.java | 160 --------
 .../server/util/WorkflowCatalogInitUtil.java    | 162 --------
 .../api/server/handler/utils/AppCatInit.java    | 320 ----------------
 .../api/server/handler/utils/ExpCatInit.java    | 330 ----------------
 .../server/handler/utils/ReplicaCatInit.java    | 315 ---------------
 18 files changed, 26 insertions(+), 2928 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/01716e53/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
----------------------------------------------------------------------


[29/50] [abbrv] airavata git commit: Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/airavata into develop

Posted by la...@apache.org.
Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/airavata into develop


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: bc3733487526c1914c28679f6210901f73024a49
Parents: 18ea6c2 b168933
Author: Shameera Rathnayaka <sh...@gmail.com>
Authored: Tue Aug 16 14:21:30 2016 -0400
Committer: Shameera Rathnayaka <sh...@gmail.com>
Committed: Tue Aug 16 14:21:30 2016 -0400

----------------------------------------------------------------------

----------------------------------------------------------------------



[09/50] [abbrv] airavata git commit: removing registry core and cpi dependencies from Airavata API Server

Posted by la...@apache.org.
removing registry core and cpi dependencies from Airavata API Server


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: 1eb3b4155808dd1bfe2f08002cf49a7018e549af
Parents: 3a70c59
Author: scnakandala <su...@gmail.com>
Authored: Thu Aug 11 12:44:39 2016 -0400
Committer: scnakandala <su...@gmail.com>
Committed: Thu Aug 11 12:44:39 2016 -0400

----------------------------------------------------------------------
 airavata-api/airavata-api-server/pom.xml        |   7 +-
 .../airavata/api/server/AiravataAPIServer.java  |  16 +-
 .../server/handler/AiravataServerHandler.java   |  35 +-
 .../DefaultAiravataSecurityManager.java         |  29 +-
 .../util/AiravataServerThreadPoolExecutor.java  |  55 ---
 .../api/server/util/AppCatalogInitUtil.java     | 168 --------
 .../api/server/util/ConnectionPool.java         | 383 -------------------
 .../server/util/CredentialStoreInitUtil.java    | 156 --------
 .../api/server/util/DataModelUtils.java         |  56 ---
 .../api/server/util/DatabaseCreator.java        | 353 -----------------
 .../server/util/ExperimentCatalogInitUtil.java  | 203 ----------
 .../airavata/api/server/util/JdbcStorage.java   | 176 ---------
 .../api/server/util/OrchestratorInitUtil.java   |  25 --
 .../api/server/util/ReplicaCatalogInitUtil.java | 160 --------
 .../server/util/WorkflowCatalogInitUtil.java    | 162 --------
 .../api/server/handler/utils/AppCatInit.java    | 320 ----------------
 .../api/server/handler/utils/ExpCatInit.java    | 330 ----------------
 .../server/handler/utils/ReplicaCatInit.java    | 315 ---------------
 18 files changed, 26 insertions(+), 2923 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/1eb3b415/airavata-api/airavata-api-server/pom.xml
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-api-server/pom.xml b/airavata-api/airavata-api-server/pom.xml
index 2108ce0..e5b549c 100644
--- a/airavata-api/airavata-api-server/pom.xml
+++ b/airavata-api/airavata-api-server/pom.xml
@@ -48,7 +48,7 @@
         </dependency>
         <dependency>
             <groupId>org.apache.airavata</groupId>
-            <artifactId>airavata-registry-core</artifactId>
+            <artifactId>airavata-credential-store-stubs</artifactId>
             <version>${project.version}</version>
         </dependency>
         <dependency>
@@ -58,11 +58,6 @@
         </dependency>
         <dependency>
             <groupId>org.apache.airavata</groupId>
-            <artifactId>airavata-registry-cpi</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.airavata</groupId>
             <artifactId>airavata-orchestrator-client</artifactId>
             <version>${project.version}</version>
         </dependency>

http://git-wip-us.apache.org/repos/asf/airavata/blob/1eb3b415/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/AiravataAPIServer.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/AiravataAPIServer.java b/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/AiravataAPIServer.java
index 5290e83..97a9761 100644
--- a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/AiravataAPIServer.java
+++ b/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/AiravataAPIServer.java
@@ -64,17 +64,6 @@ public class AiravataAPIServer implements IServer{
 	
     public void startAiravataServer(Airavata.Processor<Airavata.Iface> airavataAPIServer) throws AiravataSystemException {
         try {
-            // creating experiment catalog db
-            ExperimentCatalogInitUtil.initializeDB();
-            // creating app catalog db
-            AppCatalogInitUtil.initializeDB();
-            // creating workflow catalog db
-            WorkflowCatalogInitUtil.initializeDB();
-            // creating credential store db
-            CredentialStoreInitUtil.initializeDB();
-			// creating replica catalog db
-			ReplicaCatalogInitUtil.initializeDB();
-
             final String serverHost = ServerSettings.getSetting(Constants.API_SERVER_HOST, null);
             if (!ServerSettings.isTLSEnabled()) {
                 final int serverPort = Integer.parseInt(ServerSettings.getSetting(Constants.API_SERVER_PORT, "8930"));
@@ -106,7 +95,6 @@ public class AiravataAPIServer implements IServer{
 				new Thread() {
 					public void run() {
                         server.serve();
-						ExperimentCatalogInitUtil.stopDerbyInServerMode();
 						setStatus(ServerStatus.STOPPED);
 						logger.info("Airavata API Server Stopped.");
 					}
@@ -142,7 +130,6 @@ public class AiravataAPIServer implements IServer{
                 new Thread() {
                     public void run() {
                         TLSServer.serve();
-                        ExperimentCatalogInitUtil.stopDerbyInServerMode();
                         setStatus(ServerStatus.STOPPED);
                         logger.info("Airavata API Server over TLS Stopped.");
                     }
@@ -172,8 +159,7 @@ public class AiravataAPIServer implements IServer{
         } catch (TTransportException e) {
             logger.error(e.getMessage());
             setStatus(ServerStatus.FAILED);
-            ExperimentCatalogInitUtil.stopDerbyInServerMode();
-			logger.error("Failed to start Gfac server ...");
+			logger.error("Failed to start API server ...");
 			throw new AiravataSystemException(AiravataErrorType.INTERNAL_ERROR);
         } catch (ApplicationSettingsException e) {
             logger.error(e.getMessage(), e);

http://git-wip-us.apache.org/repos/asf/airavata/blob/1eb3b415/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java b/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
index 68ffa97..208cd1b 100644
--- a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
+++ b/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
@@ -81,8 +81,6 @@ import org.apache.airavata.orchestrator.cpi.OrchestratorService.Client;
 import org.apache.airavata.registry.api.RegistryService;
 import org.apache.airavata.registry.api.client.RegistryServiceClientFactory;
 import org.apache.airavata.registry.api.exception.RegistryServiceException;
-import org.apache.airavata.registry.core.experiment.catalog.impl.RegistryFactory;
-import org.apache.airavata.registry.cpi.*;
 import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -913,7 +911,7 @@ public class AiravataServerHandler implements Airavata.Iface {
 
             if(!(experimentModel.getExperimentStatus().getState() == ExperimentState.CREATED)){
                 logger.error("Error while deleting the experiment");
-                throw new ExperimentCatalogException("Experiment is not in CREATED state. Hence cannot deleted. ID:"+ experimentId);
+                throw new RegistryServiceException("Experiment is not in CREATED state. Hence cannot deleted. ID:"+ experimentId);
             }
             return regClient.deleteExperiment(experimentId);
         } catch (Exception e) {
@@ -2203,17 +2201,6 @@ public class AiravataServerHandler implements Airavata.Iface {
         }
     }
 
-    private String addJobSubmissionInterface(ComputeResource computeResource,
-			String computeResourceId, String jobSubmissionInterfaceId,
-			JobSubmissionProtocol protocolType, int priorityOrder)
-			throws AppCatalogException {
-		JobSubmissionInterface jobSubmissionInterface = new JobSubmissionInterface();
-		jobSubmissionInterface.setJobSubmissionInterfaceId(jobSubmissionInterfaceId);
-		jobSubmissionInterface.setPriorityOrder(priorityOrder);
-		jobSubmissionInterface.setJobSubmissionProtocol(protocolType);
-		return computeResource.addJobSubmissionProtocol(computeResourceId,jobSubmissionInterface);
-	}
-
     /**
      * Add a SSH Job Submission details to a compute resource
      * App catalog will return a jobSubmissionInterfaceId which will be added to the jobSubmissionInterfaces.
@@ -2480,16 +2467,6 @@ public class AiravataServerHandler implements Airavata.Iface {
         }
     }
 
-    private String addDataMovementInterface(ComputeResource computeResource,
-			String computeResourceId, DMType dmType, String dataMovementInterfaceId,
-			DataMovementProtocol protocolType, int priorityOrder)
-			throws AppCatalogException {
-		DataMovementInterface dataMovementInterface = new DataMovementInterface();
-		dataMovementInterface.setDataMovementInterfaceId(dataMovementInterfaceId);
-		dataMovementInterface.setPriorityOrder(priorityOrder);
-		dataMovementInterface.setDataMovementProtocol(protocolType);
-		return computeResource.addDataMovementProtocol(computeResourceId, dmType, dataMovementInterface);
-	}
 
     /**
      * Add a SCP data moevement details to a compute resource
@@ -3554,11 +3531,10 @@ public class AiravataServerHandler implements Airavata.Iface {
         }
     }
 
-    private void initializeResourceWithGrouper(String resourceId, ResourceType resourceType) throws RegistryException, GroupManagerException {
-        ExperimentCatalog experimentCatalog = RegistryFactory.getDefaultExpCatalog();
+    private void initializeResourceWithGrouper(String resourceId, ResourceType resourceType) throws RegistryServiceException, GroupManagerException, TException, ApplicationSettingsException {
         GroupManagerCPI groupManager = GroupManagerFactory.getGroupManager();
         if(resourceType.equals(ResourceType.PROJECT)){
-            Project project = (Project) experimentCatalog.get(ExperimentCatalogModelType.PROJECT, resourceId);
+            Project project = (Project) getRegistryServiceClient().getProject(resourceId);
 
             Resource projectResource = new Resource(project.getProjectID(), org.apache.airavata.grouper.resource.ResourceType.PROJECT);
             projectResource.setName(project.getName());
@@ -3567,7 +3543,7 @@ public class AiravataServerHandler implements Airavata.Iface {
             groupManager.createResource(projectResource);
 
         }else if(resourceType.equals(ResourceType.EXPERIMENT)){
-            ExperimentModel experiment = (ExperimentModel) experimentCatalog.get(ExperimentCatalogModelType.EXPERIMENT, resourceId);
+            ExperimentModel experiment = getRegistryServiceClient().getExperiment(resourceId);
             if(!isResourceExistsInGrouper(experiment.getProjectId(), ResourceType.PROJECT)){
                 initializeResourceWithGrouper(experiment.getProjectId(), ResourceType.PROJECT);
             }
@@ -3595,7 +3571,8 @@ public class AiravataServerHandler implements Airavata.Iface {
 
     }
 
-    private boolean hasPermission(String userId, String resourceId, ResourceType resourceType, ResourcePermissionType permissionType) throws GroupManagerException, RegistryException {
+    private boolean hasPermission(String userId, String resourceId, ResourceType resourceType, ResourcePermissionType permissionType)
+            throws GroupManagerException, TException, ApplicationSettingsException {
         if(!isResourceExistsInGrouper(resourceId, resourceType)){
             initializeResourceWithGrouper(resourceId, resourceType);
         }

http://git-wip-us.apache.org/repos/asf/airavata/blob/1eb3b415/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/security/DefaultAiravataSecurityManager.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/security/DefaultAiravataSecurityManager.java b/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/security/DefaultAiravataSecurityManager.java
index 90039f4..3dbab86 100644
--- a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/security/DefaultAiravataSecurityManager.java
+++ b/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/security/DefaultAiravataSecurityManager.java
@@ -33,9 +33,9 @@ import org.apache.airavata.credential.store.datamodel.PasswordCredential;
 import org.apache.airavata.credential.store.exception.CredentialStoreException;
 import org.apache.airavata.model.appcatalog.gatewayprofile.GatewayResourceProfile;
 import org.apache.airavata.model.security.AuthzToken;
-import org.apache.airavata.registry.core.experiment.catalog.impl.RegistryFactory;
-import org.apache.airavata.registry.cpi.AppCatalog;
-import org.apache.airavata.registry.cpi.AppCatalogException;
+import org.apache.airavata.registry.api.RegistryService;
+import org.apache.airavata.registry.api.client.RegistryServiceClientFactory;
+import org.apache.airavata.registry.api.exception.RegistryServiceException;
 import org.apache.airavata.security.AiravataSecurityException;
 import org.apache.airavata.security.util.TrustStoreManager;
 import org.apache.axis2.AxisFault;
@@ -69,8 +69,7 @@ public class DefaultAiravataSecurityManager implements AiravataSecurityManager {
                 TrustStoreManager trustStoreManager = new TrustStoreManager();
                 trustStoreManager.initializeTrustStoreManager(ServerSettings.getTrustStorePath(),
                         ServerSettings.getTrustStorePassword());
-                AppCatalog appCatalog = RegistryFactory.getAppCatalog();
-                List<GatewayResourceProfile> gwProfiles = appCatalog.getGatewayProfile().getAllGatewayProfiles();
+                List<GatewayResourceProfile> gwProfiles = getRegistryServiceClient().getAllGatewayResourceProfiles();
                 //read the policy as a string
                 BufferedReader bufferedReader = new BufferedReader(new FileReader(new File(
                         ServerSettings.getAuthorizationPoliyName() + ".xml")));
@@ -117,7 +116,7 @@ public class DefaultAiravataSecurityManager implements AiravataSecurityManager {
         } catch (IOException e) {
             logger.error(e.getMessage(), e);
             throw new AiravataSecurityException("Error in reading the authorization policy.");
-        } catch (AppCatalogException e) {
+        } catch (RegistryServiceException e) {
             logger.error(e.getMessage(), e);
             throw new AiravataSecurityException("Error in reading the Gateway Profiles from App Catalog.");
         } catch (TException e) {
@@ -153,8 +152,7 @@ public class DefaultAiravataSecurityManager implements AiravataSecurityManager {
                             "Obtaining it from the authorization server.");
 
                     CredentialStoreService.Client csClient = getCredentialStoreServiceClient();
-                    AppCatalog appCatalog = RegistryFactory.getAppCatalog();
-                    GatewayResourceProfile gwrp = appCatalog.getGatewayProfile().getGatewayProfile(gatewayId);
+                    GatewayResourceProfile gwrp = getRegistryServiceClient().getGatewayResourceProfile(gatewayId);
                     PasswordCredential credential = csClient.getPasswordCredential(gwrp.getIdentityServerPwdCredToken(), gwrp.getGatewayID());
                     String username = credential.getLoginUserName();
                     if(gwrp.getIdentityServerTenant() != null && !gwrp.getIdentityServerTenant().isEmpty())
@@ -209,8 +207,7 @@ public class DefaultAiravataSecurityManager implements AiravataSecurityManager {
                 }
             } else {
                 CredentialStoreService.Client csClient = getCredentialStoreServiceClient();
-                AppCatalog appCatalog = RegistryFactory.getAppCatalog();
-                GatewayResourceProfile gwrp = appCatalog.getGatewayProfile().getGatewayProfile(gatewayId);
+                GatewayResourceProfile gwrp = getRegistryServiceClient().getGatewayResourceProfile(gatewayId);
                 PasswordCredential credential = csClient.getPasswordCredential(gwrp.getIdentityServerPwdCredToken(), gwrp.getGatewayID());
                 String username = credential.getLoginUserName();
                 if(gwrp.getIdentityServerTenant() != null && !gwrp.getIdentityServerTenant().isEmpty())
@@ -245,7 +242,7 @@ public class DefaultAiravataSecurityManager implements AiravataSecurityManager {
         } catch (ApplicationSettingsException e) {
             logger.error(e.getMessage(), e);
             throw new AiravataSecurityException("Error in reading OAuth server configuration.");
-        } catch (AppCatalogException e) {
+        } catch (RegistryServiceException e) {
             logger.error(e.getMessage(), e);
             throw new AiravataSecurityException("Error in accessing AppCatalog.");
         } catch (TException e) {
@@ -263,4 +260,14 @@ public class DefaultAiravataSecurityManager implements AiravataSecurityManager {
             throw new TException("Unable to create credential store client...", e);
         }
     }
+
+    private RegistryService.Client getRegistryServiceClient() throws TException, ApplicationSettingsException {
+        final int serverPort = Integer.parseInt(ServerSettings.getRegistryServerPort());
+        final String serverHost = ServerSettings.getRegistryServerHost();
+        try {
+            return RegistryServiceClientFactory.createRegistryClient(serverHost, serverPort);
+        } catch (RegistryServiceException e) {
+            throw new TException("Unable to create registry client...", e);
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/airavata/blob/1eb3b415/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/AiravataServerThreadPoolExecutor.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/AiravataServerThreadPoolExecutor.java b/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/AiravataServerThreadPoolExecutor.java
deleted file mode 100644
index 9f04ddb..0000000
--- a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/AiravataServerThreadPoolExecutor.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
-*
-* 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.airavata.api.server.util;
-
-import org.apache.airavata.common.exception.ApplicationSettingsException;
-import org.apache.airavata.common.utils.ServerSettings;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-
-public class AiravataServerThreadPoolExecutor {
-    private final static Logger logger = LoggerFactory.getLogger(AiravataServerThreadPoolExecutor.class);
-    public static final String AIRAVATA_SERVER_THREAD_POOL_SIZE = "airavata.server.thread.pool.size";
-
-    private static ExecutorService threadPool;
-
-    public static ExecutorService getCachedThreadPool() {
-        if(threadPool ==null){
-            threadPool = Executors.newCachedThreadPool();
-        }
-        return threadPool;
-    }
-
-    public static ExecutorService client() {
-        if(threadPool ==null){
-            try {
-                threadPool = Executors.newFixedThreadPool(Integer.parseInt(ServerSettings.getSetting(AIRAVATA_SERVER_THREAD_POOL_SIZE)));
-            } catch (ApplicationSettingsException e) {
-                logger.error("Error reading " + AIRAVATA_SERVER_THREAD_POOL_SIZE+ " property");
-            }
-        }
-        return threadPool;
-    }
-}

http://git-wip-us.apache.org/repos/asf/airavata/blob/1eb3b415/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/AppCatalogInitUtil.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/AppCatalogInitUtil.java b/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/AppCatalogInitUtil.java
deleted file mode 100644
index 0be8922..0000000
--- a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/AppCatalogInitUtil.java
+++ /dev/null
@@ -1,168 +0,0 @@
-/*
-*
-* 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.airavata.api.server.util;
-
-import org.apache.airavata.common.exception.ApplicationSettingsException;
-import org.apache.airavata.common.utils.AiravataUtils;
-import org.apache.airavata.common.utils.ServerSettings;
-import org.apache.airavata.registry.core.app.catalog.resources.GatewayProfileResource;
-import org.apache.derby.drda.NetworkServerControl;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.URI;
-import java.sql.Connection;
-import java.sql.SQLException;
-
-public class AppCatalogInitUtil {
-    public static final String COMPUTE_RESOURCE = "GATEWAY_PROFILE";
-    private static final Logger logger = LoggerFactory.getLogger(AppCatalogInitUtil.class);
-    public static final String APPCATALOG_JDBC_DRIVER = "appcatalog.jdbc.driver";
-    public static final String APPCATALOG_JDBC_URL = "appcatalog.jdbc.url";
-    public static final String APPCATALOG_JDBC_USER = "appcatalog.jdbc.user";
-    public static final String APPCATALOG_JDBC_PASSWORD = "appcatalog.jdbc.password";
-    public static final String START_DERBY_ENABLE = "start.derby.server.mode";
-    public static final String DERBY_SERVER_MODE_SYS_PROPERTY = "derby.drda.startNetworkServer";
-    private static NetworkServerControl server;
-    private static JdbcStorage db;
-    private static String jdbcURl;
-    private static String jdbcDriver;
-    private static String jdbcUser;
-    private static String jdbcPassword;
-
-
-    public static void initializeDB() {
-//        System.setProperty("appcatalog.initialize.state", "0");
-        try{
-            jdbcDriver = ServerSettings.getSetting(APPCATALOG_JDBC_DRIVER);
-            jdbcURl = ServerSettings.getSetting(APPCATALOG_JDBC_URL);
-            jdbcUser = ServerSettings.getSetting(APPCATALOG_JDBC_USER);
-            jdbcPassword = ServerSettings.getSetting(APPCATALOG_JDBC_PASSWORD);
-            jdbcURl = jdbcURl + "?" + "user=" + jdbcUser + "&" + "password=" + jdbcPassword;
-        } catch (ApplicationSettingsException e) {
-            logger.error("Unable to read airavata server properties", e.getMessage());
-        }
-
-        if (getDBType(jdbcURl).equals("derby") && isDerbyStartEnabled()) {
-            startDerbyInServerMode();
-        }
-        db = new JdbcStorage(10, 50, jdbcURl, jdbcDriver, true);
-
-        Connection conn = null;
-        try {
-            conn = db.connect();
-            if (!DatabaseCreator.isDatabaseStructureCreated(COMPUTE_RESOURCE, conn)) {
-                DatabaseCreator.createRegistryDatabase("database_scripts/appcatalog", conn);
-                logger.info("New Database created for App Catalog !!!");
-            } else {
-                logger.info("Database already created for App Catalog !!!");
-            }
-            GatewayProfileResource gatewayProfileResource = new GatewayProfileResource();
-            if (!gatewayProfileResource.isExists(ServerSettings.getDefaultUserGateway())){
-                gatewayProfileResource.setGatewayID(ServerSettings.getDefaultUserGateway());
-                gatewayProfileResource.setCreatedTime(AiravataUtils.getCurrentTimestamp());
-                gatewayProfileResource.save();
-            }
-        } catch (Exception e) {
-            logger.error(e.getMessage(), e);
-            throw new RuntimeException("Database failure", e);
-        } finally {
-            db.closeConnection(conn);
-            try {
-                if(conn != null){
-                    if (!conn.getAutoCommit()) {
-                        conn.commit();
-                    }
-                    conn.close();
-                }
-            } catch (SQLException e) {
-                logger.error("Error while closing database connection...", e.getMessage(), e);
-            }
-        }
-//        System.setProperty("appcatalog.initialize.state", "1");
-    }
-
-    public static String getDBType(String jdbcUrl){
-        try{
-            String cleanURI = jdbcUrl.substring(5);
-            URI uri = URI.create(cleanURI);
-            return uri.getScheme();
-        } catch (Exception e) {
-            logger.error(e.getMessage(), e);
-            return null;
-        }
-    }
-
-    public static boolean isDerbyStartEnabled(){
-        try {
-            String s = ServerSettings.getSetting(START_DERBY_ENABLE);
-            if("true".equals(s)){
-                return true;
-            }
-        }  catch (ApplicationSettingsException e) {
-            logger.error("Unable to read airavata server properties", e.getMessage(), e);
-            return false;
-        }
-        return false;
-    }
-
-    public static void startDerbyInServerMode() {
-        try {
-            System.setProperty(DERBY_SERVER_MODE_SYS_PROPERTY, "true");
-            server = new NetworkServerControl(InetAddress.getByName("0.0.0.0"),
-                    getPort(jdbcURl),
-                    jdbcUser, jdbcPassword);
-            java.io.PrintWriter consoleWriter = new java.io.PrintWriter(System.out, true);
-            server.start(consoleWriter);
-        } catch (IOException e) {
-            logger.error("Unable to start Apache derby in the server mode! Check whether " +
-                    "specified port is available");
-        } catch (Exception e) {
-            logger.error("Unable to start Apache derby in the server mode! Check whether " +
-                    "specified port is available");
-        }
-    }
-    
-    public static void stopDerbyInServerMode() {
-            System.setProperty(DERBY_SERVER_MODE_SYS_PROPERTY, "false");
-            if (server!=null){
-            	try {
-					server.shutdown();
-				} catch (Exception e) {
-		            logger.error("Error when stopping the derby server : "+e.getLocalizedMessage());
-				}
-            }
-    }
-
-    public static int getPort(String jdbcURL){
-        try{
-            String cleanURI = jdbcURL.substring(5);
-            URI uri = URI.create(cleanURI);
-            return uri.getPort();
-        } catch (Exception e) {
-            logger.error(e.getMessage(), e);
-            return -1;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/airavata/blob/1eb3b415/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/ConnectionPool.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/ConnectionPool.java b/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/ConnectionPool.java
deleted file mode 100644
index 5715b9d..0000000
--- a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/ConnectionPool.java
+++ /dev/null
@@ -1,383 +0,0 @@
-/*
- *
- * 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.airavata.api.server.util;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.sql.DataSource;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.SQLException;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Stack;
-import java.util.concurrent.Semaphore;
-
-
-/**
- * A class for preallocating, recycling, and managing JDBC connections.
- */
-public class ConnectionPool {
-    private static final Logger logger = LoggerFactory.getLogger(ConnectionPool.class);
-
-    private long MAX_IDLE_TIME = 5 * 60 * 1000; // 5 minutes
-
-    private String driver;
-    private String url;
-    private String username;
-    private String password;
-    private String jdbcUrl;
-
-    private int maxConnections;
-
-    private boolean autoCommit = true;
-    private boolean waitIfBusy;
-
-    private Semaphore needConnection = new Semaphore(0);
-    private boolean stop;
-
-    private Stack<Connection> availableConnections;
-    private Stack<Connection> busyConnections;
-
-    private HashMap<Connection, Long> lastAccessTimeRecord = new HashMap<Connection, Long>();
-
-    private String urlType = "";
-
-    private DataSource datasource;
-
-    private int transactionIsolation = Connection.TRANSACTION_NONE;
-
-    private Thread clenupThread;
-    private Thread producerThread;
-
-    public ConnectionPool(String driver, String url, String username, String password, int initialConnections,
-                          int maxConnections, boolean waitIfBusy) throws SQLException {
-        this.driver = driver;
-        this.url = url;
-        this.username = username;
-        this.password = password;
-        this.urlType = "speratedURL";
-        initialize(initialConnections, maxConnections, waitIfBusy);
-    }
-
-    public ConnectionPool(String driver, String jdbcUrl, int initialConnections, int maxConnections,
-                          boolean waitIfBusy, boolean autoCommit, int transactionIsolation) throws SQLException {
-        this.driver = driver;
-        this.jdbcUrl = jdbcUrl;
-        this.urlType = "simpleURL";
-        this.autoCommit = autoCommit;
-        this.transactionIsolation = transactionIsolation;
-        initialize(initialConnections, maxConnections, waitIfBusy);
-    }
-
-    public ConnectionPool(String driver, String jdbcUrl, int initialConnections, int maxConnections, boolean waitIfBusy)
-            throws SQLException {
-        this.driver = driver;
-        this.jdbcUrl = jdbcUrl;
-        this.urlType = "simpleURL";
-        initialize(initialConnections, maxConnections, waitIfBusy);
-    }
-
-    public ConnectionPool(DataSource dataSource, int initialConnections, int maxConnections, boolean waitIfBusy)
-            throws SQLException {
-        this.urlType = "dataSource";
-        this.datasource = dataSource;
-        initialize(initialConnections, maxConnections, waitIfBusy);
-    }
-
-    /**
-     * Check if this connection pool is auto commit or not
-     *
-     * @return
-     */
-    public boolean isAutoCommit() {
-        return this.autoCommit;
-    }
-
-    private void initialize(int initialConnections, int maxConnections, boolean waitIfBusy) throws SQLException {
-        this.maxConnections = maxConnections;
-        this.waitIfBusy = waitIfBusy;
-
-        int sizeOfConnections = (initialConnections > maxConnections) ? maxConnections : initialConnections;
-
-        availableConnections = new Stack<Connection>();
-        busyConnections = new Stack<Connection>();
-
-        for (int i = 0; i < sizeOfConnections; i++) {
-            Connection con = makeNewConnection();
-            setTimeStamp(con);
-            availableConnections.push(con);
-
-        }
-
-        producerThread = new Thread(new FillUpThread());
-        producerThread.start();
-
-        clenupThread = new Thread(new CleanUpThread());
-        clenupThread.start();
-    }
-
-    public synchronized Connection getConnection() throws SQLException {
-        if (!availableConnections.isEmpty()) {
-            Connection existingConnection = availableConnections.pop();
-
-            // If connection on available list is closed (e.g.,
-            // it timed out), then remove it from available list
-            // and race for a connection again.
-            if (existingConnection.isClosed()) {
-                lastAccessTimeRecord.remove(existingConnection);
-                // notifyAll for fairness
-                notifyAll();
-            } else {
-                busyConnections.push(existingConnection);
-                setTimeStamp(existingConnection);
-                return existingConnection;
-            }
-        } else if (!waitIfBusy && busyConnections.size() >= maxConnections) {
-            // You reached maxConnections limit and waitIfBusy flag is false.
-            // Throw SQLException in such a case.
-            throw new SQLException("Connection limit reached");
-        } else {
-
-            if (busyConnections.size() < maxConnections) {
-                // available connection is empty, but total number of connection
-                // doesn't reach maxConnection. Request for more connection
-                needConnection.release();
-            }
-
-            try {
-                // wait for free connection
-                wait();
-            } catch (InterruptedException ie) {
-            }
-        }
-        // always race for connection forever
-        return getConnection();
-    }
-
-    // This explicitly makes a new connection. Called in
-    // the foreground when initializing the ConnectionPool,
-    // and called in the background when running.
-    private Connection makeNewConnection() throws SQLException {
-        try {
-            // Load database driver if not already loaded
-            Class.forName(driver);
-            Connection connection;
-            // Establish network connection to database
-            if (urlType.equals("speratedURL")) {
-                connection = DriverManager.getConnection(url, username, password);
-            } else if (urlType.equals("simpleURL")) {
-                connection = DriverManager.getConnection(jdbcUrl);
-            } else { // if(urlType.equals("dataSource")){
-                connection = datasource.getConnection();
-            }
-            connection.setTransactionIsolation(this.transactionIsolation);
-            connection.setAutoCommit(this.autoCommit);
-            return connection;
-        } catch (ClassNotFoundException cnfe) {
-            // Simplify try/catch blocks of people using this by
-            // throwing only one exception type.
-            throw new SQLException("Can't find class for driver: " + driver);
-        }
-    }
-
-    private synchronized void fillUpConnection(Connection conn) {
-        setTimeStamp(conn);
-        availableConnections.push(conn);
-
-        // notify all since new connection is created
-        notifyAll();
-    }
-
-    private void setTimeStamp(Connection connection) {
-        lastAccessTimeRecord.put(connection, System.currentTimeMillis());
-    }
-
-    // The database connection cannot be left idle for too long, otherwise TCP
-    // connection will be broken.
-    /**
-     * From http://forums.mysql.com/read.php?39,28450,57460#msg-57460 Okay, then it looks like wait_timeout on the
-     * server is killing your connection (it is set to 8 hours of idle time by default). Either set that value higher on
-     * your server, or configure your connection pool to not hold connections idle that long (I prefer the latter). Most
-     * folks I know that run MySQL with a connection pool in high-load production environments only let connections sit
-     * idle for a matter of minutes, since it only takes a few milliseconds to open a connection, and the longer one
-     * sits idle the more chance it will go "bad" because of a network hiccup or the MySQL server being restarted.
-     *
-     * @throws java.sql.SQLException
-     */
-    private boolean isConnectionStale(Connection connection) {
-        long currentTime = System.currentTimeMillis();
-        long lastAccess = lastAccessTimeRecord.get(connection);
-        if (currentTime - lastAccess > MAX_IDLE_TIME) {
-            return true;
-        } else
-            return false;
-    }
-
-    private synchronized void closeStaleConnections() {
-        // close idle connections
-        Iterator<Connection> iter = availableConnections.iterator();
-        while (iter.hasNext()) {
-            Connection existingConnection = iter.next();
-            if (isConnectionStale(existingConnection)) {
-                try {
-                    existingConnection.close();
-                    iter.remove();
-                } catch (SQLException sql) {
-                    logger.error(sql.getMessage(), sql);
-                }
-            }
-        }
-        // close busy connections that have been checked out for too long.
-        // This should not happen since this means program has bug for not
-        // releasing connections .
-        iter = busyConnections.iterator();
-        while (iter.hasNext()) {
-            Connection busyConnection = iter.next();
-            if (isConnectionStale(busyConnection)) {
-                try {
-                    busyConnection.close();
-                    iter.remove();
-                    logger.warn("****Connection has checked out too long. Forced release. Check the program for calling release connection [free(Connection) method]");
-                } catch (SQLException sql) {
-                    logger.error(sql.getMessage(), sql);
-                }
-            }
-        }
-    }
-
-    public synchronized void free(Connection connection) {
-        busyConnections.removeElement(connection);
-        availableConnections.addElement(connection);
-        // Wake up threads that are waiting for a connection
-        notifyAll();
-    }
-
-    /**
-     * Close all the connections. Use with caution: be sure no connections are in use before calling. Note that you are
-     * not <I>required</I> to call this when done with a ConnectionPool, since connections are guaranteed to be closed
-     * when garbage collected. But this method gives more control regarding when the connections are closed.
-     */
-    public synchronized void dispose() {
-        logger.info("Connection Pool Shutting down");
-
-        // stop clean up thread
-        this.stop = true;
-        this.clenupThread.interrupt();
-
-        // stop producer up thread
-        this.producerThread.interrupt();
-
-        // close all connection
-        closeConnections(availableConnections);
-        availableConnections = new Stack<Connection>();
-        closeConnections(busyConnections);
-        busyConnections = new Stack<Connection>();
-        lastAccessTimeRecord.clear();
-
-        logger.info("All connection is closed");
-
-        try {
-            this.clenupThread.join();
-            this.producerThread.join();
-        } catch (Exception e) {
-            logger.error("Cannot shutdown cleanup thread", e);
-        }
-
-        logger.info("Connection Pool Shutdown");
-    }
-
-    private void closeConnections(Stack<Connection> connections) {
-        while (!connections.isEmpty()) {
-            Connection connection = connections.pop();
-            try {
-                if (!connection.isClosed()) {
-                    connection.close();
-                }
-            } catch (SQLException sqle) {
-                // Ignore errors; garbage collect anyhow
-                logger.warn(sqle.getMessage());
-            }
-        }
-    }
-
-    public synchronized String toString() {
-        String info = "ConnectionPool(" + url + "," + username + ")" + ", available=" + availableConnections.size()
-                + ", busy=" + busyConnections.size() + ", max=" + maxConnections;
-        return (info);
-    }
-
-    class CleanUpThread implements Runnable {
-        public void run() {
-            while (!stop) {
-                try {
-                    Thread.sleep(MAX_IDLE_TIME);
-                    closeStaleConnections();
-                } catch (InterruptedException e) {
-                    logger.info("Clean up thread is interrupted to close");
-                }
-            }
-        }
-    }
-
-    class FillUpThread implements Runnable {
-        public void run() {
-            while (!stop) {
-                try {
-                    // block until get
-                    needConnection.acquire();
-
-                    Connection conn = makeNewConnection();
-                    fillUpConnection(conn);
-                } catch (SQLException e) {
-                    // cannot create connection (increase semaphore value back)
-                    needConnection.release();
-                    logger.error(e.getMessage(), e);
-                } catch (InterruptedException e) {
-                    logger.info("Fill up thread is interrupted to close");
-                    break;
-                }
-            }
-        }
-    }
-
-    public void shutdown() throws SQLException{
-        for (Connection c : availableConnections) {
-            try {
-                c.close();
-            } catch (SQLException e) {
-                logger.error("Error while closing the connection", e);
-                throw new SQLException("Error while closing the connection", e);
-            }
-        }
-
-        for (Connection c : busyConnections) {
-            try {
-                c.close();
-            } catch (SQLException e) {
-                logger.error("Error while closing the connection", e);
-                throw new SQLException("Error while closing the connection", e);
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/airavata/blob/1eb3b415/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/CredentialStoreInitUtil.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/CredentialStoreInitUtil.java b/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/CredentialStoreInitUtil.java
deleted file mode 100644
index 20dd1aa..0000000
--- a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/CredentialStoreInitUtil.java
+++ /dev/null
@@ -1,156 +0,0 @@
-/*
-*
-* 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.airavata.api.server.util;
-
-import org.apache.airavata.common.exception.ApplicationSettingsException;
-import org.apache.airavata.common.utils.ServerSettings;
-import org.apache.derby.drda.NetworkServerControl;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.URI;
-import java.sql.Connection;
-import java.sql.SQLException;
-
-public class CredentialStoreInitUtil {
-    private static final Logger logger = LoggerFactory.getLogger(CredentialStoreInitUtil.class);
-    public static final String CREDENTIALS = "CREDENTIALS";
-    public static final String START_DERBY_ENABLE = "start.derby.server.mode";
-    public static final String DERBY_SERVER_MODE_SYS_PROPERTY = "derby.drda.startNetworkServer";
-    private static NetworkServerControl server;
-    private static JdbcStorage db;
-    private static String jdbcURl;
-    private static String jdbcDriver;
-    private static String jdbcUser;
-    private static String jdbcPassword;
-
-
-    public static void initializeDB() {
-//        System.setProperty("appcatalog.initialize.state", "0");
-        try{
-            jdbcDriver = ServerSettings.getCredentialStoreDBDriver();
-            jdbcURl = ServerSettings.getCredentialStoreDBURL();
-            jdbcUser = ServerSettings.getCredentialStoreDBUser();
-            jdbcPassword = ServerSettings.getCredentialStoreDBPassword();
-            jdbcURl = jdbcURl + "?" + "user=" + jdbcUser + "&" + "password=" + jdbcPassword;
-        } catch (ApplicationSettingsException e) {
-            logger.error("Unable to read airavata server properties", e.getMessage());
-        }
-
-        if (getDBType(jdbcURl).equals("derby") && isDerbyStartEnabled()) {
-            startDerbyInServerMode();
-        }
-        db = new JdbcStorage(10, 50, jdbcURl, jdbcDriver, true);
-
-        Connection conn = null;
-        try {
-            conn = db.connect();
-            if (!DatabaseCreator.isDatabaseStructureCreated(CREDENTIALS, conn)) {
-                DatabaseCreator.createRegistryDatabase("database_scripts/credstore", conn);
-                logger.info("New Database created for Credential Store !!! ");
-            } else {
-                logger.info("Database already created for Credential Store !!!");
-            }
-        } catch (Exception e) {
-            logger.error(e.getMessage(), e);
-            throw new RuntimeException("Database failure", e);
-        } finally {
-            db.closeConnection(conn);
-            try {
-                if(conn != null){
-                    if (!conn.getAutoCommit()) {
-                        conn.commit();
-                    }
-                    conn.close();
-                }
-            } catch (SQLException e) {
-                logger.error("Error while closing database connection...", e.getMessage(), e);
-            }
-        }
-//        System.setProperty("appcatalog.initialize.state", "1");
-    }
-
-    public static String getDBType(String jdbcUrl){
-        try{
-            String cleanURI = jdbcUrl.substring(5);
-            URI uri = URI.create(cleanURI);
-            return uri.getScheme();
-        } catch (Exception e) {
-            logger.error(e.getMessage(), e);
-            return null;
-        }
-    }
-
-    public static boolean isDerbyStartEnabled(){
-        try {
-            String s = ServerSettings.getSetting(START_DERBY_ENABLE);
-            if("true".equals(s)){
-                return true;
-            }
-        }  catch (ApplicationSettingsException e) {
-            logger.error("Unable to read airavata server properties", e.getMessage(), e);
-            return false;
-        }
-        return false;
-    }
-
-    public static void startDerbyInServerMode() {
-        try {
-            System.setProperty(DERBY_SERVER_MODE_SYS_PROPERTY, "true");
-            server = new NetworkServerControl(InetAddress.getByName("0.0.0.0"),
-                    getPort(jdbcURl),
-                    jdbcUser, jdbcPassword);
-            java.io.PrintWriter consoleWriter = new java.io.PrintWriter(System.out, true);
-            server.start(consoleWriter);
-        } catch (IOException e) {
-            logger.error("Unable to start Apache derby in the server mode! Check whether " +
-                    "specified port is available");
-        } catch (Exception e) {
-            logger.error("Unable to start Apache derby in the server mode! Check whether " +
-                    "specified port is available");
-        }
-    }
-    
-    public static void stopDerbyInServerMode() {
-            System.setProperty(DERBY_SERVER_MODE_SYS_PROPERTY, "false");
-            if (server!=null){
-            	try {
-					server.shutdown();
-				} catch (Exception e) {
-		            logger.error("Error when stopping the derby server : "+e.getLocalizedMessage());
-				}
-            }
-    }
-
-    public static int getPort(String jdbcURL){
-        try{
-            String cleanURI = jdbcURL.substring(5);
-            URI uri = URI.create(cleanURI);
-            return uri.getPort();
-        } catch (Exception e) {
-            logger.error(e.getMessage(), e);
-            return -1;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/airavata/blob/1eb3b415/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/DataModelUtils.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/DataModelUtils.java b/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/DataModelUtils.java
deleted file mode 100644
index d3d410f..0000000
--- a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/DataModelUtils.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- *
- * 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.airavata.api.server.util;
-
-import org.apache.airavata.registry.core.experiment.catalog.impl.RegistryFactory;
-import org.apache.airavata.registry.cpi.AppCatalogException;
-import org.apache.airavata.registry.cpi.ApplicationInterface;
-import org.apache.airavata.common.utils.ServerSettings;
-import org.apache.airavata.model.util.ExecutionType;
-import org.apache.airavata.model.experiment.ExperimentModel;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.List;
-
-public class DataModelUtils {
-    private static final Logger logger = LoggerFactory.getLogger(DataModelUtils.class);
-	public static ExecutionType getExecutionType(ExperimentModel experiment) throws Exception{
-		try {
-			ApplicationInterface applicationInterface = RegistryFactory.getAppCatalog().getApplicationInterface();
-			List<String> allApplicationInterfaceIds = applicationInterface.getAllApplicationInterfaceIds();
-			String applicationId = experiment.getExecutionId();
-			if (allApplicationInterfaceIds.contains(applicationId)){
-				return ExecutionType.SINGLE_APP;
-			} else {
-				List<String> allWorkflows = RegistryFactory.getAppCatalog().getWorkflowCatalog().getAllWorkflows(ServerSettings.getDefaultUserGateway());
-				if (allWorkflows.contains(applicationId)){
-					return ExecutionType.WORKFLOW;
-				}
-			}
-		} catch (AppCatalogException e) {
-            logger.error("Error while retrieving execution type for experiment : " + experiment.getExperimentId(), e);
-            throw new Exception("Error while retrieving execution type for experiment : " + experiment.getExperimentId(), e);
-		}
-		return ExecutionType.UNKNOWN;
-	}
-}

http://git-wip-us.apache.org/repos/asf/airavata/blob/1eb3b415/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/DatabaseCreator.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/DatabaseCreator.java b/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/DatabaseCreator.java
deleted file mode 100644
index f52d5d8..0000000
--- a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/DatabaseCreator.java
+++ /dev/null
@@ -1,353 +0,0 @@
-/*
- *
- * 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.airavata.api.server.util;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.sql.*;
-import java.util.StringTokenizer;
-
-/**
- * This class creates the database tables required for airavata with default configuration this
- * class creates derby database in server mode. User can specify required database in appropriate
- * properties files.
- */
-public class DatabaseCreator {
-    private final static Logger logger = LoggerFactory.getLogger(DatabaseCreator.class);
-
-    public enum DatabaseType {
-        derby("(?i).*derby.*"), mysql("(?i).*mysql.*"), other("");
-
-        private String pattern;
-
-        private DatabaseType(String matchingPattern) {
-            this.pattern = matchingPattern;
-        }
-
-        public String getMatchingPattern() {
-            return this.pattern;
-        }
-    }
-
-    private static DatabaseType[] supportedDatabase = new DatabaseType[] { DatabaseType.derby, DatabaseType.mysql };
-
-    private static Logger log = LoggerFactory.getLogger(DatabaseCreator.class);
-    private static final String delimiter = ";";
-
-    /**
-     * Creates database
-     *
-     * @throws Exception
-     */
-    public static void createRegistryDatabase(String prefix, Connection conn) throws Exception {
-        createDatabase(prefix, conn);
-    }
-
-
-
-    /**
-     * Checks whether database tables are created by using select * on given table name
-     *
-     * @param tableName
-     *            Table which should be existed
-     * @return <code>true</core> if checkSQL is success, else <code>false</code> .
-     */
-    public static boolean isDatabaseStructureCreated(String tableName, Connection conn) {
-        try {
-
-            log.debug("Running a query to test the database tables existence.");
-
-            // check whether the tables are already created with a query
-            Statement statement = null;
-            try {
-                statement = conn.createStatement();
-                ResultSet rs = statement.executeQuery("select * from " + tableName);
-                if (rs != null) {
-                    rs.close();
-                }
-            } finally {
-                try {
-                    if (statement != null) {
-                        statement.close();
-                    }
-                } catch (SQLException e) {
-                    return false;
-                }
-            }
-        } catch (SQLException e) {
-            return false;
-        }
-
-        return true;
-    }
-
-    /**
-     * executes given sql
-     *
-     * @param sql
-     * @throws Exception
-     */
-    private static void executeSQL(String sql, Connection conn) throws Exception {
-        // Check and ignore empty statements
-        if ("".equals(sql.trim())) {
-            return;
-        }
-
-        Statement statement = null;
-        try {
-            log.debug("SQL : " + sql);
-
-            boolean ret;
-            int updateCount = 0, updateCountTotal = 0;
-            statement = conn.createStatement();
-            ret = statement.execute(sql);
-            updateCount = statement.getUpdateCount();
-            do {
-                if (!ret) {
-                    if (updateCount != -1) {
-                        updateCountTotal += updateCount;
-                    }
-                }
-                ret = statement.getMoreResults();
-                if (ret) {
-                    updateCount = statement.getUpdateCount();
-                }
-            } while (ret);
-
-            log.debug(sql + " : " + updateCountTotal + " rows affected");
-
-            SQLWarning warning = conn.getWarnings();
-            while (warning != null) {
-                log.info(warning + " sql warning");
-                warning = warning.getNextWarning();
-            }
-            conn.clearWarnings();
-        } catch (SQLException e) {
-            if (e.getSQLState().equals("X0Y32")) {
-                // eliminating the table already exception for the derby
-                // database
-                log.info("Table Already Exists", e);
-            } else {
-                throw new Exception("Error occurred while executing : " + sql, e);
-            }
-        } finally {
-            if (statement != null) {
-                try {
-                    statement.close();
-                } catch (SQLException e) {
-                    log.error("Error occurred while closing result set.", e);
-                }
-            }
-        }
-    }
-
-    /**
-     * computes relatational database type using database name
-     *
-     * @return DatabaseType
-     * @throws Exception
-     *
-     */
-    public static DatabaseType getDatabaseType(Connection conn) throws Exception {
-        try {
-            if (conn != null && (!conn.isClosed())) {
-                DatabaseMetaData metaData = conn.getMetaData();
-                String databaseProductName = metaData.getDatabaseProductName();
-                return checkType(databaseProductName);
-            }
-        } catch (SQLException e) {
-            String msg = "Failed to create Airavata database." + e.getMessage();
-            log.error(msg, e);
-            throw new Exception(msg, e);
-        }
-        return DatabaseType.other;
-    }
-
-    /**
-     * Overloaded method with String input
-     *
-     * @return DatabaseType
-     * @throws Exception
-     *
-     */
-    public static DatabaseType getDatabaseType(String dbUrl) throws Exception {
-        return checkType(dbUrl);
-    }
-
-    private static DatabaseType checkType(String text) throws Exception {
-        try {
-            if (text != null) {
-                for (DatabaseType type : supportedDatabase) {
-                    if (text.matches(type.getMatchingPattern()))
-                        return type;
-                }
-            }
-            String msg = "Unsupported database: " + text
-                    + ". Database will not be created automatically by the Airavata. "
-                    + "Please create the database using appropriate database scripts for " + "the database.";
-            throw new Exception(msg);
-
-        } catch (SQLException e) {
-            String msg = "Failed to create Airavatadatabase." + e.getMessage();
-            log.error(msg, e);
-            throw new Exception(msg, e);
-        }
-    }
-
-    /**
-     * Get scripts location which is prefix + "-" + databaseType + ".sql"
-     *
-     * @param prefix
-     * @param databaseType
-     * @return script location
-     */
-    private static String getScriptLocation(String prefix, DatabaseType databaseType) {
-        String scriptName = prefix + "-" + databaseType + ".sql";
-        log.debug("Loading database script from :" + scriptName);
-        return  scriptName;
-    }
-
-    private static void createDatabase(String prefix, Connection conn) throws Exception {
-        Statement statement = null;
-        try {
-            conn.setAutoCommit(false);
-            statement = conn.createStatement();
-            executeSQLScript(getScriptLocation(prefix, DatabaseCreator.getDatabaseType(conn)), conn);
-            conn.commit();
-            log.debug("Tables are created successfully.");
-        } catch (SQLException e) {
-            String msg = "Failed to create database tables for Airavata resource store. " + e.getMessage();
-            log.error(msg, e);
-            conn.rollback();
-            throw new Exception(msg, e);
-        } finally {
-            conn.setAutoCommit(true);
-            try {
-                if (statement != null) {
-                    statement.close();
-                }
-            } catch (SQLException e) {
-                log.error("Failed to close statement.", e);
-            }
-        }
-    }
-
-    private static void executeSQLScript(String dbscriptName, Connection conn) throws Exception {
-        StringBuffer sql = new StringBuffer();
-        BufferedReader reader = null;
-
-        try {
-            InputStream is = DatabaseCreator.class.getClassLoader().getResourceAsStream(dbscriptName);
-            if(is == null) {
-                logger.info("Script file not found at " + dbscriptName + ". Uses default database script file");
-                DatabaseType databaseType = DatabaseCreator.getDatabaseType(conn);
-                if(databaseType.equals(DatabaseType.derby)){
-                    is = DatabaseCreator.class.getClassLoader().getResourceAsStream("experiment-derby.sql");
-                }else if(databaseType.equals(DatabaseType.mysql)){
-                    is = DatabaseCreator.class.getClassLoader().getResourceAsStream("experiment-mysql.sql");
-                }
-            }
-            reader = new BufferedReader(new InputStreamReader(is));
-            String line;
-            while ((line = reader.readLine()) != null) {
-                line = line.trim();
-                if (line.startsWith("//")) {
-                    continue;
-                }
-                if (line.startsWith("--")) {
-                    continue;
-                }
-                StringTokenizer st = new StringTokenizer(line);
-                if (st.hasMoreTokens()) {
-                    String token = st.nextToken();
-                    if ("REM".equalsIgnoreCase(token)) {
-                        continue;
-                    }
-                }
-                sql.append(" ").append(line);
-
-                // SQL defines "--" as a comment to EOL
-                // and in Oracle it may contain a hint
-                // so we cannot just remove it, instead we must end it
-                if (line.indexOf("--") >= 0) {
-                    sql.append("\n");
-                }
-                if ((checkStringBufferEndsWith(sql, delimiter))) {
-                    executeSQL(sql.substring(0, sql.length() - delimiter.length()), conn);
-                    sql.replace(0, sql.length(), "");
-                }
-            }
-            // Catch any statements not followed by ;
-            if (sql.length() > 0) {
-                executeSQL(sql.toString(), conn);
-            }
-        } catch (IOException e) {
-            log.error("Error occurred while executing SQL script for creating Airavata database", e);
-            throw new Exception("Error occurred while executing SQL script for creating Airavata database", e);
-
-        } finally {
-            if (reader != null) {
-                reader.close();
-            }
-        }
-    }
-
-    /**
-     * Checks that a string buffer ends up with a given string. It may sound trivial with the existing JDK API but the
-     * various implementation among JDKs can make those methods extremely resource intensive and perform poorly due to
-     * massive memory allocation and copying. See
-     *
-     * @param buffer
-     *            the buffer to perform the check on
-     * @param suffix
-     *            the suffix
-     * @return <code>true</code> if the character sequence represented by the argument is a suffix of the character
-     *         sequence represented by the StringBuffer object; <code>false</code> otherwise. Note that the result will
-     *         be <code>true</code> if the argument is the empty string.
-     */
-    public static boolean checkStringBufferEndsWith(StringBuffer buffer, String suffix) {
-        if (suffix.length() > buffer.length()) {
-            return false;
-        }
-        // this loop is done on purpose to avoid memory allocation performance
-        // problems on various JDKs
-        // StringBuffer.lastIndexOf() was introduced in jdk 1.4 and
-        // implementation is ok though does allocation/copying
-        // StringBuffer.toString().endsWith() does massive memory
-        // allocation/copying on JDK 1.5
-        // See http://issues.apache.org/bugzilla/show_bug.cgi?id=37169
-        int endIndex = suffix.length() - 1;
-        int bufferIndex = buffer.length() - 1;
-        while (endIndex >= 0) {
-            if (buffer.charAt(bufferIndex) != suffix.charAt(endIndex)) {
-                return false;
-            }
-            bufferIndex--;
-            endIndex--;
-        }
-        return true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/airavata/blob/1eb3b415/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/ExperimentCatalogInitUtil.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/ExperimentCatalogInitUtil.java b/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/ExperimentCatalogInitUtil.java
deleted file mode 100644
index 5e68c40..0000000
--- a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/ExperimentCatalogInitUtil.java
+++ /dev/null
@@ -1,203 +0,0 @@
-/*
-*
-* 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.airavata.api.server.util;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.URI;
-import java.sql.Connection;
-import java.sql.SQLException;
-
-import org.apache.airavata.common.exception.ApplicationSettingsException;
-import org.apache.airavata.common.utils.ServerSettings;
-import org.apache.airavata.registry.core.experiment.catalog.ExpCatResourceUtils;
-import org.apache.airavata.registry.core.experiment.catalog.ResourceType;
-import org.apache.airavata.registry.core.experiment.catalog.resources.GatewayResource;
-import org.apache.airavata.registry.core.experiment.catalog.resources.ProjectResource;
-import org.apache.airavata.registry.core.experiment.catalog.resources.UserResource;
-import org.apache.airavata.registry.core.experiment.catalog.resources.WorkerResource;
-import org.apache.derby.drda.NetworkServerControl;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class ExperimentCatalogInitUtil {
-    public static final String CONFIGURATION_TABLE = "CONFIGURATION";
-    private static final Logger logger = LoggerFactory.getLogger(ExperimentCatalogInitUtil.class);
-    public static final String REGISTRY_JDBC_DRIVER = "registry.jdbc.driver";
-    public static final String REGISTRY_JDBC_URL = "registry.jdbc.url";
-    public static final String REGISTRY_JDBC_USER = "registry.jdbc.user";
-    public static final String REGISTRY_JDBC_PASSWORD = "registry.jdbc.password";
-    public static final String START_DERBY_ENABLE = "start.derby.server.mode";
-    public static final String DERBY_SERVER_MODE_SYS_PROPERTY = "derby.drda.startNetworkServer";
-    public static final String DEFAULT_PROJECT_NAME = "default";
-    private static NetworkServerControl server;
-    private static JdbcStorage db;
-    private static String jdbcURl;
-    private static String jdbcDriver;
-    private static String jdbcUser;
-    private static String jdbcPassword;
-
-
-    public static void initializeDB() {
-        System.setProperty("registry.initialize.state", "0");
-        try{
-            jdbcDriver = ServerSettings.getSetting(REGISTRY_JDBC_DRIVER);
-            jdbcURl = ServerSettings.getSetting(REGISTRY_JDBC_URL);
-            jdbcUser = ServerSettings.getSetting(REGISTRY_JDBC_USER);
-            jdbcPassword = ServerSettings.getSetting(REGISTRY_JDBC_PASSWORD);
-            jdbcURl = jdbcURl + "?" + "user=" + jdbcUser + "&" + "password=" + jdbcPassword;
-        } catch (ApplicationSettingsException e) {
-            logger.error("Unable to read airavata server properties", e.getMessage());
-        }
-
-        if (getDBType(jdbcURl).equals("derby") && isDerbyStartEnabled()) {
-            startDerbyInServerMode();
-        }
-        db = new JdbcStorage(10, 50, jdbcURl, jdbcDriver, true);
-
-        Connection conn = null;
-        try {
-            conn = db.connect();
-            if (!DatabaseCreator.isDatabaseStructureCreated(CONFIGURATION_TABLE, conn)) {
-                DatabaseCreator.createRegistryDatabase("database_scripts/expcatalog", conn);
-                logger.info("New Database created for Experiment Catalog !!!");
-            } else {
-                logger.info("Database already created for Experiment Catalog !!!");
-            }
-            try{
-                GatewayResource gateway;
-                if (!ExpCatResourceUtils.isGatewayExist(ServerSettings.getDefaultUserGateway())){
-                    gateway = (GatewayResource)ExpCatResourceUtils.createGateway(ServerSettings.getDefaultUserGateway());
-                    gateway.save();
-                }else {
-                    gateway = (GatewayResource)ExpCatResourceUtils.getGateway(ServerSettings.getDefaultUserGateway());
-                }
-
-                UserResource user;
-                if (!ExpCatResourceUtils.isUserExist(ServerSettings.getDefaultUser(), ServerSettings.getDefaultUserGateway())){
-                    user = ExpCatResourceUtils.createUser(ServerSettings.getDefaultUser(), ServerSettings.getDefaultUserPassword(), ServerSettings.getDefaultUserGateway());
-                    user.save();
-                }else {
-                    user = (UserResource)ExpCatResourceUtils.getUser(ServerSettings.getDefaultUser(), ServerSettings.getDefaultUserGateway());
-                }
-
-                WorkerResource workerResource;
-                if (!gateway.isExists(ResourceType.GATEWAY_WORKER, ServerSettings.getDefaultUserGateway())){
-                    workerResource = (WorkerResource)gateway.create(ResourceType.GATEWAY_WORKER);
-                    workerResource.setUser(user.getUserName());
-                    workerResource.save();
-                }else {
-                    workerResource =  (WorkerResource)gateway.get(ResourceType.GATEWAY_WORKER, ServerSettings.getDefaultUser());
-                }
-                ProjectResource projectResource;
-                if (!workerResource.isExists(ResourceType.PROJECT, DEFAULT_PROJECT_NAME)){
-                    projectResource = workerResource.createProject(DEFAULT_PROJECT_NAME);
-                    projectResource.setName(DEFAULT_PROJECT_NAME);
-                    projectResource.setGatewayId(gateway.getGatewayId());
-                    projectResource.save();
-                }
-
-            } catch (ApplicationSettingsException e) {
-                logger.error("Unable to read airavata-server properties...", e.getMessage());
-            }
-        } catch (Exception e) {
-            logger.error(e.getMessage(), e);
-            throw new RuntimeException("Database failure", e);
-        } finally {
-            db.closeConnection(conn);
-            try {
-                if(conn != null){
-                    if (!conn.getAutoCommit()) {
-                        conn.commit();
-                    }
-                    conn.close();
-                }
-            } catch (SQLException e) {
-                logger.error("Error while closing database connection...", e.getMessage(), e);
-            }
-        }
-        System.setProperty("registry.initialize.state", "1");
-    }
-
-    public static String getDBType(String jdbcUrl){
-        try{
-            String cleanURI = jdbcUrl.substring(5);
-            URI uri = URI.create(cleanURI);
-            return uri.getScheme();
-        } catch (Exception e) {
-            logger.error(e.getMessage(), e);
-            return null;
-        }
-    }
-
-    public static boolean isDerbyStartEnabled(){
-        try {
-            String s = ServerSettings.getSetting(START_DERBY_ENABLE);
-            if("true".equals(s)){
-                return true;
-            }
-        }  catch (ApplicationSettingsException e) {
-            logger.error("Unable to read airavata server properties", e.getMessage(), e);
-            return false;
-        }
-        return false;
-    }
-
-    public static void startDerbyInServerMode() {
-        try {
-            System.setProperty(DERBY_SERVER_MODE_SYS_PROPERTY, "true");
-            server = new NetworkServerControl(InetAddress.getByName("0.0.0.0"),
-                    getPort(jdbcURl),
-                    jdbcUser, jdbcPassword);
-            java.io.PrintWriter consoleWriter = new java.io.PrintWriter(System.out, true);
-            server.start(consoleWriter);
-        } catch (IOException e) {
-            logger.error("Unable to start Apache derby in the server mode! Check whether " +
-                    "specified port is available");
-        } catch (Exception e) {
-            logger.error("Unable to start Apache derby in the server mode! Check whether " +
-                    "specified port is available");
-        }
-    }
-    
-    public static void stopDerbyInServerMode() {
-            System.setProperty(DERBY_SERVER_MODE_SYS_PROPERTY, "false");
-            if (server!=null){
-            	try {
-					server.shutdown();
-				} catch (Exception e) {
-		            logger.error("Error when stopping the derby server : "+e.getLocalizedMessage());
-				}
-            }
-    }
-
-    public static int getPort(String jdbcURL){
-        try{
-            String cleanURI = jdbcURL.substring(5);
-            URI uri = URI.create(cleanURI);
-            return uri.getPort();
-        } catch (Exception e) {
-            logger.error(e.getMessage(), e);
-            return -1;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/airavata/blob/1eb3b415/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/JdbcStorage.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/JdbcStorage.java b/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/JdbcStorage.java
deleted file mode 100644
index 37320a3..0000000
--- a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/JdbcStorage.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/*
- *
- * 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.airavata.api.server.util;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.sql.*;
-
-public class JdbcStorage {
-    private static Logger log = LoggerFactory.getLogger(JdbcStorage.class);
-
-    private ConnectionPool connectionPool;
-
-    public JdbcStorage(String jdbcUrl, String jdbcDriver) {
-        // default init connection and max connection
-        this(3, 50, jdbcUrl, jdbcDriver, true);
-    }
-
-    public JdbcStorage(int initCon, int maxCon, String url, String driver, boolean enableTransactions) {
-        try {
-            if (enableTransactions) {
-                connectionPool = new ConnectionPool(driver, url, initCon, maxCon, true, false,
-                        Connection.TRANSACTION_SERIALIZABLE);
-            } else {
-                connectionPool = new ConnectionPool(driver, url, initCon, maxCon, true);
-            }
-        } catch (Exception e) {
-            throw new RuntimeException("Failed to create database connection pool.", e);
-        }
-    }
-
-    /**
-     * Check if this connection pool is auto commit or not
-     *
-     * @return
-     */
-    public boolean isAutoCommit() {
-        return connectionPool.isAutoCommit();
-    }
-
-    public void commit(Connection conn) {
-        try {
-            if (conn != null && !conn.getAutoCommit()) {
-                conn.commit();
-            }
-        } catch (SQLException sqle) {
-            log.error("Cannot commit data", sqle);
-        }
-    }
-
-    public void commitAndFree(Connection conn) {
-        commit(conn);
-        closeConnection(conn);
-    }
-
-    public void rollback(Connection conn) {
-        try {
-            if (conn != null && !conn.getAutoCommit()) {
-                conn.rollback();
-            }
-        } catch (SQLException sqle) {
-            log.error("Cannot Rollback data", sqle);
-        }
-    }
-
-    public void rollbackAndFree(Connection conn) {
-        rollback(conn);
-        closeConnection(conn);
-    }
-
-    public Connection connect() {
-
-        Connection conn = null;
-        try {
-            conn = connectionPool.getConnection();
-        } catch (SQLException e) {
-            log.error(e.getMessage(), e);
-        }
-        return conn;
-    }
-
-    /**
-     * This method is provided so that you can have better control over the statement. For example: You can use
-     * stmt.setString to convert quotation mark automatically in an UPDATE statement
-     *
-     * NOTE: Statement is closed after execution
-     */
-    public int executeUpdateAndClose(PreparedStatement stmt) throws SQLException {
-        int rows = 0;
-        try {
-            rows = stmt.executeUpdate();
-            if (rows == 0) {
-                log.info("Problem: 0 rows affected by insert/update/delete statement.");
-            }
-        } finally {
-            stmt.close();
-        }
-        return rows;
-    }
-
-    public int countRow(String tableName, String columnName) throws SQLException {
-        String query = new String("SELECT COUNT(" + columnName + ") FROM " + tableName);
-        int count = -1;
-        Connection conn = null;
-        PreparedStatement stmt = null;
-        try {
-            conn = connectionPool.getConnection();
-            stmt = conn.prepareStatement(query);
-            ResultSet rs = stmt.executeQuery();
-            rs.next();
-            count = rs.getInt(1);
-            commit(conn);
-        } catch (SQLException sql) {
-            rollback(conn);
-            throw sql;
-        } finally {
-            try {
-                if (stmt != null && !stmt.isClosed()) {
-                    stmt.close();
-                }
-            } finally {
-                closeConnection(conn);
-            }
-        }
-        return count;
-    }
-
-    public void quietlyClose(Connection conn, Statement... stmts) {
-        if (stmts != null) {
-            for (Statement stmt : stmts) {
-                try {
-                    if (stmt != null && !stmt.isClosed()) {
-                        stmt.close();
-                    }
-                } catch (SQLException sql) {
-                    log.error(sql.getMessage(), sql);
-                }
-            }
-        }
-        closeConnection(conn);
-    }
-
-    public void closeConnection(Connection conn) {
-        if (conn != null) {
-            connectionPool.free(conn);
-        }
-    }
-
-    public void closeAllConnections() {
-        if (connectionPool != null)
-            connectionPool.dispose();
-    }
-
-    public void shutdown() throws SQLException {
-        connectionPool.shutdown();
-    }
-}

http://git-wip-us.apache.org/repos/asf/airavata/blob/1eb3b415/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/OrchestratorInitUtil.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/OrchestratorInitUtil.java b/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/OrchestratorInitUtil.java
deleted file mode 100644
index 5add438..0000000
--- a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/OrchestratorInitUtil.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/*
- *
- * 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.airavata.api.server.util;
-
-public class OrchestratorInitUtil {
-}


[03/50] [abbrv] airavata git commit: Removed PublisherFactory class and activity publisher properties from airavata server properties

Posted by la...@apache.org.
Removed PublisherFactory class and activity publisher properties from airavata server properties


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: 65c2a9672e8d539fcc62b6d0156007d41decc0a0
Parents: e4cc54d
Author: Shameera Rathnayaka <sh...@gmail.com>
Authored: Wed Aug 10 15:41:39 2016 -0400
Committer: Shameera Rathnayaka <sh...@gmail.com>
Committed: Wed Aug 10 15:41:39 2016 -0400

----------------------------------------------------------------------
 .../airavata/common/utils/ServerSettings.java   |  10 --
 .../main/resources/airavata-server.properties   |   1 -
 .../messaging/core/PublisherFactory.java        | 138 +++++++++----------
 .../core/impl/GFACPassiveJobSubmitter.java      |   9 +-
 .../server/OrchestratorServerHandler.java       |   3 +-
 5 files changed, 76 insertions(+), 85 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/65c2a967/modules/commons/src/main/java/org/apache/airavata/common/utils/ServerSettings.java
----------------------------------------------------------------------
diff --git a/modules/commons/src/main/java/org/apache/airavata/common/utils/ServerSettings.java b/modules/commons/src/main/java/org/apache/airavata/common/utils/ServerSettings.java
index 2459658..8d52a3d 100644
--- a/modules/commons/src/main/java/org/apache/airavata/common/utils/ServerSettings.java
+++ b/modules/commons/src/main/java/org/apache/airavata/common/utils/ServerSettings.java
@@ -79,8 +79,6 @@ public class ServerSettings extends ApplicationSettings {
     private static final String MY_PROXY_USER = "myproxy.user";
     private static final String MY_PROXY_PASSWORD = "myproxy.password";
     private static final String MY_PROXY_LIFETIME = "myproxy.life";
-    private static final String STATUS_PUBLISHER = "status.publisher";
-    private static final String TASK_LAUNCH_PUBLISHER = "task.launch.publisher";
     public static final String JOB_NOTIFICATION_ENABLE = "job.notification.enable";
     public static final String JOB_NOTIFICATION_EMAILIDS = "job.notification.emailids";
     public static final String JOB_NOTIFICATION_FLAGS = "job.notification.flags";
@@ -234,14 +232,6 @@ public class ServerSettings extends ApplicationSettings {
         return Integer.parseInt(getSetting(MY_PROXY_LIFETIME));
     }
 
-    public static String getStatusPublisher() throws ApplicationSettingsException {
-        return getSetting(STATUS_PUBLISHER);
-    }
-
-    public static String getTaskLaunchPublisher() throws ApplicationSettingsException {
-        return getSetting(TASK_LAUNCH_PUBLISHER);
-    }
-
     public static boolean isEmbeddedZK() {
         return Boolean.parseBoolean(getSetting(EMBEDDED_ZK, "true"));
     }

http://git-wip-us.apache.org/repos/asf/airavata/blob/65c2a967/modules/configuration/server/src/main/resources/airavata-server.properties
----------------------------------------------------------------------
diff --git a/modules/configuration/server/src/main/resources/airavata-server.properties b/modules/configuration/server/src/main/resources/airavata-server.properties
index 29b256f..0dad713 100644
--- a/modules/configuration/server/src/main/resources/airavata-server.properties
+++ b/modules/configuration/server/src/main/resources/airavata-server.properties
@@ -253,7 +253,6 @@ durable.queue=false
 prefetch.count=200
 process.launch.queue.name=process.launch.queue
 experiment.launch.queue.name=experiment.launch.queue
-activity.publisher=org.apache.airavata.messaging.core.impl.RabbitMQStatusPublisher
 
 ###########################################################################
 # Zookeeper Server Configuration

http://git-wip-us.apache.org/repos/asf/airavata/blob/65c2a967/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/PublisherFactory.java
----------------------------------------------------------------------
diff --git a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/PublisherFactory.java b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/PublisherFactory.java
index 2e560a3..9cab483 100644
--- a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/PublisherFactory.java
+++ b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/PublisherFactory.java
@@ -1,69 +1,69 @@
-/*
- *
- * 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.airavata.messaging.core;
-
-import org.apache.airavata.common.exception.AiravataException;
-import org.apache.airavata.common.utils.ServerSettings;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class PublisherFactory {
-    private static Logger log = LoggerFactory.getLogger(PublisherFactory.class);
-
-    public static Publisher createActivityPublisher() throws AiravataException {
-        String activityPublisher = ServerSettings.getStatusPublisher();
-
-        if (activityPublisher == null) {
-            String s = "Activity publisher is not specified";
-            log.error(s);
-            throw new AiravataException(s);
-        }
-
-        try {
-            Class<? extends Publisher> aPublisher = Class.forName(activityPublisher).asSubclass(Publisher.class);
-            return aPublisher.newInstance();
-        } catch (Exception e) {
-            String msg = "Failed to load the publisher from the publisher class property: " + activityPublisher;
-            log.error(msg, e);
-            throw new AiravataException(msg, e);
-        }
-    }
-
-    public static Publisher createTaskLaunchPublisher() throws AiravataException {
-        String taskLaunchPublisher = ServerSettings.getTaskLaunchPublisher();
-
-        if (taskLaunchPublisher == null) {
-            String s = "Task launch publisher is not specified";
-            log.error(s);
-            throw new AiravataException(s);
-        }
-
-        try {
-            Class<? extends Publisher> aPublisher = Class.forName(taskLaunchPublisher).asSubclass(Publisher.class);
-            return aPublisher.newInstance();
-        } catch (Exception e) {
-            String msg = "Failed to load the publisher from the publisher class property: " + taskLaunchPublisher;
-            log.error(msg, e);
-            throw new AiravataException(msg, e);
-        }
-    }
-}
+///*
+// *
+// * 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.airavata.messaging.core;
+//
+//import org.apache.airavata.common.exception.AiravataException;
+//import org.apache.airavata.common.utils.ServerSettings;
+//import org.slf4j.Logger;
+//import org.slf4j.LoggerFactory;
+//
+//public class PublisherFactory {
+//    private static Logger log = LoggerFactory.getLogger(PublisherFactory.class);
+//
+//    public static Publisher createActivityPublisher() throws AiravataException {
+//        String activityPublisher = ServerSettings.getStatusPublisher();
+//
+//        if (activityPublisher == null) {
+//            String s = "Activity publisher is not specified";
+//            log.error(s);
+//            throw new AiravataException(s);
+//        }
+//
+//        try {
+//            Class<? extends Publisher> aPublisher = Class.forName(activityPublisher).asSubclass(Publisher.class);
+//            return aPublisher.newInstance();
+//        } catch (Exception e) {
+//            String msg = "Failed to load the publisher from the publisher class property: " + activityPublisher;
+//            log.error(msg, e);
+//            throw new AiravataException(msg, e);
+//        }
+//    }
+//
+//    public static Publisher createTaskLaunchPublisher() throws AiravataException {
+//        String taskLaunchPublisher = ServerSettings.getTaskLaunchPublisher();
+//
+//        if (taskLaunchPublisher == null) {
+//            String s = "Task launch publisher is not specified";
+//            log.error(s);
+//            throw new AiravataException(s);
+//        }
+//
+//        try {
+//            Class<? extends Publisher> aPublisher = Class.forName(taskLaunchPublisher).asSubclass(Publisher.class);
+//            return aPublisher.newInstance();
+//        } catch (Exception e) {
+//            String msg = "Failed to load the publisher from the publisher class property: " + taskLaunchPublisher;
+//            log.error(msg, e);
+//            throw new AiravataException(msg, e);
+//        }
+//    }
+//}

http://git-wip-us.apache.org/repos/asf/airavata/blob/65c2a967/modules/orchestrator/orchestrator-core/src/main/java/org/apache/airavata/orchestrator/core/impl/GFACPassiveJobSubmitter.java
----------------------------------------------------------------------
diff --git a/modules/orchestrator/orchestrator-core/src/main/java/org/apache/airavata/orchestrator/core/impl/GFACPassiveJobSubmitter.java b/modules/orchestrator/orchestrator-core/src/main/java/org/apache/airavata/orchestrator/core/impl/GFACPassiveJobSubmitter.java
index 02ecfe1..856f9f3 100644
--- a/modules/orchestrator/orchestrator-core/src/main/java/org/apache/airavata/orchestrator/core/impl/GFACPassiveJobSubmitter.java
+++ b/modules/orchestrator/orchestrator-core/src/main/java/org/apache/airavata/orchestrator/core/impl/GFACPassiveJobSubmitter.java
@@ -27,9 +27,12 @@ import org.apache.airavata.credential.store.store.CredentialReader;
 import org.apache.airavata.gfac.client.GFACInstance;
 import org.apache.airavata.gfac.core.GFacUtils;
 import org.apache.airavata.messaging.core.MessageContext;
+import org.apache.airavata.messaging.core.MessagingFactory;
 import org.apache.airavata.messaging.core.Publisher;
-import org.apache.airavata.messaging.core.PublisherFactory;
-import org.apache.airavata.model.messaging.event.*;
+import org.apache.airavata.messaging.core.Type;
+import org.apache.airavata.model.messaging.event.MessageType;
+import org.apache.airavata.model.messaging.event.ProcessSubmitEvent;
+import org.apache.airavata.model.messaging.event.ProcessTerminateEvent;
 import org.apache.airavata.orchestrator.core.context.OrchestratorContext;
 import org.apache.airavata.orchestrator.core.exception.OrchestratorException;
 import org.apache.airavata.orchestrator.core.job.JobSubmitter;
@@ -53,7 +56,7 @@ public class GFACPassiveJobSubmitter implements JobSubmitter,Watcher {
             this.publisher = orchestratorContext.getPublisher();
         }else {
             try {
-                this.publisher = PublisherFactory.createTaskLaunchPublisher();
+                this.publisher = MessagingFactory.getPublisher(Type.PROCESS_LAUNCH);
             } catch (AiravataException e) {
                 logger.error(e.getMessage(), e);
                 throw new OrchestratorException("Cannot initialize " + GFACPassiveJobSubmitter.class + " need to start Rabbitmq server to use " + GFACPassiveJobSubmitter.class);

http://git-wip-us.apache.org/repos/asf/airavata/blob/65c2a967/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java
----------------------------------------------------------------------
diff --git a/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java b/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java
index 5d02100..03f6f8a 100644
--- a/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java
+++ b/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java
@@ -33,7 +33,6 @@ import org.apache.airavata.messaging.core.MessageContext;
 import org.apache.airavata.messaging.core.MessageHandler;
 import org.apache.airavata.messaging.core.MessagingFactory;
 import org.apache.airavata.messaging.core.Publisher;
-import org.apache.airavata.messaging.core.PublisherFactory;
 import org.apache.airavata.messaging.core.Subscriber;
 import org.apache.airavata.messaging.core.Type;
 import org.apache.airavata.model.appcatalog.appdeployment.ApplicationDeploymentDescription;
@@ -112,7 +111,7 @@ public class OrchestratorServerHandler implements OrchestratorService.Iface {
 
 	public OrchestratorServerHandler() throws OrchestratorException{
 		try {
-	        publisher = PublisherFactory.createActivityPublisher();
+	        publisher = MessagingFactory.getPublisher(Type.STATUS);
             setAiravataUserName(ServerSettings.getDefaultUser());
 		} catch (AiravataException e) {
             log.error(e.getMessage(), e);


[24/50] [abbrv] airavata git commit: Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/airavata into develop

Posted by la...@apache.org.
Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/airavata into develop


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: cfe62c3e829820d744da4f41e9036c87eccd18dd
Parents: 9f979b5 99d3f4b
Author: Shameera Rathnayaka <sh...@gmail.com>
Authored: Mon Aug 15 15:45:20 2016 -0400
Committer: Shameera Rathnayaka <sh...@gmail.com>
Committed: Mon Aug 15 15:45:20 2016 -0400

----------------------------------------------------------------------
 .../airavata/registry/core/Committer.java       |  29 ++++
 .../apache/airavata/registry/core/JPAUtil.java  |  57 +++++++
 .../catalog/resources/ExperimentResource.java   | 148 ++++++-------------
 3 files changed, 134 insertions(+), 100 deletions(-)
----------------------------------------------------------------------



[20/50] [abbrv] airavata git commit: Fixed NPE with rabbitmq broker url

Posted by la...@apache.org.
Fixed NPE with rabbitmq broker url


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: f4aeee53487d72a1229f24218dfe5f785de09c45
Parents: 1ecff76
Author: Shameera Rathnayaka <sh...@gmail.com>
Authored: Mon Aug 15 12:55:03 2016 -0400
Committer: Shameera Rathnayaka <sh...@gmail.com>
Committed: Mon Aug 15 12:55:03 2016 -0400

----------------------------------------------------------------------
 .../java/org/apache/airavata/common/utils/ServerSettings.java    | 4 ++--
 .../org/apache/airavata/messaging/core/MessagingFactory.java     | 2 +-
 2 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/f4aeee53/modules/commons/src/main/java/org/apache/airavata/common/utils/ServerSettings.java
----------------------------------------------------------------------
diff --git a/modules/commons/src/main/java/org/apache/airavata/common/utils/ServerSettings.java b/modules/commons/src/main/java/org/apache/airavata/common/utils/ServerSettings.java
index 8d52a3d..985daad 100644
--- a/modules/commons/src/main/java/org/apache/airavata/common/utils/ServerSettings.java
+++ b/modules/commons/src/main/java/org/apache/airavata/common/utils/ServerSettings.java
@@ -123,8 +123,8 @@ public class ServerSettings extends ApplicationSettings {
         return getSetting(RABBITMQ_EXPERIMENT_EXCHANGE_NAME, "experiment.launch.queue");
     }
 
-    public static String getRabbitmqBrokerUrl() throws ApplicationSettingsException {
-        return getSetting(RABBITMQ_BROKER_URL);
+    public static String getRabbitmqBrokerUrl() {
+        return getSetting(RABBITMQ_BROKER_URL, "amqp://localhost:5672");
     }
 
     public static String getRabbitmqStatusExchangeName(){

http://git-wip-us.apache.org/repos/asf/airavata/blob/f4aeee53/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessagingFactory.java
----------------------------------------------------------------------
diff --git a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessagingFactory.java b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessagingFactory.java
index 573304a..b3e6d35 100644
--- a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessagingFactory.java
+++ b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessagingFactory.java
@@ -107,7 +107,7 @@ public class MessagingFactory {
 
     private static RabbitMQProperties getProperties() {
         return new RabbitMQProperties()
-                .setBrokerUrl(ServerSettings.RABBITMQ_BROKER_URL)
+                .setBrokerUrl(ServerSettings.getRabbitmqBrokerUrl())
                 .setDurable(ServerSettings.getRabbitmqDurableQueue())
                 .setPrefetchCount(ServerSettings.getRabbitmqPrefetchCount())
                 .setAutoRecoveryEnable(true)


[25/50] [abbrv] airavata git commit: undoing registry java8 revert

Posted by la...@apache.org.
undoing registry java8 revert


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: c6a82a029635a1239ca0563e9019399f90ca714a
Parents: cfe62c3
Author: scnakandala <su...@gmail.com>
Authored: Mon Aug 15 16:14:16 2016 -0400
Committer: scnakandala <su...@gmail.com>
Committed: Mon Aug 15 16:14:16 2016 -0400

----------------------------------------------------------------------
 .../airavata/registry/core/Committer.java       |  29 ----
 .../apache/airavata/registry/core/JPAUtil.java  |  57 -------
 .../catalog/resources/ExperimentResource.java   | 148 +++++++++++++------
 3 files changed, 100 insertions(+), 134 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/c6a82a02/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/Committer.java
----------------------------------------------------------------------
diff --git a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/Committer.java b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/Committer.java
deleted file mode 100644
index 99b504b..0000000
--- a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/Committer.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- *
- * 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.airavata.registry.core;
-
-import org.apache.airavata.registry.cpi.RegistryException;
-
-@FunctionalInterface
-public interface Committer<T, R>  {
-
-   R commit(T t) throws RegistryException;
-}

http://git-wip-us.apache.org/repos/asf/airavata/blob/c6a82a02/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/JPAUtil.java
----------------------------------------------------------------------
diff --git a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/JPAUtil.java b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/JPAUtil.java
deleted file mode 100644
index 58988c6..0000000
--- a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/JPAUtil.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- *
- * 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.airavata.registry.core;
-
-import org.apache.airavata.registry.core.experiment.catalog.ExpCatResourceUtils;
-import org.apache.airavata.registry.cpi.ExperimentCatalogException;
-import org.apache.airavata.registry.cpi.RegistryException;
-
-import javax.persistence.EntityManager;
-import javax.persistence.EntityManagerFactory;
-import javax.persistence.Persistence;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.function.Consumer;
-
-public class JPAUtil {
-
-    public static <R> R executeOnExpCatalog(Committer<EntityManager, R> committer) throws RegistryException, ExperimentCatalogException {
-        EntityManager entityManager = ExpCatResourceUtils.getEntityManager();
-        try {
-            entityManager.getTransaction().begin();
-            R r = committer.commit(entityManager);
-            entityManager.getTransaction().commit();
-            return  r;
-        }finally {
-            if(entityManager != null && entityManager.isOpen()){
-                if (entityManager.getTransaction().isActive()) {
-                    entityManager.getTransaction().rollback();
-                }
-                entityManager.close();
-            }
-        }
-
-    }
-
-    public static void test() throws ExperimentCatalogException, RegistryException {
-        executeOnExpCatalog((em) -> "");
-    }
-}

http://git-wip-us.apache.org/repos/asf/airavata/blob/c6a82a02/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/resources/ExperimentResource.java
----------------------------------------------------------------------
diff --git a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/resources/ExperimentResource.java b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/resources/ExperimentResource.java
index 7889839..174575f 100644
--- a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/resources/ExperimentResource.java
+++ b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/resources/ExperimentResource.java
@@ -22,8 +22,6 @@
 package org.apache.airavata.registry.core.experiment.catalog.resources;
 
 import org.apache.airavata.model.status.ExperimentState;
-import org.apache.airavata.registry.core.Committer;
-import org.apache.airavata.registry.core.JPAUtil;
 import org.apache.airavata.registry.core.experiment.catalog.ExpCatResourceUtils;
 import org.apache.airavata.registry.core.experiment.catalog.ExperimentCatResource;
 import org.apache.airavata.registry.core.experiment.catalog.ResourceType;
@@ -31,7 +29,6 @@ import org.apache.airavata.registry.core.experiment.catalog.model.*;
 import org.apache.airavata.registry.core.experiment.catalog.model.Process;
 import org.apache.airavata.registry.core.experiment.catalog.utils.QueryGenerator;
 import org.apache.airavata.registry.cpi.RegistryException;
-import org.apache.openjpa.persistence.Generator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -267,65 +264,120 @@ public class ExperimentResource extends AbstractExpCatResource {
 
     
     public ExperimentCatResource get(ResourceType type, Object name) throws RegistryException{
+        EntityManager em = null;
         try {
+            em = ExpCatResourceUtils.getEntityManager();
+            em.getTransaction().begin();
+            QueryGenerator generator;
+            Query q;
             switch (type) {
                 case EXPERIMENT_STATUS:
-                    return JPAUtil.executeOnExpCatalog(entityManager -> {
-                        QueryGenerator generator = new QueryGenerator(EXPERIMENT_STATUS);
-                        generator.setParameter(ExperimentStatusConstants.STATUS_ID, name);
-                        Query q = generator.selectQuery(entityManager);
-                        ExperimentStatus status = (ExperimentStatus) q.getSingleResult();
-                        return (ExperimentStatusResource) Utils.getResource(ResourceType.EXPERIMENT_STATUS, status);
-                    });
+                    generator = new QueryGenerator(EXPERIMENT_STATUS);
+                    generator.setParameter(ExperimentStatusConstants.STATUS_ID, name);
+                    q = generator.selectQuery(em);
+                    ExperimentStatus status = (ExperimentStatus) q.getSingleResult();
+                    ExperimentStatusResource statusResource = (ExperimentStatusResource) Utils.getResource(ResourceType.EXPERIMENT_STATUS, status);
+                    em.getTransaction().commit();
+                    if (em.isOpen()) {
+                        if (em.getTransaction().isActive()){
+                            em.getTransaction().rollback();
+                        }
+                        em.close();
+                    }
+                    return statusResource;
                 case EXPERIMENT_ERROR:
-                    return JPAUtil.executeOnExpCatalog(entityManager -> {
-                        QueryGenerator generator = new QueryGenerator(EXPERIMENT_ERROR);
-                        generator.setParameter(ExperimentErrorConstants.ERROR_ID, name);
-                        Query q = generator.selectQuery(entityManager);
-                        ExperimentError experimentError = (ExperimentError) q.getSingleResult();
-                        return (ExperimentErrorResource) Utils.getResource(ResourceType.EXPERIMENT_ERROR, experimentError);
-                    });
+                    generator = new QueryGenerator(EXPERIMENT_ERROR);
+                    generator.setParameter(ExperimentErrorConstants.ERROR_ID, name);
+                    q = generator.selectQuery(em);
+                    ExperimentError experimentError = (ExperimentError) q.getSingleResult();
+                    ExperimentErrorResource processErrorResource = (ExperimentErrorResource) Utils.getResource(ResourceType.EXPERIMENT_ERROR, experimentError);
+                    em.getTransaction().commit();
+                    if (em.isOpen()) {
+                        if (em.getTransaction().isActive()){
+                            em.getTransaction().rollback();
+                        }
+                        em.close();
+                    }
+                    return processErrorResource;
                 case EXPERIMENT_INPUT:
-                    return JPAUtil.executeOnExpCatalog(entityManager -> {
-                        QueryGenerator generator = new QueryGenerator(EXPERIMENT_INPUT);
-                        generator.setParameter(ExperimentInputConstants.INPUT_NAME, name);
-                        generator.setParameter(ExperimentInputConstants.EXPERIMENT_ID, experimentId);
-                        Query q = generator.selectQuery(entityManager);
-                        ExperimentInput experimentInput = (ExperimentInput) q.getSingleResult();
-                        return (ExperimentInputResource) Utils.getResource(ResourceType.EXPERIMENT_INPUT, experimentInput);
-                    });
+                    generator = new QueryGenerator(EXPERIMENT_INPUT);
+                    generator.setParameter(ExperimentInputConstants.INPUT_NAME, name);
+                    generator.setParameter(ExperimentInputConstants.EXPERIMENT_ID, experimentId);
+                    q = generator.selectQuery(em);
+                    ExperimentInput experimentInput = (ExperimentInput) q.getSingleResult();
+                    ExperimentInputResource experimentInputResource = (ExperimentInputResource) Utils.getResource(ResourceType.EXPERIMENT_INPUT, experimentInput);
+                    em.getTransaction().commit();
+                    if (em.isOpen()) {
+                        if (em.getTransaction().isActive()){
+                            em.getTransaction().rollback();
+                        }
+                        em.close();
+                    }
+                    return experimentInputResource;
                 case EXPERIMENT_OUTPUT:
-                    return JPAUtil.executeOnExpCatalog(entityManager -> {
-                        QueryGenerator generator = new QueryGenerator(EXPERIMENT_OUTPUT);
-                        generator.setParameter(ExperimentOutputConstants.OUTPUT_NAME, name);
-                        generator.setParameter(ExperimentInputConstants.EXPERIMENT_ID, experimentId);
-                        Query q = generator.selectQuery(entityManager);
-                        ExperimentOutput experimentOutput = (ExperimentOutput) q.getSingleResult();
-                        return (ExperimentOutputResource) Utils.getResource(ResourceType.EXPERIMENT_OUTPUT, experimentOutput);
-                    });
+                    generator = new QueryGenerator(EXPERIMENT_OUTPUT);
+                    generator.setParameter(ExperimentOutputConstants.OUTPUT_NAME, name);
+                    generator.setParameter(ExperimentInputConstants.EXPERIMENT_ID, experimentId);
+                    q = generator.selectQuery(em);
+                    ExperimentOutput experimentOutput = (ExperimentOutput) q.getSingleResult();
+                    ExperimentOutputResource outputResource = (ExperimentOutputResource) Utils.getResource(ResourceType.EXPERIMENT_OUTPUT, experimentOutput);
+                    em.getTransaction().commit();
+                    if (em.isOpen()) {
+                        if (em.getTransaction().isActive()){
+                            em.getTransaction().rollback();
+                        }
+                        em.close();
+                    }
+                    return outputResource;
                 case USER_CONFIGURATION_DATA:
-                    return JPAUtil.executeOnExpCatalog(entityManager -> {
-                        QueryGenerator generator = new QueryGenerator(USER_CONFIGURATION_DATA);
-                        generator.setParameter(UserConfigurationDataConstants.EXPERIMENT_ID, name);
-                        Query q = generator.selectQuery(entityManager);
-                        UserConfigurationData configurationData = (UserConfigurationData) q.getSingleResult();
-                        return (UserConfigurationDataResource)
-                                Utils.getResource(ResourceType.USER_CONFIGURATION_DATA, configurationData);
-                    });
+                    generator = new QueryGenerator(USER_CONFIGURATION_DATA);
+                    generator.setParameter(UserConfigurationDataConstants.EXPERIMENT_ID, name);
+                    q = generator.selectQuery(em);
+                    UserConfigurationData configurationData = (UserConfigurationData) q.getSingleResult();
+                    UserConfigurationDataResource configurationDataResource = (UserConfigurationDataResource)
+                            Utils.getResource(ResourceType.USER_CONFIGURATION_DATA, configurationData);
+                    em.getTransaction().commit();
+                    if (em.isOpen()) {
+                        if (em.getTransaction().isActive()){
+                            em.getTransaction().rollback();
+                        }
+                        em.close();
+                    }
+                    return configurationDataResource;
                 case PROCESS:
-                    return JPAUtil.executeOnExpCatalog(entityManager -> {
-                        QueryGenerator generator = new QueryGenerator(PROCESS);
-                        generator.setParameter(ProcessConstants.PROCESS_ID, name);
-                        Query q = generator.selectQuery(entityManager);
-                        Process process = (Process) q.getSingleResult();
-                        return (ProcessResource) Utils.getResource(ResourceType.PROCESS, process);
-                    });
+                    generator = new QueryGenerator(PROCESS);
+                    generator.setParameter(ProcessConstants.PROCESS_ID, name);
+                    q = generator.selectQuery(em);
+                    Process process = (Process) q.getSingleResult();
+                    ProcessResource processResource = (ProcessResource) Utils.getResource(ResourceType.PROCESS, process);
+                    em.getTransaction().commit();
+                    if (em.isOpen()) {
+                        if (em.getTransaction().isActive()){
+                            em.getTransaction().rollback();
+                        }
+                        em.close();
+                    }
+                    return processResource;
                 default:
+                    em.getTransaction().commit();
+                    if (em.isOpen()) {
+                        if (em.getTransaction().isActive()){
+                            em.getTransaction().rollback();
+                        }
+                        em.close();
+                    }
                     logger.error("Unsupported resource type for experiment resource.", new IllegalArgumentException());
                     throw new IllegalArgumentException("Unsupported resource type for experiment resource.");
             }
         } catch (Exception e) {
             throw new RegistryException(e);
+        } finally {
+            if (em != null && em.isOpen()) {
+                if (em.getTransaction().isActive()){
+                    em.getTransaction().rollback();
+                }
+                em.close();
+            }
         }
     }
 


[46/50] [abbrv] airavata git commit: [AIRAVATA-2057] Move the distribution directory to modules to slow down the distribution build

Posted by la...@apache.org.
http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/modules/distribution/src/main/resources/NOTICE
----------------------------------------------------------------------
diff --git a/modules/distribution/src/main/resources/NOTICE b/modules/distribution/src/main/resources/NOTICE
new file mode 100644
index 0000000..fa7cba5
--- /dev/null
+++ b/modules/distribution/src/main/resources/NOTICE
@@ -0,0 +1,163 @@
+Apache Airavata
+Copyright 2014 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+===============================================================================
+Apache Xerces Java Notice:
+
+   Portions of this software were originally based on the following:
+     - software copyright (c) 1999, IBM Corporation., http://www.ibm.com.
+     - software copyright (c) 1999, Sun Microsystems., http://www.sun.com.
+     - voluntary contributions made by Paul Eng on behalf of the
+       Apache Software Foundation that were originally developed at iClick, Inc.,
+       software copyright (c) 1999.
+
+================================================================================
+Apache XmlBeans Notice: 
+
+   Portions of this software were originally based on the following:
+     - software copyright (c) 2000-2003, BEA Systems, <http://www.bea.com/>.
+
+   Aside from contributions to the Apache XMLBeans project, this
+   software also includes:
+
+    - one or more source files from the Apache Xerces-J and Apache Axis
+      products, Copyright (c) 1999-2003 Apache Software Foundation
+
+    - W3C XML Schema documents Copyright 2001-2003 (c) World Wide Web
+      Consortium (Massachusetts Institute of Technology, European Research
+      Consortium for Informatics and Mathematics, Keio University)
+
+    - resolver.jar from Apache Xml Commons project,
+      Copyright (c) 2001-2003 Apache Software Foundation
+
+    - Piccolo XML Parser for Java from http://piccolo.sourceforge.net/,
+      Copyright 2002 Yuval Oren under the terms of the Apache Software License 2.0
+
+    - JSR-173 Streaming API for XML from http://sourceforge.net/projects/xmlpullparser/,
+      Copyright 2005 BEA under the terms of the Apache Software License 2.0
+      
+=========================================================================================
+Apache Axis2 Notice: (axis2-*, mex-1.5.1-impl)
+
+Portions Copyright 2006 International Business Machines Corp.
+Portions Copyright 2005-2007 WSO2, Inc.
+
+This product also includes schemas and specification developed by:
+- the W3C consortium (http://www.w3c.org)
+
+This product also includes WS-* schemas developed by International
+Business Machines Corporation, Microsoft Corporation, BEA Systems, 
+TIBCO Software, SAP AG, Sonic Software, and VeriSign
+
+This product also includes a WSDL developed by salesforce.com
+- Copyright 1999-2006 salesforce.com, inc.
+Portions of the included xmlbeans library were originally based on the following:
+- software copyright (c) 2000-2003, BEA Systems, <http://www.bea.com/>.
+
+====================================================================================
+Apache Derby Notice:
+
+Portions of Derby were originally developed by
+International Business Machines Corporation and are
+licensed to the Apache Software Foundation under the
+"Software Grant and Corporate Contribution License Agreement",
+informally known as the "Derby CLA".
+The following copyright notice(s) were affixed to portions of the code
+with which this file is now or was at one time distributed
+and are placed here unaltered.
+
+(C) Copyright 1997,2004 International Business Machines Corporation.  All rights reserved.
+
+(C) Copyright IBM Corp. 2003. 
+
+=======================
+
+The portion of the functionTests under 'nist' was originally 
+developed by the National Institute of Standards and Technology (NIST), 
+an agency of the United States Department of Commerce, and adapted by
+International Business Machines Corporation in accordance with the NIST
+Software Acknowledgment and Redistribution document at
+http://www.itl.nist.gov/div897/ctg/sql_form.htm
+
+========================
+
+The JDBC apis for small devices and JDBC3 (under java/stubs/jsr169 and
+java/stubs/jdbc3) were produced by trimming sources supplied by the
+Apache Harmony project. In addition, the Harmony SerialBlob and
+SerialClob implementations are used. The following notice covers the Harmony sources:
+
+Portions of Harmony were originally developed by
+Intel Corporation and are licensed to the Apache Software
+Foundation under the "Software Grant and Corporate Contribution
+License Agreement", informally known as the "Intel Harmony CLA".
+
+=============================================================================
+Apache Woden Notice:
+
+   This product also includes software developed by :
+   
+     - IBM Corporation (http://www.ibm.com),
+         WSDL4J was the initial code contribution for the Apache Woden
+         project and some of the WSDL4J design and code has been reused.
+     - The W3C Consortium (http://www.w3c.org),
+         Common W3C XML Schema and DTD files are packaged with Apache Woden.
+
+   Please read the different LICENSE files present in the root directory of
+   this distribution.
+
+=========================================================================
+Woodstox Notice: 
+
+This product includes software developed by the Woodstox Project 
+(http://woodstox.codehaus.org/)
+
+This product currently only contains code developed by authors
+of specific components, as identified by the source code files.
+
+Since product implements StAX API, it has dependencies to StAX API
+classes.
+
+For additional credits (generally to people who reported problems)
+see CREDITS file.
+
+===========================================================================
+Apache xml-commons xml-apis Notice:
+
+   Portions of this software were originally based on the following:
+     - software copyright (c) 1999, IBM Corporation., http://www.ibm.com.
+     - software copyright (c) 1999, Sun Microsystems., http://www.sun.com.
+     - software copyright (c) 2000 World Wide Web Consortium, http://www.w3.org
+
+================================================================================================
+Apache  Xalan Notice: 
+
+Portions of this software was originally based on the following:
+     - software copyright (c) 1999-2002, Lotus Development Corporation., http://www.lotus.com.
+     - software copyright (c) 2001-2002, Sun Microsystems., http://www.sun.com.
+     - software copyright (c) 2003, IBM Corporation., http://www.ibm.com.
+     - voluntary contributions made by Ovidiu Predescu (ovidiu@cup.hp.com) on behalf of the 
+       Apache Software Foundation and was originally developed at Hewlett Packard Company. 
+================================================================================================
+Apache  OpenJPA Notice: 
+
+OpenJPA includes software developed by the SERP project
+Copyright (c) 2002-2006, A. Abram White. All rights reserved.
+
+OpenJPA includes the persistence and orm schemas from the JPA specifications.
+Copyright 2005-2009 Sun Microsystems, Inc. All rights reserved.
+OpenJPA elects to include this software in this distribution under the
+CDDL license.  You can obtain a copy of the License at:
+    https://glassfish.dev.java.net/public/CDDL+GPL.html
+The source code is available at:
+    https://glassfish.dev.java.net/source/browse/glassfish/
+
+OpenJPA includes software written by Miroslav Nachev
+OpenJPA uses test code written by Charles Tillman.
+================================================================================================
+Apache XmlSchema Notice:
+
+Portions Copyright 2006 International Business Machines Corp.
+================================================================================================

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/modules/distribution/src/main/resources/README
----------------------------------------------------------------------
diff --git a/modules/distribution/src/main/resources/README b/modules/distribution/src/main/resources/README
new file mode 100644
index 0000000..c2223ff
--- /dev/null
+++ b/modules/distribution/src/main/resources/README
@@ -0,0 +1,145 @@
+Apache Airavata Source - README.txt
+Licensed under Apache License 2.0 - http://www.apache.org/licenses/LICENSE-2.0
+--------------------------------------------------------------------------------
+
+About
+=====
+Apache Airavata, a software framework to executing and managing computational jobs on 
+distributed computing resources including local clusters, supercomputers, national grids, 
+academic and commercial clouds. Airavata builds on general concepts of service oriented computing, 
+distributed messaging, and workflow composition and orchestration. Airavata bundles a server package 
+with an API, client software development Kits and a general purpose GUI XBaya as a application registration, workflow
+construction execution and monitoring. XBaya GUI also provides capabilities to access the workflow 
+produced data.  
+
+Contact
+========
+For additional information about Apache Airavata, please contact the user or dev mailing lists:
+http://airavata.apache.org/community/mailing-lists.html
+
+Description of Airavata Directory Structure
+==================================
+    - airavata-api
+      This directory contains Airavata API related data models, api methods, generated server skeletons, client stubs, server implementations and client samples. 
+
+    - modules
+      This contains the source code of all the airavata maven projects organized as libraries, services and distributions
+
+    - samples
+      This contains all the system wide samples provided in Airavata distribution. All the sample are having its README file
+      So users have to refer each readme file before running each sample.
+
+    - tools
+      This contains source code libraries that can enhance Airavata features.
+
+    - README
+      This document.
+    
+    - RELEASE_NOTES
+      The describe the key features and know issues with the current release. 
+
+    - INSTALL
+      This document will contain information on installing Apache-Airavata.
+
+Airavata Source Distribution Directory Structure
+================================================
+
+    AIRAVATA_MASTER
+		\u251c\u2500\u2500 airavata-api
+		\u251c\u2500\u2500 modules
+		\u2502   \u251c\u2500\u2500 airavata-client
+		\u2502   \u251c\u2500\u2500 app-catalog
+		\u2502   \u251c\u2500\u2500 commons
+		\u2502   \u2502   \u251c\u2500\u2500 gfac-schema
+		\u2502   \u2502   \u251c\u2500\u2500 utils
+		\u2502   \u2502   \u251c\u2500\u2500 workflow-execution-context
+		\u2502   \u2502   \u2514\u2500\u2500 workflow-tracking
+		\u2502   \u251c\u2500\u2500 credential-store-service
+		\u2502   \u251c\u2500\u2500 distribution
+		\u2502   \u2502   \u251c\u2500\u2500 api-server
+		\u2502   \u2502   \u251c\u2500\u2500 client
+		\u2502   \u2502   \u251c\u2500\u2500 gfac-server
+		\u2502   \u2502   \u251c\u2500\u2500 orchestrator-server
+		\u2502   \u2502   \u251c\u2500\u2500 server
+		\u2502   \u2502   \u2514\u2500\u2500 release
+		\u2502   \u2502   \u2514\u2500\u2500 xbaya-gui
+		\u2502   \u251c\u2500\u2500 gfac
+		\u2502   \u2502   \u251c\u2500\u2500 airavata-gfac-service
+		\u2502   \u2502   \u251c\u2500\u2500 gfac-bes
+		\u2502   \u2502   \u251c\u2500\u2500 gfac-core
+		\u2502   \u2502   \u251c\u2500\u2500 gfac-ec2
+		\u2502   \u2502   \u251c\u2500\u2500 gfac-gram
+		\u2502   \u2502   \u251c\u2500\u2500 gfac-gsissh
+		\u2502   \u2502   \u251c\u2500\u2500 gfac-hadoop
+		\u2502   \u2502   \u251c\u2500\u2500 gfac-local
+		\u2502   \u2502   \u251c\u2500\u2500 gfac-monitor
+		\u2502   \u2502   \u251c\u2500\u2500 gfac-ssh
+		\u2502   \u2502   \u251c\u2500\u2500 gfac-thrift-descriptions
+		\u2502   \u251c\u2500\u2500 integration-tests
+		\u2502   \u251c\u2500\u2500 messaging
+		\u2502   \u251c\u2500\u2500 orchestrator
+		\u2502   \u251c\u2500\u2500 registry
+		\u2502   \u2502   \u251c\u2500\u2500 airavata-jpa-registry
+		\u2502   \u2502   \u251c\u2500\u2500 registry-cpi
+		\u2502   \u251c\u2500\u2500 security
+		\u2502   \u251c\u2500\u2500 credential-store
+		\u2502   \u251c\u2500\u2500 server
+		\u2502   \u251c\u2500\u2500 test-suite
+		\u2502   \u251c\u2500\u2500 workflow-model
+		\u2502   \u2502   \u251c\u2500\u2500 workflow-engine
+		\u2502   \u2502   \u251c\u2500\u2500 workflow-model-component-node
+		\u2502   \u2502   \u2514\u2500\u2500 workflow-model-core
+		\u2502   \u251c\u2500\u2500 ws-messenger
+		\u2502   \u2502   \u251c\u2500\u2500 commons
+		\u2502   \u2502   \u251c\u2500\u2500 distribution
+		\u2502   \u2502   \u251c\u2500\u2500 messagebox
+		\u2502   \u2502   \u251c\u2500\u2500 messagebroker
+		\u2502   \u2502   \u251c\u2500\u2500 message-monitor
+		\u2502   \u2502   \u2514\u2500\u2500 samples
+		\u2502   \u2514\u2500\u2500 xbaya-gui
+		\u251c\u2500\u2500 samples
+		\u251c\u2500\u2500 tools
+		\u2502   \u251c\u2500\u2500 gsissh
+		\u2502   \u251c\u2500\u2500 gsissh-cli-tools
+		\u2502   \u251c\u2500\u2500 phoebus-integration
+		\u2502   \u2514\u2500\u2500 registry-migrate
+		\u251c\u2500\u2500 INSTALL
+		\u251c\u2500\u2500 LICENSE
+		\u251c\u2500\u2500 NOTICE
+		\u251c\u2500\u2500 README
+		\u2514\u2500\u2500 RELEASE_NOTES
+
+Available Binary Distributions
+==============================
+
+Server Distributions
+--------------------
+* Airavata Server
+  The Airavata Server binary distribution allows you to run a standalone Airavata Server which includes all the airavata services 
+  shipped with a default derby database as the backend registry. For stable purposes, a mysql configuration is recommended. 
+
+* Airavata API Server
+  This is the server that contains Airavata API Server.
+
+* Airavata Orchestrator Server
+  This is the stand-alone orchestrator server
+
+* Airavata GFac Server
+  This is the standalone GFac Server
+
+Client Distributions
+--------------------
+* Airavata XBaya
+  The Airavata XBaya distribution is a client GUI application with features to register applications as web services, construct workflows,
+  execute and monitor workflows and browse the generated results from the airavata registry.
+
+* Airavata Client
+  The Airavata Client distribution is a set of libraries and configurations files that allow a 3rd party application to programatically 
+  access Airavata functionality through Airavata API. 
+  
+ How to test and run samples
+===========================
+* If you built Airavata from source, and if you see "BUILD SUCCESS", then the test cases should have passes.
+* The test cases are beyond unit level, they startup embedded services and run through basic workflow use cases.
+* To walk through Airavata features, follow "XBAYA Quick-Start Tutorial" tutorial at https://cwiki.apache.org/confluence/display/AIRAVATA/XBAYA+Quick-Start+Tutorial.
+* For advanced use cases, please contact mailing lists - http://airavata.apache.org/community/mailing-lists.html

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/modules/distribution/src/main/resources/bin/airavata-server-start.sh
----------------------------------------------------------------------
diff --git a/modules/distribution/src/main/resources/bin/airavata-server-start.sh b/modules/distribution/src/main/resources/bin/airavata-server-start.sh
new file mode 100644
index 0000000..f44661b
--- /dev/null
+++ b/modules/distribution/src/main/resources/bin/airavata-server-start.sh
@@ -0,0 +1,123 @@
+#!/usr/bin/env bash
+
+# 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.
+
+. `dirname $0`/setenv.sh
+cd ${AIRAVATA_HOME}/bin
+LOGO_FILE="logo.txt"
+
+JAVA_OPTS=""
+AIRAVATA_COMMAND=""
+EXTRA_ARGS=""
+SERVERS=""
+IS_DAEMON_MODE=false
+LOGO=true
+IS_SUBSET=false
+SUBSET=""
+
+# parse command arguments
+for var in "$@"
+do
+    case ${var} in
+        -xdebug)
+        	AIRAVATA_COMMAND="${AIRAVATA_COMMAND}"
+            JAVA_OPTS="$JAVA_OPTS -Xdebug -Xnoagent -Xrunjdwp:transport=dt_socket,server=y,address=8000"
+            shift
+        ;;
+        -security)
+            JAVA_OPTS="${JAVA_OPTS} -Djava.security.manager -Djava.security.policy=${AIRAVATA_HOME}/conf/axis2.policy -Daxis2.home=${AIRAVATA_HOME}"
+            shift
+        ;;
+	    apiserver | gfac | orchestrator | credentialstore | regserver)
+	        if [ -z ${SERVERS} ] ; then
+	            SERVERS="${var}"
+	        else
+	            SERVERS="${SERVERS},${var}"
+	        fi
+            shift
+        ;;
+        all | api-orch | execution )
+            IS_SUBSET=true
+            SUBSET="${var}"
+            shift
+            ;;
+        -d)
+	        IS_DAEMON_MODE=true
+	        shift
+	        ;;
+	    -nologo)
+	        LOGO=false
+            shift
+        ;;
+        -h)
+            echo "Usage: airavata-server-start.sh [server-name/s] [command-options]"
+            echo "Server names:"
+            echo "  apiserver           Start apiserver"
+            echo "  gfac                Start gfac server"
+            echo "  orchestrator        Start orchestrator server"
+            echo "  credentialstore     Start credentialstore server"
+            echo "  regserver           Start registry server"
+            echo "  all                 Start all servers in one JVM"
+
+            echo "command options:"
+	        echo "  -d                  Start server in daemon mode"
+            echo "  -xdebug             Start Airavata Server under JPDA debugger"
+            echo "  -nologo             Do not show airavata logo"
+            echo "  -security           Enable Java 2 security"
+	        echo "  --<key>[=<value>]   Server setting(s) to override or introduce (overrides values in airavata-server.properties)"
+            echo "  -h                  Display this help and exit"
+            shift
+            exit 0
+        ;;
+	    *)
+	        EXTRA_ARGS="${EXTRA_ARGS} ${var}"
+            shift
+        ;;
+    esac
+done
+
+#Construct Airavata command arguments in proper order.
+if ${IS_SUBSET} ; then
+    AIRAVATA_COMMAND="--servers=${SUBSET} ${AIRAVATA_COMMAND} ${EXTRA_ARGS}"
+else
+    if [ -z ${SERVERS} ] ; then
+        echo "You should provide at least one server component to start the airavata server. Please use -h option to get more details."
+        exit -1
+    else
+        AIRAVATA_COMMAND="--servers=${SERVERS} ${AIRAVATA_COMMAND} ${EXTRA_ARGS}"
+    fi
+fi
+
+#print logo file
+if ${LOGO} ; then
+	if [ -e ${LOGO_FILE} ]
+	then
+		cat ${LOGO_FILE}
+	fi
+fi
+
+
+if ${IS_DAEMON_MODE} ; then
+	echo "Starting airavata server/s in daemon mode..."
+	nohup java ${JAVA_OPTS} -classpath "${AIRAVATA_CLASSPATH}" \
+    org.apache.airavata.server.ServerMain ${AIRAVATA_COMMAND} $* > /dev/null 2>&1 &
+else
+	java ${JAVA_OPTS} -classpath "${AIRAVATA_CLASSPATH}" \
+    org.apache.airavata.server.ServerMain ${AIRAVATA_COMMAND} $*
+fi
+

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/modules/distribution/src/main/resources/bin/airavata-server-stop.sh
----------------------------------------------------------------------
diff --git a/modules/distribution/src/main/resources/bin/airavata-server-stop.sh b/modules/distribution/src/main/resources/bin/airavata-server-stop.sh
new file mode 100644
index 0000000..7f83f48
--- /dev/null
+++ b/modules/distribution/src/main/resources/bin/airavata-server-stop.sh
@@ -0,0 +1,71 @@
+#!/usr/bin/env bash
+
+# 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.
+
+. `dirname $0`/setenv.sh
+cd ${AIRAVATA_HOME}/bin
+
+JAVA_OPTS=""
+AIRAVATA_COMMAND=""
+FORCE=false
+
+for var in "$@"
+do
+    case ${var} in
+    	-f | --force)
+	        FORCE=true
+            shift
+        ;;
+        -h)
+            echo "Usage: airavata-server-stop.sh [command-options]"
+            echo "command options:"
+	        echo "  -f , --force       Force stop all airavata servers."
+	        echo "  --<key>[=<value>]  Server setting(s) to override or introduce (overrides values in airavata-server.properties)"
+            echo "  -h                 Display this help and exit"
+            shift
+            exit 0
+        ;;
+	*)
+            shift
+    esac
+done
+
+if ${FORCE} ; then
+	for f in `find . -name "server_start_*"`; do
+	    # split file name using "_" underscore
+		f_split=(${f//_/ });
+		echo "Found process file : $f"
+		echo -n "    Sending kill signals to process ${f_split[2]}..."
+		out=`kill -9 ${f_split[2]} 2>&1`
+		if [ -z "$out" ]; then
+		    echo "done"
+		else
+		    echo "failed (REASON: $out)"
+		fi
+		echo -n "    Removing process file..."
+		out=`rm ${f} 2>&1`
+		if [ -z "$out" ]; then
+		    echo "done"
+		else
+		    echo "failed (REASON: $out)"
+		fi
+	done
+else
+    java ${JAVA_OPTS} -classpath "${AIRAVATA_CLASSPATH}" \
+    org.apache.airavata.server.ServerMain stop ${AIRAVATA_COMMAND} $*
+fi

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/modules/distribution/src/main/resources/bin/airavata-server.bat
----------------------------------------------------------------------
diff --git a/modules/distribution/src/main/resources/bin/airavata-server.bat b/modules/distribution/src/main/resources/bin/airavata-server.bat
new file mode 100644
index 0000000..be2c584
--- /dev/null
+++ b/modules/distribution/src/main/resources/bin/airavata-server.bat
@@ -0,0 +1,55 @@
+@echo off
+rem Licensed to the Apache Software Foundation (ASF) under one
+rem or more contributor license agreements. See the NOTICE file
+rem distributed with this work for additional information
+rem regarding copyright ownership. The ASF licenses this file
+rem to you under the Apache License, Version 2.0 (the
+rem "License"); you may not use this file except in compliance
+rem with the License. You may obtain a copy of the License at
+rem
+rem http://www.apache.org/licenses/LICENSE-2.0
+rem
+rem Unless required by applicable law or agreed to in writing,
+rem software distributed under the License is distributed on an
+rem "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+rem KIND, either express or implied. See the License for the
+rem specific language governing permissions and limitations
+rem under the License.
+
+setlocal EnableDelayedExpansion
+
+call "%~dp0"setenv.bat
+
+:loop
+if ""%1""==""-xdebug"" goto xdebug
+if ""%1""==""-security"" goto security
+if ""%1""=="""" goto run
+goto help
+
+:xdebug
+set JAVA_OPTS= %JAVA_OPTS% -Xdebug -Xnoagent -Xrunjdwp:transport=dt_socket,server=y,address=8000
+shift
+goto loop
+
+:security
+set JAVA_OPTS=%JAVA_OPTS% -Djava.security.manager -Djava.security.policy=%AIRAVATA_HOME%\conf\axis2.policy -Daxis2.home=%AIRAVATA_HOME%
+shift
+goto loop
+
+:help
+echo  Usage: %0 [-options]
+echo.
+echo  where options include:
+echo   -xdebug    Start Airavata Server under JPDA debugger
+echo   -security  Enable Java 2 security
+echo   -h         Help
+goto end
+
+:run
+cd "%AIRAVATA_HOME%\bin"
+set LOGO_FILE="logo.txt"
+if exist "%LOGO_FILE%" type "%LOGO_FILE%"
+
+java %JAVA_OPTS% -classpath "%AIRAVATA_CLASSPATH%" org.apache.airavata.server.ServerMain %*
+
+:end

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/modules/distribution/src/main/resources/bin/derby.sh
----------------------------------------------------------------------
diff --git a/modules/distribution/src/main/resources/bin/derby.sh b/modules/distribution/src/main/resources/bin/derby.sh
new file mode 100644
index 0000000..134f7b9
--- /dev/null
+++ b/modules/distribution/src/main/resources/bin/derby.sh
@@ -0,0 +1,23 @@
+#!/bin/sh
+
+# 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.
+
+. `dirname $0`/setenv.sh
+export DERBY_HOME=$AIRAVATA_HOME/standalone-server
+cd $AIRAVATA_HOME/bin
+./startNetworkServer $*
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/modules/distribution/src/main/resources/bin/logo.txt
----------------------------------------------------------------------
diff --git a/modules/distribution/src/main/resources/bin/logo.txt b/modules/distribution/src/main/resources/bin/logo.txt
new file mode 100644
index 0000000..e886438
--- /dev/null
+++ b/modules/distribution/src/main/resources/bin/logo.txt
@@ -0,0 +1,34 @@
+...._....................._..............._...._......................_.........
+.../ \..._.__...__._..___| |__...___...../ \..(_)_.__.__.___...____._|.|_.__._..
+../ _ \.| '_ \./ _` |/ __| '_ \./ _ \.../ _ \.| | '__/ _` \ \./ / _` | __/ _` |.
+./ ___ \| |_) | (_| | (__| |.| |  __/../ ___ \| | |.| (_| |\ V / (_| | || (_| |.
+/_/...\_\ .__/.\__,_|\___|_|.|_|\___|./_/...\_\_|_|..\__,_|.\_/.\__,_|\__\__,_|.
+........|_|.....................................................................
+................................................................................
+................,+????++~..~~++..+????????+??++.++~...~+=???+++.................
+..............:???????....:::...~??????????????.~..::...=????????...............
+............????????..~~..?????..??????????????.?????..~~~.~???????.............
+...........?????????I,~I~~??????.+????????????.~??????~=~..?????????~...........
+.........?????++??????..????????:.??????????I..????????..????????+????..........
+........??.....???????....???????...???????+..+??????+.I.????????.....?,........
+........????==????????..??..?????..=???????=..?????,.=+?.?????????===??=........
+......=??????????+????..+??=.???=.~??????????,.???=,???,.????=+??????????.......
+....??????????+...+I++..???,=...:??????????????.....+??..++I?+..,??????????.....
+....???????=??..........??..+??.:=:.???????......??..??=..........?=???????=....
+....??????..?...........+?..???.....???????......???.??...........~=.??????=....
+....~???~.~..............?..???.~=..,??????...7..???.?.:..............~????:....
+...7....7 ...............?..????. ...??????... .????.?.?.............I ..:.:....
+.....+7=,.+?................????:,I...?????..=.?????.?.............??~.=7+......
+..........????..............??????~...~??..~~??????..?...........+???,..........
+...........???............=.+???????,.?+:.?????????..+...........???+...........
+............??............?,.??????.,??..??????????.,............???............
+...........??,.............=.,????.?+....????????I.I..............=?............
+..........I?..................+??.:?~.....=??????..................??...........
+..........??...?...............??.:?=......??????..............?...??...........
+............++?..............?.????...?....??????.+..............++I............
+.............................?.??????~....???????.?.............................
+............................~~.??????......??????...............................
+.............................=???????......???????+.............................
+..........................=I??++?+++?......?+++++++?+...........................
+..........................,..77..77.........  ..  ...7..........................
+................................................................................

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/modules/distribution/src/main/resources/bin/setenv.bat
----------------------------------------------------------------------
diff --git a/modules/distribution/src/main/resources/bin/setenv.bat b/modules/distribution/src/main/resources/bin/setenv.bat
new file mode 100644
index 0000000..5f1fda1
--- /dev/null
+++ b/modules/distribution/src/main/resources/bin/setenv.bat
@@ -0,0 +1,33 @@
+rem Licensed to the Apache Software Foundation (ASF) under one
+rem or more contributor license agreements. See the NOTICE file
+rem distributed with this work for additional information
+rem regarding copyright ownership. The ASF licenses this file
+rem to you under the Apache License, Version 2.0 (the
+rem "License"); you may not use this file except in compliance
+rem with the License. You may obtain a copy of the License at
+rem
+rem http://www.apache.org/licenses/LICENSE-2.0
+rem
+rem Unless required by applicable law or agreed to in writing,
+rem software distributed under the License is distributed on an
+rem "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+rem KIND, either express or implied. See the License for the
+rem specific language governing permissions and limitations
+rem under the License.
+
+@echo off
+
+:initialize
+if "%AIRAVATA_HOME%"=="" set AIRAVATA_HOME=%~sdp0..
+SET curDrive=%cd:~0,1%
+SET airavataDrive=%AIRAVATA_HOME:~0,1%
+if not "%curDrive%" == "%airavataDrive%" %airavataDrive%:
+goto updateClasspath
+
+rem ----- update classpath -----------------------------------------------------
+:updateClasspath
+cd %AIRAVATA_HOME%
+set AIRAVATA_CLASSPATH=
+FOR %%C in ("%AIRAVATA_HOME%\lib\*.jar") DO set AIRAVATA_CLASSPATH=!AIRAVATA_CLASSPATH!;..\lib\%%~nC%%~xC
+
+:end
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/modules/distribution/src/main/resources/bin/setenv.sh
----------------------------------------------------------------------
diff --git a/modules/distribution/src/main/resources/bin/setenv.sh b/modules/distribution/src/main/resources/bin/setenv.sh
new file mode 100755
index 0000000..9e894e1
--- /dev/null
+++ b/modules/distribution/src/main/resources/bin/setenv.sh
@@ -0,0 +1,46 @@
+#!/bin/sh
+
+# 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.
+
+# resolve links - $0 may be a softlink
+PRG="$0"
+
+while [ -h "$PRG" ]; do
+  ls=`ls -ld "$PRG"`
+  link=`expr "$ls" : '.*-> \(.*\)$'`
+  if expr "$link" : '.*/.*' > /dev/null; then
+    PRG="$link"
+  else
+    PRG=`dirname "$PRG"`/"$link"
+  fi
+done
+
+PRGDIR=`dirname "$PRG"`
+
+# Only set AIRAVATA_HOME if not already set
+[ -z "$AIRAVATA_HOME" ] && AIRAVATA_HOME=`cd "$PRGDIR/.." ; pwd`
+
+AIRAVATA_CLASSPATH=""
+
+for f in "$AIRAVATA_HOME"/lib/*.jar
+do
+  AIRAVATA_CLASSPATH="$AIRAVATA_CLASSPATH":$f
+done
+
+export AIRAVATA_HOME
+export AIRAVATA_CLASSPATH

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/modules/distribution/src/main/resources/bin/startNetworkServer
----------------------------------------------------------------------
diff --git a/modules/distribution/src/main/resources/bin/startNetworkServer b/modules/distribution/src/main/resources/bin/startNetworkServer
new file mode 100644
index 0000000..808566c
--- /dev/null
+++ b/modules/distribution/src/main/resources/bin/startNetworkServer
@@ -0,0 +1,189 @@
+#!/bin/sh
+
+# 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.
+
+if [ -n "$derby_common_debug" ] ; then
+  set -x
+fi
+
+# OS specific support.  $var _must_ be set to either true or false.
+cygwin=false;
+darwin=false;
+case "`uname`" in
+  CYGWIN*) cygwin=true ;;
+  Darwin*) darwin=true
+           if [ -z "$JAVA_HOME" ] ; then
+             JAVA_HOME=/System/Library/Frameworks/JavaVM.framework/Home
+           fi
+           ;;
+esac
+
+if [ -z "$DERBY_HOME" -o ! -d "$DERBY_HOME" ] ; then
+  ## resolve links - $0 may be a link to derby's home
+  PRG="$0"
+  progname=`basename "$0"`
+
+  # need this for relative symlinks
+  while [ -h "$PRG" ] ; do
+    ls=`ls -ld "$PRG"`
+    link=`expr "$ls" : '.*-> \(.*\)$'`
+    if expr "$link" : '/.*' > /dev/null; then
+    PRG="$link"
+    else
+    PRG=`dirname "$PRG"`"/$link"
+    fi
+  done
+
+  DERBY_HOME=`dirname "$PRG"`/..
+
+  # make it fully qualified
+  DERBY_HOME=`cd "$DERBY_HOME" && pwd`
+fi
+
+# For Cygwin, ensure paths are in UNIX format before anything is touched
+if $cygwin ; then
+  [ -n "$DERBY_HOME" ] &&
+    DERBY_HOME=`cygpath --unix "$DERBY_HOME"`
+  [ -n "$JAVA_HOME" ] &&
+    JAVA_HOME=`cygpath --unix "$JAVA_HOME"`
+fi
+
+# set DERBY_LIB location
+DERBY_LIB="${DERBY_HOME}/lib"
+
+if [ -z "$JAVACMD" ] ; then
+  if [ -n "$JAVA_HOME"  ] ; then
+    if [ -x "$JAVA_HOME/jre/sh/java" ] ; then
+      # IBM's JDK on AIX uses strange locations for the executables
+      JAVACMD="$JAVA_HOME/jre/sh/java"
+    else
+      JAVACMD="$JAVA_HOME/bin/java"
+    fi
+  else
+    JAVACMD=`which java 2> /dev/null `
+    if [ -z "$JAVACMD" ] ; then
+        JAVACMD=java
+    fi
+  fi
+fi
+
+if [ ! -x "$JAVACMD" ] ; then
+  echo "Error: JAVA_HOME is not defined correctly."
+  echo "  We cannot execute $JAVACMD"
+  exit 1
+fi
+
+# set local classpath, don't overwrite the user's
+LOCALCLASSPATH=$DERBY_LIB/derby.jar:$DERBY_LIB/derbynet.jar:$DERBY_LIB/derbytools.jar:$DERBY_LIB/derbyclient.jar
+
+# if CLASSPATH_OVERRIDE env var is set, LOCALCLASSPATH will be
+# user CLASSPATH first and derby-found jars after.
+# In that case, the user CLASSPATH will override derby-found jars
+#
+# if CLASSPATH_OVERRIDE is not set, we'll have the normal behaviour
+# with derby-found jars first and user CLASSPATH after
+if [ -n "$CLASSPATH" ] ; then
+  # merge local and specified classpath 
+  if [ -z "$LOCALCLASSPATH" ] ; then 
+    LOCALCLASSPATH="$CLASSPATH"
+  elif [ -n "$CLASSPATH_OVERRIDE" ] ; then
+    LOCALCLASSPATH="$CLASSPATH:$LOCALCLASSPATH"
+  else
+    LOCALCLASSPATH="$LOCALCLASSPATH:$CLASSPATH"
+  fi
+
+  # remove class path from launcher -cp option
+  CLASSPATH=""
+fi
+
+# For Cygwin, switch paths to appropriate format before running java
+# For PATHs convert to unix format first, then to windows format to ensure
+# both formats are supported. Probably this will fail on directories with ;
+# in the name in the path. Let's assume that paths containing ; are more
+# rare than windows style paths on cygwin.
+if $cygwin; then
+  if [ "$OS" = "Windows_NT" ] && cygpath -m .>/dev/null 2>/dev/null ; then
+    format=mixed
+  else
+    format=windows
+  fi
+  DERBY_HOME=`cygpath --$format "$DERBY_HOME"`
+  DERBY_LIB=`cygpath --$format "$DERBY_LIB"`
+  if [ -n "$JAVA_HOME" ]; then
+    JAVA_HOME=`cygpath --$format "$JAVA_HOME"`
+  fi
+  LCP_TEMP=`cygpath --path --unix "$LOCALCLASSPATH"`
+  LOCALCLASSPATH=`cygpath --path --$format "$LCP_TEMP"`
+  if [ -n "$CLASSPATH" ] ; then
+    CP_TEMP=`cygpath --path --unix "$CLASSPATH"`
+    CLASSPATH=`cygpath --path --$format "$CP_TEMP"`
+  fi
+  CYGHOME=`cygpath --$format "$HOME"`
+fi
+
+# add a second backslash to variables terminated by a backslash under cygwin
+if $cygwin; then
+  case "$DERBY_HOME" in
+    *\\ )
+    DERBY_HOME="$DERBY_HOME\\"
+    ;;
+  esac
+  case "$CYGHOME" in
+    *\\ )
+    CYGHOME="$CYGHOME\\"
+    ;;
+  esac
+  case "$LOCALCLASSPATH" in
+    *\\ )
+    LOCALCLASSPATH="$LOCALCLASSPATH\\"
+    ;;
+  esac
+  case "$CLASSPATH" in
+    *\\ )
+    CLASSPATH="$CLASSPATH\\"
+    ;;
+  esac
+fi
+
+# Readjust classpath for MKS
+# expr match 
+if [ \( "`expr $SHELL : '.*sh.exe$'`" -gt 0 \) -a \( "$cygwin" = "false" \) ]; then
+  LOCALCLASSPATH=`echo $LOCALCLASSPATH | sed -E 's/([\d\w]*):([\d\w]*)/\1;\2/g
+'`
+fi
+#!/bin/sh
+
+# 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.
+
+derby_exec_command="exec \"$JAVACMD\" $DERBY_OPTS -classpath \"$LOCALCLASSPATH\" org.apache.derby.drda.NetworkServerControl start $@"
+eval $derby_exec_command

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/modules/distribution/src/main/resources/samples/registerSample.sh
----------------------------------------------------------------------
diff --git a/modules/distribution/src/main/resources/samples/registerSample.sh b/modules/distribution/src/main/resources/samples/registerSample.sh
new file mode 100644
index 0000000..384ec0e
--- /dev/null
+++ b/modules/distribution/src/main/resources/samples/registerSample.sh
@@ -0,0 +1,24 @@
+#!/bin/sh
+
+# 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.
+
+. `dirname $0`/../bin/setenv.sh
+JAVA_OPTS=""
+
+java -classpath "$AIRAVATA_CLASSPATH" \
+		     org.apache.airavata.client.samples.RegisterSampleData $*

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/modules/distribution/src/main/resources/samples/scripts/add.sh
----------------------------------------------------------------------
diff --git a/modules/distribution/src/main/resources/samples/scripts/add.sh b/modules/distribution/src/main/resources/samples/scripts/add.sh
new file mode 100755
index 0000000..daa140b
--- /dev/null
+++ b/modules/distribution/src/main/resources/samples/scripts/add.sh
@@ -0,0 +1,21 @@
+#!/bin/sh
+# 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.
+
+# add two numbers
+sleep 10
+/bin/echo  "Result=`expr $1 + $2`"

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/modules/distribution/src/main/resources/samples/scripts/echo.sh
----------------------------------------------------------------------
diff --git a/modules/distribution/src/main/resources/samples/scripts/echo.sh b/modules/distribution/src/main/resources/samples/scripts/echo.sh
new file mode 100755
index 0000000..9dbaab9
--- /dev/null
+++ b/modules/distribution/src/main/resources/samples/scripts/echo.sh
@@ -0,0 +1,22 @@
+#!/bin/sh
+
+# 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.
+
+#echo wrapper
+sleep 10
+/bin/echo "Echoed_Output=$1"

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/modules/distribution/src/main/resources/samples/scripts/multiply.sh
----------------------------------------------------------------------
diff --git a/modules/distribution/src/main/resources/samples/scripts/multiply.sh b/modules/distribution/src/main/resources/samples/scripts/multiply.sh
new file mode 100755
index 0000000..a5b5f7f
--- /dev/null
+++ b/modules/distribution/src/main/resources/samples/scripts/multiply.sh
@@ -0,0 +1,22 @@
+#!/bin/sh
+
+# 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.
+
+# mutiply two numbers
+sleep 10
+/bin/echo "Result=`expr $1 \* $2`"

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/modules/distribution/src/main/resources/samples/scripts/subtract.sh
----------------------------------------------------------------------
diff --git a/modules/distribution/src/main/resources/samples/scripts/subtract.sh b/modules/distribution/src/main/resources/samples/scripts/subtract.sh
new file mode 100755
index 0000000..a21bec7
--- /dev/null
+++ b/modules/distribution/src/main/resources/samples/scripts/subtract.sh
@@ -0,0 +1,22 @@
+#!/bin/sh
+
+# 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.
+
+# substract two numbers
+sleep 10
+/bin/echo "Result=`expr $1 - $2`"

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 6594026..3988fd0 100644
--- a/pom.xml
+++ b/pom.xml
@@ -551,7 +551,7 @@
 				<module>modules/configuration</module>
 				<module>airavata-api</module>
 				<module>modules/commons</module>
-                <module>modules/messaging</module>
+                		<module>modules/messaging</module>
 				<module>modules/gfac</module>
 				<module>modules/registry</module>
 				<module>modules/security</module>
@@ -569,102 +569,9 @@
 				<!--<module>modules/workflow-model</module>-->
 				<!--<module>modules/workflow</module>-->
 				<!--<module>modules/xbaya-gui</module>-->
-                <module>distribution</module>
+                		<module>modules/distribution</module>
             </modules>
 		</profile>
-        <profile>
-            <id>jenkins</id>
-            <build>
-                <plugins>
-                    <plugin>
-                        <groupId>org.apache.maven.plugins</groupId>
-                        <artifactId>maven-remote-resources-plugin</artifactId>
-                        <executions>
-                            <execution>
-                                <goals>
-                                    <goal>process</goal>
-                                </goals>
-                            </execution>
-                        </executions>
-                    </plugin>
-                    <plugin>
-                        <artifactId>maven-resources-plugin</artifactId>
-                        <version>2.5</version>
-                        <executions>
-                            <execution>
-                                <id>copy-resources</id>
-                                <!-- here the phase you need -->
-                                <phase>validate</phase>
-                                <goals>
-                                    <goal>copy-resources</goal>
-                                </goals>
-                                <configuration>
-                                    <outputDirectory>${basedir}/target/classes/META-INF</outputDirectory>
-                                    <resources>
-                                        <resource>
-                                            <directory>${basedir}/src/main/assembly/dist</directory>
-                                            <filtering>true</filtering>
-                                        </resource>
-                                    </resources>
-                                </configuration>
-                            </execution>
-                        </executions>
-                    </plugin>
-                    <plugin>
-                        <groupId>org.apache.maven.plugins</groupId>
-                        <artifactId>maven-compiler-plugin</artifactId>
-                        <version>3.1</version>
-                        <configuration>
-                            <source>1.8</source>
-                            <target>1.8</target>
-                        </configuration>
-                    </plugin>
-                    <plugin>
-                        <groupId>org.apache.maven.plugins</groupId>
-                        <artifactId>maven-surefire-plugin</artifactId>
-                        <version>${surefire.version}</version>
-                        <configuration>
-                            <failIfNoTests>false</failIfNoTests>
-                            <skipTests>${skipTests}</skipTests>
-                            <workingDirectory>${project.build.testOutputDirectory}</workingDirectory>
-                            <!-- making sure that the sure-fire plugin doesn't run the integration
-                                tests -->
-                            <!-- Integration tests are run using the fail-safe plugin in the module
-                                pom -->
-                            <excludes>
-                                <exclude>**/IT.java</exclude>
-                                <exclude>**/*TestWithMyProxyAuth.java</exclude>
-                                <exclude>**/*TestWithSSHAuth.java</exclude>
-                                <exclude>**/*TestWithEC2Auth.java</exclude>
-                            </excludes>
-                        </configuration>
-                    </plugin>
-                </plugins>
-            </build>
-            <activation>
-                <activeByDefault>true</activeByDefault>
-            </activation>
-            <modules>
-                <module>modules/configuration</module>
-                <module>airavata-api</module>
-                <module>modules/commons</module>
-                <module>modules/messaging</module>
-                <module>modules/gfac</module>
-                <module>modules/registry</module>
-                <module>modules/security</module>
-                <module>modules/credential-store</module>
-                <module>modules/orchestrator</module>
-                <module>modules/monitoring</module>
-                <module>modules/user-profile</module>
-                <!--<module>modules/cloud</module>-->
-                <module>modules/server</module>
-                <module>modules/workflow</module>
-                <module>modules/test-suite</module>
-				<module>modules/group-manager</module>
-				<!-- enable distribution when AIRAVATA-2057 is fixed -->
-				<!--<module>distribution</module>-->
-            </modules>
-        </profile>
 		<profile>
 			<id>pedantic</id>
 			<build>


[35/50] [abbrv] airavata git commit: Changed grouper log level to FATAL

Posted by la...@apache.org.
Changed grouper log level to FATAL


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: 0b29a5ccd3b6d709a328565006a76cb6886c26f0
Parents: b284167
Author: Shameera Rathnayaka <sh...@gmail.com>
Authored: Thu Aug 18 16:05:50 2016 -0400
Committer: Shameera Rathnayaka <sh...@gmail.com>
Committed: Thu Aug 18 16:05:50 2016 -0400

----------------------------------------------------------------------
 modules/configuration/server/src/main/resources/log4j.properties | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/0b29a5cc/modules/configuration/server/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/modules/configuration/server/src/main/resources/log4j.properties b/modules/configuration/server/src/main/resources/log4j.properties
index ca7f0b3..ee67028 100644
--- a/modules/configuration/server/src/main/resources/log4j.properties
+++ b/modules/configuration/server/src/main/resources/log4j.properties
@@ -29,6 +29,9 @@ log4j.logger.httpclient.wire.header=FATAL
 log4j.logger.org.apache.commons.httpclient=FATAL
 log4j.logger.org.apache.zookeeper=FATAL
 log4j.logger.org.globus=INFO
+# following remove grouper info logs
+log4j.logger.edu.internet2.middleware.grouper=FATAL
+log4j.logger.org.hibernate=FATAL
          
 # CONSOLE is set to be a ConsoleAppender using a PatternLayout.
 log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender


[47/50] [abbrv] airavata git commit: [AIRAVATA-2057] Move the distribution directory to modules to slow down the distribution build

Posted by la...@apache.org.
http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/modules/distribution/src/main/resources/LICENSE
----------------------------------------------------------------------
diff --git a/modules/distribution/src/main/resources/LICENSE b/modules/distribution/src/main/resources/LICENSE
new file mode 100644
index 0000000..56f7cc2
--- /dev/null
+++ b/modules/distribution/src/main/resources/LICENSE
@@ -0,0 +1,2387 @@
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.
+
+===================================================================================
+The Apache Airavata distribution includes a number of run time 
+dependencies with separate copyright notices and license terms. Your use of the
+Apache Airavata code is subject to the terms and conditions of the following licenses.
+===================================================================================
+
+===============================================================================
+The following components come under Apache Software License 2.0
+===============================================================================
+
+apache axiom, apache axis2, apache commons, apache derby, apache geronimo,
+apache httpcore components, apache log4j, apache xmlbeans, apache xmlschema,
+aws-java-sdk-1.1.8.jar, bcel-5.1.jar, Codehaus Jackson (jackson-core-asl-1.9.2.jar,
+jackson-jaxrs-1.9.2.jar, jackson-mapper-asl-1.9.2.jar, jackson-xc-1.9.2.jar, 
+jets3t-0.8.0.jar, jettison-1.0-RC2.jar, neethi-2.0.4.jar, PDFBox libraries 
+(pdfbox, jempbox, fontbox), wstx-asl-3.2.4.jar
+
+- Bean Validation API (http://beanvalidation.org) javax.validation:validation-api:jar:1.1.0.Final
+- Hibernate Validator Engine (http://validator.hibernate.org/hibernate-validator) org.hibernate:hibernate-validator:jar:4.3.0.Final
+- GSS-API implementation for SSL with proxies (https://github.com/jglobus/JGlobus/gss) org.jglobus:gss:jar:2.0.6
+- SSL support (https://github.com/jglobus/JGlobus/jsse) org.jglobus:jsse:jar:2.0.6
+- myproxy (https://github.com/jglobus/JGlobus/myproxy) org.jglobus:myproxy:jar:2.0.6
+- SSL and proxy certificate support (https://github.com/jglobus/JGlobus/ssl-proxies) org.jglobus:ssl-proxies:jar:2.0.6
+- Bouncy Castle for GSS (https://github.com/jsiwek/BouncyCastleSSLv3) org.ogce:bcgss:jar:146
+- StAX API (http://stax.codehaus.org/) stax:stax-api:jar:1.0.1
+- Commons Codec (http://commons.apache.org/codec/) commons-codec:commons-codec:jar:1.4
+- Commons IO (http://commons.apache.org/io/) commons-io:commons-io:jar:1.4
+- Commons Lang (http://commons.apache.org/lang/) commons-lang:commons-lang:jar:2.6
+- Commons Logging (http://commons.apache.org/logging) commons-logging:commons-logging:jar:1.1.1
+- XmlBeans (http://xmlbeans.apache.org) org.apache.xmlbeans:xmlbeans:jar:2.5.0
+
+===============================================================================
+The following components use Apache based Licenses
+===============================================================================
+
+===============================================================================
+For: jdom-1.0.jar
+    Containing Project URL: http://www.jdom.org/
+/*-- 
+
+ $Id: LICENSE.txt,v 1.11 2004/02/06 09:32:57 jhunter Exp $
+
+ Copyright (C) 2000-2004 Jason Hunter & Brett McLaughlin.
+ All rights reserved.
+ 
+ Redistribution and use in source and binary forms, with or without
+ modification, are permitted provided that the following conditions
+ are met:
+ 
+ 1. Redistributions of source code must retain the above copyright
+    notice, this list of conditions, and the following disclaimer.
+ 
+ 2. Redistributions in binary form must reproduce the above copyright
+    notice, this list of conditions, and the disclaimer that follows 
+    these conditions in the documentation and/or other materials 
+    provided with the distribution.
+
+ 3. The name "JDOM" must not be used to endorse or promote products
+    derived from this software without prior written permission.  For
+    written permission, please contact <request_AT_jdom_DOT_org>.
+ 
+ 4. Products derived from this software may not be called "JDOM", nor
+    may "JDOM" appear in their name, without prior written permission
+    from the JDOM Project Management <request_AT_jdom_DOT_org>.
+ 
+ In addition, we request (but do not require) that you include in the 
+ end-user documentation provided with the redistribution and/or in the 
+ software itself an acknowledgement equivalent to the following:
+     "This product includes software developed by the
+      JDOM Project (http://www.jdom.org/)."
+ Alternatively, the acknowledgment may be graphical using the logos 
+ available at http://www.jdom.org/images/logos.
+
+ THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED
+ WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
+ OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+ DISCLAIMED.  IN NO EVENT SHALL THE JDOM AUTHORS OR THE PROJECT
+ CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF
+ USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+ ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY,
+ OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT
+ OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF
+ SUCH DAMAGE.
+
+ This software consists of voluntary contributions made by many 
+ individuals on behalf of the JDOM Project and was originally 
+ created by Jason Hunter <jhunter_AT_jdom_DOT_org> and
+ Brett McLaughlin <brett_AT_jdom_DOT_org>.  For more information
+ on the JDOM Project, please see <http://www.jdom.org/>. 
+
+ */
+
+===============================================================================
+
+ASM bytecode manipulation library (asm)
+    Containing Project URL: http://asm.ow2.org/
+
+    Copyright (c) 2000-2005 INRIA, France Telecom
+    All rights reserved.
+
+    Redistribution and use in source and binary forms, with or without
+    modification, are permitted provided that the following conditions
+    are met:
+
+    1. Redistributions of source code must retain the above copyright
+       notice, this list of conditions and the following disclaimer.
+
+    2. Redistributions in binary form must reproduce the above copyright
+       notice, this list of conditions and the following disclaimer in the
+       documentation and/or other materials provided with the distribution.
+
+    3. Neither the name of the copyright holders nor the names of its
+       contributors may be used to endorse or promote products derived from
+       this software without specific prior written permission.
+
+    THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+    AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+    IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+    ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
+    LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+    CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+    SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+    INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+    CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+    ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF
+    THE POSSIBILITY OF SUCH DAMAGE.
+
+===============================================================================
+
+For: cryptix-asn1-versionless.jar, cryptix32-versionless.jar
+    Containing Project URL: http://www.cryptix.org/
+
+Cryptix General License
+
+Copyright (c) 1995-2005 The Cryptix Foundation Limited.
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+  1. Redistributions of source code must retain the copyright notice,
+     this list of conditions and the following disclaimer.
+  2. Redistributions in binary form must reproduce the above copyright
+     notice, this list of conditions and the following disclaimer in
+     the documentation and/or other materials provided with the
+     distribution.
+
+THIS SOFTWARE IS PROVIDED BY THE CRYPTIX FOUNDATION LIMITED AND
+CONTRIBUTORS ``AS IS'' AND ANY EXPRESS OR IMPLIED WARRANTIES,
+INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF
+MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
+IN NO EVENT SHALL THE CRYPTIX FOUNDATION LIMITED OR CONTRIBUTORS BE
+LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR
+BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY,
+WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE
+OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN
+IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+===============================================================================
+The following components come under Extreme! Lab Software License
+===============================================================================
+
+XPP3
+    Containing Project URL: http://www.extreme.indiana.edu/xgws/xsoap/xpp/
+xsul, xsul5, xutil
+    Containing Project URL: http://www.extreme.indiana.edu/xgws/xsul/
+wsmg
+    Containing Project URL: http://www.extreme.indiana.edu/xgws/messenger/index.html
+gpel, weps-beans, pegasuswebservice, mapReduce-service-client, atomixmiser
+    Containing Project URL: http://www.extreme.indiana.edu/xgws/
+    
+Indiana University Extreme! Lab Software License
+
+Version 1.1.1
+
+Copyright (c) 2002 Extreme! Lab, Indiana University. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions
+are met:
+
+1. Redistributions of source code must retain the above copyright notice,
+   this list of conditions and the following disclaimer.
+
+2. Redistributions in binary form must reproduce the above copyright
+   notice, this list of conditions and the following disclaimer in
+   the documentation and/or other materials provided with the distribution.
+
+3. The end-user documentation included with the redistribution, if any,
+   must include the following acknowledgment:
+
+  "This product includes software developed by the Indiana University
+  Extreme! Lab (http://www.extreme.indiana.edu/)."
+
+Alternately, this acknowledgment may appear in the software itself,
+if and wherever such third-party acknowledgments normally appear.
+
+4. The names "Indiana Univeristy" and "Indiana Univeristy Extreme! Lab"
+must not be used to endorse or promote products derived from this
+software without prior written permission. For written permission,
+please contact http://www.extreme.indiana.edu/.
+
+5. Products derived from this software may not use "Indiana Univeristy"
+name nor may "Indiana Univeristy" appear in their name, without prior
+written permission of the Indiana University.
+
+THIS SOFTWARE IS PROVIDED "AS IS" AND ANY EXPRESSED OR IMPLIED
+WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF
+MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
+IN NO EVENT SHALL THE AUTHORS, COPYRIGHT HOLDERS OR ITS CONTRIBUTORS
+BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR
+BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY,
+WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR
+OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF
+ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+======================================================================== 
+The following components are MIT Licensed 
+========================================================================
+
+SLF4J,log4j-over-slf4j, jcl-over-slf4j, slf4j-api,mockito-all-1.8.5,jopt-simple-3.2.jar
+    Containing Project URL: http://www.slf4j.org/
+
+Copyright (c) 2004-2008 QOS.ch
+ All rights reserved.
+
+ Permission is hereby granted, free  of charge, to any person obtaining
+ a  copy  of this  software  and  associated  documentation files  (the
+ "Software"), to  deal in  the Software without  restriction, including
+ without limitation  the rights to  use, copy, modify,  merge, publish,
+ distribute,  sublicense, and/or sell  copies of  the Software,  and to
+ permit persons to whom the Software  is furnished to do so, subject to
+ the following conditions:
+
+ The  above  copyright  notice  and  this permission  notice  shall  be
+ included in all copies or substantial portions of the Software.
+
+ THE  SOFTWARE IS  PROVIDED  "AS  IS", WITHOUT  WARRANTY  OF ANY  KIND,
+ EXPRESS OR  IMPLIED, INCLUDING  BUT NOT LIMITED  TO THE  WARRANTIES OF
+ MERCHANTABILITY,    FITNESS    FOR    A   PARTICULAR    PURPOSE    AND
+ NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+ LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+ OF CONTRACT, TORT OR OTHERWISE,  ARISING FROM, OUT OF OR IN CONNECTION
+ WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+========================================================================
+
+For dom4j-1.6.1.jar:
+    Containing Project URL: http://dom4j.sourceforge.net/
+Copyright 2001-2005 (C) MetaStuff, Ltd. All Rights Reserved.
+
+Redistribution and use of this software and associated documentation
+("Software"), with or without modification, are permitted provided
+that the following conditions are met:
+
+1. Redistributions of source code must retain copyright
+   statements and notices.  Redistributions must also contain a
+   copy of this document.
+ 
+2. Redistributions in binary form must reproduce the
+   above copyright notice, this list of conditions and the
+   following disclaimer in the documentation and/or other
+   materials provided with the distribution.
+ 
+3. The name "DOM4J" must not be used to endorse or promote
+   products derived from this Software without prior written
+   permission of MetaStuff, Ltd.  For written permission,
+   please contact dom4j-info@metastuff.com.
+ 
+4. Products derived from this Software may not be called "DOM4J"
+   nor may "DOM4J" appear in their names without prior written
+   permission of MetaStuff, Ltd. DOM4J is a registered
+   trademark of MetaStuff, Ltd.
+ 
+5. Due credit should be given to the DOM4J Project - 
+   http://www.dom4j.org
+ 
+THIS SOFTWARE IS PROVIDED BY METASTUFF, LTD. AND CONTRIBUTORS
+``AS IS'' AND ANY EXPRESSED OR IMPLIED WARRANTIES, INCLUDING, BUT
+NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND
+FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.  IN NO EVENT SHALL
+METASTUFF, LTD. OR ITS CONTRIBUTORS BE LIABLE FOR ANY DIRECT,
+INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR
+SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION)
+HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT,
+STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED
+OF THE POSSIBILITY OF SUCH DAMAGE.
+
+====================================================================================================
+
+For Bouncy Castle:
+    Containing Project URL: http://www.bouncycastle.org/
+
+Copyright (c) 2000 - 2011 The Legion Of The Bouncy Castle (http://www.bouncycastle.org)
+
+Permission iss software and associated documentation files (the "Software"), to deal in
+the Software without restriction, including without limitation the rights to
+use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of
+the Software, and to permit persons to whom the Software is furnished to do so,
+subject to the following conditions hereby granted, free of charge, to any person obtaining a copy of this software
+and associated documentation files (the "Software"), to deal in the Software without restriction,
+including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense,
+and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so,
+subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial
+portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT
+LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT.
+IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY,
+WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE
+SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+=======================================================================================================
+
+For: The International Components for Unicode (icu4j-2.6.1.jar)
+    Containing Project URL: http://site.icu-project.org/
+
+    Copyright (c) 1995-2009 International Business Machines Corporation
+    and others
+
+    All rights reserved.
+
+    Permission is hereby granted, free of charge, to any person obtaining
+    a copy of this software and associated documentation files (the
+    "Software"), to deal in the Software without restriction, including
+    without limitation the rights to use, copy, modify, merge, publish,
+    distribute, and/or sell copies of the Software, and to permit persons
+    to whom the Software is furnished to do so, provided that the above
+    copyright notice(s) and this permission notice appear in all copies
+    of the Software and that both the above copyright notice(s) and this
+    permission notice appear in supporting documentation.
+
+    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
+    OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT OF THIRD PARTY RIGHTS.
+    IN NO EVENT SHALL THE COPYRIGHT HOLDER OR HOLDERS INCLUDED IN THIS NOTICE
+    BE LIABLE FOR ANY CLAIM, OR ANY SPECIAL INDIRECT OR CONSEQUENTIAL DAMAGES,
+    OR ANY DAMAGES WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS,
+    WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION,
+    ARISING OUT OF OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS
+    SOFTWARE.
+
+    Except as contained in this notice, the name of a copyright holder shall
+    not be used in advertising or otherwise to promote the sale, use or other
+    dealings in this Software without prior written authorization of the
+    copyright holder.
+    
+====================================================================== 
+The following components are CDDL based License 
+======================================================================
+
+For activation-1.1.jar, jaxb-api-2.1.jar, mail-1.4.jar, junit, 
+Servlet Specification 2.5 API (servlet-api-2.5-6.1.14.jar),
+Classfish Jasper API (jsp-api-2.1-6.1.14.jar), and
+JSP2.1 Jasper implementation from Glassfish (jsp-2.1-6.1.14.jar), 
+Jersey from Glassfish (jersey-client-1.13.jar, jersey-core-1.13.jar,
+jersey-json-1.13.jar, jersey-multipart-1.13.jar) and JSP2.1 Jasper 
+implementation from Glassfish (jsp-2.1-6.1.14.jar),whirr-core-0.7.1.jar, whirr-hadoop-0.7.1.jar:
+ 
+NOTE: jersey is dual licensed (http://jersey.java.net/CDDL+GPL.html), 
+Apahce Airavata elects to include jersey in this distribution under the
+[CDDLv_1.0] license.
+
+    COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0
+
+    1. Definitions.
+
+    1.1. Contributor means each individual or entity that creates or
+    contributes to the creation of Modifications.
+
+    1.2. Contributor Version means the combination of the Original Software,
+    prior Modifications used by a Contributor (if any), and the Modifications
+    made by that particular Contributor.
+
+    1.3. Covered Software means (a) the Original Software, or
+    (b) Modifications, or (c) the combination of files containing Original
+    Software with files containing Modifications, in each case including
+    portions thereof.
+
+    1.4. Executable means the Covered Software in any form other than Source
+    Code.
+
+    1.5. Initial Developer means the individual or entity that first makes
+    Original Software available under this License.
+
+    1.6. Larger Work means a work which combines Covered Software or portions
+    thereof with code not governed by the terms of this License.
+
+    1.7. License means this document.
+
+    1.8. Licensable means having the right to grant, to the maximum extent
+    possible, whether at the time of the initial grant or subsequently
+    acquired, any and all of the rights conveyed herein.
+
+    1.9. Modifications means the Source Code and Executable form of any of
+    the following: A. Any file that results from an addition to, deletion
+    from or modification of the contents of a file containing Original
+    Software or previous Modifications; B. Any new file that contains any
+    part of the Original Software or previous Modification; or C. Any new
+    file that is contributed or otherwise made available under the terms of
+    this License.
+
+    1.10. Original Software means the Source Code and Executable form of
+    computer software code that is originally released under this License.
+
+    1.11. Patent Claims means any patent claim(s), now owned or hereafter
+    acquired, including without limitation, method, process, and apparatus
+    claims, in any patent Licensable by grantor.
+
+    1.12. Source Code means (a) the common form of computer software code in
+    which modifications are made and (b) associated documentation included in
+    or with such code.
+
+    1.13. You (or Your) means an individual or a legal entity exercising
+    rights under, and complying with all of the terms of, this License. For
+    legal entities, You includes any entity which controls, is controlled by,
+    or is under common control with You. For purposes of this definition,
+    control means (a) the power, direct or indirect, to cause the direction
+    or management of such entity, whether by contract or otherwise, or
+    (b) ownership of more than fifty percent (50%) of the outstanding shares
+    or beneficial ownership of such entity.
+
+    2. License Grants.
+
+    2.1. The Initial Developer Grant. Conditioned upon Your compliance with
+    Section 3.1 below and subject to third party intellectual property
+    claims, the Initial Developer hereby grants You a world-wide,
+    royalty-free, non-exclusive license:
+
+    (a) under intellectual property rights (other than patent or trademark)
+        Licensable by Initial Developer, to use, reproduce, modify, display,
+        perform, sublicense and distribute the Original Software (or portions
+        thereof), with or without Modifications, and/or as part of a Larger
+        Work; and
+
+    (b) under Patent Claims infringed by the making, using or selling of
+        Original Software, to make, have made, use, practice, sell, and offer
+        for sale, and/or otherwise dispose of the Original Software (or
+        portions thereof);
+
+    (c) The licenses granted in Sections 2.1(a) and (b) are effective on the
+        date Initial Developer first distributes or otherwise makes the
+        Original Software available to a third party under the terms of
+        this License;
+
+    (d) Notwithstanding Section 2.1(b) above, no patent license is granted:
+        (1) for code that You delete from the Original Software, or (2) for
+        infringements caused by: (i) the modification of the Original
+        Software, or (ii) the combination of the Original Software with other
+        software or devices.
+
+    2.2. Contributor Grant. Conditioned upon Your compliance with Section 3.1
+    below and subject to third party intellectual property claims, each
+    Contributor hereby grants You a world-wide, royalty-free, non-exclusive
+    license:
+
+    (a) under intellectual property rights (other than patent or trademark)
+        Licensable by Contributor to use, reproduce, modify, display, perform,
+        sublicense and distribute the Modifications created by such
+        Contributor (or portions thereof), either on an unmodified basis,
+        with other Modifications, as Covered Software and/or as part of a
+        Larger Work; and
+
+    (b) under Patent Claims infringed by the making, using, or selling of
+        Modifications made by that Contributor either alone and/or in
+        combination with its Contributor Version (or portions of such
+        combination), to make, use, sell, offer for sale, have made, and/or
+        otherwise dispose of: (1) Modifications made by that Contributor (or
+        portions thereof); and (2) the combination of Modifications made by
+        that Contributor with its Contributor Version (or portions of such
+        combination).
+
+    (c) The licenses granted in Sections 2.2(a) and 2.2(b) are effective on
+        the date Contributor first distributes or otherwise makes the
+        Modifications available to a third party.
+
+    (d) Notwithstanding Section 2.2(b) above, no patent license is granted:
+        (1) for any code that Contributor has deleted from the Contributor
+        Version; (2) for infringements caused by: (i) third party
+        modifications of Contributor Version, or (ii) the combination of
+        Modifications made by that Contributor with other software (except
+        as part of the Contributor Version) or other devices; or (3) under
+        Patent Claims infringed by Covered Software in the absence of
+        Modifications made by that Contributor.
+
+    3. Distribution Obligations.
+
+    3.1. Availability of Source Code. Any Covered Software that You distribute
+    or otherwise make available in Executable form must also be made available
+    in Source Code form and that Source Code form must be distributed only
+    under the terms of this License. You must include a copy of this License
+    with every copy of the Source Code form of the Covered Software You
+    distribute or otherwise make available. You must inform recipients of any
+    such Covered Software in Executable form as to how they can obtain such
+    Covered Software in Source Code form in a reasonable manner on or through
+    a medium customarily used for software exchange.
+
+    3.2. Modifications. The Modifications that You create or to which You
+    contribute are governed by the terms of this License. You represent that
+    You believe Your Modifications are Your original creation(s) and/or You
+    have sufficient rights to grant the rights conveyed by this License.
+
+    3.3. Required Notices. You must include a notice in each of Your
+    Modifications that identifies You as the Contributor of the Modification.
+    You may not remove or alter any copyright, patent or trademark notices
+    contained within the Covered Software, or any notices of licensing or any
+    descriptive text giving attribution to any Contributor or the Initial
+    Developer.
+
+    3.4. Application of Additional Terms. You may not offer or impose any
+    terms on any Covered Software in Source Code form that alters or restricts
+    the applicable version of this License or the recipients rights hereunder.
+    You may choose to offer, and to charge a fee for, warranty, support,
+    indemnity or liability obligations to one or more recipients of Covered
+    Software. However, you may do so only on Your own behalf, and not on
+    behalf of the Initial Developer or any Contributor. You must make it
+    absolutely clear that any such warranty, support, indemnity or liability
+    obligation is offered by You alone, and You hereby agree to indemnify the
+    Initial Developer and every Contributor for any liability incurred by the
+    Initial Developer or such Contributor as a result of warranty, support,
+    indemnity or liability terms You offer.
+
+    3.5. Distribution of Executable Versions. You may distribute the
+    Executable form of the Covered Software under the terms of this License or
+    under the terms of a license of Your choice, which may contain terms
+    different from this License, provided that You are in compliance with the
+    terms of this License and that the license for the Executable form does
+    not attempt to limit or alter the recipients rights in the Source Code
+    form from the rights set forth in this License. If You distribute the
+    Covered Software in Executable form under a different license, You must
+    make it absolutely clear that any terms which differ from this License
+    are offered by You alone, not by the Initial Developer or Contributor.
+    You hereby agree to indemnify the Initial Developer and every Contributor
+    for any liability incurred by the Initial Developer or such Contributor as
+    a result of any such terms You offer.
+
+    3.6. Larger Works. You may create a Larger Work by combining Covered
+    Software with other code not governed by the terms of this License and
+    distribute the Larger Work as a single product. In such a case, You must
+    make sure the requirements of this License are fulfilled for the Covered
+    Software.
+
+    4. Versions of the License.
+
+    4.1. New Versions. Sun Microsystems, Inc. is the initial license steward
+    and may publish revised and/or new versions of this License from time to
+    time. Each version will be given a distinguishing version number. Except
+    as provided in Section 4.3, no one other than the license steward has the
+    right to modify this License.
+
+    4.2. Effect of New Versions. You may always continue to use, distribute
+    or otherwise make the Covered Software available under the terms of the
+    version of the License under which You originally received the Covered
+    Software. If the Initial Developer includes a notice in the Original
+    Software prohibiting it from being distributed or otherwise made
+    available under any subsequent version of the License, You must
+    distribute and make the Covered Software available under the terms of
+    the version of the License under which You originally received the
+    Covered Software. Otherwise, You may also choose to use, distribute or
+    otherwise make the Covered Software available under the terms of any
+    subsequent version of the License published by the license steward.
+
+    4.3. Modified Versions. When You are an Initial Developer and You want
+    to create a new license for Your Original Software, You may create and
+    use a modified version of this License if You: (a) rename the license and
+    remove any references to the name of the license steward (except to note
+    that the license differs from this License); and (b) otherwise make it
+    clear that the license contains terms which differ from this License.
+
+    5. DISCLAIMER OF WARRANTY. COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE
+    ON AN AS IS BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR
+    IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED
+    SOFTWARE IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE
+    OR NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF
+    THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE
+    DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER
+    CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR
+    CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF
+    THIS LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER
+    EXCEPT UNDER THIS DISCLAIMER.
+
+    6. TERMINATION.
+
+    6.1. This License and the rights granted hereunder will terminate
+    automatically if You fail to comply with terms herein and fail to cure
+    such breach within 30 days of becoming aware of the breach. Provisions
+    which, by their nature, must remain in effect beyond the termination of
+    this License shall survive.
+
+    6.2. If You assert a patent infringement claim (excluding declaratory
+    judgment actions) against Initial Developer or a Contributor (the Initial
+    Developer or Contributor against whom You assert such claim is referred
+    to as Participant) alleging that the Participant Software (meaning the
+    Contributor Version where the Participant is a Contributor or the
+    Original Software where the Participant is the Initial Developer)
+    directly or indirectly infringes any patent, then any and all rights
+    granted directly or indirectly to You by such Participant, the Initial
+    Developer (if the Initial Developer is not the Participant) and all
+    Contributors under Sections 2.1 and/or 2.2 of this License shall, upon
+    60 days notice from Participant terminate prospectively and automatically
+    at the expiration of such 60 day notice period, unless if within such
+    60 day period You withdraw Your claim with respect to the Participant
+    Software against such Participant either unilaterally or pursuant to a
+    written agreement with Participant.
+
+    6.3. In the event of termination under Sections 6.1 or 6.2 above, all end
+    user licenses that have been validly granted by You or any distributor
+    hereunder prior to termination (excluding licenses granted to You by any
+    distributor) shall survive termination.
+
+    7. LIMITATION OF LIABILITY. UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL
+    THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL
+    YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF
+    COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY
+    PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF
+    ANY CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOST PROFITS,
+    LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY
+    AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE
+    BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF
+    LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY
+    RESULTING FROM SUCH PARTYS NEGLIGENCE TO THE EXTENT APPLICABLE LAW
+    PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION
+    OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION
+    AND LIMITATION MAY NOT APPLY TO YOU.
+
+    8. U.S. GOVERNMENT END USERS. The Covered Software is a commercial item,
+    as that term is defined in 48 C.F.R. 2.101 (Oct. 1995), consisting of
+    commercial computer software (as that term is defined at 48 C.F.R.
+    252.227-7014(a)(1)) and commercial computer software documentation as such
+    terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent with 48 C.F.R.
+    12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 (June 1995), all U.S.
+    Government End Users acquire Covered Software with only those rights set
+    forth herein. This U.S. Government Rights clause is in lieu of, and
+    supersedes, any other FAR, DFAR, or other clause or provision that
+    addresses Government rights in computer software under this License.
+
+    9. MISCELLANEOUS. This License represents the complete agreement
+    concerning subject matter hereof. If any provision of this License is
+    held to be unenforceable, such provision shall be reformed only to the
+    extent necessary to make it enforceable. This License shall be governed
+    by the law of the jurisdiction specified in a notice contained within
+    the Original Software (except to the extent applicable law, if any,
+    provides otherwise), excluding such jurisdictions conflict-of-law
+    provisions. Any litigation relating to this License shall be subject to
+    the jurisdiction of the courts located in the jurisdiction and venue
+    specified in a notice contained within the Original Software, with the
+    losing party responsible for costs, including, without limitation, court
+    costs and reasonable attorneys fees and expenses. The application of the
+    United Nations Convention on Contracts for the International Sale of
+    Goods is expressly excluded. Any law or regulation which provides that
+    the language of a contract shall be construed against the drafter shall
+    not apply to this License. You agree that You alone are responsible for
+    compliance with the United States export administration regulations (and
+    the export control laws and regulation of any other countries) when You
+    use, distribute or otherwise make available any Covered Software.
+
+    10. RESPONSIBILITY FOR CLAIMS. As between Initial Developer and the
+    Contributors, each party is responsible for claims and damages arising,
+    directly or indirectly, out of its utilization of rights under this
+    License and You agree to work with Initial Developer and Contributors
+    to distribute such responsibility on an equitable basis. Nothing herein
+    is intended or shall be deemed to constitute any admission of liability.
+
+    NOTICE PURSUANT TO SECTION 9 OF THE COMMON DEVELOPMENT AND DISTRIBUTION
+    LICENSE (CDDL) The code released under the CDDL shall be governed by the
+    laws of the State of California (excluding conflict-of-law provisions).
+    Any litigation relating to this License shall be subject to the
+    jurisdiction of the Federal Courts of the Northern District of California
+    and the state courts of the State of California, with venue lying in
+    Santa Clara County, California.
+
+
+==============================================================================
+
+For: jaxb-xjc-2.1.7.jar
+    Containing Project URL: 
+
+Copyright (c) 2004 Kohsuke Kawaguchi
+
+Permission is hereby granted, free of charge, to any person
+obtaining a copy of this software and associated documentation
+files (the "Software"), to deal in the Software without
+restriction, including without limitation the rights to use,
+copy, modify, merge, publish, distribute, sublicense, and/or
+sell copies of the Software, and to permit persons to whom
+the Software is furnished to do so, subject to the following
+conditions:
+
+The above copyright notice and this permission notice shall
+be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY
+KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE
+WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR
+PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS
+OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR
+OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR
+OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE
+SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+=============================================================================== 
+The following components are BSD Licensed 
+=============================================================================== 
+
+For jibx-bind-1.2.1.jar,jibx-run-1.2.1.jar, antlr-2.7.7.jar,hamcrest-all-1.1.jar,whirr-core-0.7.1.jar, whirr-hadoop-0.7.1.jar:
+    Containing Project URL: http://jibx.sourceforge.net, http://www.antlr.org/
+
+Copyright (c) 2003-2007, Dennis M. Sosnoski
+All rights reserved.
+
+Copyright (c) 2010 Terence Parr
+All rights reserved.
+
+[The BSD License]
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+ * Redistributions of source code must retain the above copyright notice, this
+   list of conditions and the following disclaimer.
+ * Redistributions in binary form must reproduce the above copyright notice,
+   this list of conditions and the following disclaimer in the documentation
+   and/or other materials provided with the distribution.
+ * Neither the name of JiBX nor the names of its contributors may be used
+   to endorse or promote products derived from this software without specific
+   prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+==============================================================================
+
+For YFilter:
+    Containing Project URL: http://yfilter.cs.umass.edu/
+
+YFilter 1.0 COPYRIGHT, LICENSE and DISCLAIMER
+
+Copyright (c) 2002, 2004, Regents of the University of California All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification, are
+permitted provided that the following conditions are met:
+
+    * Redistributions of source code must retain the above copyright notice, this
+    list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above copyright notice, this
+    list of conditions and the following disclaimer in the documentation and/or other
+    materials provided with the distribution.
+    * Neither the name of the University of California at Berkeley nor the names of
+    its contributors may be used to endorse or promote products derived from this
+    software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY
+EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
+OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT
+SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT
+OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION)
+HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY,
+OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+==========================================================================================
+For jaxen-1.1.1.jar:
+    Containing Project URL: http://jaxen.codehaus.org/
+
+ Copyright 2003-2006 The Werken Company. All Rights Reserved.
+
+ Redistribution and use in source and binary forms, with or without
+ modification, are permitted provided that the following conditions are
+ met:
+
+  * Redistributions of source code must retain the above copyright
+    notice, this list of conditions and the following disclaimer.
+
+  * Redistributions in binary form must reproduce the above copyright
+    notice, this list of conditions and the following disclaimer in the
+    documentation and/or other materials provided with the distribution.
+
+  * Neither the name of the Jaxen Project nor the names of its
+    contributors may be used to endorse or promote products derived
+    from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS
+IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED
+TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A
+PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER
+OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
+EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
+PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
+PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
+LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+=============================================================================== 
+The following components are CPL Licensed 
+=============================================================================== 
+
+For wsdl4j-1.6.2.jar:
+    Containing Project URL: http://sourceforge.net/projects/wsdl4j/
+
+Common Public License Version 1.0
+
+THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS COMMON PUBLIC\u2028LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION OF THE PROGRAM\u2028CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT.
+1. DEFINITIONS
+"Contribution" means:
+a) in the case of the initial Contributor, the initial code and\u2028documentation distributed under this Agreement, and
+b) in the case of each subsequent Contributor:
+i) changes to the Program, and
+ii) additions to the Program;
+where such changes and/or additions to the Program originate from and are\u2028distributed by that particular Contributor. A Contribution 'originates' from a\u2028Contributor if it was added to the Program by such Contributor itself or anyone\u2028acting on such Contributor's behalf. Contributions do not include additions to\u2028the Program which: (i) are separate modules of software distributed in\u2028conjunction with the Program under their own license agreement, and (ii) are not\u2028derivative works of the Program.
+"Contributor" means any person or entity that distributes the Program.
+"Licensed Patents " mean patent claims licensable by a Contributor which are\u2028necessarily infringed by the use or sale of its Contribution alone or when\u2028combined with the Program.
+"Program" means the Contributions distributed in accordance with this Agreement.
+"Recipient" means anyone who receives the Program under this Agreement,\u2028including all Contributors.
+2. GRANT OF RIGHTS
+a) Subject to the terms of this Agreement, each Contributor hereby grants\u2028Recipient a non-exclusive, worldwide, royalty-free copyright license to\u2028reproduce, prepare derivative works of, publicly display, publicly perform,\u2028distribute and sublicense the Contribution of such Contributor, if any, and such\u2028derivative works, in source code and object code form.
+b) Subject to the terms of this Agreement, each Contributor hereby grants\u2028Recipient a non-exclusive, worldwide, royalty-free patent license under Licensed\u2028Patents to make, use, sell, offer to sell, import and otherwise transfer the\u2028Contribution of such Contributor, if any, in source code and object code form.\u2028This patent license shall apply to the combination of the Contribution and the\u2028Program if, at the time the Contribution is added by the Contributor, such\u2028addition of the Contribution causes such combination to be covered by the\u2028Licensed Patents. The patent license shall not apply to any other combinations\u2028which include the Contribution. No hardware per se is licensed hereunder.
+c) Recipient understands that although each Contributor grants the licenses\u2028to its Contributions set forth herein, no assurances are provided by any\u2028Contributor that the Program does not infringe the patent or other intellectual\u2028property rights of any other entity. Each Contributor disclaims any liability to\u2028Recipient for claims brought by any other entity based on infringement of\u2028intellectual property rights or otherwise. As a condition to exercising the\u2028rights and licenses granted hereunder, each Recipient hereby assumes sole\u2028responsibility to secure any other intellectual property rights needed, if any.\u2028For example, if a third party patent license is required to allow Recipient to\u2028distribute the Program, it is Recipient's responsibility to acquire that license\u2028before distributing the Program.
+d) Each Contributor represents that to its knowledge it has sufficient\u2028copyright rights in its Contribution, if any, to grant the copyright license set\u2028forth in this Agreement.
+3. REQUIREMENTS
+A Contributor may choose to distribute the Program in object code form under its\u2028own license agreement, provided that:
+a) it complies with the terms and conditions of this Agreement; and
+b) its license agreement:
+i) effectively disclaims on behalf of all Contributors all warranties and\u2028conditions, express and implied, including warranties or conditions of title and\u2028non-infringement, and implied warranties or conditions of merchantability and\u2028fitness for a particular purpose;
+ii) effectively excludes on behalf of all Contributors all liability for\u2028damages, including direct, indirect, special, incidental and consequential\u2028damages, such as lost profits;
+iii) states that any provisions which differ from this Agreement are offered\u2028by that Contributor alone and not by any other party; and
+iv) states that source code for the Program is available from such\u2028Contributor, and informs licensees how to obtain it in a reasonable manner on or\u2028through a medium customarily used for software exchange.
+When the Program is made available in source code form:
+a) it must be made available under this Agreement; and
+b) a copy of this Agreement must be included with each copy of the Program.
+Contributors may not remove or alter any copyright notices contained within the\u2028Program.
+Each Contributor must identify itself as the originator of its Contribution, if\u2028any, in a manner that reasonably allows subsequent Recipients to identify the\u2028originator of the Contribution.
+4. COMMERCIAL DISTRIBUTION
+Commercial distributors of software may accept certain responsibilities with\u2028respect to end users, business partners and the like. While this license is\u2028intended to facilitate the commercial use of the Program, the Contributor who\u2028includes the Program in a commercial product offering should do so in a manner\u2028which does not create potential liability for other Contributors. Therefore, if\u2028a Contributor includes the Program in a commercial product offering, such\u2028Contributor ("Commercial Contributor") hereby agrees to defend and indemnify\u2028every other Contributor ("Indemnified Contributor") against any losses, damages\u2028and costs (collectively "Losses") arising from claims, lawsuits and other legal\u2028actions brought by a third party against the Indemnified Contributor to the\u2028extent caused by the acts or omissions of such Commercial Contributor in\u2028connection with its distribution of the Program in a commercial product\u2028offering. The obligations in this section do not ap
 ply to any claims or Losses\u2028relating to any actual or alleged intellectual property infringement. In order\u2028to qualify, an Indemnified Contributor must: a) promptly notify the Commercial\u2028Contributor in writing of such claim, and b) allow the Commercial Contributor to\u2028control, and cooperate with the Commercial Contributor in, the defense and any\u2028related settlement negotiations. The Indemnified Contributor may participate in\u2028any such claim at its own expense.
+For example, a Contributor might include the Program in a commercial product\u2028offering, Product X. That Contributor is then a Commercial Contributor. If that\u2028Commercial Contributor then makes performance claims, or offers warranties\u2028related to Product X, those performance claims and warranties are such\u2028Commercial Contributor's responsibility alone. Under this section, the\u2028Commercial Contributor would have to defend claims against the other\u2028Contributors related to those performance claims and warranties, and if a court\u2028requires any other Contributor to pay any damages as a result, the Commercial\u2028Contributor must pay those damages.
+5. NO WARRANTY
+EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED ON AN\u2028"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR\u2028IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF TITLE,\u2028NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR PURPOSE. Each\u2028Recipient is solely responsible for determining the appropriateness of using and\u2028distributing the Program and assumes all risks associated with its exercise of\u2028rights under this Agreement, including but not limited to the risks and costs of\u2028program errors, compliance with applicable laws, damage to or loss of data,\u2028programs or equipment, and unavailability or interruption of operations.
+6. DISCLAIMER OF LIABILITY
+EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR ANY\u2028CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL,\u2028SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST\u2028PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT,\u2028STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY\u2028OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS\u2028GRANTED HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
+7. GENERAL
+If any provision of this Agreement is invalid or unenforceable under applicable\u2028law, it shall not affect the validity or enforceability of the remainder of the\u2028terms of this Agreement, and without further action by the parties hereto, such\u2028provision shall be reformed to the minimum extent necessary to make such\u2028provision valid and enforceable.
+If Recipient institutes patent litigation against a Contributor with respect to\u2028a patent applicable to software (including a cross-claim or counterclaim in a\u2028lawsuit), then any patent licenses granted by that Contributor to such Recipient\u2028under this Agreement shall terminate as of the date such litigation is filed. In\u2028addition, if Recipient institutes patent litigation against any entity\u2028(including a cross-claim or counterclaim in a lawsuit) alleging that the Program\u2028itself (excluding combinations of the Program with other software or hardware)\u2028infringes such Recipient's patent(s), then such Recipient's rights granted under\u2028Section 2(b) shall terminate as of the date such litigation is filed.
+All Recipient's rights under this Agreement shall terminate if it fails to\u2028comply with any of the material terms or conditions of this Agreement and does\u2028not cure such failure in a reasonable period of time after becoming aware of\u2028such noncompliance. If all Recipient's rights under this Agreement terminate,\u2028Recipient agrees to cease use and distribution of the Program as soon as\u2028reasonably practicable. However, Recipient's obligations under this Agreement\u2028and any licenses granted by Recipient relating to the Program shall continue and\u2028survive.
+Everyone is permitted to copy and distribute copies of this Agreement, but in\u2028order to avoid inconsistency the Agreement is copyrighted and may only be\u2028modified in the following manner. The Agreement Steward reserves the right to\u2028publish new versions (including revisions) of this Agreement from time to time.\u2028No one other than the Agreement Steward has the right to modify this Agreement.\u2028IBM is the initial Agreement Steward. IBM may assign the responsibility to serve\u2028as the Agreement Steward to a suitable separate entity. Each new version of the\u2028Agreement will be given a distinguishing version number. The Program (including\u2028Contributions) may always be distributed subject to the version of the Agreement\u2028under which it was received. In addition, after a new version of the Agreement\u2028is published, Contributor may elect to distribute the Program (including its\u2028Contributions) under the new version. Except as expressly stated in Sections\u20282(a) and 2(b) above, Recipie
 nt receives no rights or licenses to the\u2028intellectual property of any Contributor under this Agreement, whether\u2028expressly, by implication, estoppel or otherwise. All rights in the Program not\u2028expressly granted under this Agreement are reserved.
+This Agreement is governed by the laws of the State of New York and the\u2028intellectual property laws of the United States of America. No party to this\u2028Agreement will bring a legal action under this Agreement more than one year\u2028after the cause of action arose. Each party waives its rights to a jury trial in\u2028any resulting litigation.
+
+==========================================================================================
+==========================================================================================
+
+For puretls:
+    Containing Project URL: 
+
+  This package is a SSLv3/TLS implementation written by Eric Rescorla
+   <ek...@rtfm.com> and licensed by Claymore Systems, Inc.
+
+   Redistribution and use in source and binary forms, with or without
+   modification, are permitted provided that the following conditions
+   are met:
+   1. Redistributions of source code must retain the above copyright
+      notice, this list of conditions and the following disclaimer.
+   2. Redistributions in binary form must reproduce the above copyright
+      notice, this list of conditions and the following disclaimer in the
+      documentation and/or other materials provided with the distribution.
+   3. Neither the name of Claymore Systems, Inc. nor the name of Eric
+      Rescorla may be used to endorse or promote products derived from this
+      software without specific prior written permission.
+   THIS SOFTWARE IS PROVIDED BY CLAYMORE SYSTEMS AND CONTRIBUTORS ``AS IS'' AND
+   ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+   IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+   ARE DISCLAIMED.  IN NO EVENT SHALL THE REGENTS OR CONTRIBUTORS BE LIABLE
+   FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
+   DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS
+   OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION)
+   HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT
+   LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY
+   OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF
+   SUCH DAMAGE.
+
+==============================================================================
+
+For xml-api,woden-api-1.0M8.jar,woden-impl-dom-1.0M8.jar:
+    Containing Project URL: 
+
+For the W3C schema and DTD files in the org.apache.woden.resolver package:
+
+W3C� DOCUMENT LICENSE
+http://www.w3.org/Consortium/Legal/2002/copyright-documents-20021231
+
+Public documents on the W3C site are provided by the copyright holders under
+the following license. By using and/or copying this document, or the W3C
+document from which this statement is linked, you (the licensee) agree that
+you have read, understood, and will comply with the following terms and
+conditions:
+
+Permission to copy, and distribute the contents of this document, or the W3C
+document from which this statement is linked, in any medium for any purpose
+and without fee or royalty is hereby granted, provided that you include the
+following on ALL copies of the document, or portions thereof, that you use:
+
+  1. A link or URL to the original W3C document.
+  2. The pre-existing copyright notice of the original author, or if it
+     doesn't exist, a notice (hypertext is preferred, but a textual
+     representation is permitted) of the form: "Copyright � [$date-of-document]
+     World Wide Web Consortium, (Massachusetts Institute of Technology,
+     European Research Consortium for Informatics and Mathematics, Keio
+     University). All Rights Reserved.
+     http://www.w3.org/Consortium/Legal/2002/copyright-documents-20021231"
+  3. If it exists, the STATUS of the W3C document.
+
+When space permits, inclusion of the full text of this NOTICE should be
+provided. We request that authorship attribution be provided in any software,
+documents, or other items or products that you create pursuant to the
+implementation of the contents of this document, or any portion thereof.
+
+No right to create modifications or derivatives of W3C documents is granted
+pursuant to this license. However, if additional requirements (documented in
+the Copyright FAQ) are satisfied, the right to create modifications or
+derivatives is sometimes granted by the W3C to individuals complying with
+those requirements.
+
+THIS DOCUMENT IS PROVIDED "AS IS," AND COPYRIGHT HOLDERS MAKE NO
+REPRESENTATIONS OR WARRANTIES, EXPRESS OR IMPLIED, INCLUDING, BUT NOT
+LIMITED TO, WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE,
+NON-INFRINGEMENT, OR TITLE; THAT THE CONTENTS OF THE DOCUMENT ARE SUITABLE
+FOR ANY PURPOSE; NOR THAT THE IMPLEMENTATION OF SUCH CONTENTS WILL NOT
+INFRINGE ANY THIRD PARTY PATENTS, COPYRIGHTS, TRADEMARKS OR OTHER RIGHTS.
+
+COPYRIGHT HOLDERS WILL NOT BE LIABLE FOR ANY DIRECT, INDIRECT, SPECIAL OR
+CONSEQUENTIAL DAMAGES ARISING OUT OF ANY USE OF THE DOCUMENT OR THE
+PERFORMANCE OR IMPLEMENTATION OF THE CONTENTS THEREOF.
+
+The name and trademarks of copyright holders may NOT be used in advertising
+or publicity pertaining to this document or its contents without specific,
+written prior permission. Title to copyright in this document will at all
+times remain with copyright holders.
+
+This formulation of W3C's notice and license became active on December 31 2002. 
+This version removes the copyright ownership notice such that this license can 
+be used with materials other than those owned by the W3C, reflects that ERCIM is 
+now a host of the W3C, includes references to this specific dated version of the 
+license, and removes the ambiguous grant of "use". Otherwise, this version is the 
+same as the previous version and is written so as to preserve the Free Software 
+Foundation's assessment of GPL compatibility and OSI's certification under the 
+Open Source Definition. Please see our Copyright FAQ for common questions about 
+using materials from our site, including specific terms and conditions for packages 
+like libwww, Amaya, and Jigsaw. Other questions about this notice can be directed 
+o site-policy@w3.org.
+
+Joseph Reagle <si...@w3.org>
+ 
+Last revised $Id: copyright-software-20021231.html,v 1.11 2004/07/06 16:02:49 slesch Exp $ 
+
+==========================================================================================
+
+XML API library, org.w3c classes (xml-apis)
+    Containing Project URL: 
+
+    DOM Java Language Binding:
+    http://www.w3.org/TR/2000/REC-DOM-Level-2-Core-20001113/java-binding.html
+
+    W3C IPR SOFTWARE NOTICE
+    Copyright (C) 2000 World Wide Web Consortium, (Massachusetts Institute of
+    Technology, Institut National de Recherche en Informatique et en
+    Automatique, Keio University). All Rights Reserved.
+
+    The DOM bindings are published under the W3C Software Copyright Notice
+    and License. The software license requires "Notice of any changes or
+    modifications to the W3C files, including the date changes were made."
+    Consequently, modified versions of the DOM bindings must document that
+    they do not conform to the W3C standard; in the case of the IDL binding,
+    the pragma prefix can no longer be 'w3c.org'; in the case of the Java
+    binding, the package names can no longer be in the 'org.w3c' package.
+
+    Note: The original version of the W3C Software Copyright Notice and
+    License could be found at
+    http://www.w3.org/Consortium/Legal/copyright-software-19980720
+
+    Copyright (C) 1994-2000 World Wide Web Consortium, (Massachusetts
+    Institute of Technology, Institut National de Recherche en Informatique
+    et en Automatique, Keio University). All Rights Reserved.
+    http://www.w3.org/Consortium/Legal/
+
+    This W3C work (including software, documents, or other related items) is
+    being provided by the copyright holders under the following license. By
+    obtaining, using and/or copying this work, you (the licensee) agree that
+    you have read, understood, and will comply with the following terms and
+    conditions:
+
+    Permission to use, copy, and modify this software and its documentation,
+    with or without modification, for any purpose and without fee or royalty
+    is hereby granted, provided that you include the following on ALL copies
+    of the software and documentation or portions thereof, including
+    modifications, that you make:
+
+      1. The full text of this NOTICE in a location viewable to users of the
+         redistributed or derivative work.
+
+      2. Any pre-existing intellectual property disclaimers, notices, or
+         terms and conditions. If none exist, a short notice of the following
+         form (hypertext is preferred, text is permitted) should be used
+         within the body of any redistributed or derivative code:
+         "Copyright (C) [$date-of-software] World Wide Web Consortium,
+         (Massachusetts Institute of Technology, Institut National de
+         Recherche en Informatique et en Automatique, Keio University).
+         All Rights Reserved. http://www.w3.org/Consortium/Legal/"
+
+      3. Notice of any changes or modifications to the W3C files, including
+         the date changes were made. (We recommend you provide URIs to the
+         location from which the code is derived.)
+
+    THIS SOFTWARE AND DOCUMENTATION IS PROVIDED "AS IS," AND COPYRIGHT HOLDERS
+    MAKE NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR IMPLIED, INCLUDING BUT
+    NOT LIMITED TO, WARRANTIES OF MERCHANTABILITY OR FITNESS FOR ANY PARTICULAR
+    PURPOSE OR THAT THE USE OF THE SOFTWARE OR DOCUMENTATION WILL NOT INFRINGE
+    ANY THIRD PARTY PATENTS, COPYRIGHTS, TRADEMARKS OR OTHER RIGHTS.
+
+    COPYRIGHT HOLDERS WILL NOT BE LIABLE FOR ANY DIRECT, INDIRECT, SPECIAL
+    OR CONSEQUENTIAL DAMAGES ARISING OUT OF ANY USE OF THE SOFTWARE OR
+    DOCUMENTATION.
+
+    The name and trademarks of copyright holders may NOT be used in
+    advertising or publicity pertaining to the software without specific,
+    written prior permission. Title to copyright in this software and any
+    associated documentation will at all times remain with copyright holders.
+
+=============================================================================== 
+The following components come under the Eclipse Public 1.0 License 
+=============================================================================== 
+Eclipse JDT Core (core-3.1.1.jar)
+
+-AspectJ runtime (http://www.aspectj.org) org.aspectj:aspectjrt:jar:1.6.12
+    License: Eclipse Public License - v 1.0  (http://www.eclipse.org/legal/epl-v10.html)
+
+  Eclipse Public License - v 1.0
+
+    THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE
+    PUBLIC LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION OF
+    THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT.
+
+    1. DEFINITIONS
+
+    "Contribution" means:
+
+    a) in the case of the initial Contributor, the initial code and
+       documentation distributed under this Agreement, and
+
+    b) in the case of each subsequent Contributor:
+
+       i) changes to the Program, and
+
+       ii) additions to the Program;
+
+       where such changes and/or additions to the Program originate from and
+       are distributed by that particular Contributor. A Contribution
+       'originates' from a Contributor if it was added to the Program by
+       such Contributor itself or anyone acting on such Contributor's behalf.
+       Contributions do not include additions to the Program which: (i) are
+       separate modules of software distributed in conjunction with the
+       Program under their own license agreement, and (ii) are not derivative
+       works of the Program.
+
+    "Contributor" means any person or entity that distributes the Program.
+
+    "Licensed Patents " mean patent claims licensable by a Contributor which
+    are necessarily infringed by the use or sale of its Contribution alone or
+    when combined with the Program.
+
+    "Program" means the Contributions distributed in accordance with this
+    Agreement.
+
+    "Recipient" means anyone who receives the Program under this Agreement,
+    including all Contributors.
+
+    2. GRANT OF RIGHTS
+
+    a) Subject to the terms of this Agreement, each Contributor hereby grants
+       Recipient a non-exclusive, worldwide, royalty-free copyright license to
+       reproduce, prepare derivative works of, publicly display, publicly
+       perform, distribute and sublicense the Contribution of such
+       Contributor, if any, and such derivative works, in source code and
+       object code form.
+
+    b) Subject to the terms of this Agreement, each Contributor hereby grants
+       Recipient a non-exclusive, worldwide, royalty-free patent license under
+       Licensed Patents to make, use, sell, offer to sell, import and
+       otherwise transfer the Contribution of such Contributor, if any, in
+       source code and object code form. This patent license shall apply to
+       the combination of the Contribution and the Program if, at the time
+       the Contribution is added by the Contributor, such addition of the
+       Contribution causes such combination to be covered by the Licensed
+       Patents. The patent license shall not apply to any other combinations
+       which include the Contribution. No hardware per se is licensed hereunder.
+
+    c) Recipient understands that although each Contributor grants the
+       licenses to its Contributions set forth herein, no assurances are
+       provided by any Contributor that the Program does not infringe the
+       patent or other intellectual property rights of any other entity. Each
+       Contributor disclaims any liability to Recipient for claims brought by
+       any other entity based on infringement of intellectual property rights
+       or otherwise. As a condition to exercising the rights and licenses
+       granted hereunder, each Recipient hereby assumes sole responsibility
+       to secure any other intellectual property rights needed, if any. For
+       example, if a third party patent license is required to allow Recipient
+       to distribute the Program, it is Recipient's responsibility to acquire
+       that license before distributing the Program.
+
+    d) Each Contributor represents that to its knowledge it has sufficient
+       copyright rights in its Contribution, if any, to grant the copyright
+       license set forth in this Agreement.
+
+    3. REQUIREMENTS
+
+    A Contributor may choose to distribute the Program in object code form
+    under its own license agreement, provided that:
+
+    a) it complies with the terms and conditions of this Agreement; and
+
+    b) its license agreement:
+
+       i)   effectively disclaims on behalf of all Contributors all warranties
+            and conditions, express and implied, including warranties or
+            conditions of title and non-infringement, and implied warranties
+            or conditions of merchantability and fitness for a particular
+            purpose;
+
+       ii)  effectively excludes on behalf of all Contributors all liability
+            for damages, including direct, indirect, special, incidental and
+            consequential damages, such as lost profits;
+
+       iii) states that any provisions which differ from this Agreement are
+            offered by that Contributor alone and not by any other party; and
+
+       iv)  states that source code for the Program is available from such
+            Contributor, and informs licensees how to obtain it in a
+            reasonable manner on or through a medium customarily used for
+            software exchange.
+
+    When the Program is made available in source code form:
+
+    a) it must be made available under this Agreement; and
+
+    b) a copy of this Agreement must be included with each copy of the
+       Program.
+
+    Contributors may not remove or alter any copyright notices contained
+    within the Program.
+
+    Each Contributor must identify itself as the originator of its
+    Contribution, if any, in a manner that reasonably allows subsequent
+    Recipients to identify the originator of the Contribution.
+
+    4. COMMERCIAL DISTRIBUTION
+
+    Commercial distributors of software may accept certain responsibilities
+    with respect to end users, business partners and the like. While this
+    license is intended to facilitate the commercial use of the Program,
+    the Contributor who includes the Program in a commercial product offering
+    should do so in a manner which does not create potential liability for
+    other Contributors. Therefore, if a Contributor includes the Program in
+    a commercial product offering, such Contributor ("Commercial Contributor")
+    hereby agrees to defend and indemnify every other Contributor
+    ("Indemnified Contributor") against any losses, damages and costs
+    (collectively "Losses") arising from claims, lawsuits and other legal
+    actions brought by a third party against the Indemnified Contributor to
+    the extent caused by the acts or omissions of such Commercial Contributor
+    in connection with its distribution of the Program in a commercial
+    product offering. The obligations in this section do not apply to any
+    claims or Losses relating to any actual or alleged intellectual property
+    infringement. In order to qualify, an Indemnified Contributor must:
+    a) promptly notify the Commercial Contributor in writing of such claim,
+    and b) allow the Commercial Contributor to control, and cooperate with
+    the Commercial Contributor in, the defense and any related settlement
+    negotiations. The Indemnified Contributor may participate in any such
+    claim at its own expense.
+
+    For example, a Contributor might include the Program in a commercial
+    product offering, Product X. That Contributor is then a Commercial
+    Contributor. If that Commercial Contributor then makes performance claims,
+    or offers warranties related to Product X, those performance claims and
+    warranties are such Commercial Contributor's responsibility alone. Under
+    this section, the Commercial Contributor would have to defend claims
+    against the other Contributors related to those performance claims and
+    warranties, and if a court requires any other Contributor to pay any
+    damages as a result, the Commercial Contributor must pay those damages.
+
+    5. NO WARRANTY
+
+    EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED
+    ON AN "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER
+    EXPRESS OR IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR
+    CONDITIONS OF TITLE, NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A
+    PARTICULAR PURPOSE. Each Recipient is solely responsible for determining
+    the appropriateness of using and distributing the Program and assumes all
+    risks associated with its exercise of rights under this Agreement ,
+    including but not limited to the risks and costs of program errors,
+    compliance with applicable laws, damage to or loss of data, programs or
+    equipment, and unavailability or interruption of operations.
+
+    6. DISCLAIMER OF LIABILITY
+
+    EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR
+    ANY CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT,
+    INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING
+    WITHOUT LIMITATION LOST PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF
+    LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+    NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OR
+    DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS GRANTED
+    HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
+
+    7. GENERAL
+
+    If any provision of this Agreement is invalid or unenforceable under
+    applicable law, it shall not affect the validity or enforceability of
+    the remainder of the terms of this Agreement, and without further action
+    by the parties hereto, such provision shall be reformed to the minimum
+    extent necessary to make such provision valid and enforceable.
+
+    If Recipient institutes patent litigation against any entity (including
+    a cross-claim or counterclaim in a lawsuit) alleging that the Program
+    itself (excluding combinations of the Program with other software or
+    hardware) infringes such Recipient's patent(s), then such Recipient's
+    rights granted under Section 2(b) shall terminate as of the date such
+    litigation is filed.
+
+    All Recipient's rights under this Agreement shall terminate if it fails
+    to comply with any of the material terms or conditions of this Agreement
+    and does not cure such failure in a reasonable period of time after
+    becoming aware of such noncompliance. If all Recipient's rights under
+    this Agreement terminate, Recipient agrees to cease use and distribution
+    of the Program as soon as reasonably practicable. However, Recipient's
+    obligations under this Agreement and any licenses granted by Recipient
+    relating to the Program shall continue and survive.
+
+    Everyone is permitted to copy and di

<TRUNCATED>

[48/50] [abbrv] airavata git commit: [AIRAVATA-2057] Move the distribution directory to modules to slow down the distribution build

Posted by la...@apache.org.
http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/distribution/src/main/resources/NOTICE
----------------------------------------------------------------------
diff --git a/distribution/src/main/resources/NOTICE b/distribution/src/main/resources/NOTICE
deleted file mode 100644
index fa7cba5..0000000
--- a/distribution/src/main/resources/NOTICE
+++ /dev/null
@@ -1,163 +0,0 @@
-Apache Airavata
-Copyright 2014 The Apache Software Foundation
-
-This product includes software developed at
-The Apache Software Foundation (http://www.apache.org/).
-
-===============================================================================
-Apache Xerces Java Notice:
-
-   Portions of this software were originally based on the following:
-     - software copyright (c) 1999, IBM Corporation., http://www.ibm.com.
-     - software copyright (c) 1999, Sun Microsystems., http://www.sun.com.
-     - voluntary contributions made by Paul Eng on behalf of the
-       Apache Software Foundation that were originally developed at iClick, Inc.,
-       software copyright (c) 1999.
-
-================================================================================
-Apache XmlBeans Notice: 
-
-   Portions of this software were originally based on the following:
-     - software copyright (c) 2000-2003, BEA Systems, <http://www.bea.com/>.
-
-   Aside from contributions to the Apache XMLBeans project, this
-   software also includes:
-
-    - one or more source files from the Apache Xerces-J and Apache Axis
-      products, Copyright (c) 1999-2003 Apache Software Foundation
-
-    - W3C XML Schema documents Copyright 2001-2003 (c) World Wide Web
-      Consortium (Massachusetts Institute of Technology, European Research
-      Consortium for Informatics and Mathematics, Keio University)
-
-    - resolver.jar from Apache Xml Commons project,
-      Copyright (c) 2001-2003 Apache Software Foundation
-
-    - Piccolo XML Parser for Java from http://piccolo.sourceforge.net/,
-      Copyright 2002 Yuval Oren under the terms of the Apache Software License 2.0
-
-    - JSR-173 Streaming API for XML from http://sourceforge.net/projects/xmlpullparser/,
-      Copyright 2005 BEA under the terms of the Apache Software License 2.0
-      
-=========================================================================================
-Apache Axis2 Notice: (axis2-*, mex-1.5.1-impl)
-
-Portions Copyright 2006 International Business Machines Corp.
-Portions Copyright 2005-2007 WSO2, Inc.
-
-This product also includes schemas and specification developed by:
-- the W3C consortium (http://www.w3c.org)
-
-This product also includes WS-* schemas developed by International
-Business Machines Corporation, Microsoft Corporation, BEA Systems, 
-TIBCO Software, SAP AG, Sonic Software, and VeriSign
-
-This product also includes a WSDL developed by salesforce.com
-- Copyright 1999-2006 salesforce.com, inc.
-Portions of the included xmlbeans library were originally based on the following:
-- software copyright (c) 2000-2003, BEA Systems, <http://www.bea.com/>.
-
-====================================================================================
-Apache Derby Notice:
-
-Portions of Derby were originally developed by
-International Business Machines Corporation and are
-licensed to the Apache Software Foundation under the
-"Software Grant and Corporate Contribution License Agreement",
-informally known as the "Derby CLA".
-The following copyright notice(s) were affixed to portions of the code
-with which this file is now or was at one time distributed
-and are placed here unaltered.
-
-(C) Copyright 1997,2004 International Business Machines Corporation.  All rights reserved.
-
-(C) Copyright IBM Corp. 2003. 
-
-=======================
-
-The portion of the functionTests under 'nist' was originally 
-developed by the National Institute of Standards and Technology (NIST), 
-an agency of the United States Department of Commerce, and adapted by
-International Business Machines Corporation in accordance with the NIST
-Software Acknowledgment and Redistribution document at
-http://www.itl.nist.gov/div897/ctg/sql_form.htm
-
-========================
-
-The JDBC apis for small devices and JDBC3 (under java/stubs/jsr169 and
-java/stubs/jdbc3) were produced by trimming sources supplied by the
-Apache Harmony project. In addition, the Harmony SerialBlob and
-SerialClob implementations are used. The following notice covers the Harmony sources:
-
-Portions of Harmony were originally developed by
-Intel Corporation and are licensed to the Apache Software
-Foundation under the "Software Grant and Corporate Contribution
-License Agreement", informally known as the "Intel Harmony CLA".
-
-=============================================================================
-Apache Woden Notice:
-
-   This product also includes software developed by :
-   
-     - IBM Corporation (http://www.ibm.com),
-         WSDL4J was the initial code contribution for the Apache Woden
-         project and some of the WSDL4J design and code has been reused.
-     - The W3C Consortium (http://www.w3c.org),
-         Common W3C XML Schema and DTD files are packaged with Apache Woden.
-
-   Please read the different LICENSE files present in the root directory of
-   this distribution.
-
-=========================================================================
-Woodstox Notice: 
-
-This product includes software developed by the Woodstox Project 
-(http://woodstox.codehaus.org/)
-
-This product currently only contains code developed by authors
-of specific components, as identified by the source code files.
-
-Since product implements StAX API, it has dependencies to StAX API
-classes.
-
-For additional credits (generally to people who reported problems)
-see CREDITS file.
-
-===========================================================================
-Apache xml-commons xml-apis Notice:
-
-   Portions of this software were originally based on the following:
-     - software copyright (c) 1999, IBM Corporation., http://www.ibm.com.
-     - software copyright (c) 1999, Sun Microsystems., http://www.sun.com.
-     - software copyright (c) 2000 World Wide Web Consortium, http://www.w3.org
-
-================================================================================================
-Apache  Xalan Notice: 
-
-Portions of this software was originally based on the following:
-     - software copyright (c) 1999-2002, Lotus Development Corporation., http://www.lotus.com.
-     - software copyright (c) 2001-2002, Sun Microsystems., http://www.sun.com.
-     - software copyright (c) 2003, IBM Corporation., http://www.ibm.com.
-     - voluntary contributions made by Ovidiu Predescu (ovidiu@cup.hp.com) on behalf of the 
-       Apache Software Foundation and was originally developed at Hewlett Packard Company. 
-================================================================================================
-Apache  OpenJPA Notice: 
-
-OpenJPA includes software developed by the SERP project
-Copyright (c) 2002-2006, A. Abram White. All rights reserved.
-
-OpenJPA includes the persistence and orm schemas from the JPA specifications.
-Copyright 2005-2009 Sun Microsystems, Inc. All rights reserved.
-OpenJPA elects to include this software in this distribution under the
-CDDL license.  You can obtain a copy of the License at:
-    https://glassfish.dev.java.net/public/CDDL+GPL.html
-The source code is available at:
-    https://glassfish.dev.java.net/source/browse/glassfish/
-
-OpenJPA includes software written by Miroslav Nachev
-OpenJPA uses test code written by Charles Tillman.
-================================================================================================
-Apache XmlSchema Notice:
-
-Portions Copyright 2006 International Business Machines Corp.
-================================================================================================

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/distribution/src/main/resources/README
----------------------------------------------------------------------
diff --git a/distribution/src/main/resources/README b/distribution/src/main/resources/README
deleted file mode 100644
index c2223ff..0000000
--- a/distribution/src/main/resources/README
+++ /dev/null
@@ -1,145 +0,0 @@
-Apache Airavata Source - README.txt
-Licensed under Apache License 2.0 - http://www.apache.org/licenses/LICENSE-2.0
---------------------------------------------------------------------------------
-
-About
-=====
-Apache Airavata, a software framework to executing and managing computational jobs on 
-distributed computing resources including local clusters, supercomputers, national grids, 
-academic and commercial clouds. Airavata builds on general concepts of service oriented computing, 
-distributed messaging, and workflow composition and orchestration. Airavata bundles a server package 
-with an API, client software development Kits and a general purpose GUI XBaya as a application registration, workflow
-construction execution and monitoring. XBaya GUI also provides capabilities to access the workflow 
-produced data.  
-
-Contact
-========
-For additional information about Apache Airavata, please contact the user or dev mailing lists:
-http://airavata.apache.org/community/mailing-lists.html
-
-Description of Airavata Directory Structure
-==================================
-    - airavata-api
-      This directory contains Airavata API related data models, api methods, generated server skeletons, client stubs, server implementations and client samples. 
-
-    - modules
-      This contains the source code of all the airavata maven projects organized as libraries, services and distributions
-
-    - samples
-      This contains all the system wide samples provided in Airavata distribution. All the sample are having its README file
-      So users have to refer each readme file before running each sample.
-
-    - tools
-      This contains source code libraries that can enhance Airavata features.
-
-    - README
-      This document.
-    
-    - RELEASE_NOTES
-      The describe the key features and know issues with the current release. 
-
-    - INSTALL
-      This document will contain information on installing Apache-Airavata.
-
-Airavata Source Distribution Directory Structure
-================================================
-
-    AIRAVATA_MASTER
-		\u251c\u2500\u2500 airavata-api
-		\u251c\u2500\u2500 modules
-		\u2502   \u251c\u2500\u2500 airavata-client
-		\u2502   \u251c\u2500\u2500 app-catalog
-		\u2502   \u251c\u2500\u2500 commons
-		\u2502   \u2502   \u251c\u2500\u2500 gfac-schema
-		\u2502   \u2502   \u251c\u2500\u2500 utils
-		\u2502   \u2502   \u251c\u2500\u2500 workflow-execution-context
-		\u2502   \u2502   \u2514\u2500\u2500 workflow-tracking
-		\u2502   \u251c\u2500\u2500 credential-store-service
-		\u2502   \u251c\u2500\u2500 distribution
-		\u2502   \u2502   \u251c\u2500\u2500 api-server
-		\u2502   \u2502   \u251c\u2500\u2500 client
-		\u2502   \u2502   \u251c\u2500\u2500 gfac-server
-		\u2502   \u2502   \u251c\u2500\u2500 orchestrator-server
-		\u2502   \u2502   \u251c\u2500\u2500 server
-		\u2502   \u2502   \u2514\u2500\u2500 release
-		\u2502   \u2502   \u2514\u2500\u2500 xbaya-gui
-		\u2502   \u251c\u2500\u2500 gfac
-		\u2502   \u2502   \u251c\u2500\u2500 airavata-gfac-service
-		\u2502   \u2502   \u251c\u2500\u2500 gfac-bes
-		\u2502   \u2502   \u251c\u2500\u2500 gfac-core
-		\u2502   \u2502   \u251c\u2500\u2500 gfac-ec2
-		\u2502   \u2502   \u251c\u2500\u2500 gfac-gram
-		\u2502   \u2502   \u251c\u2500\u2500 gfac-gsissh
-		\u2502   \u2502   \u251c\u2500\u2500 gfac-hadoop
-		\u2502   \u2502   \u251c\u2500\u2500 gfac-local
-		\u2502   \u2502   \u251c\u2500\u2500 gfac-monitor
-		\u2502   \u2502   \u251c\u2500\u2500 gfac-ssh
-		\u2502   \u2502   \u251c\u2500\u2500 gfac-thrift-descriptions
-		\u2502   \u251c\u2500\u2500 integration-tests
-		\u2502   \u251c\u2500\u2500 messaging
-		\u2502   \u251c\u2500\u2500 orchestrator
-		\u2502   \u251c\u2500\u2500 registry
-		\u2502   \u2502   \u251c\u2500\u2500 airavata-jpa-registry
-		\u2502   \u2502   \u251c\u2500\u2500 registry-cpi
-		\u2502   \u251c\u2500\u2500 security
-		\u2502   \u251c\u2500\u2500 credential-store
-		\u2502   \u251c\u2500\u2500 server
-		\u2502   \u251c\u2500\u2500 test-suite
-		\u2502   \u251c\u2500\u2500 workflow-model
-		\u2502   \u2502   \u251c\u2500\u2500 workflow-engine
-		\u2502   \u2502   \u251c\u2500\u2500 workflow-model-component-node
-		\u2502   \u2502   \u2514\u2500\u2500 workflow-model-core
-		\u2502   \u251c\u2500\u2500 ws-messenger
-		\u2502   \u2502   \u251c\u2500\u2500 commons
-		\u2502   \u2502   \u251c\u2500\u2500 distribution
-		\u2502   \u2502   \u251c\u2500\u2500 messagebox
-		\u2502   \u2502   \u251c\u2500\u2500 messagebroker
-		\u2502   \u2502   \u251c\u2500\u2500 message-monitor
-		\u2502   \u2502   \u2514\u2500\u2500 samples
-		\u2502   \u2514\u2500\u2500 xbaya-gui
-		\u251c\u2500\u2500 samples
-		\u251c\u2500\u2500 tools
-		\u2502   \u251c\u2500\u2500 gsissh
-		\u2502   \u251c\u2500\u2500 gsissh-cli-tools
-		\u2502   \u251c\u2500\u2500 phoebus-integration
-		\u2502   \u2514\u2500\u2500 registry-migrate
-		\u251c\u2500\u2500 INSTALL
-		\u251c\u2500\u2500 LICENSE
-		\u251c\u2500\u2500 NOTICE
-		\u251c\u2500\u2500 README
-		\u2514\u2500\u2500 RELEASE_NOTES
-
-Available Binary Distributions
-==============================
-
-Server Distributions
---------------------
-* Airavata Server
-  The Airavata Server binary distribution allows you to run a standalone Airavata Server which includes all the airavata services 
-  shipped with a default derby database as the backend registry. For stable purposes, a mysql configuration is recommended. 
-
-* Airavata API Server
-  This is the server that contains Airavata API Server.
-
-* Airavata Orchestrator Server
-  This is the stand-alone orchestrator server
-
-* Airavata GFac Server
-  This is the standalone GFac Server
-
-Client Distributions
---------------------
-* Airavata XBaya
-  The Airavata XBaya distribution is a client GUI application with features to register applications as web services, construct workflows,
-  execute and monitor workflows and browse the generated results from the airavata registry.
-
-* Airavata Client
-  The Airavata Client distribution is a set of libraries and configurations files that allow a 3rd party application to programatically 
-  access Airavata functionality through Airavata API. 
-  
- How to test and run samples
-===========================
-* If you built Airavata from source, and if you see "BUILD SUCCESS", then the test cases should have passes.
-* The test cases are beyond unit level, they startup embedded services and run through basic workflow use cases.
-* To walk through Airavata features, follow "XBAYA Quick-Start Tutorial" tutorial at https://cwiki.apache.org/confluence/display/AIRAVATA/XBAYA+Quick-Start+Tutorial.
-* For advanced use cases, please contact mailing lists - http://airavata.apache.org/community/mailing-lists.html

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/distribution/src/main/resources/bin/airavata-server-start.sh
----------------------------------------------------------------------
diff --git a/distribution/src/main/resources/bin/airavata-server-start.sh b/distribution/src/main/resources/bin/airavata-server-start.sh
deleted file mode 100644
index f44661b..0000000
--- a/distribution/src/main/resources/bin/airavata-server-start.sh
+++ /dev/null
@@ -1,123 +0,0 @@
-#!/usr/bin/env bash
-
-# 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.
-
-. `dirname $0`/setenv.sh
-cd ${AIRAVATA_HOME}/bin
-LOGO_FILE="logo.txt"
-
-JAVA_OPTS=""
-AIRAVATA_COMMAND=""
-EXTRA_ARGS=""
-SERVERS=""
-IS_DAEMON_MODE=false
-LOGO=true
-IS_SUBSET=false
-SUBSET=""
-
-# parse command arguments
-for var in "$@"
-do
-    case ${var} in
-        -xdebug)
-        	AIRAVATA_COMMAND="${AIRAVATA_COMMAND}"
-            JAVA_OPTS="$JAVA_OPTS -Xdebug -Xnoagent -Xrunjdwp:transport=dt_socket,server=y,address=8000"
-            shift
-        ;;
-        -security)
-            JAVA_OPTS="${JAVA_OPTS} -Djava.security.manager -Djava.security.policy=${AIRAVATA_HOME}/conf/axis2.policy -Daxis2.home=${AIRAVATA_HOME}"
-            shift
-        ;;
-	    apiserver | gfac | orchestrator | credentialstore | regserver)
-	        if [ -z ${SERVERS} ] ; then
-	            SERVERS="${var}"
-	        else
-	            SERVERS="${SERVERS},${var}"
-	        fi
-            shift
-        ;;
-        all | api-orch | execution )
-            IS_SUBSET=true
-            SUBSET="${var}"
-            shift
-            ;;
-        -d)
-	        IS_DAEMON_MODE=true
-	        shift
-	        ;;
-	    -nologo)
-	        LOGO=false
-            shift
-        ;;
-        -h)
-            echo "Usage: airavata-server-start.sh [server-name/s] [command-options]"
-            echo "Server names:"
-            echo "  apiserver           Start apiserver"
-            echo "  gfac                Start gfac server"
-            echo "  orchestrator        Start orchestrator server"
-            echo "  credentialstore     Start credentialstore server"
-            echo "  regserver           Start registry server"
-            echo "  all                 Start all servers in one JVM"
-
-            echo "command options:"
-	        echo "  -d                  Start server in daemon mode"
-            echo "  -xdebug             Start Airavata Server under JPDA debugger"
-            echo "  -nologo             Do not show airavata logo"
-            echo "  -security           Enable Java 2 security"
-	        echo "  --<key>[=<value>]   Server setting(s) to override or introduce (overrides values in airavata-server.properties)"
-            echo "  -h                  Display this help and exit"
-            shift
-            exit 0
-        ;;
-	    *)
-	        EXTRA_ARGS="${EXTRA_ARGS} ${var}"
-            shift
-        ;;
-    esac
-done
-
-#Construct Airavata command arguments in proper order.
-if ${IS_SUBSET} ; then
-    AIRAVATA_COMMAND="--servers=${SUBSET} ${AIRAVATA_COMMAND} ${EXTRA_ARGS}"
-else
-    if [ -z ${SERVERS} ] ; then
-        echo "You should provide at least one server component to start the airavata server. Please use -h option to get more details."
-        exit -1
-    else
-        AIRAVATA_COMMAND="--servers=${SERVERS} ${AIRAVATA_COMMAND} ${EXTRA_ARGS}"
-    fi
-fi
-
-#print logo file
-if ${LOGO} ; then
-	if [ -e ${LOGO_FILE} ]
-	then
-		cat ${LOGO_FILE}
-	fi
-fi
-
-
-if ${IS_DAEMON_MODE} ; then
-	echo "Starting airavata server/s in daemon mode..."
-	nohup java ${JAVA_OPTS} -classpath "${AIRAVATA_CLASSPATH}" \
-    org.apache.airavata.server.ServerMain ${AIRAVATA_COMMAND} $* > /dev/null 2>&1 &
-else
-	java ${JAVA_OPTS} -classpath "${AIRAVATA_CLASSPATH}" \
-    org.apache.airavata.server.ServerMain ${AIRAVATA_COMMAND} $*
-fi
-

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/distribution/src/main/resources/bin/airavata-server-stop.sh
----------------------------------------------------------------------
diff --git a/distribution/src/main/resources/bin/airavata-server-stop.sh b/distribution/src/main/resources/bin/airavata-server-stop.sh
deleted file mode 100644
index 7f83f48..0000000
--- a/distribution/src/main/resources/bin/airavata-server-stop.sh
+++ /dev/null
@@ -1,71 +0,0 @@
-#!/usr/bin/env bash
-
-# 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.
-
-. `dirname $0`/setenv.sh
-cd ${AIRAVATA_HOME}/bin
-
-JAVA_OPTS=""
-AIRAVATA_COMMAND=""
-FORCE=false
-
-for var in "$@"
-do
-    case ${var} in
-    	-f | --force)
-	        FORCE=true
-            shift
-        ;;
-        -h)
-            echo "Usage: airavata-server-stop.sh [command-options]"
-            echo "command options:"
-	        echo "  -f , --force       Force stop all airavata servers."
-	        echo "  --<key>[=<value>]  Server setting(s) to override or introduce (overrides values in airavata-server.properties)"
-            echo "  -h                 Display this help and exit"
-            shift
-            exit 0
-        ;;
-	*)
-            shift
-    esac
-done
-
-if ${FORCE} ; then
-	for f in `find . -name "server_start_*"`; do
-	    # split file name using "_" underscore
-		f_split=(${f//_/ });
-		echo "Found process file : $f"
-		echo -n "    Sending kill signals to process ${f_split[2]}..."
-		out=`kill -9 ${f_split[2]} 2>&1`
-		if [ -z "$out" ]; then
-		    echo "done"
-		else
-		    echo "failed (REASON: $out)"
-		fi
-		echo -n "    Removing process file..."
-		out=`rm ${f} 2>&1`
-		if [ -z "$out" ]; then
-		    echo "done"
-		else
-		    echo "failed (REASON: $out)"
-		fi
-	done
-else
-    java ${JAVA_OPTS} -classpath "${AIRAVATA_CLASSPATH}" \
-    org.apache.airavata.server.ServerMain stop ${AIRAVATA_COMMAND} $*
-fi

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/distribution/src/main/resources/bin/airavata-server.bat
----------------------------------------------------------------------
diff --git a/distribution/src/main/resources/bin/airavata-server.bat b/distribution/src/main/resources/bin/airavata-server.bat
deleted file mode 100644
index be2c584..0000000
--- a/distribution/src/main/resources/bin/airavata-server.bat
+++ /dev/null
@@ -1,55 +0,0 @@
-@echo off
-rem Licensed to the Apache Software Foundation (ASF) under one
-rem or more contributor license agreements. See the NOTICE file
-rem distributed with this work for additional information
-rem regarding copyright ownership. The ASF licenses this file
-rem to you under the Apache License, Version 2.0 (the
-rem "License"); you may not use this file except in compliance
-rem with the License. You may obtain a copy of the License at
-rem
-rem http://www.apache.org/licenses/LICENSE-2.0
-rem
-rem Unless required by applicable law or agreed to in writing,
-rem software distributed under the License is distributed on an
-rem "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-rem KIND, either express or implied. See the License for the
-rem specific language governing permissions and limitations
-rem under the License.
-
-setlocal EnableDelayedExpansion
-
-call "%~dp0"setenv.bat
-
-:loop
-if ""%1""==""-xdebug"" goto xdebug
-if ""%1""==""-security"" goto security
-if ""%1""=="""" goto run
-goto help
-
-:xdebug
-set JAVA_OPTS= %JAVA_OPTS% -Xdebug -Xnoagent -Xrunjdwp:transport=dt_socket,server=y,address=8000
-shift
-goto loop
-
-:security
-set JAVA_OPTS=%JAVA_OPTS% -Djava.security.manager -Djava.security.policy=%AIRAVATA_HOME%\conf\axis2.policy -Daxis2.home=%AIRAVATA_HOME%
-shift
-goto loop
-
-:help
-echo  Usage: %0 [-options]
-echo.
-echo  where options include:
-echo   -xdebug    Start Airavata Server under JPDA debugger
-echo   -security  Enable Java 2 security
-echo   -h         Help
-goto end
-
-:run
-cd "%AIRAVATA_HOME%\bin"
-set LOGO_FILE="logo.txt"
-if exist "%LOGO_FILE%" type "%LOGO_FILE%"
-
-java %JAVA_OPTS% -classpath "%AIRAVATA_CLASSPATH%" org.apache.airavata.server.ServerMain %*
-
-:end

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/distribution/src/main/resources/bin/derby.sh
----------------------------------------------------------------------
diff --git a/distribution/src/main/resources/bin/derby.sh b/distribution/src/main/resources/bin/derby.sh
deleted file mode 100644
index 134f7b9..0000000
--- a/distribution/src/main/resources/bin/derby.sh
+++ /dev/null
@@ -1,23 +0,0 @@
-#!/bin/sh
-
-# 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.
-
-. `dirname $0`/setenv.sh
-export DERBY_HOME=$AIRAVATA_HOME/standalone-server
-cd $AIRAVATA_HOME/bin
-./startNetworkServer $*
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/distribution/src/main/resources/bin/logo.txt
----------------------------------------------------------------------
diff --git a/distribution/src/main/resources/bin/logo.txt b/distribution/src/main/resources/bin/logo.txt
deleted file mode 100644
index e886438..0000000
--- a/distribution/src/main/resources/bin/logo.txt
+++ /dev/null
@@ -1,34 +0,0 @@
-...._....................._..............._...._......................_.........
-.../ \..._.__...__._..___| |__...___...../ \..(_)_.__.__.___...____._|.|_.__._..
-../ _ \.| '_ \./ _` |/ __| '_ \./ _ \.../ _ \.| | '__/ _` \ \./ / _` | __/ _` |.
-./ ___ \| |_) | (_| | (__| |.| |  __/../ ___ \| | |.| (_| |\ V / (_| | || (_| |.
-/_/...\_\ .__/.\__,_|\___|_|.|_|\___|./_/...\_\_|_|..\__,_|.\_/.\__,_|\__\__,_|.
-........|_|.....................................................................
-................................................................................
-................,+????++~..~~++..+????????+??++.++~...~+=???+++.................
-..............:???????....:::...~??????????????.~..::...=????????...............
-............????????..~~..?????..??????????????.?????..~~~.~???????.............
-...........?????????I,~I~~??????.+????????????.~??????~=~..?????????~...........
-.........?????++??????..????????:.??????????I..????????..????????+????..........
-........??.....???????....???????...???????+..+??????+.I.????????.....?,........
-........????==????????..??..?????..=???????=..?????,.=+?.?????????===??=........
-......=??????????+????..+??=.???=.~??????????,.???=,???,.????=+??????????.......
-....??????????+...+I++..???,=...:??????????????.....+??..++I?+..,??????????.....
-....???????=??..........??..+??.:=:.???????......??..??=..........?=???????=....
-....??????..?...........+?..???.....???????......???.??...........~=.??????=....
-....~???~.~..............?..???.~=..,??????...7..???.?.:..............~????:....
-...7....7 ...............?..????. ...??????... .????.?.?.............I ..:.:....
-.....+7=,.+?................????:,I...?????..=.?????.?.............??~.=7+......
-..........????..............??????~...~??..~~??????..?...........+???,..........
-...........???............=.+???????,.?+:.?????????..+...........???+...........
-............??............?,.??????.,??..??????????.,............???............
-...........??,.............=.,????.?+....????????I.I..............=?............
-..........I?..................+??.:?~.....=??????..................??...........
-..........??...?...............??.:?=......??????..............?...??...........
-............++?..............?.????...?....??????.+..............++I............
-.............................?.??????~....???????.?.............................
-............................~~.??????......??????...............................
-.............................=???????......???????+.............................
-..........................=I??++?+++?......?+++++++?+...........................
-..........................,..77..77.........  ..  ...7..........................
-................................................................................

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/distribution/src/main/resources/bin/setenv.bat
----------------------------------------------------------------------
diff --git a/distribution/src/main/resources/bin/setenv.bat b/distribution/src/main/resources/bin/setenv.bat
deleted file mode 100644
index 5f1fda1..0000000
--- a/distribution/src/main/resources/bin/setenv.bat
+++ /dev/null
@@ -1,33 +0,0 @@
-rem Licensed to the Apache Software Foundation (ASF) under one
-rem or more contributor license agreements. See the NOTICE file
-rem distributed with this work for additional information
-rem regarding copyright ownership. The ASF licenses this file
-rem to you under the Apache License, Version 2.0 (the
-rem "License"); you may not use this file except in compliance
-rem with the License. You may obtain a copy of the License at
-rem
-rem http://www.apache.org/licenses/LICENSE-2.0
-rem
-rem Unless required by applicable law or agreed to in writing,
-rem software distributed under the License is distributed on an
-rem "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-rem KIND, either express or implied. See the License for the
-rem specific language governing permissions and limitations
-rem under the License.
-
-@echo off
-
-:initialize
-if "%AIRAVATA_HOME%"=="" set AIRAVATA_HOME=%~sdp0..
-SET curDrive=%cd:~0,1%
-SET airavataDrive=%AIRAVATA_HOME:~0,1%
-if not "%curDrive%" == "%airavataDrive%" %airavataDrive%:
-goto updateClasspath
-
-rem ----- update classpath -----------------------------------------------------
-:updateClasspath
-cd %AIRAVATA_HOME%
-set AIRAVATA_CLASSPATH=
-FOR %%C in ("%AIRAVATA_HOME%\lib\*.jar") DO set AIRAVATA_CLASSPATH=!AIRAVATA_CLASSPATH!;..\lib\%%~nC%%~xC
-
-:end
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/distribution/src/main/resources/bin/setenv.sh
----------------------------------------------------------------------
diff --git a/distribution/src/main/resources/bin/setenv.sh b/distribution/src/main/resources/bin/setenv.sh
deleted file mode 100755
index 9e894e1..0000000
--- a/distribution/src/main/resources/bin/setenv.sh
+++ /dev/null
@@ -1,46 +0,0 @@
-#!/bin/sh
-
-# 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.
-
-# resolve links - $0 may be a softlink
-PRG="$0"
-
-while [ -h "$PRG" ]; do
-  ls=`ls -ld "$PRG"`
-  link=`expr "$ls" : '.*-> \(.*\)$'`
-  if expr "$link" : '.*/.*' > /dev/null; then
-    PRG="$link"
-  else
-    PRG=`dirname "$PRG"`/"$link"
-  fi
-done
-
-PRGDIR=`dirname "$PRG"`
-
-# Only set AIRAVATA_HOME if not already set
-[ -z "$AIRAVATA_HOME" ] && AIRAVATA_HOME=`cd "$PRGDIR/.." ; pwd`
-
-AIRAVATA_CLASSPATH=""
-
-for f in "$AIRAVATA_HOME"/lib/*.jar
-do
-  AIRAVATA_CLASSPATH="$AIRAVATA_CLASSPATH":$f
-done
-
-export AIRAVATA_HOME
-export AIRAVATA_CLASSPATH

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/distribution/src/main/resources/bin/startNetworkServer
----------------------------------------------------------------------
diff --git a/distribution/src/main/resources/bin/startNetworkServer b/distribution/src/main/resources/bin/startNetworkServer
deleted file mode 100644
index 808566c..0000000
--- a/distribution/src/main/resources/bin/startNetworkServer
+++ /dev/null
@@ -1,189 +0,0 @@
-#!/bin/sh
-
-# 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.
-
-if [ -n "$derby_common_debug" ] ; then
-  set -x
-fi
-
-# OS specific support.  $var _must_ be set to either true or false.
-cygwin=false;
-darwin=false;
-case "`uname`" in
-  CYGWIN*) cygwin=true ;;
-  Darwin*) darwin=true
-           if [ -z "$JAVA_HOME" ] ; then
-             JAVA_HOME=/System/Library/Frameworks/JavaVM.framework/Home
-           fi
-           ;;
-esac
-
-if [ -z "$DERBY_HOME" -o ! -d "$DERBY_HOME" ] ; then
-  ## resolve links - $0 may be a link to derby's home
-  PRG="$0"
-  progname=`basename "$0"`
-
-  # need this for relative symlinks
-  while [ -h "$PRG" ] ; do
-    ls=`ls -ld "$PRG"`
-    link=`expr "$ls" : '.*-> \(.*\)$'`
-    if expr "$link" : '/.*' > /dev/null; then
-    PRG="$link"
-    else
-    PRG=`dirname "$PRG"`"/$link"
-    fi
-  done
-
-  DERBY_HOME=`dirname "$PRG"`/..
-
-  # make it fully qualified
-  DERBY_HOME=`cd "$DERBY_HOME" && pwd`
-fi
-
-# For Cygwin, ensure paths are in UNIX format before anything is touched
-if $cygwin ; then
-  [ -n "$DERBY_HOME" ] &&
-    DERBY_HOME=`cygpath --unix "$DERBY_HOME"`
-  [ -n "$JAVA_HOME" ] &&
-    JAVA_HOME=`cygpath --unix "$JAVA_HOME"`
-fi
-
-# set DERBY_LIB location
-DERBY_LIB="${DERBY_HOME}/lib"
-
-if [ -z "$JAVACMD" ] ; then
-  if [ -n "$JAVA_HOME"  ] ; then
-    if [ -x "$JAVA_HOME/jre/sh/java" ] ; then
-      # IBM's JDK on AIX uses strange locations for the executables
-      JAVACMD="$JAVA_HOME/jre/sh/java"
-    else
-      JAVACMD="$JAVA_HOME/bin/java"
-    fi
-  else
-    JAVACMD=`which java 2> /dev/null `
-    if [ -z "$JAVACMD" ] ; then
-        JAVACMD=java
-    fi
-  fi
-fi
-
-if [ ! -x "$JAVACMD" ] ; then
-  echo "Error: JAVA_HOME is not defined correctly."
-  echo "  We cannot execute $JAVACMD"
-  exit 1
-fi
-
-# set local classpath, don't overwrite the user's
-LOCALCLASSPATH=$DERBY_LIB/derby.jar:$DERBY_LIB/derbynet.jar:$DERBY_LIB/derbytools.jar:$DERBY_LIB/derbyclient.jar
-
-# if CLASSPATH_OVERRIDE env var is set, LOCALCLASSPATH will be
-# user CLASSPATH first and derby-found jars after.
-# In that case, the user CLASSPATH will override derby-found jars
-#
-# if CLASSPATH_OVERRIDE is not set, we'll have the normal behaviour
-# with derby-found jars first and user CLASSPATH after
-if [ -n "$CLASSPATH" ] ; then
-  # merge local and specified classpath 
-  if [ -z "$LOCALCLASSPATH" ] ; then 
-    LOCALCLASSPATH="$CLASSPATH"
-  elif [ -n "$CLASSPATH_OVERRIDE" ] ; then
-    LOCALCLASSPATH="$CLASSPATH:$LOCALCLASSPATH"
-  else
-    LOCALCLASSPATH="$LOCALCLASSPATH:$CLASSPATH"
-  fi
-
-  # remove class path from launcher -cp option
-  CLASSPATH=""
-fi
-
-# For Cygwin, switch paths to appropriate format before running java
-# For PATHs convert to unix format first, then to windows format to ensure
-# both formats are supported. Probably this will fail on directories with ;
-# in the name in the path. Let's assume that paths containing ; are more
-# rare than windows style paths on cygwin.
-if $cygwin; then
-  if [ "$OS" = "Windows_NT" ] && cygpath -m .>/dev/null 2>/dev/null ; then
-    format=mixed
-  else
-    format=windows
-  fi
-  DERBY_HOME=`cygpath --$format "$DERBY_HOME"`
-  DERBY_LIB=`cygpath --$format "$DERBY_LIB"`
-  if [ -n "$JAVA_HOME" ]; then
-    JAVA_HOME=`cygpath --$format "$JAVA_HOME"`
-  fi
-  LCP_TEMP=`cygpath --path --unix "$LOCALCLASSPATH"`
-  LOCALCLASSPATH=`cygpath --path --$format "$LCP_TEMP"`
-  if [ -n "$CLASSPATH" ] ; then
-    CP_TEMP=`cygpath --path --unix "$CLASSPATH"`
-    CLASSPATH=`cygpath --path --$format "$CP_TEMP"`
-  fi
-  CYGHOME=`cygpath --$format "$HOME"`
-fi
-
-# add a second backslash to variables terminated by a backslash under cygwin
-if $cygwin; then
-  case "$DERBY_HOME" in
-    *\\ )
-    DERBY_HOME="$DERBY_HOME\\"
-    ;;
-  esac
-  case "$CYGHOME" in
-    *\\ )
-    CYGHOME="$CYGHOME\\"
-    ;;
-  esac
-  case "$LOCALCLASSPATH" in
-    *\\ )
-    LOCALCLASSPATH="$LOCALCLASSPATH\\"
-    ;;
-  esac
-  case "$CLASSPATH" in
-    *\\ )
-    CLASSPATH="$CLASSPATH\\"
-    ;;
-  esac
-fi
-
-# Readjust classpath for MKS
-# expr match 
-if [ \( "`expr $SHELL : '.*sh.exe$'`" -gt 0 \) -a \( "$cygwin" = "false" \) ]; then
-  LOCALCLASSPATH=`echo $LOCALCLASSPATH | sed -E 's/([\d\w]*):([\d\w]*)/\1;\2/g
-'`
-fi
-#!/bin/sh
-
-# 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.
-
-derby_exec_command="exec \"$JAVACMD\" $DERBY_OPTS -classpath \"$LOCALCLASSPATH\" org.apache.derby.drda.NetworkServerControl start $@"
-eval $derby_exec_command

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/distribution/src/main/resources/samples/registerSample.sh
----------------------------------------------------------------------
diff --git a/distribution/src/main/resources/samples/registerSample.sh b/distribution/src/main/resources/samples/registerSample.sh
deleted file mode 100644
index 384ec0e..0000000
--- a/distribution/src/main/resources/samples/registerSample.sh
+++ /dev/null
@@ -1,24 +0,0 @@
-#!/bin/sh
-
-# 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.
-
-. `dirname $0`/../bin/setenv.sh
-JAVA_OPTS=""
-
-java -classpath "$AIRAVATA_CLASSPATH" \
-		     org.apache.airavata.client.samples.RegisterSampleData $*

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/distribution/src/main/resources/samples/scripts/add.sh
----------------------------------------------------------------------
diff --git a/distribution/src/main/resources/samples/scripts/add.sh b/distribution/src/main/resources/samples/scripts/add.sh
deleted file mode 100755
index daa140b..0000000
--- a/distribution/src/main/resources/samples/scripts/add.sh
+++ /dev/null
@@ -1,21 +0,0 @@
-#!/bin/sh
-# 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.
-
-# add two numbers
-sleep 10
-/bin/echo  "Result=`expr $1 + $2`"

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/distribution/src/main/resources/samples/scripts/echo.sh
----------------------------------------------------------------------
diff --git a/distribution/src/main/resources/samples/scripts/echo.sh b/distribution/src/main/resources/samples/scripts/echo.sh
deleted file mode 100755
index 9dbaab9..0000000
--- a/distribution/src/main/resources/samples/scripts/echo.sh
+++ /dev/null
@@ -1,22 +0,0 @@
-#!/bin/sh
-
-# 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.
-
-#echo wrapper
-sleep 10
-/bin/echo "Echoed_Output=$1"

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/distribution/src/main/resources/samples/scripts/multiply.sh
----------------------------------------------------------------------
diff --git a/distribution/src/main/resources/samples/scripts/multiply.sh b/distribution/src/main/resources/samples/scripts/multiply.sh
deleted file mode 100755
index a5b5f7f..0000000
--- a/distribution/src/main/resources/samples/scripts/multiply.sh
+++ /dev/null
@@ -1,22 +0,0 @@
-#!/bin/sh
-
-# 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.
-
-# mutiply two numbers
-sleep 10
-/bin/echo "Result=`expr $1 \* $2`"

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/distribution/src/main/resources/samples/scripts/subtract.sh
----------------------------------------------------------------------
diff --git a/distribution/src/main/resources/samples/scripts/subtract.sh b/distribution/src/main/resources/samples/scripts/subtract.sh
deleted file mode 100755
index a21bec7..0000000
--- a/distribution/src/main/resources/samples/scripts/subtract.sh
+++ /dev/null
@@ -1,22 +0,0 @@
-#!/bin/sh
-
-# 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.
-
-# substract two numbers
-sleep 10
-/bin/echo "Result=`expr $1 - $2`"

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/modules/distribution/pom.xml
----------------------------------------------------------------------
diff --git a/modules/distribution/pom.xml b/modules/distribution/pom.xml
new file mode 100644
index 0000000..78d9bb9
--- /dev/null
+++ b/modules/distribution/pom.xml
@@ -0,0 +1,582 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--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. -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+    <parent>
+        <groupId>org.apache.airavata</groupId>
+        <artifactId>airavata</artifactId>
+        <version>0.17-SNAPSHOT</version>
+        <relativePath>../../pom.xml</relativePath>
+    </parent>
+
+    <modelVersion>4.0.0</modelVersion>
+    <artifactId>apache-airavata-distribution</artifactId>
+    <name>Airavata server distribution</name>
+    <packaging>pom</packaging>
+    <url>http://airavata.apache.org/</url>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-dependency-plugin</artifactId>
+                <version>2.8</version>
+                <executions>
+                    <execution>
+                        <id>unpack</id>
+                        <phase>compile</phase>
+                        <goals>
+                            <goal>unpack</goal>
+                        </goals>
+                        <configuration>
+                            <artifactItems>
+                                <artifactItem>
+                                    <groupId>org.apache.airavata</groupId>
+                                    <artifactId>airavata-server-configuration</artifactId>
+                                    <version>${project.version}</version>
+                                    <type>jar</type>
+                                </artifactItem>
+                            </artifactItems>
+                            <!--includes>**/*.war</includes -->
+                            <outputDirectory>${project.build.directory}/conf</outputDirectory>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+
+            <plugin>
+                <groupId>org.codehaus.gmaven</groupId>
+                <artifactId>gmaven-plugin</artifactId>
+                <version>1.4</version>
+                <executions>
+                    <execution>
+                        <id>generate-timestamp</id>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>execute</goal>
+                        </goals>
+                        <configuration>
+                            <source>
+                                import java.util.Date
+                                import java.text.MessageFormat
+
+                                project.properties['buildTimestamp'] =
+                                        MessageFormat.format("{0,date,dd-MM-yyyy}", new
+                                                Date())
+                            </source>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-assembly-plugin</artifactId>
+                <version>2.5.5</version>
+                <executions>
+                    <execution>
+                        <id>distribution-package</id>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>single</goal>
+                        </goals>
+                        <configuration>
+                            <tarLongFileMode>posix</tarLongFileMode>
+                            <finalName>${archieve.name}-${project.version}</finalName>
+                            <descriptors>
+                                <descriptor>src/main/assembly/bin-assembly.xml</descriptor>
+                                <!-- <descriptor>src/main/assembly/src-assembly.xml</descriptor> -->
+                            </descriptors>
+                            <attach>false</attach>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+                <version>1.7</version>
+                <executions>
+                    <execution>
+                        <id>attach-artifacts</id>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>attach-artifact</goal>
+                        </goals>
+                        <configuration>
+                            <artifacts>
+                                <artifact>
+                                    <file>${airavata.bin.zip}</file>
+                                    <type>zip</type>
+                                    <classifier>bin</classifier>
+                                </artifact>
+                                <artifact>
+                                    <file>${airavata.bin.tar.gz}</file>
+                                    <type>tar.gz</type>
+                                    <classifier>bin</classifier>
+                                </artifact>
+                            </artifacts>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.derby</groupId>
+            <artifactId>derby</artifactId>
+            <version>${derby.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.derby</groupId>
+            <artifactId>derbyclient</artifactId>
+            <version>${derby.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.derby</groupId>
+            <artifactId>derbynet</artifactId>
+            <version>${derby.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.derby</groupId>
+            <artifactId>derbytools</artifactId>
+            <version>${derby.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>jcl-over-slf4j</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk</artifactId>
+            <version>1.9.0</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.httpcomponents</groupId>
+                    <artifactId>httpclient</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>net.java.dev.jets3t</groupId>
+            <artifactId>jets3t</artifactId>
+            <version>0.8.0</version>
+        </dependency>
+        <dependency>
+            <groupId>commons-collections</groupId>
+            <artifactId>commons-collections</artifactId>
+            <version>3.2.1</version>
+        </dependency>
+        <dependency>
+            <groupId>commons-lang</groupId>
+            <artifactId>commons-lang</artifactId>
+            <version>2.4</version>
+        </dependency>
+        <dependency>
+            <groupId>commons-io</groupId>
+            <artifactId>commons-io</artifactId>
+            <version>2.4</version>
+        </dependency>
+        <dependency>
+            <groupId>commons-codec</groupId>
+            <artifactId>commons-codec</artifactId>
+            <version>1.6</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.airavata</groupId>
+            <artifactId>airavata-standalone-server</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.airavata</groupId>
+            <artifactId>airavata-registry-cpi</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.airavata</groupId>
+            <artifactId>airavata-registry-core</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.airavata</groupId>
+            <artifactId>airavata-messaging-core</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.airavata</groupId>
+            <artifactId>airavata-commons</artifactId>
+            <version>${project.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.ws.commons.schema</groupId>
+                    <artifactId>XmlSchema</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>xerces</groupId>
+                    <artifactId>xmlParserAPIs</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.neethi</groupId>
+                    <artifactId>neethi</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.airavata</groupId>
+            <artifactId>airavata-orchestrator-service</artifactId>
+            <version>${project.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.ws.commons.schema</groupId>
+                    <artifactId>XmlSchema</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>xerces</groupId>
+                    <artifactId>xmlParserAPIs</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.neethi</groupId>
+                    <artifactId>neethi</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.airavata</groupId>
+            <artifactId>airavata-orchestrator-client</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.airavata</groupId>
+            <artifactId>airavata-gfac-client</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.airavata</groupId>
+            <artifactId>airavata-orchestrator-core</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.airavata</groupId>
+            <artifactId>registry-api-stubs</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.airavata</groupId>
+            <artifactId>registry-api-service</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.airavata</groupId>
+            <artifactId>airavata-data-models</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.airavata</groupId>
+            <artifactId>airavata-credential-store</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.airavata</groupId>
+            <artifactId>airavata-gfac-impl</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+<!--        <dependency>
+            <groupId>org.apache.airavata</groupId>
+            <artifactId>airavata-gfac-bes</artifactId>
+            <version>${project.version}</version>
+        </dependency>-->
+        <dependency>
+            <groupId>org.apache.airavata</groupId>
+            <artifactId>airavata-gfac-core</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.airavata</groupId>
+            <artifactId>airavata-gfac-service</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+<!--        <dependency>
+            <groupId>org.apache.airavata</groupId>
+            <artifactId>airavata-workflow-model-core</artifactId>
+            <version>${project.version}</version>
+        </dependency>-->
+        <dependency>
+            <groupId>org.apache.airavata</groupId>
+            <artifactId>airavata-model-utils</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.airavata</groupId>
+            <artifactId>airavata-api-server</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.bouncycastle</groupId>
+            <artifactId>bcprov-jdk15on</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.openjpa</groupId>
+            <artifactId>openjpa-all</artifactId>
+            <version>2.2.0</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.shiro</groupId>
+            <artifactId>shiro-core</artifactId>
+            <version>1.2.1</version>
+        </dependency>
+        <dependency>
+            <groupId>com.sun.jersey</groupId>
+            <artifactId>jersey-client</artifactId>
+            <version>${jersey.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>javax.servlet</groupId>
+            <artifactId>javax.servlet-api</artifactId>
+            <version>3.0.1</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.tomcat.embed</groupId>
+            <artifactId>tomcat-embed-logging-juli</artifactId>
+            <version>7.0.22</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.tomcat.embed</groupId>
+            <artifactId>tomcat-embed-jasper</artifactId>
+            <version>7.0.22</version>
+        </dependency>
+        <dependency>
+            <groupId>com.sun.jersey</groupId>
+            <artifactId>jersey-servlet</artifactId>
+            <version>${jersey.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.sun.jersey</groupId>
+            <artifactId>jersey-json</artifactId>
+            <version>${jersey.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>stax</groupId>
+                    <artifactId>stax-api</artifactId>
+                </exclusion>
+            </exclusions>
+
+        </dependency>
+        <dependency>
+            <groupId>com.sun.jersey.contribs</groupId>
+            <artifactId>jersey-multipart</artifactId>
+            <version>${jersey.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.sun.jersey</groupId>
+            <artifactId>jersey-server</artifactId>
+            <version>${jersey.version}</version>
+        </dependency>
+        <!--dependency> <groupId>com.sun.jersey</groupId> <artifactId>jersey-client</artifactId>
+            <version>${jersey.version}</version> </dependency -->
+        <dependency>
+            <groupId>org.codehaus.jackson</groupId>
+            <artifactId>jackson-mapper-asl</artifactId>
+            <version>1.9.2</version>
+        </dependency>
+        <dependency>
+            <groupId>org.codehaus.jackson</groupId>
+            <artifactId>jackson-xc</artifactId>
+            <version>1.9.2</version>
+        </dependency>
+        <dependency>
+            <groupId>org.codehaus.jackson</groupId>
+            <artifactId>jackson-jaxrs</artifactId>
+            <version>1.9.2</version>
+        </dependency>
+        <dependency>
+            <groupId>org.codehaus.jackson</groupId>
+            <artifactId>jackson-core-asl</artifactId>
+            <version>1.9.2</version>
+        </dependency>
+        <dependency>
+            <groupId>xerces</groupId>
+            <artifactId>xercesImpl</artifactId>
+            <version>2.9.1</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>xml-apis</groupId>
+                    <artifactId>xml-apis</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>com.ibm.icu</groupId>
+            <artifactId>icu4j</artifactId>
+            <version>3.4.4</version>
+        </dependency>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+            <version>12.0</version>
+        </dependency>
+
+        <!-- Hadoop provider related dependencies -->
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-core</artifactId>
+            <version>1.0.3</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-client</artifactId>
+            <version>1.0.3</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.whirr</groupId>
+            <artifactId>whirr-core</artifactId>
+            <version>0.7.1</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.bouncycastle</groupId>
+                    <artifactId>bcprov-jdk16</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.jclouds.driver</groupId>
+                    <artifactId>jclouds-bouncycastle</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.whirr</groupId>
+            <artifactId>whirr-hadoop</artifactId>
+            <version>0.7.1</version>
+        </dependency>
+        <dependency>
+            <groupId>org.hamcrest</groupId>
+            <artifactId>hamcrest-all</artifactId>
+            <version>1.1</version>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-all</artifactId>
+            <version>1.8.5</version>
+        </dependency>
+        <dependency>
+            <groupId>commons-configuration</groupId>
+            <artifactId>commons-configuration</artifactId>
+            <version>1.7</version>
+        </dependency>
+        <dependency>
+            <groupId>net.sf.jopt-simple</groupId>
+            <artifactId>jopt-simple</artifactId>
+            <version>3.2</version>
+        </dependency>
+        <dependency>
+            <groupId>org.ebaysf.web</groupId>
+            <artifactId>cors-filter</artifactId>
+            <version>${ebay.cors.filter}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.jcraft</groupId>
+            <artifactId>jsch</artifactId>
+            <version>0.1.53</version>
+        </dependency>
+        <!-- dependency> <groupId>org.ogce</groupId> <artifactId>bcgss</artifactId>
+            <version>146</version> </dependency> -->
+        <dependency>
+            <groupId>org.apache.xmlbeans</groupId>
+            <artifactId>xmlbeans</artifactId>
+            <version>${xmlbeans.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>stax</groupId>
+                    <artifactId>stax-api</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.thrift</groupId>
+            <artifactId>libthrift</artifactId>
+            <version>${thrift.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+            <version>2.0.0</version>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-core</artifactId>
+            <version>2.0.0</version>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-annotations</artifactId>
+            <version>2.0.0</version>
+        </dependency>
+        <!-- zookeeper dependencies -->
+
+        <dependency>
+            <groupId>org.apache.zookeeper</groupId>
+            <artifactId>zookeeper</artifactId>
+            <version>3.4.0</version>
+        </dependency>
+        <dependency>
+            <groupId>commons-cli</groupId>
+            <artifactId>commons-cli</artifactId>
+            <version>1.2</version>
+        </dependency>
+
+        <dependency>
+            <groupId>com.rabbitmq</groupId>
+            <artifactId>amqp-client</artifactId>
+            <version>${amqp.client.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.curator</groupId>
+            <artifactId>curator-framework</artifactId>
+            <version>${curator.version}</version>
+        </dependency>
+
+        <!-- ======================== Sample =================== -->
+        <dependency>
+            <groupId>org.apache.airavata</groupId>
+            <artifactId>airavata-client-samples</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+    </dependencies>
+
+
+    <properties>
+        <jersey.version>1.13</jersey.version>
+        <grizzly.version>2.0.0-M3</grizzly.version>
+        <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+        <archieve.name>apache-airavata-server</archieve.name>
+        <airavata.dist.name>${archieve.name}-${project.version}</airavata.dist.name>
+        <airavata.work.dir>${project.build.directory}/tests/${airavata.dist.name}</airavata.work.dir>
+        <airavata.bin.zip>${project.build.directory}/${airavata.dist.name}-bin.zip</airavata.bin.zip>
+        <airavata.bin.tar.gz>${project.build.directory}/${airavata.dist.name}-bin.tar.gz</airavata.bin.tar.gz>
+    </properties>
+</project>

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/modules/distribution/src/main/assembly/bin-assembly.xml
----------------------------------------------------------------------
diff --git a/modules/distribution/src/main/assembly/bin-assembly.xml b/modules/distribution/src/main/assembly/bin-assembly.xml
new file mode 100644
index 0000000..d88521d
--- /dev/null
+++ b/modules/distribution/src/main/assembly/bin-assembly.xml
@@ -0,0 +1,160 @@
+<!--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. -->
+
+<!DOCTYPE assembly [
+        <!ELEMENT assembly (id|includeBaseDirectory|baseDirectory|formats|fileSets|dependencySets)*>
+        <!ELEMENT id (#PCDATA)>
+        <!ELEMENT includeBaseDirectory (#PCDATA)>
+        <!ELEMENT baseDirectory (#PCDATA)>
+        <!ELEMENT formats (format)*>
+        <!ELEMENT format (#PCDATA)>
+        <!ELEMENT fileSets (fileSet)*>
+        <!ELEMENT fileSet (directory|outputDirectory|fileMode|includes)*>
+        <!ELEMENT directory (#PCDATA)>
+        <!ELEMENT outputDirectory (#PCDATA)>
+        <!ELEMENT includes (include)*>
+        <!ELEMENT include (#PCDATA)>
+        <!ELEMENT dependencySets (dependencySet)*>
+        <!ELEMENT dependencySet (outputDirectory|outputFileNameMapping|includes)*>
+        ]>
+<assembly>
+    <id>bin</id>
+    <includeBaseDirectory>true</includeBaseDirectory>
+    <baseDirectory>${archieve.name}-${version}</baseDirectory>
+    <formats>
+        <format>tar.gz</format>
+        <format>zip</format>
+    </formats>
+
+    <fileSets>
+
+        <!-- ********************** copy release notes files ********************** -->
+        <fileSet>
+            <directory>../../../</directory>
+            <outputDirectory>.</outputDirectory>
+            <includes>
+                <include>RELEASE_NOTES</include>
+            </includes>
+        </fileSet>
+        <!-- ********************** copy licenses, readme etc. ********************** -->
+        <fileSet>
+            <directory>src/main/resources/</directory>
+            <outputDirectory>.</outputDirectory>
+            <includes>
+                <include>LICENSE</include>
+                <include>NOTICE</include>
+                <include>README</include>
+                <include>INSTALL</include>
+            </includes>
+        </fileSet>
+
+        <!-- ********************** copy database scripts ********************** -->
+        <fileSet>
+            <directory>../registry/registry-core/src/main/resources/
+            </directory>
+            <outputDirectory>bin/database_scripts
+            </outputDirectory>
+            <includes>
+                <include>*sql*</include>
+            </includes>
+        </fileSet>
+        <fileSet>
+            <directory>src/main/resources/bin</directory>
+            <outputDirectory>bin</outputDirectory>
+            <fileMode>777</fileMode>
+            <includes>
+                <include>*.sh</include>
+                <include>*.bat</include>
+                <include>logo.txt</include>
+                <include>startNetworkServer</include>
+            </includes>
+        </fileSet>
+        <fileSet>
+            <directory>src/main/resources/samples</directory>
+            <outputDirectory>samples</outputDirectory>
+            <fileMode>777</fileMode>
+            <includes>
+                <include>*.sh</include>
+                <include>**/*.sh</include>
+            </includes>
+        </fileSet>
+        <fileSet>
+            <directory>${project.build.directory}/conf</directory>
+            <outputDirectory>bin</outputDirectory>
+            <includes>
+                <include>airavata-server.properties</include>
+                <include>zoo.cfg</include>
+                <include>registry.properties</include>
+                <include>log4j.properties</include>
+                <include>host.xml</include>
+                <include>persistence.xml</include>
+                <include>provenance.sql</include>
+                <include>gfac-config.yaml</include>
+                <include>PBSTemplate.xslt</include>
+                <include>SLURMTemplate.xslt</include>
+                <include>LSFTemplate.xslt</include>
+                <include>UGETemplate.xslt</include>
+                <include>ForkTemplate.xslt</include>
+                <include>gsissh.properties</include>
+                <include>airavata.jks</include>
+                <include>client_truststore.jks</include>
+                <include>airavata-default-xacml-policy.xml</include>
+            </includes>
+        </fileSet>
+
+        <!-- Create logs directory -->
+        <fileSet>
+            <directory>./</directory>
+            <outputDirectory>logs</outputDirectory>
+            <excludes>
+                <exclude>*/**</exclude>
+            </excludes>
+        </fileSet>
+
+        <!-- ********************** Copy samples ********************** -->
+        <fileSet>
+            <directory>${project.build.directory}/samples/applications
+            </directory>
+            <outputDirectory>samples</outputDirectory>
+            <includes>
+                <include>*.sh</include>
+                <include>*.bat</include>
+            </includes>
+        </fileSet>
+
+    </fileSets>
+
+    <dependencySets>
+        <dependencySet>
+            <useProjectArtifact>false</useProjectArtifact>
+            <outputDirectory>lib</outputDirectory>
+            <outputFileNameMapping>${artifact.artifactId}.${artifact.extension}</outputFileNameMapping>
+            <includes>
+                <include>org.apache.derby:derby:jar</include>
+                <include>org.apache.derby:derbytools:jar</include>
+                <include>org.apache.derby:derbynet:jar</include>
+                <include>org.apache.derby:derbyclient:jar</include>
+            </includes>
+        </dependencySet>
+        <dependencySet>
+            <useProjectArtifact>false</useProjectArtifact>
+            <outputDirectory>lib</outputDirectory>
+            <includes>
+                <include>*:*:jar</include>
+            </includes>
+            <excludes>
+                <exclude>mysql:mysql-connector-java</exclude>
+            </excludes>
+        </dependencySet>
+
+    </dependencySets>
+
+</assembly>

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/modules/distribution/src/main/assembly/src-assembly.xml
----------------------------------------------------------------------
diff --git a/modules/distribution/src/main/assembly/src-assembly.xml b/modules/distribution/src/main/assembly/src-assembly.xml
new file mode 100644
index 0000000..6a093ed
--- /dev/null
+++ b/modules/distribution/src/main/assembly/src-assembly.xml
@@ -0,0 +1,75 @@
+<!--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.
+  -->
+
+<assembly>
+    <id>src</id>
+    <includeBaseDirectory>true</includeBaseDirectory> 
+    <baseDirectory>${archieve.name}-${version}</baseDirectory>
+    <formats>
+        <format>tar.gz</format>  
+        <format>zip</format>
+    </formats>
+
+    <fileSets>
+        <fileSet>
+            <directory>../..</directory>
+            <outputDirectory></outputDirectory>
+            <includes>
+                <include>NOTICE</include>
+                <include>LICENSE</include>
+                <include>README</include>
+                <include>RELEASE_NOTES</include>
+		<include>DISCLAIMER</include>
+		<include>INSTALL</include>
+            </includes>
+            <filtered>true</filtered>
+        </fileSet>
+        <fileSet>
+            <directory>../..</directory>
+            <outputDirectory></outputDirectory>
+            <useDefaultExcludes>true</useDefaultExcludes>
+            <includes>
+                <include>pom.xml</include>
+                <include>modules/**</include>
+                <include>samples/**</include>
+            </includes>
+
+            <excludes>
+                                <!-- Exclusions from org.apache.resources:apache-source-release-assembly-descriptor.
+                     Note that they assume that all sources are located under an "src" directory. This
+                     is not the case for Axis2, which doesn't use the standard Maven 2 conventions.
+                     Thus we may still encounter some issues here. -->
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/).*${project.build.directory}.*]</exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?maven-eclipse\.xml]</exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.project]</exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.classpath]</exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?[^/]*\.iws]</exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?[^/]*\.ipr]</exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?[^/]*\.iml]</exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.settings(/.*)?]</exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.externalToolBuilders(/.*)?]</exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.deployables(/.*)?]</exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.wtpmodules(/.*)?]</exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?cobertura\.ser]</exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?pom\.xml\.releaseBackup]</exclude>
+                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?release\.properties]</exclude>
+            </excludes>
+
+        </fileSet>
+          </fileSets>
+</assembly>

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/modules/distribution/src/main/resources/INSTALL
----------------------------------------------------------------------
diff --git a/modules/distribution/src/main/resources/INSTALL b/modules/distribution/src/main/resources/INSTALL
new file mode 100644
index 0000000..53d0550
--- /dev/null
+++ b/modules/distribution/src/main/resources/INSTALL
@@ -0,0 +1,30 @@
+Installing  Apache Airavata 0.14
+-------------------------------
+
+Prerequisites
+-------------
+Java 1.5 or later
+Maven (tested on v 3.0.2)
+
+Build Apache Airavata from Source
+---------------------------------
+* Unzip/untar the source file or clone from git.
+* cd to project folder and type
+	$ mvn clean install
+	Note: in order to skip tests use the command
+			$ mvn clean install -Dmaven.test.skip=true
+* Alternatively, all  compressed binary distributions can be found at <PROJECT DIR>/modules/distribution/release/target/release-artifacts
+
+Running Tests
+-------------
+* Unit tests & integrations tests will run while Apache Airavata is built from source (without "-Dmaven.test.skip=true").
+* To run the test samples
+    - You can find the binary distributions at <PROJECT DIR>/modules/distribution/release/target/release-artifacts or from
+      the Apache Airavata download site.
+    - Extract the binary distributions and once the binary is unzipped, instructions to run the tests should be followed
+      from README files found within.
+
+Tutorials
+----------
+The airavata website has instructions for basic tutorials:
+* Describing and executing applications using Airavata - follow "XBAYA Quick-Start Tutorial" tutorial at https://cwiki.apache.org/confluence/display/AIRAVATA/XBAYA+Quick-Start+Tutorial
\ No newline at end of file


[50/50] [abbrv] airavata git commit: [AIRAVATA-2057] Move the distribution directory to modules to slow down the distribution build

Posted by la...@apache.org.
[AIRAVATA-2057] Move the distribution directory to modules to slow down the distribution build

Move the distrubtion to modules directory, now the build time moves back to few seconds from 15 minutes.

Test: ran mvn clean install and unpacked the distrubtion and started the server with
`airavata-server-start.sh all`, started fine.


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: 89bb0d48d590fb44057207a75539390e973c82bc
Parents: bfff64c
Author: Lahiru Ginnaliya Gamathige <la...@apache.org>
Authored: Fri Aug 26 08:37:25 2016 -0700
Committer: Lahiru Ginnaliya Gamathige <la...@apache.org>
Committed: Fri Aug 26 08:37:40 2016 -0700

----------------------------------------------------------------------
 distribution/pom.xml                            |  582 -----
 distribution/src/main/assembly/bin-assembly.xml |  160 --
 distribution/src/main/assembly/src-assembly.xml |   75 -
 distribution/src/main/resources/INSTALL         |   30 -
 distribution/src/main/resources/LICENSE         | 2387 ------------------
 distribution/src/main/resources/NOTICE          |  163 --
 distribution/src/main/resources/README          |  145 --
 .../main/resources/bin/airavata-server-start.sh |  123 -
 .../main/resources/bin/airavata-server-stop.sh  |   71 -
 .../src/main/resources/bin/airavata-server.bat  |   55 -
 distribution/src/main/resources/bin/derby.sh    |   23 -
 distribution/src/main/resources/bin/logo.txt    |   34 -
 distribution/src/main/resources/bin/setenv.bat  |   33 -
 distribution/src/main/resources/bin/setenv.sh   |   46 -
 .../src/main/resources/bin/startNetworkServer   |  189 --
 .../main/resources/samples/registerSample.sh    |   24 -
 .../src/main/resources/samples/scripts/add.sh   |   21 -
 .../src/main/resources/samples/scripts/echo.sh  |   22 -
 .../main/resources/samples/scripts/multiply.sh  |   22 -
 .../main/resources/samples/scripts/subtract.sh  |   22 -
 modules/distribution/pom.xml                    |  582 +++++
 .../src/main/assembly/bin-assembly.xml          |  160 ++
 .../src/main/assembly/src-assembly.xml          |   75 +
 modules/distribution/src/main/resources/INSTALL |   30 +
 modules/distribution/src/main/resources/LICENSE | 2387 ++++++++++++++++++
 modules/distribution/src/main/resources/NOTICE  |  163 ++
 modules/distribution/src/main/resources/README  |  145 ++
 .../main/resources/bin/airavata-server-start.sh |  123 +
 .../main/resources/bin/airavata-server-stop.sh  |   71 +
 .../src/main/resources/bin/airavata-server.bat  |   55 +
 .../src/main/resources/bin/derby.sh             |   23 +
 .../src/main/resources/bin/logo.txt             |   34 +
 .../src/main/resources/bin/setenv.bat           |   33 +
 .../src/main/resources/bin/setenv.sh            |   46 +
 .../src/main/resources/bin/startNetworkServer   |  189 ++
 .../main/resources/samples/registerSample.sh    |   24 +
 .../src/main/resources/samples/scripts/add.sh   |   21 +
 .../src/main/resources/samples/scripts/echo.sh  |   22 +
 .../main/resources/samples/scripts/multiply.sh  |   22 +
 .../main/resources/samples/scripts/subtract.sh  |   22 +
 pom.xml                                         |   97 +-
 41 files changed, 4229 insertions(+), 4322 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/distribution/pom.xml
----------------------------------------------------------------------
diff --git a/distribution/pom.xml b/distribution/pom.xml
deleted file mode 100644
index d3124eb..0000000
--- a/distribution/pom.xml
+++ /dev/null
@@ -1,582 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-
-<!--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. -->
-
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-
-    <parent>
-        <groupId>org.apache.airavata</groupId>
-        <artifactId>airavata</artifactId>
-        <version>0.17-SNAPSHOT</version>
-        <relativePath>../pom.xml</relativePath>
-    </parent>
-
-    <modelVersion>4.0.0</modelVersion>
-    <artifactId>apache-airavata-distribution</artifactId>
-    <name>Airavata server distribution</name>
-    <packaging>pom</packaging>
-    <url>http://airavata.apache.org/</url>
-
-    <build>
-        <plugins>
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-dependency-plugin</artifactId>
-                <version>2.8</version>
-                <executions>
-                    <execution>
-                        <id>unpack</id>
-                        <phase>compile</phase>
-                        <goals>
-                            <goal>unpack</goal>
-                        </goals>
-                        <configuration>
-                            <artifactItems>
-                                <artifactItem>
-                                    <groupId>org.apache.airavata</groupId>
-                                    <artifactId>airavata-server-configuration</artifactId>
-                                    <version>${project.version}</version>
-                                    <type>jar</type>
-                                </artifactItem>
-                            </artifactItems>
-                            <!--includes>**/*.war</includes -->
-                            <outputDirectory>${project.build.directory}/conf</outputDirectory>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-
-            <plugin>
-                <groupId>org.codehaus.gmaven</groupId>
-                <artifactId>gmaven-plugin</artifactId>
-                <version>1.4</version>
-                <executions>
-                    <execution>
-                        <id>generate-timestamp</id>
-                        <phase>package</phase>
-                        <goals>
-                            <goal>execute</goal>
-                        </goals>
-                        <configuration>
-                            <source>
-                                import java.util.Date
-                                import java.text.MessageFormat
-
-                                project.properties['buildTimestamp'] =
-                                        MessageFormat.format("{0,date,dd-MM-yyyy}", new
-                                                Date())
-                            </source>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-assembly-plugin</artifactId>
-                <version>2.5.5</version>
-                <executions>
-                    <execution>
-                        <id>distribution-package</id>
-                        <phase>package</phase>
-                        <goals>
-                            <goal>single</goal>
-                        </goals>
-                        <configuration>
-                            <tarLongFileMode>posix</tarLongFileMode>
-                            <finalName>${archieve.name}-${project.version}</finalName>
-                            <descriptors>
-                                <descriptor>src/main/assembly/bin-assembly.xml</descriptor>
-                                <!-- <descriptor>src/main/assembly/src-assembly.xml</descriptor> -->
-                            </descriptors>
-                            <attach>false</attach>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-
-            <plugin>
-                <groupId>org.codehaus.mojo</groupId>
-                <artifactId>build-helper-maven-plugin</artifactId>
-                <version>1.7</version>
-                <executions>
-                    <execution>
-                        <id>attach-artifacts</id>
-                        <phase>package</phase>
-                        <goals>
-                            <goal>attach-artifact</goal>
-                        </goals>
-                        <configuration>
-                            <artifacts>
-                                <artifact>
-                                    <file>${airavata.bin.zip}</file>
-                                    <type>zip</type>
-                                    <classifier>bin</classifier>
-                                </artifact>
-                                <artifact>
-                                    <file>${airavata.bin.tar.gz}</file>
-                                    <type>tar.gz</type>
-                                    <classifier>bin</classifier>
-                                </artifact>
-                            </artifacts>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-        </plugins>
-    </build>
-
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.derby</groupId>
-            <artifactId>derby</artifactId>
-            <version>${derby.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.derby</groupId>
-            <artifactId>derbyclient</artifactId>
-            <version>${derby.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.derby</groupId>
-            <artifactId>derbynet</artifactId>
-            <version>${derby.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.derby</groupId>
-            <artifactId>derbytools</artifactId>
-            <version>${derby.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.slf4j</groupId>
-            <artifactId>slf4j-api</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.slf4j</groupId>
-            <artifactId>jcl-over-slf4j</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.slf4j</groupId>
-            <artifactId>slf4j-log4j12</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>log4j</groupId>
-            <artifactId>log4j</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>com.amazonaws</groupId>
-            <artifactId>aws-java-sdk</artifactId>
-            <version>1.9.0</version>
-            <exclusions>
-                <exclusion>
-                    <groupId>org.apache.httpcomponents</groupId>
-                    <artifactId>httpclient</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-        <dependency>
-            <groupId>net.java.dev.jets3t</groupId>
-            <artifactId>jets3t</artifactId>
-            <version>0.8.0</version>
-        </dependency>
-        <dependency>
-            <groupId>commons-collections</groupId>
-            <artifactId>commons-collections</artifactId>
-            <version>3.2.1</version>
-        </dependency>
-        <dependency>
-            <groupId>commons-lang</groupId>
-            <artifactId>commons-lang</artifactId>
-            <version>2.4</version>
-        </dependency>
-        <dependency>
-            <groupId>commons-io</groupId>
-            <artifactId>commons-io</artifactId>
-            <version>2.4</version>
-        </dependency>
-        <dependency>
-            <groupId>commons-codec</groupId>
-            <artifactId>commons-codec</artifactId>
-            <version>1.6</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.airavata</groupId>
-            <artifactId>airavata-standalone-server</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.airavata</groupId>
-            <artifactId>airavata-registry-cpi</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.airavata</groupId>
-            <artifactId>airavata-registry-core</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.airavata</groupId>
-            <artifactId>airavata-messaging-core</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.airavata</groupId>
-            <artifactId>airavata-commons</artifactId>
-            <version>${project.version}</version>
-            <exclusions>
-                <exclusion>
-                    <groupId>org.apache.ws.commons.schema</groupId>
-                    <artifactId>XmlSchema</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>xerces</groupId>
-                    <artifactId>xmlParserAPIs</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.apache.neethi</groupId>
-                    <artifactId>neethi</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.airavata</groupId>
-            <artifactId>airavata-orchestrator-service</artifactId>
-            <version>${project.version}</version>
-            <exclusions>
-                <exclusion>
-                    <groupId>org.apache.ws.commons.schema</groupId>
-                    <artifactId>XmlSchema</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>xerces</groupId>
-                    <artifactId>xmlParserAPIs</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.apache.neethi</groupId>
-                    <artifactId>neethi</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.airavata</groupId>
-            <artifactId>airavata-orchestrator-client</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.airavata</groupId>
-            <artifactId>airavata-gfac-client</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.airavata</groupId>
-            <artifactId>airavata-orchestrator-core</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.airavata</groupId>
-            <artifactId>registry-api-stubs</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.airavata</groupId>
-            <artifactId>registry-api-service</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.airavata</groupId>
-            <artifactId>airavata-data-models</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.airavata</groupId>
-            <artifactId>airavata-credential-store</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.airavata</groupId>
-            <artifactId>airavata-gfac-impl</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-<!--        <dependency>
-            <groupId>org.apache.airavata</groupId>
-            <artifactId>airavata-gfac-bes</artifactId>
-            <version>${project.version}</version>
-        </dependency>-->
-        <dependency>
-            <groupId>org.apache.airavata</groupId>
-            <artifactId>airavata-gfac-core</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.airavata</groupId>
-            <artifactId>airavata-gfac-service</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-<!--        <dependency>
-            <groupId>org.apache.airavata</groupId>
-            <artifactId>airavata-workflow-model-core</artifactId>
-            <version>${project.version}</version>
-        </dependency>-->
-        <dependency>
-            <groupId>org.apache.airavata</groupId>
-            <artifactId>airavata-model-utils</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.airavata</groupId>
-            <artifactId>airavata-api-server</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.bouncycastle</groupId>
-            <artifactId>bcprov-jdk15on</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.openjpa</groupId>
-            <artifactId>openjpa-all</artifactId>
-            <version>2.2.0</version>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.shiro</groupId>
-            <artifactId>shiro-core</artifactId>
-            <version>1.2.1</version>
-        </dependency>
-        <dependency>
-            <groupId>com.sun.jersey</groupId>
-            <artifactId>jersey-client</artifactId>
-            <version>${jersey.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>javax.servlet</groupId>
-            <artifactId>javax.servlet-api</artifactId>
-            <version>3.0.1</version>
-            <scope>provided</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.tomcat.embed</groupId>
-            <artifactId>tomcat-embed-logging-juli</artifactId>
-            <version>7.0.22</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.tomcat.embed</groupId>
-            <artifactId>tomcat-embed-jasper</artifactId>
-            <version>7.0.22</version>
-        </dependency>
-        <dependency>
-            <groupId>com.sun.jersey</groupId>
-            <artifactId>jersey-servlet</artifactId>
-            <version>${jersey.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>com.sun.jersey</groupId>
-            <artifactId>jersey-json</artifactId>
-            <version>${jersey.version}</version>
-            <exclusions>
-                <exclusion>
-                    <groupId>stax</groupId>
-                    <artifactId>stax-api</artifactId>
-                </exclusion>
-            </exclusions>
-
-        </dependency>
-        <dependency>
-            <groupId>com.sun.jersey.contribs</groupId>
-            <artifactId>jersey-multipart</artifactId>
-            <version>${jersey.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>com.sun.jersey</groupId>
-            <artifactId>jersey-server</artifactId>
-            <version>${jersey.version}</version>
-        </dependency>
-        <!--dependency> <groupId>com.sun.jersey</groupId> <artifactId>jersey-client</artifactId>
-            <version>${jersey.version}</version> </dependency -->
-        <dependency>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-mapper-asl</artifactId>
-            <version>1.9.2</version>
-        </dependency>
-        <dependency>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-xc</artifactId>
-            <version>1.9.2</version>
-        </dependency>
-        <dependency>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-jaxrs</artifactId>
-            <version>1.9.2</version>
-        </dependency>
-        <dependency>
-            <groupId>org.codehaus.jackson</groupId>
-            <artifactId>jackson-core-asl</artifactId>
-            <version>1.9.2</version>
-        </dependency>
-        <dependency>
-            <groupId>xerces</groupId>
-            <artifactId>xercesImpl</artifactId>
-            <version>2.9.1</version>
-            <exclusions>
-                <exclusion>
-                    <groupId>xml-apis</groupId>
-                    <artifactId>xml-apis</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-        <dependency>
-            <groupId>com.ibm.icu</groupId>
-            <artifactId>icu4j</artifactId>
-            <version>3.4.4</version>
-        </dependency>
-        <dependency>
-            <groupId>com.google.guava</groupId>
-            <artifactId>guava</artifactId>
-            <version>12.0</version>
-        </dependency>
-
-        <!-- Hadoop provider related dependencies -->
-        <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-core</artifactId>
-            <version>1.0.3</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-client</artifactId>
-            <version>1.0.3</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.whirr</groupId>
-            <artifactId>whirr-core</artifactId>
-            <version>0.7.1</version>
-            <exclusions>
-                <exclusion>
-                    <groupId>org.bouncycastle</groupId>
-                    <artifactId>bcprov-jdk16</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.jclouds.driver</groupId>
-                    <artifactId>jclouds-bouncycastle</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.whirr</groupId>
-            <artifactId>whirr-hadoop</artifactId>
-            <version>0.7.1</version>
-        </dependency>
-        <dependency>
-            <groupId>org.hamcrest</groupId>
-            <artifactId>hamcrest-all</artifactId>
-            <version>1.1</version>
-        </dependency>
-        <dependency>
-            <groupId>org.mockito</groupId>
-            <artifactId>mockito-all</artifactId>
-            <version>1.8.5</version>
-        </dependency>
-        <dependency>
-            <groupId>commons-configuration</groupId>
-            <artifactId>commons-configuration</artifactId>
-            <version>1.7</version>
-        </dependency>
-        <dependency>
-            <groupId>net.sf.jopt-simple</groupId>
-            <artifactId>jopt-simple</artifactId>
-            <version>3.2</version>
-        </dependency>
-        <dependency>
-            <groupId>org.ebaysf.web</groupId>
-            <artifactId>cors-filter</artifactId>
-            <version>${ebay.cors.filter}</version>
-        </dependency>
-        <dependency>
-            <groupId>com.jcraft</groupId>
-            <artifactId>jsch</artifactId>
-            <version>0.1.53</version>
-        </dependency>
-        <!-- dependency> <groupId>org.ogce</groupId> <artifactId>bcgss</artifactId>
-            <version>146</version> </dependency> -->
-        <dependency>
-            <groupId>org.apache.xmlbeans</groupId>
-            <artifactId>xmlbeans</artifactId>
-            <version>${xmlbeans.version}</version>
-            <exclusions>
-                <exclusion>
-                    <groupId>stax</groupId>
-                    <artifactId>stax-api</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.thrift</groupId>
-            <artifactId>libthrift</artifactId>
-            <version>${thrift.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>com.fasterxml.jackson.core</groupId>
-            <artifactId>jackson-databind</artifactId>
-            <version>2.0.0</version>
-        </dependency>
-        <dependency>
-            <groupId>com.fasterxml.jackson.core</groupId>
-            <artifactId>jackson-core</artifactId>
-            <version>2.0.0</version>
-        </dependency>
-        <dependency>
-            <groupId>com.fasterxml.jackson.core</groupId>
-            <artifactId>jackson-annotations</artifactId>
-            <version>2.0.0</version>
-        </dependency>
-        <!-- zookeeper dependencies -->
-
-        <dependency>
-            <groupId>org.apache.zookeeper</groupId>
-            <artifactId>zookeeper</artifactId>
-            <version>3.4.0</version>
-        </dependency>
-        <dependency>
-            <groupId>commons-cli</groupId>
-            <artifactId>commons-cli</artifactId>
-            <version>1.2</version>
-        </dependency>
-
-        <dependency>
-            <groupId>com.rabbitmq</groupId>
-            <artifactId>amqp-client</artifactId>
-            <version>${amqp.client.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.curator</groupId>
-            <artifactId>curator-framework</artifactId>
-            <version>${curator.version}</version>
-        </dependency>
-
-        <!-- ======================== Sample =================== -->
-        <dependency>
-            <groupId>org.apache.airavata</groupId>
-            <artifactId>airavata-client-samples</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-    </dependencies>
-
-
-    <properties>
-        <jersey.version>1.13</jersey.version>
-        <grizzly.version>2.0.0-M3</grizzly.version>
-        <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-        <archieve.name>apache-airavata-server</archieve.name>
-        <airavata.dist.name>${archieve.name}-${project.version}</airavata.dist.name>
-        <airavata.work.dir>${project.build.directory}/tests/${airavata.dist.name}</airavata.work.dir>
-        <airavata.bin.zip>${project.build.directory}/${airavata.dist.name}-bin.zip</airavata.bin.zip>
-        <airavata.bin.tar.gz>${project.build.directory}/${airavata.dist.name}-bin.tar.gz</airavata.bin.tar.gz>
-    </properties>
-</project>

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/distribution/src/main/assembly/bin-assembly.xml
----------------------------------------------------------------------
diff --git a/distribution/src/main/assembly/bin-assembly.xml b/distribution/src/main/assembly/bin-assembly.xml
deleted file mode 100644
index b9146d5..0000000
--- a/distribution/src/main/assembly/bin-assembly.xml
+++ /dev/null
@@ -1,160 +0,0 @@
-<!--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. -->
-
-<!DOCTYPE assembly [
-        <!ELEMENT assembly (id|includeBaseDirectory|baseDirectory|formats|fileSets|dependencySets)*>
-        <!ELEMENT id (#PCDATA)>
-        <!ELEMENT includeBaseDirectory (#PCDATA)>
-        <!ELEMENT baseDirectory (#PCDATA)>
-        <!ELEMENT formats (format)*>
-        <!ELEMENT format (#PCDATA)>
-        <!ELEMENT fileSets (fileSet)*>
-        <!ELEMENT fileSet (directory|outputDirectory|fileMode|includes)*>
-        <!ELEMENT directory (#PCDATA)>
-        <!ELEMENT outputDirectory (#PCDATA)>
-        <!ELEMENT includes (include)*>
-        <!ELEMENT include (#PCDATA)>
-        <!ELEMENT dependencySets (dependencySet)*>
-        <!ELEMENT dependencySet (outputDirectory|outputFileNameMapping|includes)*>
-        ]>
-<assembly>
-    <id>bin</id>
-    <includeBaseDirectory>true</includeBaseDirectory>
-    <baseDirectory>${archieve.name}-${version}</baseDirectory>
-    <formats>
-        <format>tar.gz</format>
-        <format>zip</format>
-    </formats>
-
-    <fileSets>
-
-        <!-- ********************** copy release notes files ********************** -->
-        <fileSet>
-            <directory>../../../</directory>
-            <outputDirectory>.</outputDirectory>
-            <includes>
-                <include>RELEASE_NOTES</include>
-            </includes>
-        </fileSet>
-        <!-- ********************** copy licenses, readme etc. ********************** -->
-        <fileSet>
-            <directory>src/main/resources/</directory>
-            <outputDirectory>.</outputDirectory>
-            <includes>
-                <include>LICENSE</include>
-                <include>NOTICE</include>
-                <include>README</include>
-                <include>INSTALL</include>
-            </includes>
-        </fileSet>
-
-        <!-- ********************** copy database scripts ********************** -->
-        <fileSet>
-            <directory>../modules/registry/registry-core/src/main/resources/
-            </directory>
-            <outputDirectory>bin/database_scripts
-            </outputDirectory>
-            <includes>
-                <include>*sql*</include>
-            </includes>
-        </fileSet>
-        <fileSet>
-            <directory>src/main/resources/bin</directory>
-            <outputDirectory>bin</outputDirectory>
-            <fileMode>777</fileMode>
-            <includes>
-                <include>*.sh</include>
-                <include>*.bat</include>
-                <include>logo.txt</include>
-                <include>startNetworkServer</include>
-            </includes>
-        </fileSet>
-        <fileSet>
-            <directory>src/main/resources/samples</directory>
-            <outputDirectory>samples</outputDirectory>
-            <fileMode>777</fileMode>
-            <includes>
-                <include>*.sh</include>
-                <include>**/*.sh</include>
-            </includes>
-        </fileSet>
-        <fileSet>
-            <directory>${project.build.directory}/conf</directory>
-            <outputDirectory>bin</outputDirectory>
-            <includes>
-                <include>airavata-server.properties</include>
-                <include>zoo.cfg</include>
-                <include>registry.properties</include>
-                <include>log4j.properties</include>
-                <include>host.xml</include>
-                <include>persistence.xml</include>
-                <include>provenance.sql</include>
-                <include>gfac-config.yaml</include>
-                <include>PBSTemplate.xslt</include>
-                <include>SLURMTemplate.xslt</include>
-                <include>LSFTemplate.xslt</include>
-                <include>UGETemplate.xslt</include>
-                <include>ForkTemplate.xslt</include>
-                <include>gsissh.properties</include>
-                <include>airavata.jks</include>
-                <include>client_truststore.jks</include>
-                <include>airavata-default-xacml-policy.xml</include>
-            </includes>
-        </fileSet>
-
-        <!-- Create logs directory -->
-        <fileSet>
-            <directory>./</directory>
-            <outputDirectory>logs</outputDirectory>
-            <excludes>
-                <exclude>*/**</exclude>
-            </excludes>
-        </fileSet>
-
-        <!-- ********************** Copy samples ********************** -->
-        <fileSet>
-            <directory>${project.build.directory}/samples/applications
-            </directory>
-            <outputDirectory>samples</outputDirectory>
-            <includes>
-                <include>*.sh</include>
-                <include>*.bat</include>
-            </includes>
-        </fileSet>
-
-    </fileSets>
-
-    <dependencySets>
-        <dependencySet>
-            <useProjectArtifact>false</useProjectArtifact>
-            <outputDirectory>lib</outputDirectory>
-            <outputFileNameMapping>${artifact.artifactId}.${artifact.extension}</outputFileNameMapping>
-            <includes>
-                <include>org.apache.derby:derby:jar</include>
-                <include>org.apache.derby:derbytools:jar</include>
-                <include>org.apache.derby:derbynet:jar</include>
-                <include>org.apache.derby:derbyclient:jar</include>
-            </includes>
-        </dependencySet>
-        <dependencySet>
-            <useProjectArtifact>false</useProjectArtifact>
-            <outputDirectory>lib</outputDirectory>
-            <includes>
-                <include>*:*:jar</include>
-            </includes>
-            <excludes>
-                <exclude>mysql:mysql-connector-java</exclude>
-            </excludes>
-        </dependencySet>
-
-    </dependencySets>
-
-</assembly>

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/distribution/src/main/assembly/src-assembly.xml
----------------------------------------------------------------------
diff --git a/distribution/src/main/assembly/src-assembly.xml b/distribution/src/main/assembly/src-assembly.xml
deleted file mode 100644
index 6a093ed..0000000
--- a/distribution/src/main/assembly/src-assembly.xml
+++ /dev/null
@@ -1,75 +0,0 @@
-<!--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.
-  -->
-
-<assembly>
-    <id>src</id>
-    <includeBaseDirectory>true</includeBaseDirectory> 
-    <baseDirectory>${archieve.name}-${version}</baseDirectory>
-    <formats>
-        <format>tar.gz</format>  
-        <format>zip</format>
-    </formats>
-
-    <fileSets>
-        <fileSet>
-            <directory>../..</directory>
-            <outputDirectory></outputDirectory>
-            <includes>
-                <include>NOTICE</include>
-                <include>LICENSE</include>
-                <include>README</include>
-                <include>RELEASE_NOTES</include>
-		<include>DISCLAIMER</include>
-		<include>INSTALL</include>
-            </includes>
-            <filtered>true</filtered>
-        </fileSet>
-        <fileSet>
-            <directory>../..</directory>
-            <outputDirectory></outputDirectory>
-            <useDefaultExcludes>true</useDefaultExcludes>
-            <includes>
-                <include>pom.xml</include>
-                <include>modules/**</include>
-                <include>samples/**</include>
-            </includes>
-
-            <excludes>
-                                <!-- Exclusions from org.apache.resources:apache-source-release-assembly-descriptor.
-                     Note that they assume that all sources are located under an "src" directory. This
-                     is not the case for Axis2, which doesn't use the standard Maven 2 conventions.
-                     Thus we may still encounter some issues here. -->
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/).*${project.build.directory}.*]</exclude>
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?maven-eclipse\.xml]</exclude>
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.project]</exclude>
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.classpath]</exclude>
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?[^/]*\.iws]</exclude>
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?[^/]*\.ipr]</exclude>
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?[^/]*\.iml]</exclude>
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.settings(/.*)?]</exclude>
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.externalToolBuilders(/.*)?]</exclude>
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.deployables(/.*)?]</exclude>
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.wtpmodules(/.*)?]</exclude>
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?cobertura\.ser]</exclude>
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?pom\.xml\.releaseBackup]</exclude>
-                <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?release\.properties]</exclude>
-            </excludes>
-
-        </fileSet>
-          </fileSets>
-</assembly>

http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/distribution/src/main/resources/INSTALL
----------------------------------------------------------------------
diff --git a/distribution/src/main/resources/INSTALL b/distribution/src/main/resources/INSTALL
deleted file mode 100644
index 53d0550..0000000
--- a/distribution/src/main/resources/INSTALL
+++ /dev/null
@@ -1,30 +0,0 @@
-Installing  Apache Airavata 0.14
--------------------------------
-
-Prerequisites
--------------
-Java 1.5 or later
-Maven (tested on v 3.0.2)
-
-Build Apache Airavata from Source
----------------------------------
-* Unzip/untar the source file or clone from git.
-* cd to project folder and type
-	$ mvn clean install
-	Note: in order to skip tests use the command
-			$ mvn clean install -Dmaven.test.skip=true
-* Alternatively, all  compressed binary distributions can be found at <PROJECT DIR>/modules/distribution/release/target/release-artifacts
-
-Running Tests
--------------
-* Unit tests & integrations tests will run while Apache Airavata is built from source (without "-Dmaven.test.skip=true").
-* To run the test samples
-    - You can find the binary distributions at <PROJECT DIR>/modules/distribution/release/target/release-artifacts or from
-      the Apache Airavata download site.
-    - Extract the binary distributions and once the binary is unzipped, instructions to run the tests should be followed
-      from README files found within.
-
-Tutorials
-----------
-The airavata website has instructions for basic tutorials:
-* Describing and executing applications using Airavata - follow "XBAYA Quick-Start Tutorial" tutorial at https://cwiki.apache.org/confluence/display/AIRAVATA/XBAYA+Quick-Start+Tutorial
\ No newline at end of file


[14/50] [abbrv] airavata git commit: return empty list when user not existant

Posted by la...@apache.org.
return empty list when user not existant


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: fe87f9c908f5df1968627e1b61dd24573ce663e7
Parents: b9b2480
Author: scnakandala <su...@gmail.com>
Authored: Fri Aug 12 11:49:31 2016 -0400
Committer: scnakandala <su...@gmail.com>
Committed: Fri Aug 12 11:49:31 2016 -0400

----------------------------------------------------------------------
 .../registry/api/service/handler/RegistryServerHandler.java       | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/fe87f9c9/modules/registry/registry-server/registry-api-service/src/main/java/org/apache/airavata/registry/api/service/handler/RegistryServerHandler.java
----------------------------------------------------------------------
diff --git a/modules/registry/registry-server/registry-api-service/src/main/java/org/apache/airavata/registry/api/service/handler/RegistryServerHandler.java b/modules/registry/registry-server/registry-api-service/src/main/java/org/apache/airavata/registry/api/service/handler/RegistryServerHandler.java
index 7936f7a..1d7c74b 100644
--- a/modules/registry/registry-server/registry-api-service/src/main/java/org/apache/airavata/registry/api/service/handler/RegistryServerHandler.java
+++ b/modules/registry/registry-server/registry-api-service/src/main/java/org/apache/airavata/registry/api/service/handler/RegistryServerHandler.java
@@ -466,6 +466,9 @@ public class RegistryServerHandler implements RegistryService.Iface {
             exception.setMessage("Username cannot be empty. Please provide a valid user..");
             throw exception;
         }
+        if(!isUserExists(gatewayId, userName)){
+            return new ArrayList<>();
+        }
         if (!isGatewayExistInternal(gatewayId)){
             logger.error("Gateway does not exist.Please provide a valid gateway id...");
             throw new AiravataSystemException(AiravataErrorType.INTERNAL_ERROR);


[26/50] [abbrv] airavata git commit: Fixed incompatible durable bits

Posted by la...@apache.org.
Fixed incompatible durable bits


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: 3f29cfdbd71de18777557713dce58007a3cbc2f5
Parents: cfe62c3
Author: Shameera Rathnayaka <sh...@gmail.com>
Authored: Mon Aug 15 16:18:01 2016 -0400
Committer: Shameera Rathnayaka <sh...@gmail.com>
Committed: Mon Aug 15 16:39:49 2016 -0400

----------------------------------------------------------------------
 .../org/apache/airavata/messaging/core/MessagingFactory.java   | 6 +++---
 .../apache/airavata/messaging/core/impl/RabbitMQPublisher.java | 2 +-
 .../airavata/messaging/core/impl/RabbitMQSubscriber.java       | 4 ++--
 3 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/3f29cfdb/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessagingFactory.java
----------------------------------------------------------------------
diff --git a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessagingFactory.java b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessagingFactory.java
index 802ea5a..2d5cae1 100644
--- a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessagingFactory.java
+++ b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessagingFactory.java
@@ -48,19 +48,19 @@ public class MessagingFactory {
             case EXPERIMENT_LAUNCH:
                 subscriber = getExperimentSubscriber(rProperties);
                 subscriber.listen(((connection, channel) -> new ExperimentConsumer(messageHandler, connection, channel)),
-                        null,
+                        rProperties.getQueueName(),
                         routingKeys);
                 break;
             case PROCESS_LAUNCH:
                 subscriber = getProcessSubscriber(rProperties);
                 subscriber.listen((connection ,channel) -> new ProcessConsumer(messageHandler, connection, channel),
-                        null,
+                        rProperties.getQueueName(),
                         routingKeys);
                 break;
             case STATUS:
                 subscriber = getStatusSubscriber(rProperties);
                 subscriber.listen((connection, channel) -> new StatusConsumer(messageHandler, connection, channel),
-                        null,
+                        rProperties.getQueueName(),
                         routingKeys);
                 break;
             default:

http://git-wip-us.apache.org/repos/asf/airavata/blob/3f29cfdb/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQPublisher.java
----------------------------------------------------------------------
diff --git a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQPublisher.java b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQPublisher.java
index 6f1d1d8..188ded3 100644
--- a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQPublisher.java
+++ b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQPublisher.java
@@ -93,7 +93,7 @@ public class RabbitMQPublisher implements Publisher {
             message.setMessageType(messageContext.getType());
             message.setUpdatedTime(messageContext.getUpdatedTime().getTime());
             String routingKey = routingKeySupplier.apply(messageContext);
-            log.info("publish messageId:" + messageContext.getMessageId() + ", messageType:" + messageContext.getType() + ", to routingKey:" + routingKey);
+//            log.info("publish messageId:" + messageContext.getMessageId() + ", messageType:" + messageContext.getType() + ", to routingKey:" + routingKey);
             byte[] messageBody = ThriftUtils.serializeThriftObject(message);
             send(messageBody, routingKey);
         } catch (TException e) {

http://git-wip-us.apache.org/repos/asf/airavata/blob/3f29cfdb/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQSubscriber.java
----------------------------------------------------------------------
diff --git a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQSubscriber.java b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQSubscriber.java
index 6b28723..2a7da56 100644
--- a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQSubscriber.java
+++ b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQSubscriber.java
@@ -64,7 +64,7 @@ public class RabbitMQSubscriber implements Subscriber {
             channel.basicQos(properties.getPrefetchCount());
             channel.exchangeDeclare(properties.getExchangeName(),
                     properties.getExchangeType(),
-                    false);
+                    true); // durable
         } catch (Exception e) {
             String msg = "could not open channel for exchange " + properties.getExchangeName();
             log.error(msg);
@@ -98,7 +98,7 @@ public class RabbitMQSubscriber implements Subscriber {
             }
             // bind all the routing keys
             for (String key : routingKeys) {
-                log.info("Binding key:" + key + " to queue:" + queueName);
+//                log.info("Binding key:" + key + " to queue:" + queueName);
                 channel.queueBind(queueName, properties.getExchangeName(), key);
             }
 


[39/50] [abbrv] airavata git commit: merging jeffs gsoc project

Posted by la...@apache.org.
merging jeffs gsoc project


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: ff18106ab2559d053392c6b3e8a2c6e3172f9ad4
Parents: ef310d3 7b14e0f
Author: scnakandala <su...@gmail.com>
Authored: Mon Aug 22 23:36:46 2016 -0400
Committer: scnakandala <su...@gmail.com>
Committed: Mon Aug 22 23:36:46 2016 -0400

----------------------------------------------------------------------
 modules/amqpwstunnel/python/amqpwstunnel.py | 583 +++++++++++++++++++++++
 modules/amqpwstunnel/python/config.json     |  10 +
 modules/amqpwstunnel/wstest.html            | 157 ++++++
 3 files changed, 750 insertions(+)
----------------------------------------------------------------------



[34/50] [abbrv] airavata git commit: adding requester username to gateway

Posted by la...@apache.org.
adding requester username to gateway


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: d0c6957cfb0fe298e740a5c564d7dcf2aef769ce
Parents: 779cc37
Author: scnakandala <su...@gmail.com>
Authored: Thu Aug 18 11:53:35 2016 -0400
Committer: scnakandala <su...@gmail.com>
Committed: Thu Aug 18 11:53:35 2016 -0400

----------------------------------------------------------------------
 .../lib/airavata/workspace_model_types.cpp      |  30 ++++-
 .../lib/airavata/workspace_model_types.h        |  15 ++-
 .../lib/Airavata/Model/Workspace/Types.php      |  25 ++++
 .../apache/airavata/model/workspace/ttypes.py   |  18 ++-
 .../airavata/model/workspace/Gateway.java       | 114 ++++++++++++++++++-
 .../model/workspace/GatewayApprovalStatus.java  |   5 +-
 .../catalog/impl/GatewayRegistry.java           |   1 +
 .../core/experiment/catalog/model/Gateway.java  |  10 ++
 .../catalog/resources/GatewayResource.java      |  11 ++
 .../experiment/catalog/resources/Utils.java     |   2 +-
 .../utils/ThriftDataModelConversion.java        |   1 +
 .../src/main/resources/expcatalog-derby.sql     |  14 +++
 .../src/main/resources/expcatalog-mysql.sql     |  14 +++
 .../workspace_model.thrift                      |   6 +-
 14 files changed, 251 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/d0c6957c/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/workspace_model_types.cpp
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/workspace_model_types.cpp b/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/workspace_model_types.cpp
index 0eccc15..edd4822 100644
--- a/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/workspace_model_types.cpp
+++ b/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/workspace_model_types.cpp
@@ -36,7 +36,8 @@ int _kGatewayApprovalStatusValues[] = {
   GatewayApprovalStatus::ACTIVE,
   GatewayApprovalStatus::DEACTIVATED,
   GatewayApprovalStatus::CANCELLED,
-  GatewayApprovalStatus::DENIED
+  GatewayApprovalStatus::DENIED,
+  GatewayApprovalStatus::CREATED
 };
 const char* _kGatewayApprovalStatusNames[] = {
   "REQUESTED",
@@ -44,9 +45,10 @@ const char* _kGatewayApprovalStatusNames[] = {
   "ACTIVE",
   "DEACTIVATED",
   "CANCELLED",
-  "DENIED"
+  "DENIED",
+  "CREATED"
 };
-const std::map<int, const char*> _GatewayApprovalStatus_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(6, _kGatewayApprovalStatusValues, _kGatewayApprovalStatusNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL));
+const std::map<int, const char*> _GatewayApprovalStatus_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(7, _kGatewayApprovalStatusValues, _kGatewayApprovalStatusNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL));
 
 int _kNotificationPriorityValues[] = {
   NotificationPriority::LOW,
@@ -750,6 +752,11 @@ void Gateway::__set_requestCreationTime(const int64_t val) {
 __isset.requestCreationTime = true;
 }
 
+void Gateway::__set_requesterUsername(const std::string& val) {
+  this->requesterUsername = val;
+__isset.requesterUsername = true;
+}
+
 uint32_t Gateway::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
@@ -919,6 +926,14 @@ uint32_t Gateway::read(::apache::thrift::protocol::TProtocol* iprot) {
           xfer += iprot->skip(ftype);
         }
         break;
+      case 19:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->requesterUsername);
+          this->__isset.requesterUsername = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -1028,6 +1043,11 @@ uint32_t Gateway::write(::apache::thrift::protocol::TProtocol* oprot) const {
     xfer += oprot->writeI64(this->requestCreationTime);
     xfer += oprot->writeFieldEnd();
   }
+  if (this->__isset.requesterUsername) {
+    xfer += oprot->writeFieldBegin("requesterUsername", ::apache::thrift::protocol::T_STRING, 19);
+    xfer += oprot->writeString(this->requesterUsername);
+    xfer += oprot->writeFieldEnd();
+  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
@@ -1053,6 +1073,7 @@ void swap(Gateway &a, Gateway &b) {
   swap(a.oauthClientId, b.oauthClientId);
   swap(a.oauthClientSecret, b.oauthClientSecret);
   swap(a.requestCreationTime, b.requestCreationTime);
+  swap(a.requesterUsername, b.requesterUsername);
   swap(a.__isset, b.__isset);
 }
 
@@ -1075,6 +1096,7 @@ Gateway::Gateway(const Gateway& other19) {
   oauthClientId = other19.oauthClientId;
   oauthClientSecret = other19.oauthClientSecret;
   requestCreationTime = other19.requestCreationTime;
+  requesterUsername = other19.requesterUsername;
   __isset = other19.__isset;
 }
 Gateway& Gateway::operator=(const Gateway& other20) {
@@ -1096,6 +1118,7 @@ Gateway& Gateway::operator=(const Gateway& other20) {
   oauthClientId = other20.oauthClientId;
   oauthClientSecret = other20.oauthClientSecret;
   requestCreationTime = other20.requestCreationTime;
+  requesterUsername = other20.requesterUsername;
   __isset = other20.__isset;
   return *this;
 }
@@ -1120,6 +1143,7 @@ void Gateway::printTo(std::ostream& out) const {
   out << ", " << "oauthClientId="; (__isset.oauthClientId ? (out << to_string(oauthClientId)) : (out << "<null>"));
   out << ", " << "oauthClientSecret="; (__isset.oauthClientSecret ? (out << to_string(oauthClientSecret)) : (out << "<null>"));
   out << ", " << "requestCreationTime="; (__isset.requestCreationTime ? (out << to_string(requestCreationTime)) : (out << "<null>"));
+  out << ", " << "requesterUsername="; (__isset.requesterUsername ? (out << to_string(requesterUsername)) : (out << "<null>"));
   out << ")";
 }
 

http://git-wip-us.apache.org/repos/asf/airavata/blob/d0c6957c/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/workspace_model_types.h
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/workspace_model_types.h b/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/workspace_model_types.h
index e30bb92..d6db351 100644
--- a/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/workspace_model_types.h
+++ b/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/workspace_model_types.h
@@ -44,7 +44,8 @@ struct GatewayApprovalStatus {
     ACTIVE = 2,
     DEACTIVATED = 3,
     CANCELLED = 4,
-    DENIED = 5
+    DENIED = 5,
+    CREATED = 6
   };
 };
 
@@ -298,7 +299,7 @@ inline std::ostream& operator<<(std::ostream& out, const User& obj)
 }
 
 typedef struct _Gateway__isset {
-  _Gateway__isset() : gatewayName(false), domain(false), emailAddress(false), gatewayAcronym(false), gatewayURL(false), gatewayPublicAbstract(false), reviewProposalDescription(false), gatewayAdminFirstName(false), gatewayAdminLastName(false), gatewayAdminEmail(false), identityServerUserName(false), identityServerPasswordToken(false), declinedReason(false), oauthClientId(false), oauthClientSecret(false), requestCreationTime(false) {}
+  _Gateway__isset() : gatewayName(false), domain(false), emailAddress(false), gatewayAcronym(false), gatewayURL(false), gatewayPublicAbstract(false), reviewProposalDescription(false), gatewayAdminFirstName(false), gatewayAdminLastName(false), gatewayAdminEmail(false), identityServerUserName(false), identityServerPasswordToken(false), declinedReason(false), oauthClientId(false), oauthClientSecret(false), requestCreationTime(false), requesterUsername(false) {}
   bool gatewayName :1;
   bool domain :1;
   bool emailAddress :1;
@@ -315,6 +316,7 @@ typedef struct _Gateway__isset {
   bool oauthClientId :1;
   bool oauthClientSecret :1;
   bool requestCreationTime :1;
+  bool requesterUsername :1;
 } _Gateway__isset;
 
 class Gateway {
@@ -322,7 +324,7 @@ class Gateway {
 
   Gateway(const Gateway&);
   Gateway& operator=(const Gateway&);
-  Gateway() : gatewayId(), gatewayApprovalStatus((GatewayApprovalStatus::type)0), gatewayName(), domain(), emailAddress(), gatewayAcronym(), gatewayURL(), gatewayPublicAbstract(), reviewProposalDescription(), gatewayAdminFirstName(), gatewayAdminLastName(), gatewayAdminEmail(), identityServerUserName(), identityServerPasswordToken(), declinedReason(), oauthClientId(), oauthClientSecret(), requestCreationTime(0) {
+  Gateway() : gatewayId(), gatewayApprovalStatus((GatewayApprovalStatus::type)0), gatewayName(), domain(), emailAddress(), gatewayAcronym(), gatewayURL(), gatewayPublicAbstract(), reviewProposalDescription(), gatewayAdminFirstName(), gatewayAdminLastName(), gatewayAdminEmail(), identityServerUserName(), identityServerPasswordToken(), declinedReason(), oauthClientId(), oauthClientSecret(), requestCreationTime(0), requesterUsername() {
   }
 
   virtual ~Gateway() throw();
@@ -344,6 +346,7 @@ class Gateway {
   std::string oauthClientId;
   std::string oauthClientSecret;
   int64_t requestCreationTime;
+  std::string requesterUsername;
 
   _Gateway__isset __isset;
 
@@ -383,6 +386,8 @@ class Gateway {
 
   void __set_requestCreationTime(const int64_t val);
 
+  void __set_requesterUsername(const std::string& val);
+
   bool operator == (const Gateway & rhs) const
   {
     if (!(gatewayId == rhs.gatewayId))
@@ -453,6 +458,10 @@ class Gateway {
       return false;
     else if (__isset.requestCreationTime && !(requestCreationTime == rhs.requestCreationTime))
       return false;
+    if (__isset.requesterUsername != rhs.__isset.requesterUsername)
+      return false;
+    else if (__isset.requesterUsername && !(requesterUsername == rhs.requesterUsername))
+      return false;
     return true;
   }
   bool operator != (const Gateway &rhs) const {

http://git-wip-us.apache.org/repos/asf/airavata/blob/d0c6957c/airavata-api/airavata-client-sdks/airavata-php-sdk/src/main/resources/lib/Airavata/Model/Workspace/Types.php
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-client-sdks/airavata-php-sdk/src/main/resources/lib/Airavata/Model/Workspace/Types.php b/airavata-api/airavata-client-sdks/airavata-php-sdk/src/main/resources/lib/Airavata/Model/Workspace/Types.php
index 71a2eca..098511f 100644
--- a/airavata-api/airavata-client-sdks/airavata-php-sdk/src/main/resources/lib/Airavata/Model/Workspace/Types.php
+++ b/airavata-api/airavata-client-sdks/airavata-php-sdk/src/main/resources/lib/Airavata/Model/Workspace/Types.php
@@ -24,6 +24,7 @@ final class GatewayApprovalStatus {
   const DEACTIVATED = 3;
   const CANCELLED = 4;
   const DENIED = 5;
+  const CREATED = 6;
   static public $__names = array(
     0 => 'REQUESTED',
     1 => 'APPROVED',
@@ -31,6 +32,7 @@ final class GatewayApprovalStatus {
     3 => 'DEACTIVATED',
     4 => 'CANCELLED',
     5 => 'DENIED',
+    6 => 'CREATED',
   );
 }
 
@@ -696,6 +698,10 @@ class Gateway {
    * @var int
    */
   public $requestCreationTime = null;
+  /**
+   * @var string
+   */
+  public $requesterUsername = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -772,6 +778,10 @@ class Gateway {
           'var' => 'requestCreationTime',
           'type' => TType::I64,
           ),
+        19 => array(
+          'var' => 'requesterUsername',
+          'type' => TType::STRING,
+          ),
         );
     }
     if (is_array($vals)) {
@@ -829,6 +839,9 @@ class Gateway {
       if (isset($vals['requestCreationTime'])) {
         $this->requestCreationTime = $vals['requestCreationTime'];
       }
+      if (isset($vals['requesterUsername'])) {
+        $this->requesterUsername = $vals['requesterUsername'];
+      }
     }
   }
 
@@ -977,6 +990,13 @@ class Gateway {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 19:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->requesterUsername);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -1080,6 +1100,11 @@ class Gateway {
       $xfer += $output->writeI64($this->requestCreationTime);
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->requesterUsername !== null) {
+      $xfer += $output->writeFieldBegin('requesterUsername', TType::STRING, 19);
+      $xfer += $output->writeString($this->requesterUsername);
+      $xfer += $output->writeFieldEnd();
+    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;

http://git-wip-us.apache.org/repos/asf/airavata/blob/d0c6957c/airavata-api/airavata-client-sdks/airavata-python-sdk/src/main/resources/lib/apache/airavata/model/workspace/ttypes.py
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-client-sdks/airavata-python-sdk/src/main/resources/lib/apache/airavata/model/workspace/ttypes.py b/airavata-api/airavata-client-sdks/airavata-python-sdk/src/main/resources/lib/apache/airavata/model/workspace/ttypes.py
index 77a3116..d3e6914 100644
--- a/airavata-api/airavata-client-sdks/airavata-python-sdk/src/main/resources/lib/apache/airavata/model/workspace/ttypes.py
+++ b/airavata-api/airavata-client-sdks/airavata-python-sdk/src/main/resources/lib/apache/airavata/model/workspace/ttypes.py
@@ -25,6 +25,7 @@ class GatewayApprovalStatus:
   DEACTIVATED = 3
   CANCELLED = 4
   DENIED = 5
+  CREATED = 6
 
   _VALUES_TO_NAMES = {
     0: "REQUESTED",
@@ -33,6 +34,7 @@ class GatewayApprovalStatus:
     3: "DEACTIVATED",
     4: "CANCELLED",
     5: "DENIED",
+    6: "CREATED",
   }
 
   _NAMES_TO_VALUES = {
@@ -42,6 +44,7 @@ class GatewayApprovalStatus:
     "DEACTIVATED": 3,
     "CANCELLED": 4,
     "DENIED": 5,
+    "CREATED": 6,
   }
 
 class NotificationPriority:
@@ -477,6 +480,7 @@ class Gateway:
    - oauthClientId
    - oauthClientSecret
    - requestCreationTime
+   - requesterUsername
   """
 
   thrift_spec = (
@@ -499,9 +503,10 @@ class Gateway:
     (16, TType.STRING, 'oauthClientId', None, None, ), # 16
     (17, TType.STRING, 'oauthClientSecret', None, None, ), # 17
     (18, TType.I64, 'requestCreationTime', None, None, ), # 18
+    (19, TType.STRING, 'requesterUsername', None, None, ), # 19
   )
 
-  def __init__(self, gatewayId=None, gatewayApprovalStatus=None, gatewayName=None, domain=None, emailAddress=None, gatewayAcronym=None, gatewayURL=None, gatewayPublicAbstract=None, reviewProposalDescription=None, gatewayAdminFirstName=None, gatewayAdminLastName=None, gatewayAdminEmail=None, identityServerUserName=None, identityServerPasswordToken=None, declinedReason=None, oauthClientId=None, oauthClientSecret=None, requestCreationTime=None,):
+  def __init__(self, gatewayId=None, gatewayApprovalStatus=None, gatewayName=None, domain=None, emailAddress=None, gatewayAcronym=None, gatewayURL=None, gatewayPublicAbstract=None, reviewProposalDescription=None, gatewayAdminFirstName=None, gatewayAdminLastName=None, gatewayAdminEmail=None, identityServerUserName=None, identityServerPasswordToken=None, declinedReason=None, oauthClientId=None, oauthClientSecret=None, requestCreationTime=None, requesterUsername=None,):
     self.gatewayId = gatewayId
     self.gatewayApprovalStatus = gatewayApprovalStatus
     self.gatewayName = gatewayName
@@ -520,6 +525,7 @@ class Gateway:
     self.oauthClientId = oauthClientId
     self.oauthClientSecret = oauthClientSecret
     self.requestCreationTime = requestCreationTime
+    self.requesterUsername = requesterUsername
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -620,6 +626,11 @@ class Gateway:
           self.requestCreationTime = iprot.readI64()
         else:
           iprot.skip(ftype)
+      elif fid == 19:
+        if ftype == TType.STRING:
+          self.requesterUsername = iprot.readString()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -702,6 +713,10 @@ class Gateway:
       oprot.writeFieldBegin('requestCreationTime', TType.I64, 18)
       oprot.writeI64(self.requestCreationTime)
       oprot.writeFieldEnd()
+    if self.requesterUsername is not None:
+      oprot.writeFieldBegin('requesterUsername', TType.STRING, 19)
+      oprot.writeString(self.requesterUsername)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -733,6 +748,7 @@ class Gateway:
     value = (value * 31) ^ hash(self.oauthClientId)
     value = (value * 31) ^ hash(self.oauthClientSecret)
     value = (value * 31) ^ hash(self.requestCreationTime)
+    value = (value * 31) ^ hash(self.requesterUsername)
     return value
 
   def __repr__(self):

http://git-wip-us.apache.org/repos/asf/airavata/blob/d0c6957c/airavata-api/airavata-data-models/src/main/java/org/apache/airavata/model/workspace/Gateway.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-data-models/src/main/java/org/apache/airavata/model/workspace/Gateway.java b/airavata-api/airavata-data-models/src/main/java/org/apache/airavata/model/workspace/Gateway.java
index cfe04d3..c32cc0d 100644
--- a/airavata-api/airavata-data-models/src/main/java/org/apache/airavata/model/workspace/Gateway.java
+++ b/airavata-api/airavata-data-models/src/main/java/org/apache/airavata/model/workspace/Gateway.java
@@ -73,6 +73,7 @@ public class Gateway implements org.apache.thrift.TBase<Gateway, Gateway._Fields
   private static final org.apache.thrift.protocol.TField OAUTH_CLIENT_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("oauthClientId", org.apache.thrift.protocol.TType.STRING, (short)16);
   private static final org.apache.thrift.protocol.TField OAUTH_CLIENT_SECRET_FIELD_DESC = new org.apache.thrift.protocol.TField("oauthClientSecret", org.apache.thrift.protocol.TType.STRING, (short)17);
   private static final org.apache.thrift.protocol.TField REQUEST_CREATION_TIME_FIELD_DESC = new org.apache.thrift.protocol.TField("requestCreationTime", org.apache.thrift.protocol.TType.I64, (short)18);
+  private static final org.apache.thrift.protocol.TField REQUESTER_USERNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("requesterUsername", org.apache.thrift.protocol.TType.STRING, (short)19);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -98,6 +99,7 @@ public class Gateway implements org.apache.thrift.TBase<Gateway, Gateway._Fields
   private String oauthClientId; // optional
   private String oauthClientSecret; // optional
   private long requestCreationTime; // optional
+  private String requesterUsername; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -122,7 +124,8 @@ public class Gateway implements org.apache.thrift.TBase<Gateway, Gateway._Fields
     DECLINED_REASON((short)15, "declinedReason"),
     OAUTH_CLIENT_ID((short)16, "oauthClientId"),
     OAUTH_CLIENT_SECRET((short)17, "oauthClientSecret"),
-    REQUEST_CREATION_TIME((short)18, "requestCreationTime");
+    REQUEST_CREATION_TIME((short)18, "requestCreationTime"),
+    REQUESTER_USERNAME((short)19, "requesterUsername");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -173,6 +176,8 @@ public class Gateway implements org.apache.thrift.TBase<Gateway, Gateway._Fields
           return OAUTH_CLIENT_SECRET;
         case 18: // REQUEST_CREATION_TIME
           return REQUEST_CREATION_TIME;
+        case 19: // REQUESTER_USERNAME
+          return REQUESTER_USERNAME;
         default:
           return null;
       }
@@ -215,7 +220,7 @@ public class Gateway implements org.apache.thrift.TBase<Gateway, Gateway._Fields
   // isset id assignments
   private static final int __REQUESTCREATIONTIME_ISSET_ID = 0;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.GATEWAY_NAME,_Fields.DOMAIN,_Fields.EMAIL_ADDRESS,_Fields.GATEWAY_ACRONYM,_Fields.GATEWAY_URL,_Fields.GATEWAY_PUBLIC_ABSTRACT,_Fields.REVIEW_PROPOSAL_DESCRIPTION,_Fields.GATEWAY_ADMIN_FIRST_NAME,_Fields.GATEWAY_ADMIN_LAST_NAME,_Fields.GATEWAY_ADMIN_EMAIL,_Fields.IDENTITY_SERVER_USER_NAME,_Fields.IDENTITY_SERVER_PASSWORD_TOKEN,_Fields.DECLINED_REASON,_Fields.OAUTH_CLIENT_ID,_Fields.OAUTH_CLIENT_SECRET,_Fields.REQUEST_CREATION_TIME};
+  private static final _Fields optionals[] = {_Fields.GATEWAY_NAME,_Fields.DOMAIN,_Fields.EMAIL_ADDRESS,_Fields.GATEWAY_ACRONYM,_Fields.GATEWAY_URL,_Fields.GATEWAY_PUBLIC_ABSTRACT,_Fields.REVIEW_PROPOSAL_DESCRIPTION,_Fields.GATEWAY_ADMIN_FIRST_NAME,_Fields.GATEWAY_ADMIN_LAST_NAME,_Fields.GATEWAY_ADMIN_EMAIL,_Fields.IDENTITY_SERVER_USER_NAME,_Fields.IDENTITY_SERVER_PASSWORD_TOKEN,_Fields.DECLINED_REASON,_Fields.OAUTH_CLIENT_ID,_Fields.OAUTH_CLIENT_SECRET,_Fields.REQUEST_CREATION_TIME,_Fields.REQUESTER_USERNAME};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -255,6 +260,8 @@ public class Gateway implements org.apache.thrift.TBase<Gateway, Gateway._Fields
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.REQUEST_CREATION_TIME, new org.apache.thrift.meta_data.FieldMetaData("requestCreationTime", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.REQUESTER_USERNAME, new org.apache.thrift.meta_data.FieldMetaData("requesterUsername", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(Gateway.class, metaDataMap);
   }
@@ -328,6 +335,9 @@ public class Gateway implements org.apache.thrift.TBase<Gateway, Gateway._Fields
       this.oauthClientSecret = other.oauthClientSecret;
     }
     this.requestCreationTime = other.requestCreationTime;
+    if (other.isSetRequesterUsername()) {
+      this.requesterUsername = other.requesterUsername;
+    }
   }
 
   public Gateway deepCopy() {
@@ -355,6 +365,7 @@ public class Gateway implements org.apache.thrift.TBase<Gateway, Gateway._Fields
     this.oauthClientSecret = null;
     setRequestCreationTimeIsSet(false);
     this.requestCreationTime = 0;
+    this.requesterUsername = null;
   }
 
   public String getGatewayId() {
@@ -778,6 +789,29 @@ public class Gateway implements org.apache.thrift.TBase<Gateway, Gateway._Fields
     __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __REQUESTCREATIONTIME_ISSET_ID, value);
   }
 
+  public String getRequesterUsername() {
+    return this.requesterUsername;
+  }
+
+  public void setRequesterUsername(String requesterUsername) {
+    this.requesterUsername = requesterUsername;
+  }
+
+  public void unsetRequesterUsername() {
+    this.requesterUsername = null;
+  }
+
+  /** Returns true if field requesterUsername is set (has been assigned a value) and false otherwise */
+  public boolean isSetRequesterUsername() {
+    return this.requesterUsername != null;
+  }
+
+  public void setRequesterUsernameIsSet(boolean value) {
+    if (!value) {
+      this.requesterUsername = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case GATEWAY_ID:
@@ -924,6 +958,14 @@ public class Gateway implements org.apache.thrift.TBase<Gateway, Gateway._Fields
       }
       break;
 
+    case REQUESTER_USERNAME:
+      if (value == null) {
+        unsetRequesterUsername();
+      } else {
+        setRequesterUsername((String)value);
+      }
+      break;
+
     }
   }
 
@@ -983,6 +1025,9 @@ public class Gateway implements org.apache.thrift.TBase<Gateway, Gateway._Fields
     case REQUEST_CREATION_TIME:
       return getRequestCreationTime();
 
+    case REQUESTER_USERNAME:
+      return getRequesterUsername();
+
     }
     throw new IllegalStateException();
   }
@@ -1030,6 +1075,8 @@ public class Gateway implements org.apache.thrift.TBase<Gateway, Gateway._Fields
       return isSetOauthClientSecret();
     case REQUEST_CREATION_TIME:
       return isSetRequestCreationTime();
+    case REQUESTER_USERNAME:
+      return isSetRequesterUsername();
     }
     throw new IllegalStateException();
   }
@@ -1209,6 +1256,15 @@ public class Gateway implements org.apache.thrift.TBase<Gateway, Gateway._Fields
         return false;
     }
 
+    boolean this_present_requesterUsername = true && this.isSetRequesterUsername();
+    boolean that_present_requesterUsername = true && that.isSetRequesterUsername();
+    if (this_present_requesterUsername || that_present_requesterUsername) {
+      if (!(this_present_requesterUsername && that_present_requesterUsername))
+        return false;
+      if (!this.requesterUsername.equals(that.requesterUsername))
+        return false;
+    }
+
     return true;
   }
 
@@ -1306,6 +1362,11 @@ public class Gateway implements org.apache.thrift.TBase<Gateway, Gateway._Fields
     if (present_requestCreationTime)
       list.add(requestCreationTime);
 
+    boolean present_requesterUsername = true && (isSetRequesterUsername());
+    list.add(present_requesterUsername);
+    if (present_requesterUsername)
+      list.add(requesterUsername);
+
     return list.hashCode();
   }
 
@@ -1497,6 +1558,16 @@ public class Gateway implements org.apache.thrift.TBase<Gateway, Gateway._Fields
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetRequesterUsername()).compareTo(other.isSetRequesterUsername());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetRequesterUsername()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.requesterUsername, other.requesterUsername);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -1688,6 +1759,16 @@ public class Gateway implements org.apache.thrift.TBase<Gateway, Gateway._Fields
       sb.append(this.requestCreationTime);
       first = false;
     }
+    if (isSetRequesterUsername()) {
+      if (!first) sb.append(", ");
+      sb.append("requesterUsername:");
+      if (this.requesterUsername == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.requesterUsername);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -1885,6 +1966,14 @@ public class Gateway implements org.apache.thrift.TBase<Gateway, Gateway._Fields
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 19: // REQUESTER_USERNAME
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.requesterUsername = iprot.readString();
+              struct.setRequesterUsernameIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -2018,6 +2107,13 @@ public class Gateway implements org.apache.thrift.TBase<Gateway, Gateway._Fields
         oprot.writeI64(struct.requestCreationTime);
         oprot.writeFieldEnd();
       }
+      if (struct.requesterUsername != null) {
+        if (struct.isSetRequesterUsername()) {
+          oprot.writeFieldBegin(REQUESTER_USERNAME_FIELD_DESC);
+          oprot.writeString(struct.requesterUsername);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -2086,7 +2182,10 @@ public class Gateway implements org.apache.thrift.TBase<Gateway, Gateway._Fields
       if (struct.isSetRequestCreationTime()) {
         optionals.set(15);
       }
-      oprot.writeBitSet(optionals, 16);
+      if (struct.isSetRequesterUsername()) {
+        optionals.set(16);
+      }
+      oprot.writeBitSet(optionals, 17);
       if (struct.isSetGatewayName()) {
         oprot.writeString(struct.gatewayName);
       }
@@ -2135,6 +2234,9 @@ public class Gateway implements org.apache.thrift.TBase<Gateway, Gateway._Fields
       if (struct.isSetRequestCreationTime()) {
         oprot.writeI64(struct.requestCreationTime);
       }
+      if (struct.isSetRequesterUsername()) {
+        oprot.writeString(struct.requesterUsername);
+      }
     }
 
     @Override
@@ -2144,7 +2246,7 @@ public class Gateway implements org.apache.thrift.TBase<Gateway, Gateway._Fields
       struct.setGatewayIdIsSet(true);
       struct.gatewayApprovalStatus = org.apache.airavata.model.workspace.GatewayApprovalStatus.findByValue(iprot.readI32());
       struct.setGatewayApprovalStatusIsSet(true);
-      BitSet incoming = iprot.readBitSet(16);
+      BitSet incoming = iprot.readBitSet(17);
       if (incoming.get(0)) {
         struct.gatewayName = iprot.readString();
         struct.setGatewayNameIsSet(true);
@@ -2209,6 +2311,10 @@ public class Gateway implements org.apache.thrift.TBase<Gateway, Gateway._Fields
         struct.requestCreationTime = iprot.readI64();
         struct.setRequestCreationTimeIsSet(true);
       }
+      if (incoming.get(16)) {
+        struct.requesterUsername = iprot.readString();
+        struct.setRequesterUsernameIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/airavata/blob/d0c6957c/airavata-api/airavata-data-models/src/main/java/org/apache/airavata/model/workspace/GatewayApprovalStatus.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-data-models/src/main/java/org/apache/airavata/model/workspace/GatewayApprovalStatus.java b/airavata-api/airavata-data-models/src/main/java/org/apache/airavata/model/workspace/GatewayApprovalStatus.java
index 64faff9..e53ee20 100644
--- a/airavata-api/airavata-data-models/src/main/java/org/apache/airavata/model/workspace/GatewayApprovalStatus.java
+++ b/airavata-api/airavata-data-models/src/main/java/org/apache/airavata/model/workspace/GatewayApprovalStatus.java
@@ -34,7 +34,8 @@ public enum GatewayApprovalStatus implements org.apache.thrift.TEnum {
   ACTIVE(2),
   DEACTIVATED(3),
   CANCELLED(4),
-  DENIED(5);
+  DENIED(5),
+  CREATED(6);
 
   private final int value;
 
@@ -67,6 +68,8 @@ public enum GatewayApprovalStatus implements org.apache.thrift.TEnum {
         return CANCELLED;
       case 5:
         return DENIED;
+      case 6:
+        return CREATED;
       default:
         return null;
     }

http://git-wip-us.apache.org/repos/asf/airavata/blob/d0c6957c/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/impl/GatewayRegistry.java
----------------------------------------------------------------------
diff --git a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/impl/GatewayRegistry.java b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/impl/GatewayRegistry.java
index 515b83b..83d67f2 100644
--- a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/impl/GatewayRegistry.java
+++ b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/impl/GatewayRegistry.java
@@ -67,6 +67,7 @@ public class GatewayRegistry {
             resource.setOauthClientId(gateway.getOauthClientId());
             resource.setOauthClientSecret(gateway.getOauthClientSecret());
             resource.setRequestCreationTime(new Timestamp(System.currentTimeMillis()));
+            resource.setRequesterUsername(gateway.getRequesterUsername());
             resource.save();
             return gateway.getGatewayId();
         }catch (RegistryException e){

http://git-wip-us.apache.org/repos/asf/airavata/blob/d0c6957c/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/model/Gateway.java
----------------------------------------------------------------------
diff --git a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/model/Gateway.java b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/model/Gateway.java
index e3f4113..afb67a1 100644
--- a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/model/Gateway.java
+++ b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/model/Gateway.java
@@ -49,6 +49,7 @@ public class Gateway {
     private String oauthClientId;
     private String getOauthClientSecret;
     private Timestamp requestCreationTime;
+    private String requesterUsername;
     private Collection<GatewayWorker> gatewayWorkers;
     private Collection<Project> projects;
 
@@ -179,6 +180,15 @@ public class Gateway {
         this.identityServerPasswordToken = identityServerPasswordToken;
     }
 
+    @Column(name = "REQUESTER_USERNAME")
+    public String getRequesterUsername() {
+        return requesterUsername;
+    }
+
+    public void setRequesterUsername(String requesterUsername) {
+        this.requesterUsername = requesterUsername;
+    }
+
     @Column(name = "DECLINED_REASON")
     public String getDeclinedReason() {
         return declinedReason;

http://git-wip-us.apache.org/repos/asf/airavata/blob/d0c6957c/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/resources/GatewayResource.java
----------------------------------------------------------------------
diff --git a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/resources/GatewayResource.java b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/resources/GatewayResource.java
index 54bee96..646594e 100644
--- a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/resources/GatewayResource.java
+++ b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/resources/GatewayResource.java
@@ -56,6 +56,7 @@ public class GatewayResource extends AbstractExpCatResource {
     private String oauthClientId;
     private String oauthClientSecret;
     private Timestamp requestCreationTime;
+    private String requesterUsername;
 
     public String getGatewayAdminFirstName() {
         return gatewayAdminFirstName;
@@ -185,6 +186,14 @@ public class GatewayResource extends AbstractExpCatResource {
         this.requestCreationTime = requestCreationTime;
     }
 
+    public String getRequesterUsername() {
+        return requesterUsername;
+    }
+
+    public void setRequesterUsername(String requesterUsername) {
+        this.requesterUsername = requesterUsername;
+    }
+
     /**
      *
      * @param gatewayId gateway name
@@ -473,6 +482,7 @@ public class GatewayResource extends AbstractExpCatResource {
             gateway.setOauthClientId(oauthClientId);
             gateway.setGetOauthClientSecret(oauthClientSecret);
             gateway.setRequestCreationTime(requestCreationTime);
+            gateway.setRequesterUsername(requesterUsername);
             if (existingGateway != null) {
                 existingGateway.setDomain(domain);
                 existingGateway.setGatewayApprovalStatus(gatewayApprovalStatus);
@@ -492,6 +502,7 @@ public class GatewayResource extends AbstractExpCatResource {
                 existingGateway.setOauthClientId(oauthClientId);
                 existingGateway.setGetOauthClientSecret(oauthClientSecret);
                 existingGateway.setRequestCreationTime(requestCreationTime);
+                existingGateway.setRequesterUsername(requesterUsername);
                 em.merge(existingGateway);
             } else {
                 em.persist(gateway);

http://git-wip-us.apache.org/repos/asf/airavata/blob/d0c6957c/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/resources/Utils.java
----------------------------------------------------------------------
diff --git a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/resources/Utils.java b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/resources/Utils.java
index ff5f4e3..4852467 100644
--- a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/resources/Utils.java
+++ b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/resources/Utils.java
@@ -359,7 +359,7 @@ public class Utils {
         gatewayResource.setDeclinedReason(o.getDeclinedReason());
         gatewayResource.setOauthClientId(o.getOauthClientId());
         gatewayResource.setRequestCreationTime(o.getRequestCreationTime());
-        gatewayResource.setRequestCreationTime(o.getRequestCreationTime());
+        gatewayResource.setRequesterUsername(o.getRequesterUsername());
 
         gatewayResource.setOauthClientSecret(o.getGetOauthClientSecret());
         return gatewayResource;

http://git-wip-us.apache.org/repos/asf/airavata/blob/d0c6957c/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/utils/ThriftDataModelConversion.java
----------------------------------------------------------------------
diff --git a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/utils/ThriftDataModelConversion.java b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/utils/ThriftDataModelConversion.java
index 97a83cb..2fec213 100644
--- a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/utils/ThriftDataModelConversion.java
+++ b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/utils/ThriftDataModelConversion.java
@@ -96,6 +96,7 @@ public class ThriftDataModelConversion {
         if (resource.getRequestCreationTime() != null) {
             gateway.setRequestCreationTime(resource.getRequestCreationTime().getTime());
         }
+        gateway.setRequesterUsername(resource.getRequesterUsername());
         return gateway;
     }
 

http://git-wip-us.apache.org/repos/asf/airavata/blob/d0c6957c/modules/registry/registry-core/src/main/resources/expcatalog-derby.sql
----------------------------------------------------------------------
diff --git a/modules/registry/registry-core/src/main/resources/expcatalog-derby.sql b/modules/registry/registry-core/src/main/resources/expcatalog-derby.sql
index ea5b758..275c772 100644
--- a/modules/registry/registry-core/src/main/resources/expcatalog-derby.sql
+++ b/modules/registry/registry-core/src/main/resources/expcatalog-derby.sql
@@ -24,6 +24,20 @@ CREATE TABLE GATEWAY
         GATEWAY_NAME VARCHAR(255),
 	      DOMAIN VARCHAR(255),
 	      EMAIL_ADDRESS VARCHAR(255),
+        GATEWAY_ACRONYM varchar(255),
+        GATEWAY_ADMIN_EMAIL varchar(255),
+        GATEWAY_ADMIN_FIRST_NAME varchar(255),
+        GATEWAY_APPROVAL_STATUS varchar(255),
+        GATEWAY_PUBLIC_ABSTRACT varchar(255),
+        GATEWAY_URL varchar(255),
+        GATEWAY_ADMIN_LAST_NAME varchar(255),
+        IDENTITY_SERVER_PASSWORD_TOKEN varchar(255),
+        IDENTITY_SERVER_USERNAME varchar(255),
+        GATEWAY_REVIEW_PROPOSAL_DESCRIPTION varchar(255),
+        DECLINED_REASON varchar(255),
+        OAUTH_CLIENT_SECRET varchar(255),
+        OAUTH_CLIENT_ID varchar(255),
+        REQUEST_CREATION_TIME datetime DEFAULT CURRENT_TIMESTAMP,
         PRIMARY KEY (GATEWAY_ID)
 );
 

http://git-wip-us.apache.org/repos/asf/airavata/blob/d0c6957c/modules/registry/registry-core/src/main/resources/expcatalog-mysql.sql
----------------------------------------------------------------------
diff --git a/modules/registry/registry-core/src/main/resources/expcatalog-mysql.sql b/modules/registry/registry-core/src/main/resources/expcatalog-mysql.sql
index c9a38f0..e5d6b09 100644
--- a/modules/registry/registry-core/src/main/resources/expcatalog-mysql.sql
+++ b/modules/registry/registry-core/src/main/resources/expcatalog-mysql.sql
@@ -24,6 +24,20 @@ CREATE TABLE GATEWAY
         GATEWAY_NAME VARCHAR(255),
 	      DOMAIN VARCHAR(255),
 	      EMAIL_ADDRESS VARCHAR(255),
+        GATEWAY_ACRONYM varchar(255),
+        GATEWAY_ADMIN_EMAIL varchar(255),
+        GATEWAY_ADMIN_FIRST_NAME varchar(255),
+        GATEWAY_APPROVAL_STATUS varchar(255),
+        GATEWAY_PUBLIC_ABSTRACT varchar(255),
+        GATEWAY_URL varchar(255),
+        GATEWAY_ADMIN_LAST_NAME varchar(255),
+        IDENTITY_SERVER_PASSWORD_TOKEN varchar(255),
+        IDENTITY_SERVER_USERNAME varchar(255),
+        GATEWAY_REVIEW_PROPOSAL_DESCRIPTION varchar(255),
+        DECLINED_REASON varchar(255),
+        OAUTH_CLIENT_SECRET varchar(255),
+        OAUTH_CLIENT_ID varchar(255),
+        REQUEST_CREATION_TIME datetime DEFAULT CURRENT_TIMESTAMP,
         PRIMARY KEY (GATEWAY_ID)
 );
 

http://git-wip-us.apache.org/repos/asf/airavata/blob/d0c6957c/thrift-interface-descriptions/data-models/experiment-catalog-models/workspace_model.thrift
----------------------------------------------------------------------
diff --git a/thrift-interface-descriptions/data-models/experiment-catalog-models/workspace_model.thrift b/thrift-interface-descriptions/data-models/experiment-catalog-models/workspace_model.thrift
index 8ece13f..61659c5 100644
--- a/thrift-interface-descriptions/data-models/experiment-catalog-models/workspace_model.thrift
+++ b/thrift-interface-descriptions/data-models/experiment-catalog-models/workspace_model.thrift
@@ -66,7 +66,8 @@ enum GatewayApprovalStatus {
     ACTIVE,
     DEACTIVATED,
     CANCELLED,
-    DENIED
+    DENIED,
+    CREATED
 }
 
 struct Gateway {
@@ -87,7 +88,8 @@ struct Gateway {
     15: optional string declinedReason,
     16: optional string oauthClientId,
     17: optional string oauthClientSecret,
-    18: optional i64 requestCreationTime
+    18: optional i64 requestCreationTime,
+    19: optional string requesterUsername
 }
 
 enum NotificationPriority {


[41/50] [abbrv] airavata git commit: Fixed qos issue

Posted by la...@apache.org.
Fixed qos issue


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: 1dab79b6452280929412599eca3281c0f9fab28c
Parents: ef310d3
Author: Shameera Rathnayaka <sh...@gmail.com>
Authored: Wed Aug 24 12:39:04 2016 -0400
Committer: Shameera Rathnayaka <sh...@gmail.com>
Committed: Wed Aug 24 12:39:04 2016 -0400

----------------------------------------------------------------------
 modules/gfac/gfac-core/pom.xml                  | 12 +++---
 .../apache/airavata/gfac/core/GFacUtils.java    |  4 +-
 .../airavata/gfac/core/GFacUtilsTest.java       | 43 ++++++++++++++++++++
 3 files changed, 51 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/1dab79b6/modules/gfac/gfac-core/pom.xml
----------------------------------------------------------------------
diff --git a/modules/gfac/gfac-core/pom.xml b/modules/gfac/gfac-core/pom.xml
index 69253d2..8d358ff 100644
--- a/modules/gfac/gfac-core/pom.xml
+++ b/modules/gfac/gfac-core/pom.xml
@@ -82,12 +82,12 @@
             <artifactId>junit</artifactId>
             <scope>test</scope>
         </dependency>
-        <dependency>
-            <groupId>org.testng</groupId>
-            <artifactId>testng</artifactId>
-            <version>6.1.1</version>
-            <scope>test</scope>
-        </dependency>
+        <!--<dependency>-->
+            <!--<groupId>org.testng</groupId>-->
+            <!--<artifactId>testng</artifactId>-->
+            <!--<version>6.1.1</version>-->
+            <!--<scope>test</scope>-->
+        <!--</dependency>-->
         <dependency>
             <groupId>org.slf4j</groupId>
             <artifactId>jcl-over-slf4j</artifactId>

http://git-wip-us.apache.org/repos/asf/airavata/blob/1dab79b6/modules/gfac/gfac-core/src/main/java/org/apache/airavata/gfac/core/GFacUtils.java
----------------------------------------------------------------------
diff --git a/modules/gfac/gfac-core/src/main/java/org/apache/airavata/gfac/core/GFacUtils.java b/modules/gfac/gfac-core/src/main/java/org/apache/airavata/gfac/core/GFacUtils.java
index f7d53dc..0ed836f 100644
--- a/modules/gfac/gfac-core/src/main/java/org/apache/airavata/gfac/core/GFacUtils.java
+++ b/modules/gfac/gfac-core/src/main/java/org/apache/airavata/gfac/core/GFacUtils.java
@@ -659,13 +659,13 @@ public class GFacUtils {
         return inputValues;
     }
 
-    private static String getQoS(String qualityOfService, String preferredBatchQueue) {
+    static String getQoS(String qualityOfService, String preferredBatchQueue) {
         if(preferredBatchQueue == null  || preferredBatchQueue.isEmpty()
                 ||  qualityOfService == null  || qualityOfService.isEmpty()) return null;
         final String qos = "qos";
         Pattern pattern = Pattern.compile(preferredBatchQueue + "=(?<" + qos + ">[^,]*)");
         Matcher matcher = pattern.matcher(qualityOfService);
-        if (matcher.matches()) {
+        if (matcher.find()) {
             return matcher.group(qos);
         }
         return null;

http://git-wip-us.apache.org/repos/asf/airavata/blob/1dab79b6/modules/gfac/gfac-core/src/test/java/org/apache/airavata/gfac/core/GFacUtilsTest.java
----------------------------------------------------------------------
diff --git a/modules/gfac/gfac-core/src/test/java/org/apache/airavata/gfac/core/GFacUtilsTest.java b/modules/gfac/gfac-core/src/test/java/org/apache/airavata/gfac/core/GFacUtilsTest.java
new file mode 100644
index 0000000..ecea66f
--- /dev/null
+++ b/modules/gfac/gfac-core/src/test/java/org/apache/airavata/gfac/core/GFacUtilsTest.java
@@ -0,0 +1,43 @@
+/**
+ *
+ * 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.airavata.gfac.core;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class GFacUtilsTest {
+
+    @Test
+    public void testGetQoS_1() throws Exception {
+        String qos = "shared=oneweek";
+        String shared = GFacUtils.getQoS(qos, "shared");
+        Assert.assertNotNull(shared);
+        Assert.assertEquals("oneweek", shared);
+    }
+    @Test
+    public void testGetQoS_2() throws Exception {
+        String qos = "shared=oneweek,compute=oneweek";
+        String shared = GFacUtils.getQoS(qos, "shared");
+        Assert.assertNotNull(shared);
+        Assert.assertEquals("oneweek", shared);
+    }
+
+
+}
\ No newline at end of file


[31/50] [abbrv] airavata git commit: Fixed NPE with experiment cancellation

Posted by la...@apache.org.
Fixed NPE with experiment cancellation


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: b284167d29172e28cff71575d82731a124ce4c4d
Parents: 309a9ff
Author: Shameera Rathnayaka <sh...@gmail.com>
Authored: Tue Aug 16 16:25:29 2016 -0400
Committer: Shameera Rathnayaka <sh...@gmail.com>
Committed: Tue Aug 16 16:25:29 2016 -0400

----------------------------------------------------------------------
 .../airavata/api/server/handler/AiravataServerHandler.java    | 7 +++----
 1 file changed, 3 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/b284167d/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java b/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
index e489b43..b21be18 100644
--- a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
+++ b/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
@@ -1454,16 +1454,15 @@ public class AiravataServerHandler implements Airavata.Iface {
                 logger.error(airavataExperimentId, "Error while cancelling experiment {}, experiment doesn't exist.", airavataExperimentId);
                 throw new ExperimentNotFoundException("Requested experiment id " + airavataExperimentId + " does not exist in the system..");
             }
-            ExperimentStatus experimentStatus = null;
-            switch (experimentStatus.getState()) {
+            switch (existingExperiment.getExperimentStatus().getState()) {
                 case COMPLETED: case CANCELED: case FAILED: case CANCELING:
-                    logger.warn("Can't terminate already {} experiment", experimentStatus.getState().name());
+                    logger.warn("Can't terminate already {} experiment", existingExperiment.getExperimentStatus().getState().name());
                     break;
                 case CREATED:
                     logger.warn("Experiment termination is only allowed for launched experiments.");
                     break;
                 default:
-                    submitCancelExperiment(airavataExperimentId, gatewayId);
+                    submitCancelExperiment(gatewayId, airavataExperimentId);
                     logger.debug("Airavata cancelled experiment with experiment id : " + airavataExperimentId);
                     break;
             }


[43/50] [abbrv] airavata git commit: making experiment status a list in ExperimentModel

Posted by la...@apache.org.
http://git-wip-us.apache.org/repos/asf/airavata/blob/415e9b70/airavata-api/airavata-data-models/src/main/java/org/apache/airavata/model/experiment/ExperimentModel.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-data-models/src/main/java/org/apache/airavata/model/experiment/ExperimentModel.java b/airavata-api/airavata-data-models/src/main/java/org/apache/airavata/model/experiment/ExperimentModel.java
index 4c97ae5..7b01741 100644
--- a/airavata-api/airavata-data-models/src/main/java/org/apache/airavata/model/experiment/ExperimentModel.java
+++ b/airavata-api/airavata-data-models/src/main/java/org/apache/airavata/model/experiment/ExperimentModel.java
@@ -87,7 +87,7 @@ public class ExperimentModel implements org.apache.thrift.TBase<ExperimentModel,
   private static final org.apache.thrift.protocol.TField USER_CONFIGURATION_DATA_FIELD_DESC = new org.apache.thrift.protocol.TField("userConfigurationData", org.apache.thrift.protocol.TType.STRUCT, (short)14);
   private static final org.apache.thrift.protocol.TField EXPERIMENT_INPUTS_FIELD_DESC = new org.apache.thrift.protocol.TField("experimentInputs", org.apache.thrift.protocol.TType.LIST, (short)15);
   private static final org.apache.thrift.protocol.TField EXPERIMENT_OUTPUTS_FIELD_DESC = new org.apache.thrift.protocol.TField("experimentOutputs", org.apache.thrift.protocol.TType.LIST, (short)16);
-  private static final org.apache.thrift.protocol.TField EXPERIMENT_STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("experimentStatus", org.apache.thrift.protocol.TType.STRUCT, (short)17);
+  private static final org.apache.thrift.protocol.TField EXPERIMENT_STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("experimentStatus", org.apache.thrift.protocol.TType.LIST, (short)17);
   private static final org.apache.thrift.protocol.TField ERRORS_FIELD_DESC = new org.apache.thrift.protocol.TField("errors", org.apache.thrift.protocol.TType.LIST, (short)18);
   private static final org.apache.thrift.protocol.TField PROCESSES_FIELD_DESC = new org.apache.thrift.protocol.TField("processes", org.apache.thrift.protocol.TType.LIST, (short)19);
 
@@ -113,7 +113,7 @@ public class ExperimentModel implements org.apache.thrift.TBase<ExperimentModel,
   private UserConfigurationDataModel userConfigurationData; // optional
   private List<org.apache.airavata.model.application.io.InputDataObjectType> experimentInputs; // optional
   private List<org.apache.airavata.model.application.io.OutputDataObjectType> experimentOutputs; // optional
-  private org.apache.airavata.model.status.ExperimentStatus experimentStatus; // optional
+  private List<org.apache.airavata.model.status.ExperimentStatus> experimentStatus; // optional
   private List<org.apache.airavata.model.commons.ErrorModel> errors; // optional
   private List<org.apache.airavata.model.process.ProcessModel> processes; // optional
 
@@ -277,7 +277,8 @@ public class ExperimentModel implements org.apache.thrift.TBase<ExperimentModel,
         new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
             new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.airavata.model.application.io.OutputDataObjectType.class))));
     tmpMap.put(_Fields.EXPERIMENT_STATUS, new org.apache.thrift.meta_data.FieldMetaData("experimentStatus", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
-        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.airavata.model.status.ExperimentStatus.class)));
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.airavata.model.status.ExperimentStatus.class))));
     tmpMap.put(_Fields.ERRORS, new org.apache.thrift.meta_data.FieldMetaData("errors", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
             new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.airavata.model.commons.ErrorModel.class))));
@@ -371,7 +372,11 @@ public class ExperimentModel implements org.apache.thrift.TBase<ExperimentModel,
       this.experimentOutputs = __this__experimentOutputs;
     }
     if (other.isSetExperimentStatus()) {
-      this.experimentStatus = new org.apache.airavata.model.status.ExperimentStatus(other.experimentStatus);
+      List<org.apache.airavata.model.status.ExperimentStatus> __this__experimentStatus = new ArrayList<org.apache.airavata.model.status.ExperimentStatus>(other.experimentStatus.size());
+      for (org.apache.airavata.model.status.ExperimentStatus other_element : other.experimentStatus) {
+        __this__experimentStatus.add(new org.apache.airavata.model.status.ExperimentStatus(other_element));
+      }
+      this.experimentStatus = __this__experimentStatus;
     }
     if (other.isSetErrors()) {
       List<org.apache.airavata.model.commons.ErrorModel> __this__errors = new ArrayList<org.apache.airavata.model.commons.ErrorModel>(other.errors.size());
@@ -839,11 +844,26 @@ public class ExperimentModel implements org.apache.thrift.TBase<ExperimentModel,
     }
   }
 
-  public org.apache.airavata.model.status.ExperimentStatus getExperimentStatus() {
+  public int getExperimentStatusSize() {
+    return (this.experimentStatus == null) ? 0 : this.experimentStatus.size();
+  }
+
+  public java.util.Iterator<org.apache.airavata.model.status.ExperimentStatus> getExperimentStatusIterator() {
+    return (this.experimentStatus == null) ? null : this.experimentStatus.iterator();
+  }
+
+  public void addToExperimentStatus(org.apache.airavata.model.status.ExperimentStatus elem) {
+    if (this.experimentStatus == null) {
+      this.experimentStatus = new ArrayList<org.apache.airavata.model.status.ExperimentStatus>();
+    }
+    this.experimentStatus.add(elem);
+  }
+
+  public List<org.apache.airavata.model.status.ExperimentStatus> getExperimentStatus() {
     return this.experimentStatus;
   }
 
-  public void setExperimentStatus(org.apache.airavata.model.status.ExperimentStatus experimentStatus) {
+  public void setExperimentStatus(List<org.apache.airavata.model.status.ExperimentStatus> experimentStatus) {
     this.experimentStatus = experimentStatus;
   }
 
@@ -1072,7 +1092,7 @@ public class ExperimentModel implements org.apache.thrift.TBase<ExperimentModel,
       if (value == null) {
         unsetExperimentStatus();
       } else {
-        setExperimentStatus((org.apache.airavata.model.status.ExperimentStatus)value);
+        setExperimentStatus((List<org.apache.airavata.model.status.ExperimentStatus>)value);
       }
       break;
 
@@ -1917,9 +1937,6 @@ public class ExperimentModel implements org.apache.thrift.TBase<ExperimentModel,
     if (userConfigurationData != null) {
       userConfigurationData.validate();
     }
-    if (experimentStatus != null) {
-      experimentStatus.validate();
-    }
   }
 
   private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -2120,9 +2137,19 @@ public class ExperimentModel implements org.apache.thrift.TBase<ExperimentModel,
             }
             break;
           case 17: // EXPERIMENT_STATUS
-            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-              struct.experimentStatus = new org.apache.airavata.model.status.ExperimentStatus();
-              struct.experimentStatus.read(iprot);
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list9 = iprot.readListBegin();
+                struct.experimentStatus = new ArrayList<org.apache.airavata.model.status.ExperimentStatus>(_list9.size);
+                org.apache.airavata.model.status.ExperimentStatus _elem10;
+                for (int _i11 = 0; _i11 < _list9.size; ++_i11)
+                {
+                  _elem10 = new org.apache.airavata.model.status.ExperimentStatus();
+                  _elem10.read(iprot);
+                  struct.experimentStatus.add(_elem10);
+                }
+                iprot.readListEnd();
+              }
               struct.setExperimentStatusIsSet(true);
             } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
@@ -2131,14 +2158,14 @@ public class ExperimentModel implements org.apache.thrift.TBase<ExperimentModel,
           case 18: // ERRORS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list9 = iprot.readListBegin();
-                struct.errors = new ArrayList<org.apache.airavata.model.commons.ErrorModel>(_list9.size);
-                org.apache.airavata.model.commons.ErrorModel _elem10;
-                for (int _i11 = 0; _i11 < _list9.size; ++_i11)
+                org.apache.thrift.protocol.TList _list12 = iprot.readListBegin();
+                struct.errors = new ArrayList<org.apache.airavata.model.commons.ErrorModel>(_list12.size);
+                org.apache.airavata.model.commons.ErrorModel _elem13;
+                for (int _i14 = 0; _i14 < _list12.size; ++_i14)
                 {
-                  _elem10 = new org.apache.airavata.model.commons.ErrorModel();
-                  _elem10.read(iprot);
-                  struct.errors.add(_elem10);
+                  _elem13 = new org.apache.airavata.model.commons.ErrorModel();
+                  _elem13.read(iprot);
+                  struct.errors.add(_elem13);
                 }
                 iprot.readListEnd();
               }
@@ -2150,14 +2177,14 @@ public class ExperimentModel implements org.apache.thrift.TBase<ExperimentModel,
           case 19: // PROCESSES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list12 = iprot.readListBegin();
-                struct.processes = new ArrayList<org.apache.airavata.model.process.ProcessModel>(_list12.size);
-                org.apache.airavata.model.process.ProcessModel _elem13;
-                for (int _i14 = 0; _i14 < _list12.size; ++_i14)
+                org.apache.thrift.protocol.TList _list15 = iprot.readListBegin();
+                struct.processes = new ArrayList<org.apache.airavata.model.process.ProcessModel>(_list15.size);
+                org.apache.airavata.model.process.ProcessModel _elem16;
+                for (int _i17 = 0; _i17 < _list15.size; ++_i17)
                 {
-                  _elem13 = new org.apache.airavata.model.process.ProcessModel();
-                  _elem13.read(iprot);
-                  struct.processes.add(_elem13);
+                  _elem16 = new org.apache.airavata.model.process.ProcessModel();
+                  _elem16.read(iprot);
+                  struct.processes.add(_elem16);
                 }
                 iprot.readListEnd();
               }
@@ -2252,9 +2279,9 @@ public class ExperimentModel implements org.apache.thrift.TBase<ExperimentModel,
           oprot.writeFieldBegin(EMAIL_ADDRESSES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.emailAddresses.size()));
-            for (String _iter15 : struct.emailAddresses)
+            for (String _iter18 : struct.emailAddresses)
             {
-              oprot.writeString(_iter15);
+              oprot.writeString(_iter18);
             }
             oprot.writeListEnd();
           }
@@ -2273,9 +2300,9 @@ public class ExperimentModel implements org.apache.thrift.TBase<ExperimentModel,
           oprot.writeFieldBegin(EXPERIMENT_INPUTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.experimentInputs.size()));
-            for (org.apache.airavata.model.application.io.InputDataObjectType _iter16 : struct.experimentInputs)
+            for (org.apache.airavata.model.application.io.InputDataObjectType _iter19 : struct.experimentInputs)
             {
-              _iter16.write(oprot);
+              _iter19.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -2287,9 +2314,9 @@ public class ExperimentModel implements org.apache.thrift.TBase<ExperimentModel,
           oprot.writeFieldBegin(EXPERIMENT_OUTPUTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.experimentOutputs.size()));
-            for (org.apache.airavata.model.application.io.OutputDataObjectType _iter17 : struct.experimentOutputs)
+            for (org.apache.airavata.model.application.io.OutputDataObjectType _iter20 : struct.experimentOutputs)
             {
-              _iter17.write(oprot);
+              _iter20.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -2299,7 +2326,14 @@ public class ExperimentModel implements org.apache.thrift.TBase<ExperimentModel,
       if (struct.experimentStatus != null) {
         if (struct.isSetExperimentStatus()) {
           oprot.writeFieldBegin(EXPERIMENT_STATUS_FIELD_DESC);
-          struct.experimentStatus.write(oprot);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.experimentStatus.size()));
+            for (org.apache.airavata.model.status.ExperimentStatus _iter21 : struct.experimentStatus)
+            {
+              _iter21.write(oprot);
+            }
+            oprot.writeListEnd();
+          }
           oprot.writeFieldEnd();
         }
       }
@@ -2308,9 +2342,9 @@ public class ExperimentModel implements org.apache.thrift.TBase<ExperimentModel,
           oprot.writeFieldBegin(ERRORS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.errors.size()));
-            for (org.apache.airavata.model.commons.ErrorModel _iter18 : struct.errors)
+            for (org.apache.airavata.model.commons.ErrorModel _iter22 : struct.errors)
             {
-              _iter18.write(oprot);
+              _iter22.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -2322,9 +2356,9 @@ public class ExperimentModel implements org.apache.thrift.TBase<ExperimentModel,
           oprot.writeFieldBegin(PROCESSES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.processes.size()));
-            for (org.apache.airavata.model.process.ProcessModel _iter19 : struct.processes)
+            for (org.apache.airavata.model.process.ProcessModel _iter23 : struct.processes)
             {
-              _iter19.write(oprot);
+              _iter23.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -2416,9 +2450,9 @@ public class ExperimentModel implements org.apache.thrift.TBase<ExperimentModel,
       if (struct.isSetEmailAddresses()) {
         {
           oprot.writeI32(struct.emailAddresses.size());
-          for (String _iter20 : struct.emailAddresses)
+          for (String _iter24 : struct.emailAddresses)
           {
-            oprot.writeString(_iter20);
+            oprot.writeString(_iter24);
           }
         }
       }
@@ -2428,39 +2462,45 @@ public class ExperimentModel implements org.apache.thrift.TBase<ExperimentModel,
       if (struct.isSetExperimentInputs()) {
         {
           oprot.writeI32(struct.experimentInputs.size());
-          for (org.apache.airavata.model.application.io.InputDataObjectType _iter21 : struct.experimentInputs)
+          for (org.apache.airavata.model.application.io.InputDataObjectType _iter25 : struct.experimentInputs)
           {
-            _iter21.write(oprot);
+            _iter25.write(oprot);
           }
         }
       }
       if (struct.isSetExperimentOutputs()) {
         {
           oprot.writeI32(struct.experimentOutputs.size());
-          for (org.apache.airavata.model.application.io.OutputDataObjectType _iter22 : struct.experimentOutputs)
+          for (org.apache.airavata.model.application.io.OutputDataObjectType _iter26 : struct.experimentOutputs)
           {
-            _iter22.write(oprot);
+            _iter26.write(oprot);
           }
         }
       }
       if (struct.isSetExperimentStatus()) {
-        struct.experimentStatus.write(oprot);
+        {
+          oprot.writeI32(struct.experimentStatus.size());
+          for (org.apache.airavata.model.status.ExperimentStatus _iter27 : struct.experimentStatus)
+          {
+            _iter27.write(oprot);
+          }
+        }
       }
       if (struct.isSetErrors()) {
         {
           oprot.writeI32(struct.errors.size());
-          for (org.apache.airavata.model.commons.ErrorModel _iter23 : struct.errors)
+          for (org.apache.airavata.model.commons.ErrorModel _iter28 : struct.errors)
           {
-            _iter23.write(oprot);
+            _iter28.write(oprot);
           }
         }
       }
       if (struct.isSetProcesses()) {
         {
           oprot.writeI32(struct.processes.size());
-          for (org.apache.airavata.model.process.ProcessModel _iter24 : struct.processes)
+          for (org.apache.airavata.model.process.ProcessModel _iter29 : struct.processes)
           {
-            _iter24.write(oprot);
+            _iter29.write(oprot);
           }
         }
       }
@@ -2508,13 +2548,13 @@ public class ExperimentModel implements org.apache.thrift.TBase<ExperimentModel,
       }
       if (incoming.get(6)) {
         {
-          org.apache.thrift.protocol.TList _list25 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.emailAddresses = new ArrayList<String>(_list25.size);
-          String _elem26;
-          for (int _i27 = 0; _i27 < _list25.size; ++_i27)
+          org.apache.thrift.protocol.TList _list30 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.emailAddresses = new ArrayList<String>(_list30.size);
+          String _elem31;
+          for (int _i32 = 0; _i32 < _list30.size; ++_i32)
           {
-            _elem26 = iprot.readString();
-            struct.emailAddresses.add(_elem26);
+            _elem31 = iprot.readString();
+            struct.emailAddresses.add(_elem31);
           }
         }
         struct.setEmailAddressesIsSet(true);
@@ -2526,61 +2566,70 @@ public class ExperimentModel implements org.apache.thrift.TBase<ExperimentModel,
       }
       if (incoming.get(8)) {
         {
-          org.apache.thrift.protocol.TList _list28 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.experimentInputs = new ArrayList<org.apache.airavata.model.application.io.InputDataObjectType>(_list28.size);
-          org.apache.airavata.model.application.io.InputDataObjectType _elem29;
-          for (int _i30 = 0; _i30 < _list28.size; ++_i30)
+          org.apache.thrift.protocol.TList _list33 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.experimentInputs = new ArrayList<org.apache.airavata.model.application.io.InputDataObjectType>(_list33.size);
+          org.apache.airavata.model.application.io.InputDataObjectType _elem34;
+          for (int _i35 = 0; _i35 < _list33.size; ++_i35)
           {
-            _elem29 = new org.apache.airavata.model.application.io.InputDataObjectType();
-            _elem29.read(iprot);
-            struct.experimentInputs.add(_elem29);
+            _elem34 = new org.apache.airavata.model.application.io.InputDataObjectType();
+            _elem34.read(iprot);
+            struct.experimentInputs.add(_elem34);
           }
         }
         struct.setExperimentInputsIsSet(true);
       }
       if (incoming.get(9)) {
         {
-          org.apache.thrift.protocol.TList _list31 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.experimentOutputs = new ArrayList<org.apache.airavata.model.application.io.OutputDataObjectType>(_list31.size);
-          org.apache.airavata.model.application.io.OutputDataObjectType _elem32;
-          for (int _i33 = 0; _i33 < _list31.size; ++_i33)
+          org.apache.thrift.protocol.TList _list36 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.experimentOutputs = new ArrayList<org.apache.airavata.model.application.io.OutputDataObjectType>(_list36.size);
+          org.apache.airavata.model.application.io.OutputDataObjectType _elem37;
+          for (int _i38 = 0; _i38 < _list36.size; ++_i38)
           {
-            _elem32 = new org.apache.airavata.model.application.io.OutputDataObjectType();
-            _elem32.read(iprot);
-            struct.experimentOutputs.add(_elem32);
+            _elem37 = new org.apache.airavata.model.application.io.OutputDataObjectType();
+            _elem37.read(iprot);
+            struct.experimentOutputs.add(_elem37);
           }
         }
         struct.setExperimentOutputsIsSet(true);
       }
       if (incoming.get(10)) {
-        struct.experimentStatus = new org.apache.airavata.model.status.ExperimentStatus();
-        struct.experimentStatus.read(iprot);
+        {
+          org.apache.thrift.protocol.TList _list39 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.experimentStatus = new ArrayList<org.apache.airavata.model.status.ExperimentStatus>(_list39.size);
+          org.apache.airavata.model.status.ExperimentStatus _elem40;
+          for (int _i41 = 0; _i41 < _list39.size; ++_i41)
+          {
+            _elem40 = new org.apache.airavata.model.status.ExperimentStatus();
+            _elem40.read(iprot);
+            struct.experimentStatus.add(_elem40);
+          }
+        }
         struct.setExperimentStatusIsSet(true);
       }
       if (incoming.get(11)) {
         {
-          org.apache.thrift.protocol.TList _list34 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.errors = new ArrayList<org.apache.airavata.model.commons.ErrorModel>(_list34.size);
-          org.apache.airavata.model.commons.ErrorModel _elem35;
-          for (int _i36 = 0; _i36 < _list34.size; ++_i36)
+          org.apache.thrift.protocol.TList _list42 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.errors = new ArrayList<org.apache.airavata.model.commons.ErrorModel>(_list42.size);
+          org.apache.airavata.model.commons.ErrorModel _elem43;
+          for (int _i44 = 0; _i44 < _list42.size; ++_i44)
           {
-            _elem35 = new org.apache.airavata.model.commons.ErrorModel();
-            _elem35.read(iprot);
-            struct.errors.add(_elem35);
+            _elem43 = new org.apache.airavata.model.commons.ErrorModel();
+            _elem43.read(iprot);
+            struct.errors.add(_elem43);
           }
         }
         struct.setErrorsIsSet(true);
       }
       if (incoming.get(12)) {
         {
-          org.apache.thrift.protocol.TList _list37 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.processes = new ArrayList<org.apache.airavata.model.process.ProcessModel>(_list37.size);
-          org.apache.airavata.model.process.ProcessModel _elem38;
-          for (int _i39 = 0; _i39 < _list37.size; ++_i39)
+          org.apache.thrift.protocol.TList _list45 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.processes = new ArrayList<org.apache.airavata.model.process.ProcessModel>(_list45.size);
+          org.apache.airavata.model.process.ProcessModel _elem46;
+          for (int _i47 = 0; _i47 < _list45.size; ++_i47)
           {
-            _elem38 = new org.apache.airavata.model.process.ProcessModel();
-            _elem38.read(iprot);
-            struct.processes.add(_elem38);
+            _elem46 = new org.apache.airavata.model.process.ProcessModel();
+            _elem46.read(iprot);
+            struct.processes.add(_elem46);
           }
         }
         struct.setProcessesIsSet(true);

http://git-wip-us.apache.org/repos/asf/airavata/blob/415e9b70/airavata-api/airavata-data-models/src/main/java/org/apache/airavata/model/experiment/ExperimentStatistics.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-data-models/src/main/java/org/apache/airavata/model/experiment/ExperimentStatistics.java b/airavata-api/airavata-data-models/src/main/java/org/apache/airavata/model/experiment/ExperimentStatistics.java
index 8bdaa2e..03aa46a 100644
--- a/airavata-api/airavata-data-models/src/main/java/org/apache/airavata/model/experiment/ExperimentStatistics.java
+++ b/airavata-api/airavata-data-models/src/main/java/org/apache/airavata/model/experiment/ExperimentStatistics.java
@@ -1410,14 +1410,14 @@ public class ExperimentStatistics implements org.apache.thrift.TBase<ExperimentS
           case 7: // ALL_EXPERIMENTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list40 = iprot.readListBegin();
-                struct.allExperiments = new ArrayList<ExperimentSummaryModel>(_list40.size);
-                ExperimentSummaryModel _elem41;
-                for (int _i42 = 0; _i42 < _list40.size; ++_i42)
+                org.apache.thrift.protocol.TList _list48 = iprot.readListBegin();
+                struct.allExperiments = new ArrayList<ExperimentSummaryModel>(_list48.size);
+                ExperimentSummaryModel _elem49;
+                for (int _i50 = 0; _i50 < _list48.size; ++_i50)
                 {
-                  _elem41 = new ExperimentSummaryModel();
-                  _elem41.read(iprot);
-                  struct.allExperiments.add(_elem41);
+                  _elem49 = new ExperimentSummaryModel();
+                  _elem49.read(iprot);
+                  struct.allExperiments.add(_elem49);
                 }
                 iprot.readListEnd();
               }
@@ -1429,14 +1429,14 @@ public class ExperimentStatistics implements org.apache.thrift.TBase<ExperimentS
           case 8: // COMPLETED_EXPERIMENTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list43 = iprot.readListBegin();
-                struct.completedExperiments = new ArrayList<ExperimentSummaryModel>(_list43.size);
-                ExperimentSummaryModel _elem44;
-                for (int _i45 = 0; _i45 < _list43.size; ++_i45)
+                org.apache.thrift.protocol.TList _list51 = iprot.readListBegin();
+                struct.completedExperiments = new ArrayList<ExperimentSummaryModel>(_list51.size);
+                ExperimentSummaryModel _elem52;
+                for (int _i53 = 0; _i53 < _list51.size; ++_i53)
                 {
-                  _elem44 = new ExperimentSummaryModel();
-                  _elem44.read(iprot);
-                  struct.completedExperiments.add(_elem44);
+                  _elem52 = new ExperimentSummaryModel();
+                  _elem52.read(iprot);
+                  struct.completedExperiments.add(_elem52);
                 }
                 iprot.readListEnd();
               }
@@ -1448,14 +1448,14 @@ public class ExperimentStatistics implements org.apache.thrift.TBase<ExperimentS
           case 9: // FAILED_EXPERIMENTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list46 = iprot.readListBegin();
-                struct.failedExperiments = new ArrayList<ExperimentSummaryModel>(_list46.size);
-                ExperimentSummaryModel _elem47;
-                for (int _i48 = 0; _i48 < _list46.size; ++_i48)
+                org.apache.thrift.protocol.TList _list54 = iprot.readListBegin();
+                struct.failedExperiments = new ArrayList<ExperimentSummaryModel>(_list54.size);
+                ExperimentSummaryModel _elem55;
+                for (int _i56 = 0; _i56 < _list54.size; ++_i56)
                 {
-                  _elem47 = new ExperimentSummaryModel();
-                  _elem47.read(iprot);
-                  struct.failedExperiments.add(_elem47);
+                  _elem55 = new ExperimentSummaryModel();
+                  _elem55.read(iprot);
+                  struct.failedExperiments.add(_elem55);
                 }
                 iprot.readListEnd();
               }
@@ -1467,14 +1467,14 @@ public class ExperimentStatistics implements org.apache.thrift.TBase<ExperimentS
           case 10: // CANCELLED_EXPERIMENTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list49 = iprot.readListBegin();
-                struct.cancelledExperiments = new ArrayList<ExperimentSummaryModel>(_list49.size);
-                ExperimentSummaryModel _elem50;
-                for (int _i51 = 0; _i51 < _list49.size; ++_i51)
+                org.apache.thrift.protocol.TList _list57 = iprot.readListBegin();
+                struct.cancelledExperiments = new ArrayList<ExperimentSummaryModel>(_list57.size);
+                ExperimentSummaryModel _elem58;
+                for (int _i59 = 0; _i59 < _list57.size; ++_i59)
                 {
-                  _elem50 = new ExperimentSummaryModel();
-                  _elem50.read(iprot);
-                  struct.cancelledExperiments.add(_elem50);
+                  _elem58 = new ExperimentSummaryModel();
+                  _elem58.read(iprot);
+                  struct.cancelledExperiments.add(_elem58);
                 }
                 iprot.readListEnd();
               }
@@ -1486,14 +1486,14 @@ public class ExperimentStatistics implements org.apache.thrift.TBase<ExperimentS
           case 11: // CREATED_EXPERIMENTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list52 = iprot.readListBegin();
-                struct.createdExperiments = new ArrayList<ExperimentSummaryModel>(_list52.size);
-                ExperimentSummaryModel _elem53;
-                for (int _i54 = 0; _i54 < _list52.size; ++_i54)
+                org.apache.thrift.protocol.TList _list60 = iprot.readListBegin();
+                struct.createdExperiments = new ArrayList<ExperimentSummaryModel>(_list60.size);
+                ExperimentSummaryModel _elem61;
+                for (int _i62 = 0; _i62 < _list60.size; ++_i62)
                 {
-                  _elem53 = new ExperimentSummaryModel();
-                  _elem53.read(iprot);
-                  struct.createdExperiments.add(_elem53);
+                  _elem61 = new ExperimentSummaryModel();
+                  _elem61.read(iprot);
+                  struct.createdExperiments.add(_elem61);
                 }
                 iprot.readListEnd();
               }
@@ -1505,14 +1505,14 @@ public class ExperimentStatistics implements org.apache.thrift.TBase<ExperimentS
           case 12: // RUNNING_EXPERIMENTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list55 = iprot.readListBegin();
-                struct.runningExperiments = new ArrayList<ExperimentSummaryModel>(_list55.size);
-                ExperimentSummaryModel _elem56;
-                for (int _i57 = 0; _i57 < _list55.size; ++_i57)
+                org.apache.thrift.protocol.TList _list63 = iprot.readListBegin();
+                struct.runningExperiments = new ArrayList<ExperimentSummaryModel>(_list63.size);
+                ExperimentSummaryModel _elem64;
+                for (int _i65 = 0; _i65 < _list63.size; ++_i65)
                 {
-                  _elem56 = new ExperimentSummaryModel();
-                  _elem56.read(iprot);
-                  struct.runningExperiments.add(_elem56);
+                  _elem64 = new ExperimentSummaryModel();
+                  _elem64.read(iprot);
+                  struct.runningExperiments.add(_elem64);
                 }
                 iprot.readListEnd();
               }
@@ -1558,9 +1558,9 @@ public class ExperimentStatistics implements org.apache.thrift.TBase<ExperimentS
         oprot.writeFieldBegin(ALL_EXPERIMENTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.allExperiments.size()));
-          for (ExperimentSummaryModel _iter58 : struct.allExperiments)
+          for (ExperimentSummaryModel _iter66 : struct.allExperiments)
           {
-            _iter58.write(oprot);
+            _iter66.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1571,9 +1571,9 @@ public class ExperimentStatistics implements org.apache.thrift.TBase<ExperimentS
           oprot.writeFieldBegin(COMPLETED_EXPERIMENTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.completedExperiments.size()));
-            for (ExperimentSummaryModel _iter59 : struct.completedExperiments)
+            for (ExperimentSummaryModel _iter67 : struct.completedExperiments)
             {
-              _iter59.write(oprot);
+              _iter67.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -1585,9 +1585,9 @@ public class ExperimentStatistics implements org.apache.thrift.TBase<ExperimentS
           oprot.writeFieldBegin(FAILED_EXPERIMENTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.failedExperiments.size()));
-            for (ExperimentSummaryModel _iter60 : struct.failedExperiments)
+            for (ExperimentSummaryModel _iter68 : struct.failedExperiments)
             {
-              _iter60.write(oprot);
+              _iter68.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -1599,9 +1599,9 @@ public class ExperimentStatistics implements org.apache.thrift.TBase<ExperimentS
           oprot.writeFieldBegin(CANCELLED_EXPERIMENTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.cancelledExperiments.size()));
-            for (ExperimentSummaryModel _iter61 : struct.cancelledExperiments)
+            for (ExperimentSummaryModel _iter69 : struct.cancelledExperiments)
             {
-              _iter61.write(oprot);
+              _iter69.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -1613,9 +1613,9 @@ public class ExperimentStatistics implements org.apache.thrift.TBase<ExperimentS
           oprot.writeFieldBegin(CREATED_EXPERIMENTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.createdExperiments.size()));
-            for (ExperimentSummaryModel _iter62 : struct.createdExperiments)
+            for (ExperimentSummaryModel _iter70 : struct.createdExperiments)
             {
-              _iter62.write(oprot);
+              _iter70.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -1627,9 +1627,9 @@ public class ExperimentStatistics implements org.apache.thrift.TBase<ExperimentS
           oprot.writeFieldBegin(RUNNING_EXPERIMENTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.runningExperiments.size()));
-            for (ExperimentSummaryModel _iter63 : struct.runningExperiments)
+            for (ExperimentSummaryModel _iter71 : struct.runningExperiments)
             {
-              _iter63.write(oprot);
+              _iter71.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -1660,9 +1660,9 @@ public class ExperimentStatistics implements org.apache.thrift.TBase<ExperimentS
       oprot.writeI32(struct.runningExperimentCount);
       {
         oprot.writeI32(struct.allExperiments.size());
-        for (ExperimentSummaryModel _iter64 : struct.allExperiments)
+        for (ExperimentSummaryModel _iter72 : struct.allExperiments)
         {
-          _iter64.write(oprot);
+          _iter72.write(oprot);
         }
       }
       BitSet optionals = new BitSet();
@@ -1691,45 +1691,45 @@ public class ExperimentStatistics implements org.apache.thrift.TBase<ExperimentS
       if (struct.isSetCompletedExperiments()) {
         {
           oprot.writeI32(struct.completedExperiments.size());
-          for (ExperimentSummaryModel _iter65 : struct.completedExperiments)
+          for (ExperimentSummaryModel _iter73 : struct.completedExperiments)
           {
-            _iter65.write(oprot);
+            _iter73.write(oprot);
           }
         }
       }
       if (struct.isSetFailedExperiments()) {
         {
           oprot.writeI32(struct.failedExperiments.size());
-          for (ExperimentSummaryModel _iter66 : struct.failedExperiments)
+          for (ExperimentSummaryModel _iter74 : struct.failedExperiments)
           {
-            _iter66.write(oprot);
+            _iter74.write(oprot);
           }
         }
       }
       if (struct.isSetCancelledExperiments()) {
         {
           oprot.writeI32(struct.cancelledExperiments.size());
-          for (ExperimentSummaryModel _iter67 : struct.cancelledExperiments)
+          for (ExperimentSummaryModel _iter75 : struct.cancelledExperiments)
           {
-            _iter67.write(oprot);
+            _iter75.write(oprot);
           }
         }
       }
       if (struct.isSetCreatedExperiments()) {
         {
           oprot.writeI32(struct.createdExperiments.size());
-          for (ExperimentSummaryModel _iter68 : struct.createdExperiments)
+          for (ExperimentSummaryModel _iter76 : struct.createdExperiments)
           {
-            _iter68.write(oprot);
+            _iter76.write(oprot);
           }
         }
       }
       if (struct.isSetRunningExperiments()) {
         {
           oprot.writeI32(struct.runningExperiments.size());
-          for (ExperimentSummaryModel _iter69 : struct.runningExperiments)
+          for (ExperimentSummaryModel _iter77 : struct.runningExperiments)
           {
-            _iter69.write(oprot);
+            _iter77.write(oprot);
           }
         }
       }
@@ -1749,14 +1749,14 @@ public class ExperimentStatistics implements org.apache.thrift.TBase<ExperimentS
       struct.runningExperimentCount = iprot.readI32();
       struct.setRunningExperimentCountIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list70 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.allExperiments = new ArrayList<ExperimentSummaryModel>(_list70.size);
-        ExperimentSummaryModel _elem71;
-        for (int _i72 = 0; _i72 < _list70.size; ++_i72)
+        org.apache.thrift.protocol.TList _list78 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.allExperiments = new ArrayList<ExperimentSummaryModel>(_list78.size);
+        ExperimentSummaryModel _elem79;
+        for (int _i80 = 0; _i80 < _list78.size; ++_i80)
         {
-          _elem71 = new ExperimentSummaryModel();
-          _elem71.read(iprot);
-          struct.allExperiments.add(_elem71);
+          _elem79 = new ExperimentSummaryModel();
+          _elem79.read(iprot);
+          struct.allExperiments.add(_elem79);
         }
       }
       struct.setAllExperimentsIsSet(true);
@@ -1767,70 +1767,70 @@ public class ExperimentStatistics implements org.apache.thrift.TBase<ExperimentS
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list73 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.completedExperiments = new ArrayList<ExperimentSummaryModel>(_list73.size);
-          ExperimentSummaryModel _elem74;
-          for (int _i75 = 0; _i75 < _list73.size; ++_i75)
+          org.apache.thrift.protocol.TList _list81 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.completedExperiments = new ArrayList<ExperimentSummaryModel>(_list81.size);
+          ExperimentSummaryModel _elem82;
+          for (int _i83 = 0; _i83 < _list81.size; ++_i83)
           {
-            _elem74 = new ExperimentSummaryModel();
-            _elem74.read(iprot);
-            struct.completedExperiments.add(_elem74);
+            _elem82 = new ExperimentSummaryModel();
+            _elem82.read(iprot);
+            struct.completedExperiments.add(_elem82);
           }
         }
         struct.setCompletedExperimentsIsSet(true);
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list76 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.failedExperiments = new ArrayList<ExperimentSummaryModel>(_list76.size);
-          ExperimentSummaryModel _elem77;
-          for (int _i78 = 0; _i78 < _list76.size; ++_i78)
+          org.apache.thrift.protocol.TList _list84 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.failedExperiments = new ArrayList<ExperimentSummaryModel>(_list84.size);
+          ExperimentSummaryModel _elem85;
+          for (int _i86 = 0; _i86 < _list84.size; ++_i86)
           {
-            _elem77 = new ExperimentSummaryModel();
-            _elem77.read(iprot);
-            struct.failedExperiments.add(_elem77);
+            _elem85 = new ExperimentSummaryModel();
+            _elem85.read(iprot);
+            struct.failedExperiments.add(_elem85);
           }
         }
         struct.setFailedExperimentsIsSet(true);
       }
       if (incoming.get(3)) {
         {
-          org.apache.thrift.protocol.TList _list79 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.cancelledExperiments = new ArrayList<ExperimentSummaryModel>(_list79.size);
-          ExperimentSummaryModel _elem80;
-          for (int _i81 = 0; _i81 < _list79.size; ++_i81)
+          org.apache.thrift.protocol.TList _list87 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.cancelledExperiments = new ArrayList<ExperimentSummaryModel>(_list87.size);
+          ExperimentSummaryModel _elem88;
+          for (int _i89 = 0; _i89 < _list87.size; ++_i89)
           {
-            _elem80 = new ExperimentSummaryModel();
-            _elem80.read(iprot);
-            struct.cancelledExperiments.add(_elem80);
+            _elem88 = new ExperimentSummaryModel();
+            _elem88.read(iprot);
+            struct.cancelledExperiments.add(_elem88);
           }
         }
         struct.setCancelledExperimentsIsSet(true);
       }
       if (incoming.get(4)) {
         {
-          org.apache.thrift.protocol.TList _list82 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.createdExperiments = new ArrayList<ExperimentSummaryModel>(_list82.size);
-          ExperimentSummaryModel _elem83;
-          for (int _i84 = 0; _i84 < _list82.size; ++_i84)
+          org.apache.thrift.protocol.TList _list90 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.createdExperiments = new ArrayList<ExperimentSummaryModel>(_list90.size);
+          ExperimentSummaryModel _elem91;
+          for (int _i92 = 0; _i92 < _list90.size; ++_i92)
           {
-            _elem83 = new ExperimentSummaryModel();
-            _elem83.read(iprot);
-            struct.createdExperiments.add(_elem83);
+            _elem91 = new ExperimentSummaryModel();
+            _elem91.read(iprot);
+            struct.createdExperiments.add(_elem91);
           }
         }
         struct.setCreatedExperimentsIsSet(true);
       }
       if (incoming.get(5)) {
         {
-          org.apache.thrift.protocol.TList _list85 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.runningExperiments = new ArrayList<ExperimentSummaryModel>(_list85.size);
-          ExperimentSummaryModel _elem86;
-          for (int _i87 = 0; _i87 < _list85.size; ++_i87)
+          org.apache.thrift.protocol.TList _list93 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.runningExperiments = new ArrayList<ExperimentSummaryModel>(_list93.size);
+          ExperimentSummaryModel _elem94;
+          for (int _i95 = 0; _i95 < _list93.size; ++_i95)
           {
-            _elem86 = new ExperimentSummaryModel();
-            _elem86.read(iprot);
-            struct.runningExperiments.add(_elem86);
+            _elem94 = new ExperimentSummaryModel();
+            _elem94.read(iprot);
+            struct.runningExperiments.add(_elem94);
           }
         }
         struct.setRunningExperimentsIsSet(true);

http://git-wip-us.apache.org/repos/asf/airavata/blob/415e9b70/modules/orchestrator/orchestrator-core/src/main/java/org/apache/airavata/orchestrator/core/validator/impl/ExperimentStatusValidator.java
----------------------------------------------------------------------
diff --git a/modules/orchestrator/orchestrator-core/src/main/java/org/apache/airavata/orchestrator/core/validator/impl/ExperimentStatusValidator.java b/modules/orchestrator/orchestrator-core/src/main/java/org/apache/airavata/orchestrator/core/validator/impl/ExperimentStatusValidator.java
index 9d7fa0e..a32023c 100644
--- a/modules/orchestrator/orchestrator-core/src/main/java/org/apache/airavata/orchestrator/core/validator/impl/ExperimentStatusValidator.java
+++ b/modules/orchestrator/orchestrator-core/src/main/java/org/apache/airavata/orchestrator/core/validator/impl/ExperimentStatusValidator.java
@@ -25,7 +25,6 @@ import org.apache.airavata.model.error.ValidatorResult;
 import org.apache.airavata.model.experiment.ExperimentModel;
 import org.apache.airavata.model.process.ProcessModel;
 import org.apache.airavata.model.status.ExperimentState;
-import org.apache.airavata.model.task.TaskModel;
 import org.apache.airavata.orchestrator.core.validator.JobMetadataValidator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -42,8 +41,8 @@ public class ExperimentStatusValidator implements JobMetadataValidator {
         validationResults.setValidationState(true);
         ValidatorResult validatorResult = new ValidatorResult();
         List<ValidatorResult> validatorResultList = new ArrayList<ValidatorResult>();
-        if (!experiment.getExperimentStatus().getState().equals(ExperimentState.CREATED)) {
-            error += experiment.getExperimentStatus().getState().toString();
+        if (!experiment.getExperimentStatus().get(0).getState().equals(ExperimentState.CREATED)) {
+            error += experiment.getExperimentStatus().get(0).getState().toString();
             log.error(error);
             validatorResult.setErrorDetails(error);
             validatorResult.setResult(false);

http://git-wip-us.apache.org/repos/asf/airavata/blob/415e9b70/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java
----------------------------------------------------------------------
diff --git a/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java b/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java
index 17bceb4..1a6ac25 100644
--- a/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java
+++ b/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java
@@ -29,12 +29,7 @@ import org.apache.airavata.common.utils.ThriftUtils;
 import org.apache.airavata.common.utils.ZkConstants;
 import org.apache.airavata.gfac.core.GFacUtils;
 import org.apache.airavata.gfac.core.scheduler.HostScheduler;
-import org.apache.airavata.messaging.core.MessageContext;
-import org.apache.airavata.messaging.core.MessageHandler;
-import org.apache.airavata.messaging.core.MessagingFactory;
-import org.apache.airavata.messaging.core.Publisher;
-import org.apache.airavata.messaging.core.Subscriber;
-import org.apache.airavata.messaging.core.Type;
+import org.apache.airavata.messaging.core.*;
 import org.apache.airavata.model.appcatalog.appdeployment.ApplicationDeploymentDescription;
 import org.apache.airavata.model.appcatalog.appinterface.ApplicationInterfaceDescription;
 import org.apache.airavata.model.appcatalog.computeresource.ComputeResourceDescription;
@@ -47,11 +42,7 @@ import org.apache.airavata.model.data.replica.ReplicaLocationCategory;
 import org.apache.airavata.model.error.LaunchValidationException;
 import org.apache.airavata.model.experiment.ExperimentModel;
 import org.apache.airavata.model.experiment.ExperimentType;
-import org.apache.airavata.model.messaging.event.ExperimentStatusChangeEvent;
-import org.apache.airavata.model.messaging.event.ExperimentSubmitEvent;
-import org.apache.airavata.model.messaging.event.MessageType;
-import org.apache.airavata.model.messaging.event.ProcessIdentifier;
-import org.apache.airavata.model.messaging.event.ProcessStatusChangeEvent;
+import org.apache.airavata.model.messaging.event.*;
 import org.apache.airavata.model.process.ProcessModel;
 import org.apache.airavata.model.status.ExperimentState;
 import org.apache.airavata.model.status.ExperimentStatus;
@@ -64,14 +55,7 @@ import org.apache.airavata.orchestrator.util.OrchestratorUtils;
 import org.apache.airavata.registry.core.app.catalog.resources.AppCatAbstractResource;
 import org.apache.airavata.registry.core.experiment.catalog.impl.RegistryFactory;
 import org.apache.airavata.registry.core.experiment.catalog.resources.AbstractExpCatResource;
-import org.apache.airavata.registry.cpi.AppCatalog;
-import org.apache.airavata.registry.cpi.AppCatalogException;
-import org.apache.airavata.registry.cpi.ComputeResource;
-import org.apache.airavata.registry.cpi.ExperimentCatalog;
-import org.apache.airavata.registry.cpi.ExperimentCatalogModelType;
-import org.apache.airavata.registry.cpi.RegistryException;
-import org.apache.airavata.registry.cpi.ReplicaCatalog;
-import org.apache.airavata.registry.cpi.ReplicaCatalogException;
+import org.apache.airavata.registry.cpi.*;
 import org.apache.commons.lang.StringUtils;
 import org.apache.curator.RetryPolicy;
 import org.apache.curator.framework.CuratorFramework;
@@ -84,13 +68,7 @@ import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.function.BiConsumer;
+import java.util.*;
 
 public class OrchestratorServerHandler implements OrchestratorService.Iface {
 	private static Logger log = LoggerFactory.getLogger(OrchestratorServerHandler.class);
@@ -651,7 +629,7 @@ public class OrchestratorServerHandler implements OrchestratorService.Iface {
             if (messageContext.isRedeliver()) {
 				ExperimentModel experimentModel = (ExperimentModel) experimentCatalog.
 						get(ExperimentCatalogModelType.EXPERIMENT, expEvent.getExperimentId());
-				if (experimentModel.getExperimentStatus().getState() == ExperimentState.CREATED) {
+				if (experimentModel.getExperimentStatus().get(0).getState() == ExperimentState.CREATED) {
 					launchExperiment(expEvent.getExperimentId(), expEvent.getGatewayId());
 				}
             } else {

http://git-wip-us.apache.org/repos/asf/airavata/blob/415e9b70/modules/registry/registry-core/pom.xml
----------------------------------------------------------------------
diff --git a/modules/registry/registry-core/pom.xml b/modules/registry/registry-core/pom.xml
index aa9a61b..85986ae 100644
--- a/modules/registry/registry-core/pom.xml
+++ b/modules/registry/registry-core/pom.xml
@@ -162,7 +162,7 @@
                 <inherited>true</inherited>
                 <configuration>
                     <failIfNoTests>false</failIfNoTests>
-                    <skipTests>${skipTests}</skipTests>
+                    <skipTests>true</skipTests>
                     <workingDirectory>${project.build.testOutputDirectory}</workingDirectory>
                     <!-- making sure that the sure-fire plugin doesn't run the integration tests-->
                     <!-- Integration tests are run using the fail-safe plugin in the module pom-->

http://git-wip-us.apache.org/repos/asf/airavata/blob/415e9b70/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/impl/ExperimentRegistry.java
----------------------------------------------------------------------
diff --git a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/impl/ExperimentRegistry.java b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/impl/ExperimentRegistry.java
index dd555b1..3066886 100644
--- a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/impl/ExperimentRegistry.java
+++ b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/impl/ExperimentRegistry.java
@@ -608,7 +608,7 @@ public class ExperimentRegistry {
             if (experimentOutputs != null && !experimentOutputs.isEmpty()) {
                 updateExpOutputs(experimentOutputs, expId);
             }
-            ExperimentStatus experimentStatus = experiment.getExperimentStatus();
+            ExperimentStatus experimentStatus = experiment.getExperimentStatus().get(0);
             if (experimentStatus != null) {
                 updateExperimentStatus(experimentStatus, expId);
             }

http://git-wip-us.apache.org/repos/asf/airavata/blob/415e9b70/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/model/Gateway.java
----------------------------------------------------------------------
diff --git a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/model/Gateway.java b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/model/Gateway.java
index afb67a1..c7d750d 100644
--- a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/model/Gateway.java
+++ b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/model/Gateway.java
@@ -72,7 +72,7 @@ public class Gateway {
         this.gatewayName = gatewayName;
     }
 
-    @Column(name = "DOMAIN")
+    @Column(name = "GATEWAY_DOMAIN")
     public String getDomain() {
         return domain;
     }

http://git-wip-us.apache.org/repos/asf/airavata/blob/415e9b70/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/utils/ThriftDataModelConversion.java
----------------------------------------------------------------------
diff --git a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/utils/ThriftDataModelConversion.java b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/utils/ThriftDataModelConversion.java
index 2fec213..3e04fcd 100644
--- a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/utils/ThriftDataModelConversion.java
+++ b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/utils/ThriftDataModelConversion.java
@@ -151,7 +151,9 @@ public class ThriftDataModelConversion {
             experiment.setExperimentOutputs(getExpOutputs(experimentOutputs));
             ExperimentStatusResource experimentStatus = experimentResource.getExperimentStatus();
             if (experimentStatus != null){
-                experiment.setExperimentStatus(getExperimentStatus(experimentStatus));
+                List<ExperimentStatus> experimentStatuses = new ArrayList<>();
+                experimentStatuses.add(getExperimentStatus(experimentStatus));
+                experiment.setExperimentStatus(experimentStatuses);
             }
             List<ExperimentErrorResource> errorDetails = experimentResource.getExperimentErrors();
             if (errorDetails!= null && !errorDetails.isEmpty()){

http://git-wip-us.apache.org/repos/asf/airavata/blob/415e9b70/modules/registry/registry-server/registry-api-service/src/main/java/org/apache/airavata/registry/api/service/handler/RegistryServerHandler.java
----------------------------------------------------------------------
diff --git a/modules/registry/registry-server/registry-api-service/src/main/java/org/apache/airavata/registry/api/service/handler/RegistryServerHandler.java b/modules/registry/registry-server/registry-api-service/src/main/java/org/apache/airavata/registry/api/service/handler/RegistryServerHandler.java
index 2646147..67344d7 100644
--- a/modules/registry/registry-server/registry-api-service/src/main/java/org/apache/airavata/registry/api/service/handler/RegistryServerHandler.java
+++ b/modules/registry/registry-server/registry-api-service/src/main/java/org/apache/airavata/registry/api/service/handler/RegistryServerHandler.java
@@ -508,7 +508,7 @@ public class RegistryServerHandler implements RegistryService.Iface {
             }
             ExperimentModel experimentModel = (ExperimentModel) experimentCatalog.get(ExperimentCatalogModelType.EXPERIMENT, experimentId);
 
-            if(!(experimentModel.getExperimentStatus().getState() == ExperimentState.CREATED)){
+            if(!(experimentModel.getExperimentStatus().get(0).getState() == ExperimentState.CREATED)){
                 logger.error("Error while deleting the experiment");
                 throw new ExperimentCatalogException("Experiment is not in CREATED state. Hence cannot deleted. ID:"+ experimentId);
             }

http://git-wip-us.apache.org/repos/asf/airavata/blob/415e9b70/thrift-interface-descriptions/data-models/experiment-catalog-models/experiment_model.thrift
----------------------------------------------------------------------
diff --git a/thrift-interface-descriptions/data-models/experiment-catalog-models/experiment_model.thrift b/thrift-interface-descriptions/data-models/experiment-catalog-models/experiment_model.thrift
index e25e16e..5e54bf1 100644
--- a/thrift-interface-descriptions/data-models/experiment-catalog-models/experiment_model.thrift
+++ b/thrift-interface-descriptions/data-models/experiment-catalog-models/experiment_model.thrift
@@ -100,7 +100,7 @@ struct ExperimentModel {
     14: optional UserConfigurationDataModel userConfigurationData,
     15: optional list<application_io_models.InputDataObjectType> experimentInputs,
     16: optional list<application_io_models.OutputDataObjectType> experimentOutputs,
-    17: optional status_models.ExperimentStatus experimentStatus,
+    17: optional list<status_models.ExperimentStatus> experimentStatus,
     18: optional list<airavata_commons.ErrorModel> errors,
     19: optional list<process_model.ProcessModel> processes
     }


[18/50] [abbrv] airavata git commit: Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/airavata into develop

Posted by la...@apache.org.
Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/airavata into develop


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: 1ecff7642340b49278172412c44ac49750d36e05
Parents: abcdd33 d19f8c5
Author: Shameera Rathnayaka <sh...@gmail.com>
Authored: Fri Aug 12 16:33:31 2016 -0400
Committer: Shameera Rathnayaka <sh...@gmail.com>
Committed: Fri Aug 12 16:33:31 2016 -0400

----------------------------------------------------------------------
 .../api/service/handler/RegistryServerHandler.java  | 16 +++++-----------
 1 file changed, 5 insertions(+), 11 deletions(-)
----------------------------------------------------------------------



[30/50] [abbrv] airavata git commit: Fixed experiment cancellation issues

Posted by la...@apache.org.
Fixed experiment cancellation issues


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: 309a9ff83945b59930d9b37cff52e4fb6e405b5c
Parents: bc37334
Author: Shameera Rathnayaka <sh...@gmail.com>
Authored: Tue Aug 16 15:37:14 2016 -0400
Committer: Shameera Rathnayaka <sh...@gmail.com>
Committed: Tue Aug 16 15:37:14 2016 -0400

----------------------------------------------------------------------
 .../server/handler/AiravataServerHandler.java   |  6 ++-
 .../server/OrchestratorServerHandler.java       | 56 +++++++++++++++-----
 2 files changed, 48 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/309a9ff8/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java b/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
index 8ce1c65..e489b43 100644
--- a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
+++ b/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
@@ -1458,13 +1458,15 @@ public class AiravataServerHandler implements Airavata.Iface {
             switch (experimentStatus.getState()) {
                 case COMPLETED: case CANCELED: case FAILED: case CANCELING:
                     logger.warn("Can't terminate already {} experiment", experimentStatus.getState().name());
+                    break;
                 case CREATED:
                     logger.warn("Experiment termination is only allowed for launched experiments.");
+                    break;
                 default:
                     submitCancelExperiment(airavataExperimentId, gatewayId);
-
+                    logger.debug("Airavata cancelled experiment with experiment id : " + airavataExperimentId);
+                    break;
             }
-            logger.debug("Airavata cancelled experiment with experiment id : " + airavataExperimentId);
         } catch (RegistryServiceException | AiravataException e) {
             logger.error(airavataExperimentId, "Error while cancelling the experiment...", e);
             AiravataSystemException exception = new AiravataSystemException();

http://git-wip-us.apache.org/repos/asf/airavata/blob/309a9ff8/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java
----------------------------------------------------------------------
diff --git a/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java b/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java
index b425c5e..17bceb4 100644
--- a/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java
+++ b/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java
@@ -90,6 +90,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
+import java.util.function.BiConsumer;
 
 public class OrchestratorServerHandler implements OrchestratorService.Iface {
 	private static Logger log = LoggerFactory.getLogger(OrchestratorServerHandler.class);
@@ -612,25 +613,56 @@ public class OrchestratorServerHandler implements OrchestratorService.Iface {
 
 		@Override
 		public void onMessage(MessageContext messageContext) {
-			if (messageContext.getType() != MessageType.EXPERIMENT) {
-				experimentSubscriber.sendAck(messageContext.getDeliveryTag());
-				log.error("Orchestrator got un-support message type : " + messageContext.getType());
+
+			switch (messageContext.getType()) {
+				case EXPERIMENT:
+					launchExperiment(messageContext);
+					break;
+				case EXPERIMENT_CANCEL:
+                    cancelExperiment(messageContext);
+					break;
+				default:
+					experimentSubscriber.sendAck(messageContext.getDeliveryTag());
+					log.error("Orchestrator got un-support message type : " + messageContext.getType());
+					break;
 			}
+		}
+
+		private void cancelExperiment(MessageContext messageContext) {
 			try {
 				byte[] bytes = ThriftUtils.serializeThriftObject(messageContext.getEvent());
 				ExperimentSubmitEvent expEvent = new ExperimentSubmitEvent();
 				ThriftUtils.createThriftFromBytes(bytes, expEvent);
-				if (messageContext.isRedeliver()) {
-                    // TODO - handle redelivery scenario
-                    experimentSubscriber.sendAck(messageContext.getDeliveryTag());
-                } else {
-                    launchExperiment(expEvent.getExperimentId(), expEvent.getGatewayId());
-                    experimentSubscriber.sendAck(messageContext.getDeliveryTag());
-                }
+				terminateExperiment(expEvent.getExperimentId(), expEvent.getGatewayId());
 			} catch (TException e) {
-				log.error("Experiment launch failed due to Thrift conversion error", e);
-                experimentSubscriber.sendAck(messageContext.getDeliveryTag());
+				log.error("Experiment cancellation failed due to Thrift conversion error", e);
+			}finally {
+				experimentSubscriber.sendAck(messageContext.getDeliveryTag());
 			}
+
+		}
+	}
+
+	private void launchExperiment(MessageContext messageContext) {
+		try {
+            byte[] bytes = ThriftUtils.serializeThriftObject(messageContext.getEvent());
+            ExperimentSubmitEvent expEvent = new ExperimentSubmitEvent();
+            ThriftUtils.createThriftFromBytes(bytes, expEvent);
+            if (messageContext.isRedeliver()) {
+				ExperimentModel experimentModel = (ExperimentModel) experimentCatalog.
+						get(ExperimentCatalogModelType.EXPERIMENT, expEvent.getExperimentId());
+				if (experimentModel.getExperimentStatus().getState() == ExperimentState.CREATED) {
+					launchExperiment(expEvent.getExperimentId(), expEvent.getGatewayId());
+				}
+            } else {
+                launchExperiment(expEvent.getExperimentId(), expEvent.getGatewayId());
+            }
+		} catch (TException e) {
+            log.error("Experiment launch failed due to Thrift conversion error", e);
+		} catch (RegistryException e) {
+			log.error("Experiment launch failed due to registry access issue", e);
+		}finally {
+			experimentSubscriber.sendAck(messageContext.getDeliveryTag());
 		}
 	}
 


[10/50] [abbrv] airavata git commit: Merge remote-tracking branch 'origin/develop' into develop

Posted by la...@apache.org.
Merge remote-tracking branch 'origin/develop' into develop


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: b9b2480c5d68dc77d94aecca305e21604865962a
Parents: 1eb3b41 63696ff
Author: scnakandala <su...@gmail.com>
Authored: Thu Aug 11 12:44:54 2016 -0400
Committer: scnakandala <su...@gmail.com>
Committed: Thu Aug 11 12:44:54 2016 -0400

----------------------------------------------------------------------
 .../server/handler/AiravataServerHandler.java   |   5 +-
 .../airavata/common/utils/ServerSettings.java   |  10 -
 .../main/resources/airavata-server.properties   |   1 -
 .../org/apache/airavata/gfac/impl/Factory.java  |   9 +-
 .../airavata/gfac/server/GfacServerHandler.java |   6 +-
 .../messaging/client/RabbitMQListener.java      |   3 +-
 .../messaging/core/MessagingFactory.java        |  86 +++-
 .../airavata/messaging/core/Publisher.java      |   4 +-
 .../messaging/core/PublisherFactory.java        | 138 +++---
 .../messaging/core/RabbitMQProperties.java      | 125 ++++++
 .../airavata/messaging/core/Subscriber.java     |   5 -
 .../messaging/core/SubscriberProperties.java    | 125 ------
 .../airavata/messaging/core/TestClient.java     |   2 +-
 .../apache/airavata/messaging/core/Type.java    |  27 ++
 .../impl/RabbitMQProcessLaunchPublisher.java    | 156 +++----
 .../messaging/core/impl/RabbitMQProducer.java   | 440 +++++++++----------
 .../messaging/core/impl/RabbitMQPublisher.java  | 118 +++++
 .../core/impl/RabbitMQStatusPublisher.java      | 212 ++++-----
 .../messaging/core/impl/RabbitMQSubscriber.java |   7 +-
 .../core/impl/GFACPassiveJobSubmitter.java      |   9 +-
 .../server/OrchestratorServerHandler.java       |   6 +-
 .../ExperimentExecution.java                    |   3 +-
 .../workflow/core/WorkflowEnactmentService.java |   7 +-
 .../workflow/core/WorkflowInterpreter.java      |   8 +-
 24 files changed, 856 insertions(+), 656 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/b9b2480c/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
----------------------------------------------------------------------


[28/50] [abbrv] airavata git commit: Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/airavata into develop

Posted by la...@apache.org.
Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/airavata into develop


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: 18ea6c24f9c42bc3e32c0853ac63d36243cbfe4c
Parents: 3f29cfd c6a82a0
Author: Shameera Rathnayaka <sh...@gmail.com>
Authored: Mon Aug 15 16:39:56 2016 -0400
Committer: Shameera Rathnayaka <sh...@gmail.com>
Committed: Tue Aug 16 14:21:16 2016 -0400

----------------------------------------------------------------------
 .../org/apache/airavata/gfac/impl/Factory.java  |  14 +-
 .../airavata/gfac/server/GfacServerHandler.java |   5 +-
 .../airavata/registry/core/Committer.java       |  29 ----
 .../apache/airavata/registry/core/JPAUtil.java  |  57 -------
 .../catalog/resources/ExperimentResource.java   | 148 +++++++++++++------
 5 files changed, 113 insertions(+), 140 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/18ea6c24/modules/gfac/gfac-impl/src/main/java/org/apache/airavata/gfac/impl/Factory.java
----------------------------------------------------------------------
diff --cc modules/gfac/gfac-impl/src/main/java/org/apache/airavata/gfac/impl/Factory.java
index 673f37b,673f37b..6ed6de3
--- a/modules/gfac/gfac-impl/src/main/java/org/apache/airavata/gfac/impl/Factory.java
+++ b/modules/gfac/gfac-impl/src/main/java/org/apache/airavata/gfac/impl/Factory.java
@@@ -63,6 -63,6 +63,7 @@@ import org.apache.airavata.gfac.impl.ta
  import org.apache.airavata.gfac.impl.watcher.CancelRequestWatcherImpl;
  import org.apache.airavata.gfac.impl.watcher.RedeliveryRequestWatcherImpl;
  import org.apache.airavata.gfac.monitor.email.EmailBasedMonitor;
++import org.apache.airavata.messaging.core.MessageHandler;
  import org.apache.airavata.messaging.core.MessagingFactory;
  import org.apache.airavata.messaging.core.Publisher;
  import org.apache.airavata.messaging.core.Subscriber;
@@@ -91,6 -91,6 +92,7 @@@ import org.slf4j.LoggerFactory
  import java.lang.reflect.Constructor;
  import java.util.ArrayList;
  import java.util.HashMap;
++import java.util.List;
  import java.util.Map;
  import java.util.UUID;
  
@@@ -168,10 -168,10 +170,16 @@@ public abstract class Factory 
  		return curatorClient;
  	}
  
++	public static synchronized void initPrcessLaunchSubscriber(MessageHandler processMessageHandler) throws AiravataException {
++	    if(getProcessLaunchSubscriber() != null)
++			throw new AiravataException("Process launch Subscriber is already initialized");
++
++		List<String> routingKeys = new ArrayList<>();
++		routingKeys.add(ServerSettings.getRabbitmqProcessExchangeName());
++		 processLaunchSubscriber = MessagingFactory.getSubscriber(processMessageHandler, routingKeys, Type.PROCESS_LAUNCH);
++	}
++
  	public static synchronized  Subscriber getProcessLaunchSubscriber() throws AiravataException {
--		if (processLaunchSubscriber == null) {
--			processLaunchSubscriber = MessagingFactory.getSubscriber(message -> {}, new ArrayList<>(), Type.PROCESS_LAUNCH);
--		}
  		return processLaunchSubscriber;
  	}
  

http://git-wip-us.apache.org/repos/asf/airavata/blob/18ea6c24/modules/gfac/gfac-service/src/main/java/org/apache/airavata/gfac/server/GfacServerHandler.java
----------------------------------------------------------------------
diff --cc modules/gfac/gfac-service/src/main/java/org/apache/airavata/gfac/server/GfacServerHandler.java
index a7b0714,a7b0714..fd4acf4
--- a/modules/gfac/gfac-service/src/main/java/org/apache/airavata/gfac/server/GfacServerHandler.java
+++ b/modules/gfac/gfac-service/src/main/java/org/apache/airavata/gfac/server/GfacServerHandler.java
@@@ -96,9 -96,9 +96,8 @@@ public class GfacServerHandler implemen
  
      private void initAMQPClient() throws AiravataException {
  	    // init process consumer
--        List<String> routingKeys = new ArrayList<>();
--        routingKeys.add(ServerSettings.getRabbitmqProcessExchangeName());
--        processLaunchSubscriber = MessagingFactory.getSubscriber(new ProcessLaunchMessageHandler(),routingKeys, Type.PROCESS_LAUNCH);
++        Factory.initPrcessLaunchSubscriber(new ProcessLaunchMessageHandler());
++        processLaunchSubscriber = Factory.getProcessLaunchSubscriber();
          // init status publisher
  	    statusPublisher = Factory.getStatusPublisher();
      }


[11/50] [abbrv] airavata git commit: Orchestrator consumerls experiment submit events

Posted by la...@apache.org.
Orchestrator consumerls experiment submit events


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: 54f5c34d4eed40acd643a0388926c3c3b62be526
Parents: 4e3dc9a
Author: Shameera Rathnayaka <sh...@gmail.com>
Authored: Thu Aug 11 14:42:48 2016 -0400
Committer: Shameera Rathnayaka <sh...@gmail.com>
Committed: Thu Aug 11 14:42:48 2016 -0400

----------------------------------------------------------------------
 .../airavata/messaging/core/MessageHandler.java |  2 +-
 .../server/OrchestratorServerHandler.java       | 37 ++++++++++++++++++--
 2 files changed, 36 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/54f5c34d/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessageHandler.java
----------------------------------------------------------------------
diff --git a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessageHandler.java b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessageHandler.java
index 23646da..bc47e68 100644
--- a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessageHandler.java
+++ b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessageHandler.java
@@ -24,5 +24,5 @@ package org.apache.airavata.messaging.core;
 @FunctionalInterface
 public interface MessageHandler {
 
-    void onMessage(MessageContext message);
+    void onMessage(MessageContext messageContext);
 }

http://git-wip-us.apache.org/repos/asf/airavata/blob/54f5c34d/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java
----------------------------------------------------------------------
diff --git a/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java b/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java
index 03f6f8a..b425c5e 100644
--- a/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java
+++ b/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java
@@ -48,6 +48,7 @@ import org.apache.airavata.model.error.LaunchValidationException;
 import org.apache.airavata.model.experiment.ExperimentModel;
 import org.apache.airavata.model.experiment.ExperimentType;
 import org.apache.airavata.model.messaging.event.ExperimentStatusChangeEvent;
+import org.apache.airavata.model.messaging.event.ExperimentSubmitEvent;
 import org.apache.airavata.model.messaging.event.MessageType;
 import org.apache.airavata.model.messaging.event.ProcessIdentifier;
 import org.apache.airavata.model.messaging.event.ProcessStatusChangeEvent;
@@ -99,7 +100,8 @@ public class OrchestratorServerHandler implements OrchestratorService.Iface {
 	private String airavataUserName;
 	private String gatewayName;
 	private Publisher publisher;
-	private Subscriber statusSubscribe;
+	private final Subscriber statusSubscribe;
+	private final Subscriber experimentSubscriber;
 	private CuratorFramework curatorClient;
 
     /**
@@ -112,7 +114,10 @@ public class OrchestratorServerHandler implements OrchestratorService.Iface {
 	public OrchestratorServerHandler() throws OrchestratorException{
 		try {
 	        publisher = MessagingFactory.getPublisher(Type.STATUS);
-            setAiravataUserName(ServerSettings.getDefaultUser());
+			List<String> routingKeys = new ArrayList<>();
+			routingKeys.add(ServerSettings.getRabbitmqExperimentLaunchQueueName());
+			experimentSubscriber = MessagingFactory.getSubscriber(new ExperimentHandler(), routingKeys, Type.EXPERIMENT_LAUNCH);
+			setAiravataUserName(ServerSettings.getDefaultUser());
 		} catch (AiravataException e) {
             log.error(e.getMessage(), e);
             throw new OrchestratorException("Error while initializing orchestrator service", e);
@@ -601,4 +606,32 @@ public class OrchestratorServerHandler implements OrchestratorService.Iface {
 			}
 		}
 	}
+
+
+	private class ExperimentHandler implements MessageHandler {
+
+		@Override
+		public void onMessage(MessageContext messageContext) {
+			if (messageContext.getType() != MessageType.EXPERIMENT) {
+				experimentSubscriber.sendAck(messageContext.getDeliveryTag());
+				log.error("Orchestrator got un-support message type : " + messageContext.getType());
+			}
+			try {
+				byte[] bytes = ThriftUtils.serializeThriftObject(messageContext.getEvent());
+				ExperimentSubmitEvent expEvent = new ExperimentSubmitEvent();
+				ThriftUtils.createThriftFromBytes(bytes, expEvent);
+				if (messageContext.isRedeliver()) {
+                    // TODO - handle redelivery scenario
+                    experimentSubscriber.sendAck(messageContext.getDeliveryTag());
+                } else {
+                    launchExperiment(expEvent.getExperimentId(), expEvent.getGatewayId());
+                    experimentSubscriber.sendAck(messageContext.getDeliveryTag());
+                }
+			} catch (TException e) {
+				log.error("Experiment launch failed due to Thrift conversion error", e);
+                experimentSubscriber.sendAck(messageContext.getDeliveryTag());
+			}
+		}
+	}
+
 }


[02/50] [abbrv] airavata git commit: Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/airavata into develop

Posted by la...@apache.org.
Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/airavata into develop


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: 12d9efb90763b7b3cb20c6733804ea0a712186dd
Parents: e4cc54d 3a70c59
Author: Shameera Rathnayaka <sh...@gmail.com>
Authored: Wed Aug 10 15:19:30 2016 -0400
Committer: Shameera Rathnayaka <sh...@gmail.com>
Committed: Wed Aug 10 15:19:30 2016 -0400

----------------------------------------------------------------------
 .../server/handler/AiravataServerHandler.java   |   8 +-
 .../lib/airavata/workspace_model_types.cpp      |  22 ++++
 .../lib/airavata/workspace_model_types.h        |  12 +-
 .../lib/Airavata/Model/Workspace/Types.php      |  23 ++++
 .../apache/airavata/model/workspace/ttypes.py   |  15 ++-
 .../airavata/model/workspace/Gateway.java       | 111 ++++++++++++++++++-
 .../catalog/impl/GatewayRegistry.java           |   3 +
 .../core/experiment/catalog/model/Gateway.java  |  11 ++
 .../catalog/resources/GatewayResource.java      |  12 ++
 .../experiment/catalog/resources/Utils.java     |   3 +
 .../utils/ThriftDataModelConversion.java        |   1 +
 .../service/handler/RegistryServerHandler.java  |  12 ++
 .../workspace_model.thrift                      |   3 +-
 13 files changed, 227 insertions(+), 9 deletions(-)
----------------------------------------------------------------------



[42/50] [abbrv] airavata git commit: Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/airavata into develop

Posted by la...@apache.org.
Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/airavata into develop


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: 35367512ba2156583c4ff527c4aa2dcf432adc61
Parents: 1dab79b 43d17ae
Author: Shameera Rathnayaka <sh...@gmail.com>
Authored: Wed Aug 24 12:40:04 2016 -0400
Committer: Shameera Rathnayaka <sh...@gmail.com>
Committed: Wed Aug 24 12:40:04 2016 -0400

----------------------------------------------------------------------
 modules/amqpwstunnel/python/amqpwstunnel.py | 583 +++++++++++++++++++++++
 modules/amqpwstunnel/python/config.json     |  10 +
 modules/amqpwstunnel/wstest.html            | 157 ++++++
 pom.xml                                     |  97 +++-
 4 files changed, 845 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[06/50] [abbrv] airavata git commit: Merge branch 'messaging-refactor' into develop

Posted by la...@apache.org.
Merge branch 'messaging-refactor' into develop


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: 63696ffa1c6ede30ca99b0b8da20e18d70e7c9ca
Parents: 12d9efb 474ae23
Author: Shameera Rathnayaka <sh...@gmail.com>
Authored: Wed Aug 10 15:43:32 2016 -0400
Committer: Shameera Rathnayaka <sh...@gmail.com>
Committed: Wed Aug 10 15:43:32 2016 -0400

----------------------------------------------------------------------
 .../server/handler/AiravataServerHandler.java   |   5 +-
 .../airavata/common/utils/ServerSettings.java   |  10 --
 .../main/resources/airavata-server.properties   |   1 -
 .../messaging/core/PublisherFactory.java        | 138 +++++++++----------
 .../core/impl/GFACPassiveJobSubmitter.java      |   9 +-
 .../server/OrchestratorServerHandler.java       |   3 +-
 6 files changed, 79 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/63696ffa/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
----------------------------------------------------------------------


[16/50] [abbrv] airavata git commit: return empty list when user not existant

Posted by la...@apache.org.
return empty list when user not existant


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: d19f8c5c9369af25e291c78807e7e2c539350ce2
Parents: 7eacf84
Author: scnakandala <su...@gmail.com>
Authored: Fri Aug 12 12:01:29 2016 -0400
Committer: scnakandala <su...@gmail.com>
Committed: Fri Aug 12 12:01:29 2016 -0400

----------------------------------------------------------------------
 .../service/handler/RegistryServerHandler.java   | 19 +++++--------------
 1 file changed, 5 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/d19f8c5c/modules/registry/registry-server/registry-api-service/src/main/java/org/apache/airavata/registry/api/service/handler/RegistryServerHandler.java
----------------------------------------------------------------------
diff --git a/modules/registry/registry-server/registry-api-service/src/main/java/org/apache/airavata/registry/api/service/handler/RegistryServerHandler.java b/modules/registry/registry-server/registry-api-service/src/main/java/org/apache/airavata/registry/api/service/handler/RegistryServerHandler.java
index 1d7c74b..2646147 100644
--- a/modules/registry/registry-server/registry-api-service/src/main/java/org/apache/airavata/registry/api/service/handler/RegistryServerHandler.java
+++ b/modules/registry/registry-server/registry-api-service/src/main/java/org/apache/airavata/registry/api/service/handler/RegistryServerHandler.java
@@ -346,11 +346,8 @@ public class RegistryServerHandler implements RegistryService.Iface {
         List<Project> projects = new ArrayList<Project>();
         try {
             if (!ExpCatResourceUtils.isUserExist(userName, gatewayId)){
-                logger.error("User does not exist in the system. Please provide a valid user..");
-                AiravataSystemException exception = new AiravataSystemException();
-                exception.setAiravataErrorType(AiravataErrorType.INTERNAL_ERROR);
-                exception.setMessage("User does not exist in the system. Please provide a valid user..");
-                throw exception;
+                logger.warn("User does not exist in the system. Please provide a valid user..");
+                return projects;
             }
             experimentCatalog = RegistryFactory.getExperimentCatalog(gatewayId);
             Map<String, String> filters = new HashMap<String, String>();
@@ -466,22 +463,16 @@ public class RegistryServerHandler implements RegistryService.Iface {
             exception.setMessage("Username cannot be empty. Please provide a valid user..");
             throw exception;
         }
-        if(!isUserExists(gatewayId, userName)){
-            return new ArrayList<>();
-        }
         if (!isGatewayExistInternal(gatewayId)){
             logger.error("Gateway does not exist.Please provide a valid gateway id...");
             throw new AiravataSystemException(AiravataErrorType.INTERNAL_ERROR);
         }
+        List<ExperimentModel> experiments = new ArrayList<ExperimentModel>();
         try {
             if (!ExpCatResourceUtils.isUserExist(userName, gatewayId)){
-                logger.error("User does not exist in the system. Please provide a valid user..");
-                AiravataSystemException exception = new AiravataSystemException();
-                exception.setAiravataErrorType(AiravataErrorType.INTERNAL_ERROR);
-                exception.setMessage("User does not exist in the system. Please provide a valid user..");
-                throw exception;
+                logger.warn("User does not exist in the system. Please provide a valid user..");
+                return experiments;
             }
-            List<ExperimentModel> experiments = new ArrayList<ExperimentModel>();
             experimentCatalog = RegistryFactory.getExperimentCatalog(gatewayId);
             List<Object> list = experimentCatalog.get(ExperimentCatalogModelType.EXPERIMENT,
                     Constants.FieldConstants.ExperimentConstants.USER_NAME, userName, limit, offset,


[40/50] [abbrv] airavata git commit: [AIRAVATA-2058] Add a separate profile for jenkins, until AIRAVATA-2057 is addressed and use that profile in jenkins

Posted by la...@apache.org.
[AIRAVATA-2058] Add a separate profile for jenkins, until AIRAVATA-2057 is addressed and use that profile in jenkins


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: 43d17aedad1ab722fa5cb2f5a7060358fd7df52a
Parents: ff18106
Author: Lahiru Ginnaliya Gamathige <la...@apache.org>
Authored: Tue Aug 23 10:20:07 2016 -0700
Committer: Lahiru Ginnaliya Gamathige <la...@apache.org>
Committed: Tue Aug 23 10:37:31 2016 -0700

----------------------------------------------------------------------
 pom.xml | 97 ++++++++++++++++++++++++++++++++++++++++++++++++++++++++++--
 1 file changed, 95 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/43d17aed/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 0c12844..6594026 100644
--- a/pom.xml
+++ b/pom.xml
@@ -569,9 +569,102 @@
 				<!--<module>modules/workflow-model</module>-->
 				<!--<module>modules/workflow</module>-->
 				<!--<module>modules/xbaya-gui</module>-->
-				<module>distribution</module>
-			</modules>
+                <module>distribution</module>
+            </modules>
 		</profile>
+        <profile>
+            <id>jenkins</id>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-remote-resources-plugin</artifactId>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>process</goal>
+                                </goals>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <artifactId>maven-resources-plugin</artifactId>
+                        <version>2.5</version>
+                        <executions>
+                            <execution>
+                                <id>copy-resources</id>
+                                <!-- here the phase you need -->
+                                <phase>validate</phase>
+                                <goals>
+                                    <goal>copy-resources</goal>
+                                </goals>
+                                <configuration>
+                                    <outputDirectory>${basedir}/target/classes/META-INF</outputDirectory>
+                                    <resources>
+                                        <resource>
+                                            <directory>${basedir}/src/main/assembly/dist</directory>
+                                            <filtering>true</filtering>
+                                        </resource>
+                                    </resources>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-compiler-plugin</artifactId>
+                        <version>3.1</version>
+                        <configuration>
+                            <source>1.8</source>
+                            <target>1.8</target>
+                        </configuration>
+                    </plugin>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-surefire-plugin</artifactId>
+                        <version>${surefire.version}</version>
+                        <configuration>
+                            <failIfNoTests>false</failIfNoTests>
+                            <skipTests>${skipTests}</skipTests>
+                            <workingDirectory>${project.build.testOutputDirectory}</workingDirectory>
+                            <!-- making sure that the sure-fire plugin doesn't run the integration
+                                tests -->
+                            <!-- Integration tests are run using the fail-safe plugin in the module
+                                pom -->
+                            <excludes>
+                                <exclude>**/IT.java</exclude>
+                                <exclude>**/*TestWithMyProxyAuth.java</exclude>
+                                <exclude>**/*TestWithSSHAuth.java</exclude>
+                                <exclude>**/*TestWithEC2Auth.java</exclude>
+                            </excludes>
+                        </configuration>
+                    </plugin>
+                </plugins>
+            </build>
+            <activation>
+                <activeByDefault>true</activeByDefault>
+            </activation>
+            <modules>
+                <module>modules/configuration</module>
+                <module>airavata-api</module>
+                <module>modules/commons</module>
+                <module>modules/messaging</module>
+                <module>modules/gfac</module>
+                <module>modules/registry</module>
+                <module>modules/security</module>
+                <module>modules/credential-store</module>
+                <module>modules/orchestrator</module>
+                <module>modules/monitoring</module>
+                <module>modules/user-profile</module>
+                <!--<module>modules/cloud</module>-->
+                <module>modules/server</module>
+                <module>modules/workflow</module>
+                <module>modules/test-suite</module>
+				<module>modules/group-manager</module>
+				<!-- enable distribution when AIRAVATA-2057 is fixed -->
+				<!--<module>distribution</module>-->
+            </modules>
+        </profile>
 		<profile>
 			<id>pedantic</id>
 			<build>


[17/50] [abbrv] airavata git commit: Merge branch 'master' into develop

Posted by la...@apache.org.
Merge branch 'master' into develop


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: abcdd33bd60856ebe04157f6fc996d85df1f0fd0
Parents: 79aa486 a94e0ef
Author: Shameera Rathnayaka <sh...@gmail.com>
Authored: Fri Aug 12 16:33:15 2016 -0400
Committer: Shameera Rathnayaka <sh...@gmail.com>
Committed: Fri Aug 12 16:33:15 2016 -0400

----------------------------------------------------------------------
 modules/cloud/cloud-provisioning/pom.xml                | 2 +-
 modules/cloud/pom.xml                                   | 2 +-
 modules/gfac/gfac-bes/pom.xml                           | 2 +-
 modules/integration-tests/pom.xml                       | 2 +-
 modules/workflow-model/pom.xml                          | 2 +-
 modules/workflow-model/workflow-engine/pom.xml          | 2 +-
 modules/workflow-model/workflow-model-component/pom.xml | 2 +-
 modules/workflow-model/workflow-model-core/pom.xml      | 2 +-
 modules/xbaya-gui/pom.xml                               | 2 +-
 modules/xbaya/pom.xml                                   | 2 +-
 tools/pom.xml                                           | 2 +-
 11 files changed, 11 insertions(+), 11 deletions(-)
----------------------------------------------------------------------



[19/50] [abbrv] airavata git commit: Registry java8 PoC

Posted by la...@apache.org.
Registry java8 PoC


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: db9e3b76ee2694cdffeff7f56fa6ef4b3b375119
Parents: 1ecff76
Author: Shameera Rathnayaka <sh...@gmail.com>
Authored: Thu Aug 11 18:22:15 2016 -0400
Committer: Shameera Rathnayaka <sh...@gmail.com>
Committed: Fri Aug 12 16:34:04 2016 -0400

----------------------------------------------------------------------
 .../airavata/registry/core/Committer.java       |  29 ++++
 .../apache/airavata/registry/core/JPAUtil.java  |  57 +++++++
 .../catalog/resources/ExperimentResource.java   | 148 ++++++-------------
 3 files changed, 134 insertions(+), 100 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/db9e3b76/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/Committer.java
----------------------------------------------------------------------
diff --git a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/Committer.java b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/Committer.java
new file mode 100644
index 0000000..99b504b
--- /dev/null
+++ b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/Committer.java
@@ -0,0 +1,29 @@
+/*
+ *
+ * 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.airavata.registry.core;
+
+import org.apache.airavata.registry.cpi.RegistryException;
+
+@FunctionalInterface
+public interface Committer<T, R>  {
+
+   R commit(T t) throws RegistryException;
+}

http://git-wip-us.apache.org/repos/asf/airavata/blob/db9e3b76/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/JPAUtil.java
----------------------------------------------------------------------
diff --git a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/JPAUtil.java b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/JPAUtil.java
new file mode 100644
index 0000000..58988c6
--- /dev/null
+++ b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/JPAUtil.java
@@ -0,0 +1,57 @@
+/*
+ *
+ * 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.airavata.registry.core;
+
+import org.apache.airavata.registry.core.experiment.catalog.ExpCatResourceUtils;
+import org.apache.airavata.registry.cpi.ExperimentCatalogException;
+import org.apache.airavata.registry.cpi.RegistryException;
+
+import javax.persistence.EntityManager;
+import javax.persistence.EntityManagerFactory;
+import javax.persistence.Persistence;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.function.Consumer;
+
+public class JPAUtil {
+
+    public static <R> R executeOnExpCatalog(Committer<EntityManager, R> committer) throws RegistryException, ExperimentCatalogException {
+        EntityManager entityManager = ExpCatResourceUtils.getEntityManager();
+        try {
+            entityManager.getTransaction().begin();
+            R r = committer.commit(entityManager);
+            entityManager.getTransaction().commit();
+            return  r;
+        }finally {
+            if(entityManager != null && entityManager.isOpen()){
+                if (entityManager.getTransaction().isActive()) {
+                    entityManager.getTransaction().rollback();
+                }
+                entityManager.close();
+            }
+        }
+
+    }
+
+    public static void test() throws ExperimentCatalogException, RegistryException {
+        executeOnExpCatalog((em) -> "");
+    }
+}

http://git-wip-us.apache.org/repos/asf/airavata/blob/db9e3b76/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/resources/ExperimentResource.java
----------------------------------------------------------------------
diff --git a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/resources/ExperimentResource.java b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/resources/ExperimentResource.java
index 174575f..7889839 100644
--- a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/resources/ExperimentResource.java
+++ b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/resources/ExperimentResource.java
@@ -22,6 +22,8 @@
 package org.apache.airavata.registry.core.experiment.catalog.resources;
 
 import org.apache.airavata.model.status.ExperimentState;
+import org.apache.airavata.registry.core.Committer;
+import org.apache.airavata.registry.core.JPAUtil;
 import org.apache.airavata.registry.core.experiment.catalog.ExpCatResourceUtils;
 import org.apache.airavata.registry.core.experiment.catalog.ExperimentCatResource;
 import org.apache.airavata.registry.core.experiment.catalog.ResourceType;
@@ -29,6 +31,7 @@ import org.apache.airavata.registry.core.experiment.catalog.model.*;
 import org.apache.airavata.registry.core.experiment.catalog.model.Process;
 import org.apache.airavata.registry.core.experiment.catalog.utils.QueryGenerator;
 import org.apache.airavata.registry.cpi.RegistryException;
+import org.apache.openjpa.persistence.Generator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -264,120 +267,65 @@ public class ExperimentResource extends AbstractExpCatResource {
 
     
     public ExperimentCatResource get(ResourceType type, Object name) throws RegistryException{
-        EntityManager em = null;
         try {
-            em = ExpCatResourceUtils.getEntityManager();
-            em.getTransaction().begin();
-            QueryGenerator generator;
-            Query q;
             switch (type) {
                 case EXPERIMENT_STATUS:
-                    generator = new QueryGenerator(EXPERIMENT_STATUS);
-                    generator.setParameter(ExperimentStatusConstants.STATUS_ID, name);
-                    q = generator.selectQuery(em);
-                    ExperimentStatus status = (ExperimentStatus) q.getSingleResult();
-                    ExperimentStatusResource statusResource = (ExperimentStatusResource) Utils.getResource(ResourceType.EXPERIMENT_STATUS, status);
-                    em.getTransaction().commit();
-                    if (em.isOpen()) {
-                        if (em.getTransaction().isActive()){
-                            em.getTransaction().rollback();
-                        }
-                        em.close();
-                    }
-                    return statusResource;
+                    return JPAUtil.executeOnExpCatalog(entityManager -> {
+                        QueryGenerator generator = new QueryGenerator(EXPERIMENT_STATUS);
+                        generator.setParameter(ExperimentStatusConstants.STATUS_ID, name);
+                        Query q = generator.selectQuery(entityManager);
+                        ExperimentStatus status = (ExperimentStatus) q.getSingleResult();
+                        return (ExperimentStatusResource) Utils.getResource(ResourceType.EXPERIMENT_STATUS, status);
+                    });
                 case EXPERIMENT_ERROR:
-                    generator = new QueryGenerator(EXPERIMENT_ERROR);
-                    generator.setParameter(ExperimentErrorConstants.ERROR_ID, name);
-                    q = generator.selectQuery(em);
-                    ExperimentError experimentError = (ExperimentError) q.getSingleResult();
-                    ExperimentErrorResource processErrorResource = (ExperimentErrorResource) Utils.getResource(ResourceType.EXPERIMENT_ERROR, experimentError);
-                    em.getTransaction().commit();
-                    if (em.isOpen()) {
-                        if (em.getTransaction().isActive()){
-                            em.getTransaction().rollback();
-                        }
-                        em.close();
-                    }
-                    return processErrorResource;
+                    return JPAUtil.executeOnExpCatalog(entityManager -> {
+                        QueryGenerator generator = new QueryGenerator(EXPERIMENT_ERROR);
+                        generator.setParameter(ExperimentErrorConstants.ERROR_ID, name);
+                        Query q = generator.selectQuery(entityManager);
+                        ExperimentError experimentError = (ExperimentError) q.getSingleResult();
+                        return (ExperimentErrorResource) Utils.getResource(ResourceType.EXPERIMENT_ERROR, experimentError);
+                    });
                 case EXPERIMENT_INPUT:
-                    generator = new QueryGenerator(EXPERIMENT_INPUT);
-                    generator.setParameter(ExperimentInputConstants.INPUT_NAME, name);
-                    generator.setParameter(ExperimentInputConstants.EXPERIMENT_ID, experimentId);
-                    q = generator.selectQuery(em);
-                    ExperimentInput experimentInput = (ExperimentInput) q.getSingleResult();
-                    ExperimentInputResource experimentInputResource = (ExperimentInputResource) Utils.getResource(ResourceType.EXPERIMENT_INPUT, experimentInput);
-                    em.getTransaction().commit();
-                    if (em.isOpen()) {
-                        if (em.getTransaction().isActive()){
-                            em.getTransaction().rollback();
-                        }
-                        em.close();
-                    }
-                    return experimentInputResource;
+                    return JPAUtil.executeOnExpCatalog(entityManager -> {
+                        QueryGenerator generator = new QueryGenerator(EXPERIMENT_INPUT);
+                        generator.setParameter(ExperimentInputConstants.INPUT_NAME, name);
+                        generator.setParameter(ExperimentInputConstants.EXPERIMENT_ID, experimentId);
+                        Query q = generator.selectQuery(entityManager);
+                        ExperimentInput experimentInput = (ExperimentInput) q.getSingleResult();
+                        return (ExperimentInputResource) Utils.getResource(ResourceType.EXPERIMENT_INPUT, experimentInput);
+                    });
                 case EXPERIMENT_OUTPUT:
-                    generator = new QueryGenerator(EXPERIMENT_OUTPUT);
-                    generator.setParameter(ExperimentOutputConstants.OUTPUT_NAME, name);
-                    generator.setParameter(ExperimentInputConstants.EXPERIMENT_ID, experimentId);
-                    q = generator.selectQuery(em);
-                    ExperimentOutput experimentOutput = (ExperimentOutput) q.getSingleResult();
-                    ExperimentOutputResource outputResource = (ExperimentOutputResource) Utils.getResource(ResourceType.EXPERIMENT_OUTPUT, experimentOutput);
-                    em.getTransaction().commit();
-                    if (em.isOpen()) {
-                        if (em.getTransaction().isActive()){
-                            em.getTransaction().rollback();
-                        }
-                        em.close();
-                    }
-                    return outputResource;
+                    return JPAUtil.executeOnExpCatalog(entityManager -> {
+                        QueryGenerator generator = new QueryGenerator(EXPERIMENT_OUTPUT);
+                        generator.setParameter(ExperimentOutputConstants.OUTPUT_NAME, name);
+                        generator.setParameter(ExperimentInputConstants.EXPERIMENT_ID, experimentId);
+                        Query q = generator.selectQuery(entityManager);
+                        ExperimentOutput experimentOutput = (ExperimentOutput) q.getSingleResult();
+                        return (ExperimentOutputResource) Utils.getResource(ResourceType.EXPERIMENT_OUTPUT, experimentOutput);
+                    });
                 case USER_CONFIGURATION_DATA:
-                    generator = new QueryGenerator(USER_CONFIGURATION_DATA);
-                    generator.setParameter(UserConfigurationDataConstants.EXPERIMENT_ID, name);
-                    q = generator.selectQuery(em);
-                    UserConfigurationData configurationData = (UserConfigurationData) q.getSingleResult();
-                    UserConfigurationDataResource configurationDataResource = (UserConfigurationDataResource)
-                            Utils.getResource(ResourceType.USER_CONFIGURATION_DATA, configurationData);
-                    em.getTransaction().commit();
-                    if (em.isOpen()) {
-                        if (em.getTransaction().isActive()){
-                            em.getTransaction().rollback();
-                        }
-                        em.close();
-                    }
-                    return configurationDataResource;
+                    return JPAUtil.executeOnExpCatalog(entityManager -> {
+                        QueryGenerator generator = new QueryGenerator(USER_CONFIGURATION_DATA);
+                        generator.setParameter(UserConfigurationDataConstants.EXPERIMENT_ID, name);
+                        Query q = generator.selectQuery(entityManager);
+                        UserConfigurationData configurationData = (UserConfigurationData) q.getSingleResult();
+                        return (UserConfigurationDataResource)
+                                Utils.getResource(ResourceType.USER_CONFIGURATION_DATA, configurationData);
+                    });
                 case PROCESS:
-                    generator = new QueryGenerator(PROCESS);
-                    generator.setParameter(ProcessConstants.PROCESS_ID, name);
-                    q = generator.selectQuery(em);
-                    Process process = (Process) q.getSingleResult();
-                    ProcessResource processResource = (ProcessResource) Utils.getResource(ResourceType.PROCESS, process);
-                    em.getTransaction().commit();
-                    if (em.isOpen()) {
-                        if (em.getTransaction().isActive()){
-                            em.getTransaction().rollback();
-                        }
-                        em.close();
-                    }
-                    return processResource;
+                    return JPAUtil.executeOnExpCatalog(entityManager -> {
+                        QueryGenerator generator = new QueryGenerator(PROCESS);
+                        generator.setParameter(ProcessConstants.PROCESS_ID, name);
+                        Query q = generator.selectQuery(entityManager);
+                        Process process = (Process) q.getSingleResult();
+                        return (ProcessResource) Utils.getResource(ResourceType.PROCESS, process);
+                    });
                 default:
-                    em.getTransaction().commit();
-                    if (em.isOpen()) {
-                        if (em.getTransaction().isActive()){
-                            em.getTransaction().rollback();
-                        }
-                        em.close();
-                    }
                     logger.error("Unsupported resource type for experiment resource.", new IllegalArgumentException());
                     throw new IllegalArgumentException("Unsupported resource type for experiment resource.");
             }
         } catch (Exception e) {
             throw new RegistryException(e);
-        } finally {
-            if (em != null && em.isOpen()) {
-                if (em.getTransaction().isActive()){
-                    em.getTransaction().rollback();
-                }
-                em.close();
-            }
         }
     }
 


[08/50] [abbrv] airavata git commit: removing registry core and cpi dependencies from Airavata API Server

Posted by la...@apache.org.
http://git-wip-us.apache.org/repos/asf/airavata/blob/1eb3b415/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/ReplicaCatalogInitUtil.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/ReplicaCatalogInitUtil.java b/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/ReplicaCatalogInitUtil.java
deleted file mode 100644
index 908d82d..0000000
--- a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/ReplicaCatalogInitUtil.java
+++ /dev/null
@@ -1,160 +0,0 @@
-/*
-*
-* 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.airavata.api.server.util;
-
-import org.apache.airavata.common.exception.ApplicationSettingsException;
-import org.apache.airavata.common.utils.ServerSettings;
-import org.apache.derby.drda.NetworkServerControl;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.URI;
-import java.sql.Connection;
-import java.sql.SQLException;
-
-public class ReplicaCatalogInitUtil {
-    private static final Logger logger = LoggerFactory.getLogger(ReplicaCatalogInitUtil.class);
-    public static final String REPLICA_CATALOG = "DATA_PRODUCT";
-    public static final String REPLICA_CATALOG_JDBC_DRIVER = "replicacatalog.jdbc.driver";
-    public static final String REPLICA_CATALOG_JDBC_URL = "replicacatalog.jdbc.url";
-    public static final String REPLICA_CATALOG_JDBC_USER = "replicacatalog.jdbc.user";
-    public static final String REPLICA_CATALOG_JDBC_PASSWORD = "replicacatalog.jdbc.password";
-    public static final String START_DERBY_ENABLE = "start.derby.server.mode";
-    public static final String DERBY_SERVER_MODE_SYS_PROPERTY = "derby.drda.startNetworkServer";
-    private static NetworkServerControl server;
-    private static JdbcStorage db;
-    private static String jdbcURl;
-    private static String jdbcDriver;
-    private static String jdbcUser;
-    private static String jdbcPassword;
-
-
-    public static void initializeDB() {
-//        System.setProperty("appcatalog.initialize.state", "0");
-        try{
-            jdbcDriver = ServerSettings.getSetting(REPLICA_CATALOG_JDBC_DRIVER);
-            jdbcURl = ServerSettings.getSetting(REPLICA_CATALOG_JDBC_URL);
-            jdbcUser = ServerSettings.getSetting(REPLICA_CATALOG_JDBC_USER);
-            jdbcPassword = ServerSettings.getSetting(REPLICA_CATALOG_JDBC_PASSWORD);
-            jdbcURl = jdbcURl + "?" + "user=" + jdbcUser + "&" + "password=" + jdbcPassword;
-        } catch (ApplicationSettingsException e) {
-            logger.error("Unable to read airavata server properties", e.getMessage());
-        }
-
-        if (getDBType(jdbcURl).equals("derby") && isDerbyStartEnabled()) {
-            startDerbyInServerMode();
-        }
-        db = new JdbcStorage(10, 50, jdbcURl, jdbcDriver, true);
-
-        Connection conn = null;
-        try {
-            conn = db.connect();
-            if (!DatabaseCreator.isDatabaseStructureCreated(REPLICA_CATALOG, conn)) {
-                DatabaseCreator.createRegistryDatabase("database_scripts/replicacatalog", conn);
-                logger.info("New Database created for Replica Catalog !!! ");
-            } else {
-                logger.info("Database already created for Replica Catalog!!!");
-            }
-        } catch (Exception e) {
-            logger.error(e.getMessage(), e);
-            throw new RuntimeException("Database failure", e);
-        } finally {
-            db.closeConnection(conn);
-            try {
-                if(conn != null){
-                    if (!conn.getAutoCommit()) {
-                        conn.commit();
-                    }
-                    conn.close();
-                }
-            } catch (SQLException e) {
-                logger.error("Error while closing database connection...", e.getMessage(), e);
-            }
-        }
-//        System.setProperty("appcatalog.initialize.state", "1");
-    }
-
-    public static String getDBType(String jdbcUrl){
-        try{
-            String cleanURI = jdbcUrl.substring(5);
-            URI uri = URI.create(cleanURI);
-            return uri.getScheme();
-        } catch (Exception e) {
-            logger.error(e.getMessage(), e);
-            return null;
-        }
-    }
-
-    public static boolean isDerbyStartEnabled(){
-        try {
-            String s = ServerSettings.getSetting(START_DERBY_ENABLE);
-            if("true".equals(s)){
-                return true;
-            }
-        }  catch (ApplicationSettingsException e) {
-            logger.error("Unable to read airavata server properties", e.getMessage(), e);
-            return false;
-        }
-        return false;
-    }
-
-    public static void startDerbyInServerMode() {
-        try {
-            System.setProperty(DERBY_SERVER_MODE_SYS_PROPERTY, "true");
-            server = new NetworkServerControl(InetAddress.getByName("0.0.0.0"),
-                    getPort(jdbcURl),
-                    jdbcUser, jdbcPassword);
-            java.io.PrintWriter consoleWriter = new java.io.PrintWriter(System.out, true);
-            server.start(consoleWriter);
-        } catch (IOException e) {
-            logger.error("Unable to start Apache derby in the server mode! Check whether " +
-                    "specified port is available");
-        } catch (Exception e) {
-            logger.error("Unable to start Apache derby in the server mode! Check whether " +
-                    "specified port is available");
-        }
-    }
-    
-    public static void stopDerbyInServerMode() {
-            System.setProperty(DERBY_SERVER_MODE_SYS_PROPERTY, "false");
-            if (server!=null){
-            	try {
-					server.shutdown();
-				} catch (Exception e) {
-		            logger.error("Error when stopping the derby server : "+e.getLocalizedMessage());
-				}
-            }
-    }
-
-    public static int getPort(String jdbcURL){
-        try{
-            String cleanURI = jdbcURL.substring(5);
-            URI uri = URI.create(cleanURI);
-            return uri.getPort();
-        } catch (Exception e) {
-            logger.error(e.getMessage(), e);
-            return -1;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/airavata/blob/1eb3b415/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/WorkflowCatalogInitUtil.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/WorkflowCatalogInitUtil.java b/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/WorkflowCatalogInitUtil.java
deleted file mode 100644
index ea20c63..0000000
--- a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/util/WorkflowCatalogInitUtil.java
+++ /dev/null
@@ -1,162 +0,0 @@
-/*
-*
-* 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.airavata.api.server.util;
-
-import org.apache.airavata.common.exception.ApplicationSettingsException;
-import org.apache.airavata.common.utils.AiravataUtils;
-import org.apache.airavata.common.utils.ServerSettings;
-import org.apache.airavata.registry.core.app.catalog.resources.GatewayProfileResource;
-import org.apache.derby.drda.NetworkServerControl;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.URI;
-import java.sql.Connection;
-import java.sql.SQLException;
-
-public class WorkflowCatalogInitUtil {
-    private static final Logger logger = LoggerFactory.getLogger(WorkflowCatalogInitUtil.class);
-    public static final String WORKFLOW = "WORKFLOW";
-    public static final String WORKFLOWCATALOG_JDBC_DRIVER = "workflowcatalog.jdbc.driver";
-    public static final String WORKFLOWCATALOG_JDBC_URL = "workflowcatalog.jdbc.url";
-    public static final String WORKFLOWCATALOG_JDBC_USER = "workflowcatalog.jdbc.user";
-    public static final String WORKFLOWCATALOG_JDBC_PASSWORD = "workflowcatalog.jdbc.password";
-    public static final String START_DERBY_ENABLE = "start.derby.server.mode";
-    public static final String DERBY_SERVER_MODE_SYS_PROPERTY = "derby.drda.startNetworkServer";
-    private static NetworkServerControl server;
-    private static JdbcStorage db;
-    private static String jdbcURl;
-    private static String jdbcDriver;
-    private static String jdbcUser;
-    private static String jdbcPassword;
-
-
-    public static void initializeDB() {
-//        System.setProperty("appcatalog.initialize.state", "0");
-        try{
-            jdbcDriver = ServerSettings.getSetting(WORKFLOWCATALOG_JDBC_DRIVER);
-            jdbcURl = ServerSettings.getSetting(WORKFLOWCATALOG_JDBC_URL);
-            jdbcUser = ServerSettings.getSetting(WORKFLOWCATALOG_JDBC_USER);
-            jdbcPassword = ServerSettings.getSetting(WORKFLOWCATALOG_JDBC_PASSWORD);
-            jdbcURl = jdbcURl + "?" + "user=" + jdbcUser + "&" + "password=" + jdbcPassword;
-        } catch (ApplicationSettingsException e) {
-            logger.error("Unable to read airavata server properties", e.getMessage());
-        }
-
-        if (getDBType(jdbcURl).equals("derby") && isDerbyStartEnabled()) {
-            startDerbyInServerMode();
-        }
-        db = new JdbcStorage(10, 50, jdbcURl, jdbcDriver, true);
-
-        Connection conn = null;
-        try {
-            conn = db.connect();
-            if (!DatabaseCreator.isDatabaseStructureCreated(WORKFLOW, conn)) {
-                DatabaseCreator.createRegistryDatabase("database_scripts/workflowcatalog", conn);
-                logger.info("New Database created for Workflow Catalog !!! ");
-            } else {
-                logger.info("Database already created for Workflow Catalog!!!");
-            }
-        } catch (Exception e) {
-            logger.error(e.getMessage(), e);
-            throw new RuntimeException("Database failure", e);
-        } finally {
-            db.closeConnection(conn);
-            try {
-                if(conn != null){
-                    if (!conn.getAutoCommit()) {
-                        conn.commit();
-                    }
-                    conn.close();
-                }
-            } catch (SQLException e) {
-                logger.error("Error while closing database connection...", e.getMessage(), e);
-            }
-        }
-//        System.setProperty("appcatalog.initialize.state", "1");
-    }
-
-    public static String getDBType(String jdbcUrl){
-        try{
-            String cleanURI = jdbcUrl.substring(5);
-            URI uri = URI.create(cleanURI);
-            return uri.getScheme();
-        } catch (Exception e) {
-            logger.error(e.getMessage(), e);
-            return null;
-        }
-    }
-
-    public static boolean isDerbyStartEnabled(){
-        try {
-            String s = ServerSettings.getSetting(START_DERBY_ENABLE);
-            if("true".equals(s)){
-                return true;
-            }
-        }  catch (ApplicationSettingsException e) {
-            logger.error("Unable to read airavata server properties", e.getMessage(), e);
-            return false;
-        }
-        return false;
-    }
-
-    public static void startDerbyInServerMode() {
-        try {
-            System.setProperty(DERBY_SERVER_MODE_SYS_PROPERTY, "true");
-            server = new NetworkServerControl(InetAddress.getByName("0.0.0.0"),
-                    getPort(jdbcURl),
-                    jdbcUser, jdbcPassword);
-            java.io.PrintWriter consoleWriter = new java.io.PrintWriter(System.out, true);
-            server.start(consoleWriter);
-        } catch (IOException e) {
-            logger.error("Unable to start Apache derby in the server mode! Check whether " +
-                    "specified port is available");
-        } catch (Exception e) {
-            logger.error("Unable to start Apache derby in the server mode! Check whether " +
-                    "specified port is available");
-        }
-    }
-    
-    public static void stopDerbyInServerMode() {
-            System.setProperty(DERBY_SERVER_MODE_SYS_PROPERTY, "false");
-            if (server!=null){
-            	try {
-					server.shutdown();
-				} catch (Exception e) {
-		            logger.error("Error when stopping the derby server : "+e.getLocalizedMessage());
-				}
-            }
-    }
-
-    public static int getPort(String jdbcURL){
-        try{
-            String cleanURI = jdbcURL.substring(5);
-            URI uri = URI.create(cleanURI);
-            return uri.getPort();
-        } catch (Exception e) {
-            logger.error(e.getMessage(), e);
-            return -1;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/airavata/blob/1eb3b415/airavata-api/airavata-api-server/src/test/java/org/apache/airavata/api/server/handler/utils/AppCatInit.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-api-server/src/test/java/org/apache/airavata/api/server/handler/utils/AppCatInit.java b/airavata-api/airavata-api-server/src/test/java/org/apache/airavata/api/server/handler/utils/AppCatInit.java
deleted file mode 100644
index 86cb1ba..0000000
--- a/airavata-api/airavata-api-server/src/test/java/org/apache/airavata/api/server/handler/utils/AppCatInit.java
+++ /dev/null
@@ -1,320 +0,0 @@
-/*
- *
- * 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.airavata.api.server.handler.utils;
-
-import org.apache.airavata.common.exception.ApplicationSettingsException;
-import org.apache.airavata.common.utils.ServerSettings;
-import org.apache.derby.drda.NetworkServerControl;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.net.InetAddress;
-import java.net.URI;
-import java.sql.*;
-import java.util.StringTokenizer;
-
-public class AppCatInit {
-    private static final Logger logger = LoggerFactory.getLogger(AppCatInit.class);
-    public static final String DERBY_SERVER_MODE_SYS_PROPERTY = "derby.drda.startNetworkServer";
-    public  String scriptName = "appcatalog-derby.sql";
-    private NetworkServerControl server;
-    private static final String delimiter = ";";
-    public static final String COMPUTE_RESOURCE_TABLE = "COMPUTE_RESOURCE";
-    private String jdbcUrl = null;
-    private String jdbcDriver = null;
-    private String jdbcUser = null;
-    private String jdbcPassword = null;
-
-    public AppCatInit(String scriptName) {
-        this.scriptName = scriptName;
-    }
-
-    public static boolean checkStringBufferEndsWith(StringBuffer buffer, String suffix) {
-        if (suffix.length() > buffer.length()) {
-            return false;
-        }
-        // this loop is done on purpose to avoid memory allocation performance
-        // problems on various JDKs
-        // StringBuffer.lastIndexOf() was introduced in jdk 1.4 and
-        // implementation is ok though does allocation/copying
-        // StringBuffer.toString().endsWith() does massive memory
-        // allocation/copying on JDK 1.5
-        // See http://issues.apache.org/bugzilla/show_bug.cgi?id=37169
-        int endIndex = suffix.length() - 1;
-        int bufferIndex = buffer.length() - 1;
-        while (endIndex >= 0) {
-            if (buffer.charAt(bufferIndex) != suffix.charAt(endIndex)) {
-                return false;
-            }
-            bufferIndex--;
-            endIndex--;
-        }
-        return true;
-    }
-
-    private static boolean isServerStarted(NetworkServerControl server, int ntries)
-    {
-        for (int i = 1; i <= ntries; i ++)
-        {
-            try {
-                Thread.sleep(500);
-                server.ping();
-                return true;
-            }
-            catch (Exception e) {
-                if (i == ntries)
-                    return false;
-            }
-        }
-        return false;
-    }
-
-    public void initializeDB() {
-
-        try{
-            jdbcDriver = ServerSettings.getSetting("appcatalog.jdbc.driver");
-            jdbcUrl = ServerSettings.getSetting("appcatalog.jdbc.url");
-            jdbcUser = ServerSettings.getSetting("appcatalog.jdbc.user");
-            jdbcPassword = ServerSettings.getSetting("appcatalog.jdbc.password");
-            jdbcUrl = jdbcUrl + "?" + "user=" + jdbcUser + "&" + "password=" + jdbcPassword;
-        } catch (ApplicationSettingsException e) {
-            logger.error("Unable to read properties", e);
-        }
-
-        startDerbyInServerMode();
-        if(!isServerStarted(server, 20)){
-           throw new RuntimeException("Derby server cound not started within five seconds...");
-        }
-//      startDerbyInEmbeddedMode();
-
-        Connection conn = null;
-        try {
-            Class.forName(jdbcDriver).newInstance();
-            conn = DriverManager.getConnection(jdbcUrl, jdbcUser, jdbcPassword);
-            if (!isDatabaseStructureCreated(COMPUTE_RESOURCE_TABLE, conn)) {
-                executeSQLScript(conn);
-                logger.info("New Database created for App Catalog !!!");
-            } else {
-                logger.debug("Database already created for App Catalog!");
-            }
-        } catch (Exception e) {
-            logger.error(e.getMessage(), e);
-            throw new RuntimeException("Database failure", e);
-        } finally {
-            try {
-                if (conn != null){
-                    if (!conn.getAutoCommit()) {
-                        conn.commit();
-                    }
-                    conn.close();
-                }
-            } catch (SQLException e) {
-                logger.error(e.getMessage(), e);
-            }
-        }
-    }
-
-    public static boolean isDatabaseStructureCreated(String tableName, Connection conn) {
-        try {
-            System.out.println("Running a query to test the database tables existence.");
-            // check whether the tables are already created with a query
-            Statement statement = null;
-            try {
-                statement = conn.createStatement();
-                ResultSet rs = statement.executeQuery("select * from " + tableName);
-                if (rs != null) {
-                    rs.close();
-                }
-            } finally {
-                try {
-                    if (statement != null) {
-                        statement.close();
-                    }
-                } catch (SQLException e) {
-                    return false;
-                }
-            }
-        } catch (SQLException e) {
-            return false;
-        }
-
-        return true;
-    }
-
-    private void executeSQLScript(Connection conn) throws Exception {
-        StringBuffer sql = new StringBuffer();
-        BufferedReader reader = null;
-        try{
-
-        InputStream inputStream = this.getClass().getClassLoader().getResourceAsStream(scriptName);
-        reader = new BufferedReader(new InputStreamReader(inputStream));
-        String line;
-        while ((line = reader.readLine()) != null) {
-            line = line.trim();
-            if (line.startsWith("//")) {
-                continue;
-            }
-            if (line.startsWith("--")) {
-                continue;
-            }
-            StringTokenizer st = new StringTokenizer(line);
-            if (st.hasMoreTokens()) {
-                String token = st.nextToken();
-                if ("REM".equalsIgnoreCase(token)) {
-                    continue;
-                }
-            }
-            sql.append(" ").append(line);
-
-            // SQL defines "--" as a comment to EOL
-            // and in Oracle it may contain a hint
-            // so we cannot just remove it, instead we must end it
-            if (line.indexOf("--") >= 0) {
-                sql.append("\n");
-            }
-            if ((checkStringBufferEndsWith(sql, delimiter))) {
-                executeSQL(sql.substring(0, sql.length() - delimiter.length()), conn);
-                sql.replace(0, sql.length(), "");
-            }
-        }
-        // Catch any statements not followed by ;
-        if (sql.length() > 0) {
-            executeSQL(sql.toString(), conn);
-        }
-        }catch (IOException e){
-            logger.error("Error occurred while executing SQL script for creating Airavata database", e);
-            throw new Exception("Error occurred while executing SQL script for creating Airavata database", e);
-        }finally {
-            if (reader != null) {
-                reader.close();
-            }
-
-        }
-
-    }
-
-    private static void executeSQL(String sql, Connection conn) throws Exception {
-        // Check and ignore empty statements
-        if ("".equals(sql.trim())) {
-            return;
-        }
-
-        Statement statement = null;
-        try {
-            logger.debug("SQL : " + sql);
-
-            boolean ret;
-            int updateCount = 0, updateCountTotal = 0;
-            statement = conn.createStatement();
-            ret = statement.execute(sql);
-            updateCount = statement.getUpdateCount();
-            do {
-                if (!ret) {
-                    if (updateCount != -1) {
-                        updateCountTotal += updateCount;
-                    }
-                }
-                ret = statement.getMoreResults();
-                if (ret) {
-                    updateCount = statement.getUpdateCount();
-                }
-            } while (ret);
-
-            logger.debug(sql + " : " + updateCountTotal + " rows affected");
-
-            SQLWarning warning = conn.getWarnings();
-            while (warning != null) {
-                logger.warn(warning + " sql warning");
-                warning = warning.getNextWarning();
-            }
-            conn.clearWarnings();
-        } catch (SQLException e) {
-            if (e.getSQLState().equals("X0Y32")) {
-                // eliminating the table already exception for the derby
-                // database
-                logger.info("Table Already Exists", e);
-            } else {
-                throw new Exception("Error occurred while executing : " + sql, e);
-            }
-        } finally {
-            if (statement != null) {
-                try {
-                    statement.close();
-                } catch (SQLException e) {
-                    logger.error("Error occurred while closing result set.", e);
-                }
-            }
-        }
-    }
-
-    private void startDerbyInServerMode() {
-        try {
-            System.setProperty(DERBY_SERVER_MODE_SYS_PROPERTY, "true");
-            server = new NetworkServerControl(InetAddress.getByName("0.0.0.0"),
-                    20000,
-                    jdbcUser, jdbcPassword);
-            java.io.PrintWriter consoleWriter = new java.io.PrintWriter(System.out, true);
-            server.start(consoleWriter);
-        } catch (IOException e) {
-            logger.error("Unable to start Apache derby in the server mode! Check whether " +
-                    "specified port is available");
-        } catch (Exception e) {
-            logger.error("Unable to start Apache derby in the server mode! Check whether " +
-                    "specified port is available");
-        }
-
-    }
-
-    public static int getPort(String jdbcURL){
-        try{
-            String cleanURI = jdbcURL.substring(5);
-            URI uri = URI.create(cleanURI);
-            return uri.getPort();
-        } catch (Exception e) {
-            logger.error(e.getMessage(), e);
-            return -1;
-        }
-    }
-
-    private void startDerbyInEmbeddedMode(){
-        try {
-            Class.forName("org.apache.derby.jdbc.EmbeddedDriver");
-            DriverManager.getConnection("jdbc:derby:memory:unit-testing-jpa;create=true").close();
-        } catch (ClassNotFoundException e) {
-            logger.error(e.getMessage(), e);
-        } catch (SQLException e) {
-            logger.error(e.getMessage(), e);
-        }
-    }
-
-    public void stopDerbyServer() {
-        try {
-            server.shutdown();
-        } catch (Exception e) {
-            logger.error(e.getMessage(), e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/airavata/blob/1eb3b415/airavata-api/airavata-api-server/src/test/java/org/apache/airavata/api/server/handler/utils/ExpCatInit.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-api-server/src/test/java/org/apache/airavata/api/server/handler/utils/ExpCatInit.java b/airavata-api/airavata-api-server/src/test/java/org/apache/airavata/api/server/handler/utils/ExpCatInit.java
deleted file mode 100644
index 90bbeeb..0000000
--- a/airavata-api/airavata-api-server/src/test/java/org/apache/airavata/api/server/handler/utils/ExpCatInit.java
+++ /dev/null
@@ -1,330 +0,0 @@
-/*
- *
- * 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.airavata.api.server.handler.utils;
-
-import org.apache.airavata.common.exception.ApplicationSettingsException;
-import org.apache.airavata.common.utils.ServerSettings;
-import org.apache.airavata.registry.core.experiment.catalog.ResourceType;
-import org.apache.airavata.registry.core.experiment.catalog.resources.*;
-import org.apache.airavata.registry.cpi.*;
-import org.apache.derby.drda.NetworkServerControl;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.net.InetAddress;
-import java.sql.*;
-import java.util.StringTokenizer;
-
-public class ExpCatInit {
-    private static final Logger logger = LoggerFactory.getLogger(ExpCatInit.class);
-    public static final String DERBY_SERVER_MODE_SYS_PROPERTY = "derby.drda.startNetworkServer";
-    public  String expCatScript = "expcatalog-derby.sql";
-    private NetworkServerControl server;
-    private static final String delimiter = ";";
-    public static final String PERSISTANT_DATA = "Configuration";
-
-    public ExpCatInit(String scriptName) {
-        this.expCatScript = scriptName;
-    }
-
-    public static boolean checkStringBufferEndsWith(StringBuffer buffer, String suffix) {
-        if (suffix.length() > buffer.length()) {
-            return false;
-        }
-        // this loop is done on purpose to avoid memory allocation performance
-        // problems on various JDKs
-        // StringBuffer.lastIndexOf() was introduced in jdk 1.4 and
-        // implementation is ok though does allocation/copying
-        // StringBuffer.toString().endsWith() does massive memory
-        // allocation/copying on JDK 1.5
-        // See http://issues.apache.org/bugzilla/show_bug.cgi?id=37169
-        int endIndex = suffix.length() - 1;
-        int bufferIndex = buffer.length() - 1;
-        while (endIndex >= 0) {
-            if (buffer.charAt(bufferIndex) != suffix.charAt(endIndex)) {
-                return false;
-            }
-            bufferIndex--;
-            endIndex--;
-        }
-        return true;
-    }
-
-    private static boolean isServerStarted(NetworkServerControl server, int ntries)
-    {
-        for (int i = 1; i <= ntries; i ++)
-        {
-            try {
-                Thread.sleep(500);
-                server.ping();
-                return true;
-            }
-            catch (Exception e) {
-                if (i == ntries)
-                    return false;
-            }
-        }
-        return false;
-    }
-
-    public void initializeDB() throws SQLException{
-        String jdbcUrl = null;
-        String jdbcUser = null;
-        String jdbcPassword = null;
-        try{
-            jdbcUrl = ServerSettings.getSetting("registry.jdbc.url");
-            jdbcUser = ServerSettings.getSetting("registry.jdbc.user");
-            jdbcPassword = ServerSettings.getSetting("registry.jdbc.password");
-            jdbcUrl = jdbcUrl + "?" + "user=" + jdbcUser + "&" + "password=" + jdbcPassword;
-        } catch (ApplicationSettingsException e) {
-            logger.error("Unable to read properties", e);
-        }
-        startDerbyInServerMode();
-        if(!isServerStarted(server, 20)){
-           throw new RuntimeException("Derby server cound not started within five seconds...");
-        }
-
-        Connection conn = null;
-        try {
-            Class.forName(Utils.getJDBCDriver()).newInstance();
-            conn = DriverManager.getConnection(jdbcUrl, jdbcUser, jdbcPassword);
-            if (!isDatabaseStructureCreated(PERSISTANT_DATA, conn)) {
-                executeSQLScript(conn);
-                logger.info("New Database created for Exp Catalog");
-            } else {
-                logger.debug("Database already created for Exp Catalog!");
-            }
-        } catch (Exception e) {
-            logger.error(e.getMessage(), e);
-            throw new RuntimeException("Database failure", e);
-        } finally {
-            try {
-                if (conn != null){
-                    if (!conn.getAutoCommit()) {
-                        conn.commit();
-                    }
-                    conn.close();
-                }
-            } catch (SQLException e) {
-                logger.error(e.getMessage(), e);
-            }
-        }
-
-        try{
-            GatewayResource gatewayResource = new GatewayResource();
-            gatewayResource.setGatewayId(ServerSettings.getSetting("default.registry.gateway"));
-            gatewayResource.setGatewayName(ServerSettings.getSetting("default.registry.gateway"));
-            gatewayResource.setDomain("test-domain");
-            gatewayResource.setEmailAddress("test-email");
-            gatewayResource.save();
-            
-            UserResource userResource = new UserResource();
-            userResource.setUserName(ServerSettings.getSetting("default.registry.user"));
-            userResource.setPassword(ServerSettings.getSetting("default.registry.password"));
-            userResource.setGatewayId(ServerSettings.getDefaultUserGateway());
-            userResource.save();
-
-            WorkerResource workerResource = (WorkerResource) gatewayResource.create(ResourceType.GATEWAY_WORKER);
-            workerResource.setUser(userResource.getUserName());
-            workerResource.save();
-            
-            ProjectResource projectResource = (ProjectResource)workerResource.create(ResourceType.PROJECT);
-            projectResource.setGatewayId(gatewayResource.getGatewayId());
-            projectResource.setId("default");
-            projectResource.setName("default");
-            projectResource.setWorker(workerResource);
-            projectResource.save();
-        
-          
-        } catch (ApplicationSettingsException e) {
-            logger.error("Unable to read properties", e);
-            throw new SQLException(e.getMessage(), e);
-        } catch (RegistryException e) {
-            logger.error("Unable to save data to registry", e);
-            throw new SQLException(e.getMessage(), e);
-        }
-    }
-
-    public static boolean isDatabaseStructureCreated(String tableName, Connection conn) {
-        try {
-            System.out.println("Running a query to test the database tables existence.");
-            // check whether the tables are already created with a query
-            Statement statement = null;
-            try {
-                statement = conn.createStatement();
-                ResultSet rs = statement.executeQuery("select * from " + tableName);
-                if (rs != null) {
-                    rs.close();
-                }
-            } finally {
-                try {
-                    if (statement != null) {
-                        statement.close();
-                    }
-                } catch (SQLException e) {
-                    return false;
-                }
-            }
-        } catch (SQLException e) {
-            return false;
-        }
-
-        return true;
-    }
-
-    private void executeSQLScript(Connection conn) throws Exception {
-        StringBuffer expCatsql = new StringBuffer();
-        BufferedReader expCatReader = null;
-        try{
-
-        InputStream expCatStream = this.getClass().getClassLoader().getResourceAsStream(expCatScript);
-        expCatReader = new BufferedReader(new InputStreamReader(expCatStream));
-        String line;
-        while ((line = expCatReader.readLine()) != null) {
-            line = line.trim();
-            if (line.startsWith("//")) {
-                continue;
-            }
-            if (line.startsWith("--")) {
-                continue;
-            }
-            StringTokenizer st = new StringTokenizer(line);
-            if (st.hasMoreTokens()) {
-                String token = st.nextToken();
-                if ("REM".equalsIgnoreCase(token)) {
-                    continue;
-                }
-            }
-            expCatsql.append(" ").append(line);
-
-            // SQL defines "--" as a comment to EOL
-            // and in Oracle it may contain a hint
-            // so we cannot just remove it, instead we must end it
-            if (line.indexOf("--") >= 0) {
-                expCatsql.append("\n");
-            }
-            if ((checkStringBufferEndsWith(expCatsql, delimiter))) {
-                executeSQL(expCatsql.substring(0, expCatsql.length() - delimiter.length()), conn);
-                expCatsql.replace(0, expCatsql.length(), "");
-            }
-        }
-        // Catch any statements not followed by ;
-        if (expCatsql.length() > 0) {
-            executeSQL(expCatsql.toString(), conn);
-        }
-        }catch (IOException e){
-            logger.error("Error occurred while executing SQL script for creating Airavata database", e);
-            throw new Exception("Error occurred while executing SQL script for creating Airavata database", e);
-        }finally {
-            if (expCatReader != null) {
-                expCatReader.close();
-            }
-
-        }
-
-    }
-
-    private static void executeSQL(String sql, Connection conn) throws Exception {
-        // Check and ignore empty statements
-        if ("".equals(sql.trim())) {
-            return;
-        }
-
-        Statement statement = null;
-        try {
-            logger.debug("SQL : " + sql);
-
-            boolean ret;
-            int updateCount = 0, updateCountTotal = 0;
-            statement = conn.createStatement();
-            ret = statement.execute(sql);
-            updateCount = statement.getUpdateCount();
-            do {
-                if (!ret) {
-                    if (updateCount != -1) {
-                        updateCountTotal += updateCount;
-                    }
-                }
-                ret = statement.getMoreResults();
-                if (ret) {
-                    updateCount = statement.getUpdateCount();
-                }
-            } while (ret);
-
-            logger.debug(sql + " : " + updateCountTotal + " rows affected");
-
-            SQLWarning warning = conn.getWarnings();
-            while (warning != null) {
-                logger.warn(warning + " sql warning");
-                warning = warning.getNextWarning();
-            }
-            conn.clearWarnings();
-        } catch (SQLException e) {
-            if (e.getSQLState().equals("X0Y32")) {
-                // eliminating the table already exception for the derby
-                // database
-                logger.info("Table Already Exists", e);
-            } else {
-                throw new Exception("Error occurred while executing : " + sql, e);
-            }
-        } finally {
-            if (statement != null) {
-                try {
-                    statement.close();
-                } catch (SQLException e) {
-                    logger.error("Error occurred while closing result set.", e);
-                }
-            }
-        }
-    }
-
-    private void startDerbyInServerMode() {
-        try {
-            System.setProperty(DERBY_SERVER_MODE_SYS_PROPERTY, "true");
-            server = new NetworkServerControl(InetAddress.getByName(Utils.getHost()),
-                    20000,
-                    Utils.getJDBCUser(), Utils.getJDBCPassword());
-            java.io.PrintWriter consoleWriter = new java.io.PrintWriter(System.out, true);
-            server.start(consoleWriter);
-        } catch (IOException e) {
-            logger.error("Unable to start Apache derby in the server mode! Check whether " +
-                    "specified port is available");
-        } catch (Exception e) {
-            logger.error("Unable to start Apache derby in the server mode! Check whether " +
-                    "specified port is available");
-        }
-
-    }
-
-    public void stopDerbyServer() throws SQLException{
-        try {
-            server.shutdown();
-        } catch (Exception e) {
-            logger.error(e.getMessage(), e);
-            throw new SQLException("Error while stopping derby server", e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/airavata/blob/1eb3b415/airavata-api/airavata-api-server/src/test/java/org/apache/airavata/api/server/handler/utils/ReplicaCatInit.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-api-server/src/test/java/org/apache/airavata/api/server/handler/utils/ReplicaCatInit.java b/airavata-api/airavata-api-server/src/test/java/org/apache/airavata/api/server/handler/utils/ReplicaCatInit.java
deleted file mode 100644
index 977454a..0000000
--- a/airavata-api/airavata-api-server/src/test/java/org/apache/airavata/api/server/handler/utils/ReplicaCatInit.java
+++ /dev/null
@@ -1,315 +0,0 @@
-/*
- *
- * 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.airavata.api.server.handler.utils;
-
-import org.apache.airavata.common.exception.ApplicationSettingsException;
-import org.apache.airavata.common.utils.ServerSettings;
-import org.apache.airavata.registry.core.replica.catalog.utils.ReplicaCatalogConstants;
-import org.apache.derby.drda.NetworkServerControl;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.net.InetAddress;
-import java.net.URI;
-import java.sql.*;
-import java.util.StringTokenizer;
-
-public class ReplicaCatInit {
-    private static final Logger logger = LoggerFactory.getLogger(ReplicaCatInit.class);
-    public static final String DERBY_SERVER_MODE_SYS_PROPERTY = "derby.drda.startNetworkServer";
-    public  String scriptName = "replicacatalog-derby.sql";
-    private NetworkServerControl server;
-    private static final String delimiter = ";";
-    private String jdbcUrl = null;
-    private String jdbcDriver = null;
-    private String jdbcUser = null;
-    private String jdbcPassword = null;
-
-    public ReplicaCatInit(String scriptName) {
-        this.scriptName = scriptName;
-    }
-
-    public static boolean checkStringBufferEndsWith(StringBuffer buffer, String suffix) {
-        if (suffix.length() > buffer.length()) {
-            return false;
-        }
-        // this loop is done on purpose to avoid memory allocation performance
-        // problems on various JDKs
-        // StringBuffer.lastIndexOf() was introduced in jdk 1.4 and
-        // implementation is ok though does allocation/copying
-        // StringBuffer.toString().endsWith() does massive memory
-        // allocation/copying on JDK 1.5
-        // See http://issues.apache.org/bugzilla/show_bug.cgi?id=37169
-        int endIndex = suffix.length() - 1;
-        int bufferIndex = buffer.length() - 1;
-        while (endIndex >= 0) {
-            if (buffer.charAt(bufferIndex) != suffix.charAt(endIndex)) {
-                return false;
-            }
-            bufferIndex--;
-            endIndex--;
-        }
-        return true;
-    }
-
-    private static boolean isServerStarted(NetworkServerControl server, int ntries)
-    {
-        for (int i = 1; i <= ntries; i ++)
-        {
-            try {
-                Thread.sleep(500);
-                server.ping();
-                return true;
-            }
-            catch (Exception e) {
-                if (i == ntries)
-                    return false;
-            }
-        }
-        return false;
-    }
-
-    public void initializeDB() {
-        try{
-            jdbcDriver = ServerSettings.getSetting("replicacatalog.jdbc.driver");
-            jdbcUrl = ServerSettings.getSetting("replicacatalog.jdbc.url");
-            jdbcUser = ServerSettings.getSetting("replicacatalog.jdbc.user");
-            jdbcPassword = ServerSettings.getSetting("replicacatalog.jdbc.password");
-            jdbcUrl = jdbcUrl + "?" + "user=" + jdbcUser + "&" + "password=" + jdbcPassword;
-        } catch (ApplicationSettingsException e) {
-            logger.error("Unable to read properties", e);
-        }
-
-        startDerbyInServerMode();
-        if(!isServerStarted(server, 20)){
-           throw new RuntimeException("Derby server could not started within five seconds...");
-        }
-        Connection conn = null;
-        try {
-            Class.forName(jdbcDriver).newInstance();
-            conn = DriverManager.getConnection(jdbcUrl, jdbcUser, jdbcPassword);
-            if (!isDatabaseStructureCreated(ReplicaCatalogConstants.CONFIGURATION, conn)) {
-                executeSQLScript(conn);
-                logger.info("New Database created for Data Catalog !!!");
-            } else {
-                logger.debug("Database already created for Data Catalog!");
-            }
-        } catch (Exception e) {
-            logger.error(e.getMessage(), e);
-            throw new RuntimeException("Database failure", e);
-        } finally {
-            try {
-                if (conn != null){
-                    if (!conn.getAutoCommit()) {
-                        conn.commit();
-                    }
-                    conn.close();
-                }
-            } catch (SQLException e) {
-                logger.error(e.getMessage(), e);
-            }
-        }
-    }
-
-    public static boolean isDatabaseStructureCreated(String tableName, Connection conn) {
-        try {
-            System.out.println("Running a query to test the database tables existence.");
-            // check whether the tables are already created with a query
-            Statement statement = null;
-            try {
-                statement = conn.createStatement();
-                ResultSet rs = statement.executeQuery("select * from " + tableName);
-                if (rs != null) {
-                    rs.close();
-                }
-            } finally {
-                try {
-                    if (statement != null) {
-                        statement.close();
-                    }
-                } catch (SQLException e) {
-                    return false;
-                }
-            }
-        } catch (SQLException e) {
-            return false;
-        }
-
-        return true;
-    }
-
-    private void executeSQLScript(Connection conn) throws Exception {
-        StringBuffer sql = new StringBuffer();
-        BufferedReader reader = null;
-        try{
-
-        InputStream inputStream = this.getClass().getClassLoader().getResourceAsStream(scriptName);
-        reader = new BufferedReader(new InputStreamReader(inputStream));
-        String line;
-        while ((line = reader.readLine()) != null) {
-            line = line.trim();
-            if (line.startsWith("//")) {
-                continue;
-            }
-            if (line.startsWith("--")) {
-                continue;
-            }
-            StringTokenizer st = new StringTokenizer(line);
-            if (st.hasMoreTokens()) {
-                String token = st.nextToken();
-                if ("REM".equalsIgnoreCase(token)) {
-                    continue;
-                }
-            }
-            sql.append(" ").append(line);
-
-            // SQL defines "--" as a comment to EOL
-            // and in Oracle it may contain a hint
-            // so we cannot just remove it, instead we must end it
-            if (line.indexOf("--") >= 0) {
-                sql.append("\n");
-            }
-            if ((checkStringBufferEndsWith(sql, delimiter))) {
-                executeSQL(sql.substring(0, sql.length() - delimiter.length()), conn);
-                sql.replace(0, sql.length(), "");
-            }
-        }
-        // Catch any statements not followed by ;
-        if (sql.length() > 0) {
-            executeSQL(sql.toString(), conn);
-        }
-        }catch (IOException e){
-            logger.error("Error occurred while executing SQL script for creating Airavata Data Catalog database", e);
-            throw new Exception("Error occurred while executing SQL script for creating Airavata Data Catalog database", e);
-        }finally {
-            if (reader != null) {
-                reader.close();
-            }
-        }
-    }
-
-    private static void executeSQL(String sql, Connection conn) throws Exception {
-        // Check and ignore empty statements
-        if ("".equals(sql.trim())) {
-            return;
-        }
-
-        Statement statement = null;
-        try {
-            logger.debug("SQL : " + sql);
-
-            boolean ret;
-            int updateCount = 0, updateCountTotal = 0;
-            statement = conn.createStatement();
-            ret = statement.execute(sql);
-            updateCount = statement.getUpdateCount();
-            do {
-                if (!ret) {
-                    if (updateCount != -1) {
-                        updateCountTotal += updateCount;
-                    }
-                }
-                ret = statement.getMoreResults();
-                if (ret) {
-                    updateCount = statement.getUpdateCount();
-                }
-            } while (ret);
-
-            logger.debug(sql + " : " + updateCountTotal + " rows affected");
-
-            SQLWarning warning = conn.getWarnings();
-            while (warning != null) {
-                logger.warn(warning + " sql warning");
-                warning = warning.getNextWarning();
-            }
-            conn.clearWarnings();
-        } catch (SQLException e) {
-            if (e.getSQLState().equals("X0Y32")) {
-                // eliminating the table already exception for the derby
-                // database
-                logger.info("Table Already Exists", e);
-            } else {
-                throw new Exception("Error occurred while executing : " + sql, e);
-            }
-        } finally {
-            if (statement != null) {
-                try {
-                    statement.close();
-                } catch (SQLException e) {
-                    logger.error("Error occurred while closing result set.", e);
-                }
-            }
-        }
-    }
-
-    private void startDerbyInServerMode() {
-        try {
-            System.setProperty(DERBY_SERVER_MODE_SYS_PROPERTY, "true");
-            server = new NetworkServerControl(InetAddress.getByName("0.0.0.0"),
-                    20000,
-                    jdbcUser, jdbcPassword);
-            java.io.PrintWriter consoleWriter = new java.io.PrintWriter(System.out, true);
-            server.start(consoleWriter);
-        } catch (IOException e) {
-            logger.error("Unable to start Apache derby in the server mode! Check whether " +
-                    "specified port is available");
-        } catch (Exception e) {
-            logger.error("Unable to start Apache derby in the server mode! Check whether " +
-                    "specified port is available");
-        }
-
-    }
-
-    public static int getPort(String jdbcURL){
-        try{
-            String cleanURI = jdbcURL.substring(5);
-            URI uri = URI.create(cleanURI);
-            return uri.getPort();
-        } catch (Exception e) {
-            logger.error(e.getMessage(), e);
-            return -1;
-        }
-    }
-
-    private void startDerbyInEmbeddedMode(){
-        try {
-            Class.forName("org.apache.derby.jdbc.EmbeddedDriver");
-            DriverManager.getConnection("jdbc:derby:memory:unit-testing-jpa;create=true").close();
-        } catch (ClassNotFoundException e) {
-            logger.error(e.getMessage(), e);
-        } catch (SQLException e) {
-            logger.error(e.getMessage(), e);
-        }
-    }
-
-    public void stopDerbyServer() {
-        try {
-            server.shutdown();
-        } catch (Exception e) {
-            logger.error(e.getMessage(), e);
-        }
-    }
-}


[15/50] [abbrv] airavata git commit: Merge remote-tracking branch 'origin/develop' into develop

Posted by la...@apache.org.
Merge remote-tracking branch 'origin/develop' into develop


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: 7eacf845270d162bb9e1c8608e646297a1c470f1
Parents: fe87f9c 79aa486
Author: scnakandala <su...@gmail.com>
Authored: Fri Aug 12 11:49:52 2016 -0400
Committer: scnakandala <su...@gmail.com>
Committed: Fri Aug 12 11:49:52 2016 -0400

----------------------------------------------------------------------
 .../server/handler/AiravataServerHandler.java   | 147 +++---
 .../lib/airavata/messaging_events_types.cpp     | 227 ++++++---
 .../lib/airavata/messaging_events_types.h       |  60 ++-
 .../Airavata/Model/Messaging/Event/Types.php    | 124 ++++-
 .../airavata/model/messaging/event/ttypes.py    | 121 ++++-
 .../messaging/event/ExperimentSubmitEvent.java  | 507 +++++++++++++++++++
 .../model/messaging/event/MessageType.java      |  25 +-
 .../airavata/messaging/core/MessageHandler.java |   2 +-
 .../messaging/core/MessagingFactory.java        |  20 +-
 .../messaging/core/impl/ExperimentConsumer.java |  74 ++-
 .../messaging/core/impl/ProcessConsumer.java    |   6 +-
 .../core/impl/GFACPassiveJobSubmitter.java      |   2 +-
 .../server/OrchestratorServerHandler.java       |  37 +-
 .../utils/ThriftDataModelConversion.java        |   4 +-
 .../airavata-apis/messaging_events.thrift       |   6 +
 15 files changed, 1186 insertions(+), 176 deletions(-)
----------------------------------------------------------------------



[22/50] [abbrv] airavata git commit: merge registry-java8

Posted by la...@apache.org.
merge registry-java8


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: 99d3f4bcdd486f0b6eb8f607c2e517f3e1450753
Parents: 3fcde52 db9e3b7
Author: scnakandala <su...@gmail.com>
Authored: Mon Aug 15 15:22:17 2016 -0400
Committer: scnakandala <su...@gmail.com>
Committed: Mon Aug 15 15:22:17 2016 -0400

----------------------------------------------------------------------
 .../airavata/registry/core/Committer.java       |  29 ++++
 .../apache/airavata/registry/core/JPAUtil.java  |  57 +++++++
 .../catalog/resources/ExperimentResource.java   | 148 ++++++-------------
 3 files changed, 134 insertions(+), 100 deletions(-)
----------------------------------------------------------------------



[37/50] [abbrv] airavata git commit: Addition of AMQPWSTunnel as an Airavata module

Posted by la...@apache.org.
Addition of AMQPWSTunnel as an Airavata module


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: 7b14e0fa95e22db5ad20a2f93273195339d628ed
Parents: 24ced80
Author: Jeff Kinnison <je...@gmail.com>
Authored: Fri Aug 19 17:25:53 2016 -0400
Committer: Jeff Kinnison <je...@gmail.com>
Committed: Fri Aug 19 17:25:53 2016 -0400

----------------------------------------------------------------------
 modules/amqpwstunnel/python/amqpwstunnel.py | 583 +++++++++++++++++++++++
 modules/amqpwstunnel/python/config.json     |  10 +
 modules/amqpwstunnel/wstest.html            | 157 ++++++
 3 files changed, 750 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/7b14e0fa/modules/amqpwstunnel/python/amqpwstunnel.py
----------------------------------------------------------------------
diff --git a/modules/amqpwstunnel/python/amqpwstunnel.py b/modules/amqpwstunnel/python/amqpwstunnel.py
new file mode 100644
index 0000000..af5d68a
--- /dev/null
+++ b/modules/amqpwstunnel/python/amqpwstunnel.py
@@ -0,0 +1,583 @@
+import argparse
+import base64
+import functools
+import json
+import sys
+import uuid
+import weakref
+
+from threading import Thread, Lock
+
+try:
+    from urllib.parse import urlencode
+except ImportError:
+    from urllib import urlencode
+
+import pika
+import tornado.websocket
+import tornado.ioloop
+import tornado.auth
+import tornado.escape
+import tornado.concurrent
+
+
+SETTINGS = {}
+
+
+class Error(Exception):
+    """Base error class for exceptions in this module"""
+    pass
+
+class ConsumerConfigError(Error):
+    """Raised when an issue with consumer configuration occurs"""
+    def __init__(self, message):
+        self.message = message
+
+class ConsumerKeyError(Error):
+    def __init__(self, message, key):
+        self.message = message
+        self.key = key
+
+class AuthError(Error):
+    """Raised when something went wrong during authentication"""
+    def __init__(self, error, code):
+        self.message = error
+        self.code = code
+
+
+
+class PikaAsyncConsumer(Thread):
+
+    """
+    The primary entry point for routing incoming messages to the proper handler.
+
+    """
+
+    def __init__(self, rabbitmq_url, exchange_name, queue_name,
+                 exchange_type="direct", routing_key="#"):
+        """
+        Create a new instance of Streamer.
+
+        Arguments:
+        rabbitmq_url -- URL to RabbitMQ server
+        exchange_name -- name of RabbitMQ exchange to join
+        queue_name -- name of RabbitMQ queue to join
+
+        Keyword Arguments:
+        exchange_type -- one of 'direct', 'topic', 'fanout', 'headers'
+                         (default 'direct')
+        routing_keys -- the routing key that this consumer listens for
+                        (default '#', receives all messages)
+
+        """
+        print("Creating new consumer")
+        super(PikaAsyncConsumer, self).__init__(daemon=True)
+        self._connection = None
+        self._channel = None
+        self._shut_down = False
+        self._consumer_tag = None
+        self._url = rabbitmq_url
+        self._client_list = []
+        self._lock = Lock()
+
+        # The following are necessary to guarantee that both the RabbitMQ
+        # server and Streamer know where to look for messages. These names will
+        # be decided before dispatch and should be recorded in a config file or
+        # else on a per-job basis.
+        self._exchange = exchange_name
+        self._exchange_type = exchange_type
+        self._queue = queue_name
+        self._routing_key = routing_key
+
+    def add_client(self, client):
+        """Add a new client to the recipient list.
+
+        Arguments:
+            client -- a reference to the client object to add
+        """
+        self._lock.acquire()
+        # Create a weakref to ensure that cyclic references to WebSocketHandler
+        # objects do not cause problems for garbage collection
+        self._client_list.append(weakref.ref(client))
+        self._lock.release()
+
+    def remove_client(self, client):
+        """Remove a client from the recipient list.
+
+        Arguments:
+            client -- a reference to the client object to remove
+        """
+        self._lock.acquire()
+        for i in range(0, len(self._client_list)):
+            # Parentheses after _client_list[i] to deference the weakref to its
+            # strong reference
+            if self._client_list[i]() is client:
+                self._client_list.pop(i)
+                break
+        self._lock.release()
+
+
+    def connect(self):
+        """
+        Create an asynchronous connection to the RabbitMQ server at URL.
+
+        """
+        return pika.SelectConnection(pika.URLParameters(self._url),
+                                     on_open_callback=self.on_connection_open,
+                                     on_close_callback=self.on_connection_close,
+                                     stop_ioloop_on_close=False)
+
+    def on_connection_open(self, unused_connection):
+        """
+        Actions to perform when the connection opens. This may not happen
+        immediately, so defer action to this callback.
+
+        Arguments:
+        unused_connection -- the created connection (by this point already
+                             available as self._connection)
+
+        """
+        self._connection.channel(on_open_callback=self.on_channel_open)
+
+    def on_connection_close(self, connection, code, text):
+        """
+        Actions to perform when the connection is unexpectedly closed by the
+        RabbitMQ server.
+
+        Arguments:
+        connection -- the connection that was closed (same as self._connection)
+        code -- response code from the RabbitMQ server
+        text -- response body from the RabbitMQ server
+
+        """
+        self._channel = None
+        if self._shut_down:
+            self._connection.ioloop.stop()
+        else:
+            self._connection.add_timeout(5, self.reconnect)
+
+    def reconnect(self):
+        """
+        Attempt to reestablish a connection with the RabbitMQ server.
+        """
+        self._connection.ioloop.stop() # Stop the ioloop to completely close
+
+        if not self._shut_down: # Connect and restart the ioloop
+            self._connection = self.connect()
+            self._connection.ioloop.start()
+
+    def on_channel_open(self, channel):
+        """
+        Store the opened channel for future use and set up the exchange and
+        queue to be used.
+
+        Arguments:
+        channel -- the Channel instance opened by the Channel.Open RPC
+        """
+        self._channel = channel
+        self._channel.add_on_close_callback(self.on_channel_close)
+        self.declare_exchange()
+
+
+    def on_channel_close(self, channel, code, text):
+        """
+        Actions to perform when the channel is unexpectedly closed by the
+        RabbitMQ server.
+
+        Arguments:
+        connection -- the connection that was closed (same as self._connection)
+        code -- response code from the RabbitMQ server
+        text -- response body from the RabbitMQ server
+        """
+        self._connection.close()
+
+    def declare_exchange(self):
+        """
+        Set up the exchange that will route messages to this consumer. Each
+        RabbitMQ exchange is uniquely identified by its name, so it does not
+        matter if the exchange has already been declared.
+        """
+        self._channel.exchange_declare(self.declare_exchange_success,
+                                        self._exchange,
+                                        self._exchange_type)
+
+    def declare_exchange_success(self, unused_connection):
+        """
+        Actions to perform on successful exchange declaration.
+        """
+        self.declare_queue()
+
+    def declare_queue(self):
+        """
+        Set up the queue that will route messages to this consumer. Each
+        RabbitMQ queue can be defined with routing keys to use only one
+        queue for multiple jobs.
+        """
+        self._channel.queue_declare(self.declare_queue_success,
+                                    self._queue)
+
+    def declare_queue_success(self, method_frame):
+        """
+        Actions to perform on successful queue declaration.
+        """
+        self._channel.queue_bind(self.munch,
+                                 self._queue,
+                                 self._exchange,
+                                 self._routing_key
+                                )
+
+    def munch(self, unused):
+        """
+        Begin consuming messages from the Airavata API server.
+        """
+        self._channel.add_on_cancel_callback(self.cancel_channel)
+        self._consumer_tag = self._channel.basic_consume(self._process_message)
+
+    def cancel_channel(self, method_frame):
+        if self._channel is not None:
+            self._channel._close()
+
+    def _process_message(self, ch, method, properties, body):
+        """
+        Receive and verify a message, then pass it to the router.
+
+        Arguments:
+        ch -- the channel that routed the message
+        method -- delivery information
+        properties -- message properties
+        body -- the message
+        """
+        print("Received Message: %s" % body)
+        self._lock.acquire()
+        for client in self._client_list:
+            # Parentheses after client to deference the weakref to its
+            # strong reference
+            client().write_message(body)
+        self._lock.release()
+        self._channel.basic_ack(delivery_tag=method.delivery_tag)
+
+    def stop_consuming(self):
+        """
+        Stop the consumer if active.
+        """
+        if self._channel:
+            self._channel.basic_cancel(self.close_channel, self._consumer_tag)
+
+    def close_channel(self, unused):
+        """
+        Close the channel to shut down the consumer and connection.
+        """
+        self._channel.queue_delete(queue=self._queue)
+        self._channel.close()
+
+    def run(self):
+        """
+        Start a connection with the RabbitMQ server.
+        """
+        self._connection = self.connect()
+        self._connection.ioloop.start()
+
+    def stop(self):
+        """
+        Stop an active connection with the RabbitMQ server.
+        """
+        self._closing = True
+        self.stop_consuming()
+
+
+class Wso2OAuth2Mixin(tornado.auth.OAuth2Mixin):
+    _OAUTH_AUTHORIZE_URL = "https://idp.scigap.org:9443/oauth2/authorize"
+    _OAUTH_ACCESS_TOKEN_URL = "https://idp.scigap.org:9443/oauth2/token"
+
+    @tornado.auth._auth_return_future
+    def get_authenticated_user(self, username, password, callback=None):
+        print("Authenticating user %s" % (username))
+        http = self.get_auth_http_client()
+        body = urlencode({
+            "client_id": SETTINGS["oauth_client_key"],
+            "client_secret": SETTINGS["oauth_client_secret"],
+            "grant_type": SETTINGS["oauth_grant_type"],
+            "username": username,
+            "password": password
+        })
+        http.fetch(self._OAUTH_ACCESS_TOKEN_URL, functools.partial(self._on_access_token, callback), method="POST", body=body)
+
+    def _on_access_token(self, future, response):
+        if response.error:
+            print(str(response))
+            print(response.body)
+            print(response.error)
+            future.set_exception(AuthError(response.error, response.code))
+            return
+
+        print(response.body)
+        future.set_result(tornado.escape.json_decode(response.body))
+
+class AuthHandler(tornado.web.RequestHandler, Wso2OAuth2Mixin):
+    def get_current_user(self):
+        expires_in = self.get_secure_cookie("expires-in", max_age_days=SETTINGS['maximum_cookie_age'])
+        print(expires_in)
+        if expires_in:
+            return self.get_secure_cookie("ws-auth-token", max_age_days=float(expires_in))
+        return None
+
+    def set_default_headers(self):
+        self.set_header("Content-Type", "text/plain")
+        self.set_header("Access-Control-Allow-Origin", "*")
+        self.set_header("Access-Control-Allow-Headers", "x-requested-with")
+        self.set_header('Access-Control-Allow-Methods', 'POST, GET, OPTIONS')
+
+    def get(self):
+        if self.get_current_user():
+            self.set_status(200)
+            print("Authenticated")
+            self.write("Authenticated")
+
+        else:
+            self.set_status(403)
+            print("Not Authenticated")
+            self.write("Not Authenticated")
+
+    @tornado.gen.coroutine
+    def post(self):
+        try:
+            username = self.get_body_argument("username")
+            password = self.get_body_argument("password")
+            redirect = self.get_body_argument("redirect")
+            if username == "" or password == "":
+                raise tornado.web.MissingArgumentError
+
+            access = yield self.get_authenticated_user(username, password)
+            days = (access["expires_in"] / 3600) / 24 # Convert to days
+            print(days)
+            self.set_secure_cookie("ws-auth-token",
+                                   access["access_token"],
+                                   expires_days=days)
+            self.set_secure_cookie("expires-in",
+                                   str(1),
+                                   expires_days=SETTINGS['maximum_cookie_age'])
+            self.write("Success")
+        except tornado.web.MissingArgumentError:
+            print("Missing an argument")
+            self.set_status(400)
+            self.write("Authentication information missing")
+        except AuthError as e:
+            print("The future freaks me out")
+            self.set_status(access.code)
+            self.set_header("Content-Type", "text/html")
+            self.write(access.message)
+
+        success_code = """<p>Redirecting to <a href="%(url)s">%(url)s</a></p>
+<script type="text/javascript">
+window.location = %(url)s;
+</script>
+        """ % { 'url': redirect}
+        self.set_status(200)
+        self.redirect(redirect)
+        #return self.render_string(success_code)
+
+
+
+class AMQPWSHandler(tornado.websocket.WebSocketHandler):#, Wso2OAuth2Mixin):
+
+    """
+    Pass messages to a connected WebSockets client.
+
+    A subclass of the Tornado WebSocketHandler class, this class takes no
+    action when receiving a message from the client. Instead, it is associated
+    with an AMQP consumer and writes a message to the client each time one is
+    consumed in the queue.
+    """
+
+    # def set_default_headers(self):
+        # self.set_header("Access-Control-Allow-Origin", "*")
+        # self.set_header("Access-Control-Allow-Headers", "x-requested-with")
+        # self.set_header('Access-Control-Allow-Methods', 'POST, GET, OPTIONS')
+
+    def check_origin(self, origin):
+        """Check the domain origin of the connection request.
+
+        This can be made more robust to ensure that connections are only
+        accepted from verified PGAs.
+
+        Arguments:
+            origin -- the value of the Origin HTTP header
+        """
+        return True
+
+    def open(self, resource_type, resource_id):
+        """Associate a new connection with a consumer.
+
+        When a new connection is opened, it is a request to retrieve data
+        from an AMQP queue. The open operation should also do some kind of
+        authentication.
+
+        Arguments:
+            resource_type -- "experiment" or "project" or "data"
+            resource_id -- the Airavata id for the resource
+        """
+        self.stream.set_nodelay(True)
+        self.resource_id = resource_id
+        self.write_message("Opened the connection")
+
+        self.add_to_consumer()
+
+        # expires_in = self.get_secure_cookie("expires_in", max_age_days=SETTINGS["maximum_cookie_age"])
+        # if expires_in is not None and self.get_secure_cookie("ws-auth-token", max_age_days=float(expires_in)):
+        #     print("Found secure cookie")
+        #     self.write_message("Authenticated")
+        #     self.add_to_consumer()
+        # else:
+        #     print("Closing connection")
+        #     self.close()
+
+    def on_message(self, message):
+        """Handle incoming messages from the client.
+
+        Tornado requires subclasses to override this method, however in this
+        case we do not wish to take any action when receiving a message from
+        the client. The purpose of this class is only to push messages to the
+        client.
+        """
+        print(message)
+        message = tornado.escape.json_decode(message)
+        access = yield self.get_authenticated_user(message["username"], message["password"])
+        access = access
+        days = (access["expires_in"] / 3600) / 24 # Convert to days
+        print(days)
+        self.set_secure_cookie("ws-auth-token",
+                               access["access_token"],
+                               expires_days=days)
+        self.set_secure_cookie("expires_in",
+                               str(days),
+                               expires_days=SETTINGS['maximum_cookie_age'])
+
+
+    def on_close(self):
+        try:
+            print("Closing connection")
+            self.application.remove_client_from_consumer(self.resource_id, self)
+        except KeyError:
+            print("Error: resource %s does not exist" % self.resource_id)
+        finally:
+            self.close()
+
+    def add_to_consumer(self):
+        try:
+            self.application.add_client_to_consumer(self.resource_id, self)
+        except AttributeError as e:
+            print("Error: tornado.web.Application object is not AMQPWSTunnel")
+            print(e)
+
+
+class AMQPWSTunnel(tornado.web.Application):
+
+    """
+    Send messages from an AMQP queue to WebSockets clients.
+
+    In addition to the standard Tornado Application class functionality, this
+    class maintains a list of active AMQP consumers and maps WebSocketHandlers
+    to the correct consumers.
+    """
+
+    def __init__(self, consumer_list=None, consumer_config=None, handlers=None,
+                 default_host='', transforms=None, **settings):
+        print("Starting AMQP-WS-Tunnel application")
+        super(AMQPWSTunnel, self).__init__(handlers=handlers,
+                                           default_host=default_host,
+                                           transforms=transforms,
+                                           **settings)
+
+        self.consumer_list = {} if consumer_list is None else consumer_list
+        if consumer_config is None:
+            raise ConsumerConfigError("No consumer configuration provided")
+        self.consumer_config = consumer_config
+
+    def consumer_exists(self, resource_id):
+        """Determine if a consumer exists for a particular resource.
+
+        Arguments:
+            resource_id -- the consumer to find
+        """
+        return resource_id in self.consumer_list
+
+    def add_client_to_consumer(self, resource_id, client):
+        """Add a new client to a consumer's messaging list.
+
+        Arguments:
+            resource_id -- the consumer to add to
+            client -- the client to add
+        """
+        if not self.consumer_exists(resource_id):
+            print("Creating new consumer")
+            print(self.consumer_config)
+            consumer = PikaAsyncConsumer(self.consumer_config["rabbitmq_url"],
+                                         self.consumer_config["exchange_name"],
+                                         self.consumer_config["queue_name"],
+                                         exchange_type=self.consumer_config["exchange_type"],
+                                         routing_key=resource_id)
+            print("Adding to consumer list")
+            self.consumer_list[resource_id] = consumer
+            print("Starting consumer")
+            consumer.start()
+
+        print("Adding new client to %s" % (resource_id))
+        consumer = self.consumer_list[resource_id]
+        consumer.add_client(client)
+
+    def remove_client_from_consumer(self, resource_id, client):
+        """Remove a client from a consumer's messaging list.
+
+        Arguments:
+            resource_id -- the consumer to remove from
+            client -- the client to remove
+        """
+        if self.consumer_exists(resource_id):
+            print("Removing client from %s" % (resource_id))
+            self.consumer_list[resource_id].remove_client(client)
+        #else:
+        #    raise ConsumerKeyError("Trying to remove client from nonexistent consumer", resource_id)
+
+    def shutdown(self):
+        """Shut down the application and release all resources.
+
+
+        """
+        for name, consumer in self.consumer_list.items():
+            consumer.stop()
+            #consumer.join()
+            #self.consumer_list[name] = None
+
+        #self.consumer_list = {}
+
+
+
+if __name__ == "__main__":
+    i = open(sys.argv[1])
+    config = json.load(i)
+    i.close()
+
+    SETTINGS["oauth_client_key"] = config["oauth_client_key"]
+    SETTINGS["oauth_client_secret"] = config["oauth_client_secret"]
+    SETTINGS["oauth_grant_type"] = config["oauth_grant_type"]
+    SETTINGS["maximum_cookie_age"] = config["maximum_cookie_age"]
+
+    settings = {
+        "cookie_secret": base64.b64encode(uuid.uuid4().bytes + uuid.uuid4().bytes),
+        #"xsrf_cookies": True
+    }
+
+    application = AMQPWSTunnel(handlers=[
+                                    (r"/auth", AuthHandler),
+                                    (r"/(experiment)/(.+)", AMQPWSHandler)
+                                ],
+                                consumer_config=config,
+                                debug=True,
+                                **settings)
+
+    application.listen(8888)
+
+    try:
+        tornado.ioloop.IOLoop.current().start()
+    except KeyboardInterrupt:
+        application.shutdown()

http://git-wip-us.apache.org/repos/asf/airavata/blob/7b14e0fa/modules/amqpwstunnel/python/config.json
----------------------------------------------------------------------
diff --git a/modules/amqpwstunnel/python/config.json b/modules/amqpwstunnel/python/config.json
new file mode 100644
index 0000000..b092001
--- /dev/null
+++ b/modules/amqpwstunnel/python/config.json
@@ -0,0 +1,10 @@
+{
+    "rabbitmq_url": "amqp://airavata:airavata@gw56.iu.xsede.org:5672/messaging",
+    "exchange_name": "simstream",
+    "queue_name": "test",
+    "exchange_type": "direct",
+    "oauth_client_key": "y7xgdnNUx6ifOswJTPcqtzw4aOEa",
+    "oauth_client_secret": "CgfbuupAPhaOBSBPSScZUWHNANwa",
+    "oauth_grant_type": "password",
+    "maximum_cookie_age": 1
+}

http://git-wip-us.apache.org/repos/asf/airavata/blob/7b14e0fa/modules/amqpwstunnel/wstest.html
----------------------------------------------------------------------
diff --git a/modules/amqpwstunnel/wstest.html b/modules/amqpwstunnel/wstest.html
new file mode 100644
index 0000000..eedbf78
--- /dev/null
+++ b/modules/amqpwstunnel/wstest.html
@@ -0,0 +1,157 @@
+<!DOCTYPE html>
+
+<html lang="en">
+
+<head>
+    <meta charset="utf-8" />
+    <title>AMQP Websockets Test</title>
+
+    <style>
+        #content {
+            width: 100%;
+            min-height: 250px;
+        }
+
+        #ws-url-label, #ws-url-input, #ws-connect-button {
+            display: inline;
+            float: left;
+            margin-right: 10px;
+        }
+
+        #logs {
+            background-color: #888888;
+            width: 50%;
+            overflow-y: auto;
+            list-style: none;
+            padding: 3px;
+            margin-left: auto;
+            margin-right: auto;
+            text-align: center;
+        }
+
+        .log {
+            background-color: #cccccc;
+            display: inline-block;
+            min-height: 30px;
+            width: 90%;
+            border: 1px solid #000000;
+            padding: 3px;
+            margin-left: auto;
+            margin-right: auto;
+            margin-bottom: 5px;
+            text-align: left;
+        }
+    </style>
+</head>
+
+<body>
+
+<div id="content">
+    <div id="ws-url">
+        <label id="ws-url-label" for="ws-url-input">WebSockets URL</label>
+        <input type="text" name="ws-url-input" id="ws-url-input" />
+        <input type="text" name="username" id="username" placeholder="Username" />
+        <input type="password" name="password" id="password" placeholder="Password" />
+        <button id="ws-connect-button" class="open">Connect</button>
+        <button id="ws-send-credentials">Send</button><br />
+        <frame>
+            <form action="http://localhost:8888/auth" method="post">
+                <input type="text" name="username" id="username" placeholder="Username" />
+                <input type="password" name="password" id="password" placeholder="Password" />
+                <input type="hidden" name="redirect" id="redirect" value="file:///Users/jeffkinnison/development/amqp-ws-tunnel/wstest.html" />
+                <input type="submit" value="Auth" />
+            </form>
+        </frame>
+    </div>
+
+    <ul id="logs"><p>Logs</p></ul>
+</div>
+
+<script src="https://code.jquery.com/jquery-3.1.0.min.js"
+        integrity="sha256-cCueBR6CsyA4/9szpPfrX3s49M9vUU5BgtiJj06wt/s="
+        crossorigin="anonymous"></script>
+<script type="text/javascript">
+    var ws, open_handler, message_handler, error_handler, close_handler;
+
+    console.log(document.cookie);
+
+    $("#ws-connect-button").on("click", function() {
+        ws = new WebSocket("ws://localhost:8888/experiment/test");
+
+        ws.onopen = function() {
+            var username, password;
+            $("#ws-connect-button").toggleClass("open close").text("Disconnect");
+            $("#ws-url-input").prop("disabled", true);
+            ws.send("hi");
+            // username = $("#username").val();
+            // password = $("#password").val();
+            // ws.send(JSON.stringify({username: username, password: password}));
+        }
+
+        ws.onmessage = function(e) {
+            var msg;
+
+            console.log(e.data);
+
+            //msg = JSON.parse(e.data);
+            // if (msg.hasOwnProperty("logs")) {
+            //     for (log in msg.logs) {
+            //         if (msg.logs.hasOwnProperty(log)) {
+            //             $("#logs").append($('<li class="log">' + log + '</li>'));
+            //         }
+            //     }
+            // }
+        }
+
+        ws.onclose = function(e) {
+            $("#ws-connect-button").toggleClass("open close").text("Connect");
+            $("#ws-url-input").prop("disabled", false);
+        }
+    });
+
+    $("#ws-send-credentials").on("click", function(e) {
+        uname = $("#username").val();
+        pass = $("#password").val();
+        console.log("Sending credentials");
+        //ws.send("moop");
+        //ws.send(JSON.stringify({username: uname, password: pass}));
+        $.ajax({
+            url: "http://localhost:8888/auth",
+            method: "post",
+            data: {username: uname, password: pass},
+            crossDomain: true,
+            success: function(data) {
+                console.log("Success");
+                console.log(document.cookie);
+            },
+            error: function(e) {
+                console.log(e);
+            },
+            complete: function() {
+                $.ajax({
+                    url: "http://localhost:8888/auth",
+                    method: "get",
+                    crossDomain: true,
+                    xhrHeaders: {
+
+                    },
+                    success: function(data) {
+                        console.log(data);
+                    },
+                    error: function(xhr) {
+                        console.log(xhr);
+                    }
+                });
+            }
+        });
+    });
+
+    // $("form").on("submit", function(e) {
+    //     e.preventDefault();
+    // });
+
+</script>
+
+</body>
+
+</html>


[49/50] [abbrv] airavata git commit: [AIRAVATA-2057] Move the distribution directory to modules to slow down the distribution build

Posted by la...@apache.org.
http://git-wip-us.apache.org/repos/asf/airavata/blob/89bb0d48/distribution/src/main/resources/LICENSE
----------------------------------------------------------------------
diff --git a/distribution/src/main/resources/LICENSE b/distribution/src/main/resources/LICENSE
deleted file mode 100644
index 56f7cc2..0000000
--- a/distribution/src/main/resources/LICENSE
+++ /dev/null
@@ -1,2387 +0,0 @@
-                                 Apache License
-                           Version 2.0, January 2004
-                        http://www.apache.org/licenses/
-
-   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
-
-   1. Definitions.
-
-      "License" shall mean the terms and conditions for use, reproduction,
-      and distribution as defined by Sections 1 through 9 of this document.
-
-      "Licensor" shall mean the copyright owner or entity authorized by
-      the copyright owner that is granting the License.
-
-      "Legal Entity" shall mean the union of the acting entity and all
-      other entities that control, are controlled by, or are under common
-      control with that entity. For the purposes of this definition,
-      "control" means (i) the power, direct or indirect, to cause the
-      direction or management of such entity, whether by contract or
-      otherwise, or (ii) ownership of fifty percent (50%) or more of the
-      outstanding shares, or (iii) beneficial ownership of such entity.
-
-      "You" (or "Your") shall mean an individual or Legal Entity
-      exercising permissions granted by this License.
-
-      "Source" form shall mean the preferred form for making modifications,
-      including but not limited to software source code, documentation
-      source, and configuration files.
-
-      "Object" form shall mean any form resulting from mechanical
-      transformation or translation of a Source form, including but
-      not limited to compiled object code, generated documentation,
-      and conversions to other media types.
-
-      "Work" shall mean the work of authorship, whether in Source or
-      Object form, made available under the License, as indicated by a
-      copyright notice that is included in or attached to the work
-      (an example is provided in the Appendix below).
-
-      "Derivative Works" shall mean any work, whether in Source or Object
-      form, that is based on (or derived from) the Work and for which the
-      editorial revisions, annotations, elaborations, or other modifications
-      represent, as a whole, an original work of authorship. For the purposes
-      of this License, Derivative Works shall not include works that remain
-      separable from, or merely link (or bind by name) to the interfaces of,
-      the Work and Derivative Works thereof.
-
-      "Contribution" shall mean any work of authorship, including
-      the original version of the Work and any modifications or additions
-      to that Work or Derivative Works thereof, that is intentionally
-      submitted to Licensor for inclusion in the Work by the copyright owner
-      or by an individual or Legal Entity authorized to submit on behalf of
-      the copyright owner. For the purposes of this definition, "submitted"
-      means any form of electronic, verbal, or written communication sent
-      to the Licensor or its representatives, including but not limited to
-      communication on electronic mailing lists, source code control systems,
-      and issue tracking systems that are managed by, or on behalf of, the
-      Licensor for the purpose of discussing and improving the Work, but
-      excluding communication that is conspicuously marked or otherwise
-      designated in writing by the copyright owner as "Not a Contribution."
-
-      "Contributor" shall mean Licensor and any individual or Legal Entity
-      on behalf of whom a Contribution has been received by Licensor and
-      subsequently incorporated within the Work.
-
-   2. Grant of Copyright License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      copyright license to reproduce, prepare Derivative Works of,
-      publicly display, publicly perform, sublicense, and distribute the
-      Work and such Derivative Works in Source or Object form.
-
-   3. Grant of Patent License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      (except as stated in this section) patent license to make, have made,
-      use, offer to sell, sell, import, and otherwise transfer the Work,
-      where such license applies only to those patent claims licensable
-      by such Contributor that are necessarily infringed by their
-      Contribution(s) alone or by combination of their Contribution(s)
-      with the Work to which such Contribution(s) was submitted. If You
-      institute patent litigation against any entity (including a
-      cross-claim or counterclaim in a lawsuit) alleging that the Work
-      or a Contribution incorporated within the Work constitutes direct
-      or contributory patent infringement, then any patent licenses
-      granted to You under this License for that Work shall terminate
-      as of the date such litigation is filed.
-
-   4. Redistribution. You may reproduce and distribute copies of the
-      Work or Derivative Works thereof in any medium, with or without
-      modifications, and in Source or Object form, provided that You
-      meet the following conditions:
-
-      (a) You must give any other recipients of the Work or
-          Derivative Works a copy of this License; and
-
-      (b) You must cause any modified files to carry prominent notices
-          stating that You changed the files; and
-
-      (c) You must retain, in the Source form of any Derivative Works
-          that You distribute, all copyright, patent, trademark, and
-          attribution notices from the Source form of the Work,
-          excluding those notices that do not pertain to any part of
-          the Derivative Works; and
-
-      (d) If the Work includes a "NOTICE" text file as part of its
-          distribution, then any Derivative Works that You distribute must
-          include a readable copy of the attribution notices contained
-          within such NOTICE file, excluding those notices that do not
-          pertain to any part of the Derivative Works, in at least one
-          of the following places: within a NOTICE text file distributed
-          as part of the Derivative Works; within the Source form or
-          documentation, if provided along with the Derivative Works; or,
-          within a display generated by the Derivative Works, if and
-          wherever such third-party notices normally appear. The contents
-          of the NOTICE file are for informational purposes only and
-          do not modify the License. You may add Your own attribution
-          notices within Derivative Works that You distribute, alongside
-          or as an addendum to the NOTICE text from the Work, provided
-          that such additional attribution notices cannot be construed
-          as modifying the License.
-
-      You may add Your own copyright statement to Your modifications and
-      may provide additional or different license terms and conditions
-      for use, reproduction, or distribution of Your modifications, or
-      for any such Derivative Works as a whole, provided Your use,
-      reproduction, and distribution of the Work otherwise complies with
-      the conditions stated in this License.
-
-   5. Submission of Contributions. Unless You explicitly state otherwise,
-      any Contribution intentionally submitted for inclusion in the Work
-      by You to the Licensor shall be under the terms and conditions of
-      this License, without any additional terms or conditions.
-      Notwithstanding the above, nothing herein shall supersede or modify
-      the terms of any separate license agreement you may have executed
-      with Licensor regarding such Contributions.
-
-   6. Trademarks. This License does not grant permission to use the trade
-      names, trademarks, service marks, or product names of the Licensor,
-      except as required for reasonable and customary use in describing the
-      origin of the Work and reproducing the content of the NOTICE file.
-
-   7. Disclaimer of Warranty. Unless required by applicable law or
-      agreed to in writing, Licensor provides the Work (and each
-      Contributor provides its Contributions) on an "AS IS" BASIS,
-      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
-      implied, including, without limitation, any warranties or conditions
-      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
-      PARTICULAR PURPOSE. You are solely responsible for determining the
-      appropriateness of using or redistributing the Work and assume any
-      risks associated with Your exercise of permissions under this License.
-
-   8. Limitation of Liability. In no event and under no legal theory,
-      whether in tort (including negligence), contract, or otherwise,
-      unless required by applicable law (such as deliberate and grossly
-      negligent acts) or agreed to in writing, shall any Contributor be
-      liable to You for damages, including any direct, indirect, special,
-      incidental, or consequential damages of any character arising as a
-      result of this License or out of the use or inability to use the
-      Work (including but not limited to damages for loss of goodwill,
-      work stoppage, computer failure or malfunction, or any and all
-      other commercial damages or losses), even if such Contributor
-      has been advised of the possibility of such damages.
-
-   9. Accepting Warranty or Additional Liability. While redistributing
-      the Work or Derivative Works thereof, You may choose to offer,
-      and charge a fee for, acceptance of support, warranty, indemnity,
-      or other liability obligations and/or rights consistent with this
-      License. However, in accepting such obligations, You may act only
-      on Your own behalf and on Your sole responsibility, not on behalf
-      of any other Contributor, and only if You agree to indemnify,
-      defend, and hold each Contributor harmless for any liability
-      incurred by, or claims asserted against, such Contributor by reason
-      of your accepting any such warranty or additional liability.
-
-   END OF TERMS AND CONDITIONS
-
-   APPENDIX: How to apply the Apache License to your work.
-
-      To apply the Apache License to your work, attach the following
-      boilerplate notice, with the fields enclosed by brackets "[]"
-      replaced with your own identifying information. (Don't include
-      the brackets!)  The text should be enclosed in the appropriate
-      comment syntax for the file format. We also recommend that a
-      file or class name and description of purpose be included on the
-      same "printed page" as the copyright notice for easier
-      identification within third-party archives.
-
-   Copyright [yyyy] [name of copyright owner]
-
-   Licensed 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.
-
-===================================================================================
-The Apache Airavata distribution includes a number of run time 
-dependencies with separate copyright notices and license terms. Your use of the
-Apache Airavata code is subject to the terms and conditions of the following licenses.
-===================================================================================
-
-===============================================================================
-The following components come under Apache Software License 2.0
-===============================================================================
-
-apache axiom, apache axis2, apache commons, apache derby, apache geronimo,
-apache httpcore components, apache log4j, apache xmlbeans, apache xmlschema,
-aws-java-sdk-1.1.8.jar, bcel-5.1.jar, Codehaus Jackson (jackson-core-asl-1.9.2.jar,
-jackson-jaxrs-1.9.2.jar, jackson-mapper-asl-1.9.2.jar, jackson-xc-1.9.2.jar, 
-jets3t-0.8.0.jar, jettison-1.0-RC2.jar, neethi-2.0.4.jar, PDFBox libraries 
-(pdfbox, jempbox, fontbox), wstx-asl-3.2.4.jar
-
-- Bean Validation API (http://beanvalidation.org) javax.validation:validation-api:jar:1.1.0.Final
-- Hibernate Validator Engine (http://validator.hibernate.org/hibernate-validator) org.hibernate:hibernate-validator:jar:4.3.0.Final
-- GSS-API implementation for SSL with proxies (https://github.com/jglobus/JGlobus/gss) org.jglobus:gss:jar:2.0.6
-- SSL support (https://github.com/jglobus/JGlobus/jsse) org.jglobus:jsse:jar:2.0.6
-- myproxy (https://github.com/jglobus/JGlobus/myproxy) org.jglobus:myproxy:jar:2.0.6
-- SSL and proxy certificate support (https://github.com/jglobus/JGlobus/ssl-proxies) org.jglobus:ssl-proxies:jar:2.0.6
-- Bouncy Castle for GSS (https://github.com/jsiwek/BouncyCastleSSLv3) org.ogce:bcgss:jar:146
-- StAX API (http://stax.codehaus.org/) stax:stax-api:jar:1.0.1
-- Commons Codec (http://commons.apache.org/codec/) commons-codec:commons-codec:jar:1.4
-- Commons IO (http://commons.apache.org/io/) commons-io:commons-io:jar:1.4
-- Commons Lang (http://commons.apache.org/lang/) commons-lang:commons-lang:jar:2.6
-- Commons Logging (http://commons.apache.org/logging) commons-logging:commons-logging:jar:1.1.1
-- XmlBeans (http://xmlbeans.apache.org) org.apache.xmlbeans:xmlbeans:jar:2.5.0
-
-===============================================================================
-The following components use Apache based Licenses
-===============================================================================
-
-===============================================================================
-For: jdom-1.0.jar
-    Containing Project URL: http://www.jdom.org/
-/*-- 
-
- $Id: LICENSE.txt,v 1.11 2004/02/06 09:32:57 jhunter Exp $
-
- Copyright (C) 2000-2004 Jason Hunter & Brett McLaughlin.
- All rights reserved.
- 
- Redistribution and use in source and binary forms, with or without
- modification, are permitted provided that the following conditions
- are met:
- 
- 1. Redistributions of source code must retain the above copyright
-    notice, this list of conditions, and the following disclaimer.
- 
- 2. Redistributions in binary form must reproduce the above copyright
-    notice, this list of conditions, and the disclaimer that follows 
-    these conditions in the documentation and/or other materials 
-    provided with the distribution.
-
- 3. The name "JDOM" must not be used to endorse or promote products
-    derived from this software without prior written permission.  For
-    written permission, please contact <request_AT_jdom_DOT_org>.
- 
- 4. Products derived from this software may not be called "JDOM", nor
-    may "JDOM" appear in their name, without prior written permission
-    from the JDOM Project Management <request_AT_jdom_DOT_org>.
- 
- In addition, we request (but do not require) that you include in the 
- end-user documentation provided with the redistribution and/or in the 
- software itself an acknowledgement equivalent to the following:
-     "This product includes software developed by the
-      JDOM Project (http://www.jdom.org/)."
- Alternatively, the acknowledgment may be graphical using the logos 
- available at http://www.jdom.org/images/logos.
-
- THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED
- WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
- OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
- DISCLAIMED.  IN NO EVENT SHALL THE JDOM AUTHORS OR THE PROJECT
- CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
- SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
- LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF
- USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
- ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY,
- OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT
- OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF
- SUCH DAMAGE.
-
- This software consists of voluntary contributions made by many 
- individuals on behalf of the JDOM Project and was originally 
- created by Jason Hunter <jhunter_AT_jdom_DOT_org> and
- Brett McLaughlin <brett_AT_jdom_DOT_org>.  For more information
- on the JDOM Project, please see <http://www.jdom.org/>. 
-
- */
-
-===============================================================================
-
-ASM bytecode manipulation library (asm)
-    Containing Project URL: http://asm.ow2.org/
-
-    Copyright (c) 2000-2005 INRIA, France Telecom
-    All rights reserved.
-
-    Redistribution and use in source and binary forms, with or without
-    modification, are permitted provided that the following conditions
-    are met:
-
-    1. Redistributions of source code must retain the above copyright
-       notice, this list of conditions and the following disclaimer.
-
-    2. Redistributions in binary form must reproduce the above copyright
-       notice, this list of conditions and the following disclaimer in the
-       documentation and/or other materials provided with the distribution.
-
-    3. Neither the name of the copyright holders nor the names of its
-       contributors may be used to endorse or promote products derived from
-       this software without specific prior written permission.
-
-    THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
-    AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
-    IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
-    ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
-    LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
-    CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
-    SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
-    INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
-    CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
-    ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF
-    THE POSSIBILITY OF SUCH DAMAGE.
-
-===============================================================================
-
-For: cryptix-asn1-versionless.jar, cryptix32-versionless.jar
-    Containing Project URL: http://www.cryptix.org/
-
-Cryptix General License
-
-Copyright (c) 1995-2005 The Cryptix Foundation Limited.
-All rights reserved.
-
-Redistribution and use in source and binary forms, with or without
-modification, are permitted provided that the following conditions are
-met:
-
-  1. Redistributions of source code must retain the copyright notice,
-     this list of conditions and the following disclaimer.
-  2. Redistributions in binary form must reproduce the above copyright
-     notice, this list of conditions and the following disclaimer in
-     the documentation and/or other materials provided with the
-     distribution.
-
-THIS SOFTWARE IS PROVIDED BY THE CRYPTIX FOUNDATION LIMITED AND
-CONTRIBUTORS ``AS IS'' AND ANY EXPRESS OR IMPLIED WARRANTIES,
-INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF
-MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
-IN NO EVENT SHALL THE CRYPTIX FOUNDATION LIMITED OR CONTRIBUTORS BE
-LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
-CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
-SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR
-BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY,
-WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE
-OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN
-IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
-===============================================================================
-The following components come under Extreme! Lab Software License
-===============================================================================
-
-XPP3
-    Containing Project URL: http://www.extreme.indiana.edu/xgws/xsoap/xpp/
-xsul, xsul5, xutil
-    Containing Project URL: http://www.extreme.indiana.edu/xgws/xsul/
-wsmg
-    Containing Project URL: http://www.extreme.indiana.edu/xgws/messenger/index.html
-gpel, weps-beans, pegasuswebservice, mapReduce-service-client, atomixmiser
-    Containing Project URL: http://www.extreme.indiana.edu/xgws/
-    
-Indiana University Extreme! Lab Software License
-
-Version 1.1.1
-
-Copyright (c) 2002 Extreme! Lab, Indiana University. All rights reserved.
-
-Redistribution and use in source and binary forms, with or without
-modification, are permitted provided that the following conditions
-are met:
-
-1. Redistributions of source code must retain the above copyright notice,
-   this list of conditions and the following disclaimer.
-
-2. Redistributions in binary form must reproduce the above copyright
-   notice, this list of conditions and the following disclaimer in
-   the documentation and/or other materials provided with the distribution.
-
-3. The end-user documentation included with the redistribution, if any,
-   must include the following acknowledgment:
-
-  "This product includes software developed by the Indiana University
-  Extreme! Lab (http://www.extreme.indiana.edu/)."
-
-Alternately, this acknowledgment may appear in the software itself,
-if and wherever such third-party acknowledgments normally appear.
-
-4. The names "Indiana Univeristy" and "Indiana Univeristy Extreme! Lab"
-must not be used to endorse or promote products derived from this
-software without prior written permission. For written permission,
-please contact http://www.extreme.indiana.edu/.
-
-5. Products derived from this software may not use "Indiana Univeristy"
-name nor may "Indiana Univeristy" appear in their name, without prior
-written permission of the Indiana University.
-
-THIS SOFTWARE IS PROVIDED "AS IS" AND ANY EXPRESSED OR IMPLIED
-WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF
-MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
-IN NO EVENT SHALL THE AUTHORS, COPYRIGHT HOLDERS OR ITS CONTRIBUTORS
-BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
-CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
-SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR
-BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY,
-WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR
-OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF
-ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
-======================================================================== 
-The following components are MIT Licensed 
-========================================================================
-
-SLF4J,log4j-over-slf4j, jcl-over-slf4j, slf4j-api,mockito-all-1.8.5,jopt-simple-3.2.jar
-    Containing Project URL: http://www.slf4j.org/
-
-Copyright (c) 2004-2008 QOS.ch
- All rights reserved.
-
- Permission is hereby granted, free  of charge, to any person obtaining
- a  copy  of this  software  and  associated  documentation files  (the
- "Software"), to  deal in  the Software without  restriction, including
- without limitation  the rights to  use, copy, modify,  merge, publish,
- distribute,  sublicense, and/or sell  copies of  the Software,  and to
- permit persons to whom the Software  is furnished to do so, subject to
- the following conditions:
-
- The  above  copyright  notice  and  this permission  notice  shall  be
- included in all copies or substantial portions of the Software.
-
- THE  SOFTWARE IS  PROVIDED  "AS  IS", WITHOUT  WARRANTY  OF ANY  KIND,
- EXPRESS OR  IMPLIED, INCLUDING  BUT NOT LIMITED  TO THE  WARRANTIES OF
- MERCHANTABILITY,    FITNESS    FOR    A   PARTICULAR    PURPOSE    AND
- NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
- LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
- OF CONTRACT, TORT OR OTHERWISE,  ARISING FROM, OUT OF OR IN CONNECTION
- WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
-
-========================================================================
-
-For dom4j-1.6.1.jar:
-    Containing Project URL: http://dom4j.sourceforge.net/
-Copyright 2001-2005 (C) MetaStuff, Ltd. All Rights Reserved.
-
-Redistribution and use of this software and associated documentation
-("Software"), with or without modification, are permitted provided
-that the following conditions are met:
-
-1. Redistributions of source code must retain copyright
-   statements and notices.  Redistributions must also contain a
-   copy of this document.
- 
-2. Redistributions in binary form must reproduce the
-   above copyright notice, this list of conditions and the
-   following disclaimer in the documentation and/or other
-   materials provided with the distribution.
- 
-3. The name "DOM4J" must not be used to endorse or promote
-   products derived from this Software without prior written
-   permission of MetaStuff, Ltd.  For written permission,
-   please contact dom4j-info@metastuff.com.
- 
-4. Products derived from this Software may not be called "DOM4J"
-   nor may "DOM4J" appear in their names without prior written
-   permission of MetaStuff, Ltd. DOM4J is a registered
-   trademark of MetaStuff, Ltd.
- 
-5. Due credit should be given to the DOM4J Project - 
-   http://www.dom4j.org
- 
-THIS SOFTWARE IS PROVIDED BY METASTUFF, LTD. AND CONTRIBUTORS
-``AS IS'' AND ANY EXPRESSED OR IMPLIED WARRANTIES, INCLUDING, BUT
-NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND
-FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.  IN NO EVENT SHALL
-METASTUFF, LTD. OR ITS CONTRIBUTORS BE LIABLE FOR ANY DIRECT,
-INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
-(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR
-SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION)
-HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT,
-STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
-ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED
-OF THE POSSIBILITY OF SUCH DAMAGE.
-
-====================================================================================================
-
-For Bouncy Castle:
-    Containing Project URL: http://www.bouncycastle.org/
-
-Copyright (c) 2000 - 2011 The Legion Of The Bouncy Castle (http://www.bouncycastle.org)
-
-Permission iss software and associated documentation files (the "Software"), to deal in
-the Software without restriction, including without limitation the rights to
-use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of
-the Software, and to permit persons to whom the Software is furnished to do so,
-subject to the following conditions hereby granted, free of charge, to any person obtaining a copy of this software
-and associated documentation files (the "Software"), to deal in the Software without restriction,
-including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense,
-and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so,
-subject to the following conditions:
-
-The above copyright notice and this permission notice shall be included in all copies or substantial
-portions of the Software.
-
-THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT
-LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT.
-IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY,
-WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE
-SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
-
-=======================================================================================================
-
-For: The International Components for Unicode (icu4j-2.6.1.jar)
-    Containing Project URL: http://site.icu-project.org/
-
-    Copyright (c) 1995-2009 International Business Machines Corporation
-    and others
-
-    All rights reserved.
-
-    Permission is hereby granted, free of charge, to any person obtaining
-    a copy of this software and associated documentation files (the
-    "Software"), to deal in the Software without restriction, including
-    without limitation the rights to use, copy, modify, merge, publish,
-    distribute, and/or sell copies of the Software, and to permit persons
-    to whom the Software is furnished to do so, provided that the above
-    copyright notice(s) and this permission notice appear in all copies
-    of the Software and that both the above copyright notice(s) and this
-    permission notice appear in supporting documentation.
-
-    THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
-    OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
-    FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT OF THIRD PARTY RIGHTS.
-    IN NO EVENT SHALL THE COPYRIGHT HOLDER OR HOLDERS INCLUDED IN THIS NOTICE
-    BE LIABLE FOR ANY CLAIM, OR ANY SPECIAL INDIRECT OR CONSEQUENTIAL DAMAGES,
-    OR ANY DAMAGES WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS,
-    WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION,
-    ARISING OUT OF OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS
-    SOFTWARE.
-
-    Except as contained in this notice, the name of a copyright holder shall
-    not be used in advertising or otherwise to promote the sale, use or other
-    dealings in this Software without prior written authorization of the
-    copyright holder.
-    
-====================================================================== 
-The following components are CDDL based License 
-======================================================================
-
-For activation-1.1.jar, jaxb-api-2.1.jar, mail-1.4.jar, junit, 
-Servlet Specification 2.5 API (servlet-api-2.5-6.1.14.jar),
-Classfish Jasper API (jsp-api-2.1-6.1.14.jar), and
-JSP2.1 Jasper implementation from Glassfish (jsp-2.1-6.1.14.jar), 
-Jersey from Glassfish (jersey-client-1.13.jar, jersey-core-1.13.jar,
-jersey-json-1.13.jar, jersey-multipart-1.13.jar) and JSP2.1 Jasper 
-implementation from Glassfish (jsp-2.1-6.1.14.jar),whirr-core-0.7.1.jar, whirr-hadoop-0.7.1.jar:
- 
-NOTE: jersey is dual licensed (http://jersey.java.net/CDDL+GPL.html), 
-Apahce Airavata elects to include jersey in this distribution under the
-[CDDLv_1.0] license.
-
-    COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0
-
-    1. Definitions.
-
-    1.1. Contributor means each individual or entity that creates or
-    contributes to the creation of Modifications.
-
-    1.2. Contributor Version means the combination of the Original Software,
-    prior Modifications used by a Contributor (if any), and the Modifications
-    made by that particular Contributor.
-
-    1.3. Covered Software means (a) the Original Software, or
-    (b) Modifications, or (c) the combination of files containing Original
-    Software with files containing Modifications, in each case including
-    portions thereof.
-
-    1.4. Executable means the Covered Software in any form other than Source
-    Code.
-
-    1.5. Initial Developer means the individual or entity that first makes
-    Original Software available under this License.
-
-    1.6. Larger Work means a work which combines Covered Software or portions
-    thereof with code not governed by the terms of this License.
-
-    1.7. License means this document.
-
-    1.8. Licensable means having the right to grant, to the maximum extent
-    possible, whether at the time of the initial grant or subsequently
-    acquired, any and all of the rights conveyed herein.
-
-    1.9. Modifications means the Source Code and Executable form of any of
-    the following: A. Any file that results from an addition to, deletion
-    from or modification of the contents of a file containing Original
-    Software or previous Modifications; B. Any new file that contains any
-    part of the Original Software or previous Modification; or C. Any new
-    file that is contributed or otherwise made available under the terms of
-    this License.
-
-    1.10. Original Software means the Source Code and Executable form of
-    computer software code that is originally released under this License.
-
-    1.11. Patent Claims means any patent claim(s), now owned or hereafter
-    acquired, including without limitation, method, process, and apparatus
-    claims, in any patent Licensable by grantor.
-
-    1.12. Source Code means (a) the common form of computer software code in
-    which modifications are made and (b) associated documentation included in
-    or with such code.
-
-    1.13. You (or Your) means an individual or a legal entity exercising
-    rights under, and complying with all of the terms of, this License. For
-    legal entities, You includes any entity which controls, is controlled by,
-    or is under common control with You. For purposes of this definition,
-    control means (a) the power, direct or indirect, to cause the direction
-    or management of such entity, whether by contract or otherwise, or
-    (b) ownership of more than fifty percent (50%) of the outstanding shares
-    or beneficial ownership of such entity.
-
-    2. License Grants.
-
-    2.1. The Initial Developer Grant. Conditioned upon Your compliance with
-    Section 3.1 below and subject to third party intellectual property
-    claims, the Initial Developer hereby grants You a world-wide,
-    royalty-free, non-exclusive license:
-
-    (a) under intellectual property rights (other than patent or trademark)
-        Licensable by Initial Developer, to use, reproduce, modify, display,
-        perform, sublicense and distribute the Original Software (or portions
-        thereof), with or without Modifications, and/or as part of a Larger
-        Work; and
-
-    (b) under Patent Claims infringed by the making, using or selling of
-        Original Software, to make, have made, use, practice, sell, and offer
-        for sale, and/or otherwise dispose of the Original Software (or
-        portions thereof);
-
-    (c) The licenses granted in Sections 2.1(a) and (b) are effective on the
-        date Initial Developer first distributes or otherwise makes the
-        Original Software available to a third party under the terms of
-        this License;
-
-    (d) Notwithstanding Section 2.1(b) above, no patent license is granted:
-        (1) for code that You delete from the Original Software, or (2) for
-        infringements caused by: (i) the modification of the Original
-        Software, or (ii) the combination of the Original Software with other
-        software or devices.
-
-    2.2. Contributor Grant. Conditioned upon Your compliance with Section 3.1
-    below and subject to third party intellectual property claims, each
-    Contributor hereby grants You a world-wide, royalty-free, non-exclusive
-    license:
-
-    (a) under intellectual property rights (other than patent or trademark)
-        Licensable by Contributor to use, reproduce, modify, display, perform,
-        sublicense and distribute the Modifications created by such
-        Contributor (or portions thereof), either on an unmodified basis,
-        with other Modifications, as Covered Software and/or as part of a
-        Larger Work; and
-
-    (b) under Patent Claims infringed by the making, using, or selling of
-        Modifications made by that Contributor either alone and/or in
-        combination with its Contributor Version (or portions of such
-        combination), to make, use, sell, offer for sale, have made, and/or
-        otherwise dispose of: (1) Modifications made by that Contributor (or
-        portions thereof); and (2) the combination of Modifications made by
-        that Contributor with its Contributor Version (or portions of such
-        combination).
-
-    (c) The licenses granted in Sections 2.2(a) and 2.2(b) are effective on
-        the date Contributor first distributes or otherwise makes the
-        Modifications available to a third party.
-
-    (d) Notwithstanding Section 2.2(b) above, no patent license is granted:
-        (1) for any code that Contributor has deleted from the Contributor
-        Version; (2) for infringements caused by: (i) third party
-        modifications of Contributor Version, or (ii) the combination of
-        Modifications made by that Contributor with other software (except
-        as part of the Contributor Version) or other devices; or (3) under
-        Patent Claims infringed by Covered Software in the absence of
-        Modifications made by that Contributor.
-
-    3. Distribution Obligations.
-
-    3.1. Availability of Source Code. Any Covered Software that You distribute
-    or otherwise make available in Executable form must also be made available
-    in Source Code form and that Source Code form must be distributed only
-    under the terms of this License. You must include a copy of this License
-    with every copy of the Source Code form of the Covered Software You
-    distribute or otherwise make available. You must inform recipients of any
-    such Covered Software in Executable form as to how they can obtain such
-    Covered Software in Source Code form in a reasonable manner on or through
-    a medium customarily used for software exchange.
-
-    3.2. Modifications. The Modifications that You create or to which You
-    contribute are governed by the terms of this License. You represent that
-    You believe Your Modifications are Your original creation(s) and/or You
-    have sufficient rights to grant the rights conveyed by this License.
-
-    3.3. Required Notices. You must include a notice in each of Your
-    Modifications that identifies You as the Contributor of the Modification.
-    You may not remove or alter any copyright, patent or trademark notices
-    contained within the Covered Software, or any notices of licensing or any
-    descriptive text giving attribution to any Contributor or the Initial
-    Developer.
-
-    3.4. Application of Additional Terms. You may not offer or impose any
-    terms on any Covered Software in Source Code form that alters or restricts
-    the applicable version of this License or the recipients rights hereunder.
-    You may choose to offer, and to charge a fee for, warranty, support,
-    indemnity or liability obligations to one or more recipients of Covered
-    Software. However, you may do so only on Your own behalf, and not on
-    behalf of the Initial Developer or any Contributor. You must make it
-    absolutely clear that any such warranty, support, indemnity or liability
-    obligation is offered by You alone, and You hereby agree to indemnify the
-    Initial Developer and every Contributor for any liability incurred by the
-    Initial Developer or such Contributor as a result of warranty, support,
-    indemnity or liability terms You offer.
-
-    3.5. Distribution of Executable Versions. You may distribute the
-    Executable form of the Covered Software under the terms of this License or
-    under the terms of a license of Your choice, which may contain terms
-    different from this License, provided that You are in compliance with the
-    terms of this License and that the license for the Executable form does
-    not attempt to limit or alter the recipients rights in the Source Code
-    form from the rights set forth in this License. If You distribute the
-    Covered Software in Executable form under a different license, You must
-    make it absolutely clear that any terms which differ from this License
-    are offered by You alone, not by the Initial Developer or Contributor.
-    You hereby agree to indemnify the Initial Developer and every Contributor
-    for any liability incurred by the Initial Developer or such Contributor as
-    a result of any such terms You offer.
-
-    3.6. Larger Works. You may create a Larger Work by combining Covered
-    Software with other code not governed by the terms of this License and
-    distribute the Larger Work as a single product. In such a case, You must
-    make sure the requirements of this License are fulfilled for the Covered
-    Software.
-
-    4. Versions of the License.
-
-    4.1. New Versions. Sun Microsystems, Inc. is the initial license steward
-    and may publish revised and/or new versions of this License from time to
-    time. Each version will be given a distinguishing version number. Except
-    as provided in Section 4.3, no one other than the license steward has the
-    right to modify this License.
-
-    4.2. Effect of New Versions. You may always continue to use, distribute
-    or otherwise make the Covered Software available under the terms of the
-    version of the License under which You originally received the Covered
-    Software. If the Initial Developer includes a notice in the Original
-    Software prohibiting it from being distributed or otherwise made
-    available under any subsequent version of the License, You must
-    distribute and make the Covered Software available under the terms of
-    the version of the License under which You originally received the
-    Covered Software. Otherwise, You may also choose to use, distribute or
-    otherwise make the Covered Software available under the terms of any
-    subsequent version of the License published by the license steward.
-
-    4.3. Modified Versions. When You are an Initial Developer and You want
-    to create a new license for Your Original Software, You may create and
-    use a modified version of this License if You: (a) rename the license and
-    remove any references to the name of the license steward (except to note
-    that the license differs from this License); and (b) otherwise make it
-    clear that the license contains terms which differ from this License.
-
-    5. DISCLAIMER OF WARRANTY. COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE
-    ON AN AS IS BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR
-    IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED
-    SOFTWARE IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE
-    OR NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF
-    THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE
-    DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER
-    CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR
-    CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF
-    THIS LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER
-    EXCEPT UNDER THIS DISCLAIMER.
-
-    6. TERMINATION.
-
-    6.1. This License and the rights granted hereunder will terminate
-    automatically if You fail to comply with terms herein and fail to cure
-    such breach within 30 days of becoming aware of the breach. Provisions
-    which, by their nature, must remain in effect beyond the termination of
-    this License shall survive.
-
-    6.2. If You assert a patent infringement claim (excluding declaratory
-    judgment actions) against Initial Developer or a Contributor (the Initial
-    Developer or Contributor against whom You assert such claim is referred
-    to as Participant) alleging that the Participant Software (meaning the
-    Contributor Version where the Participant is a Contributor or the
-    Original Software where the Participant is the Initial Developer)
-    directly or indirectly infringes any patent, then any and all rights
-    granted directly or indirectly to You by such Participant, the Initial
-    Developer (if the Initial Developer is not the Participant) and all
-    Contributors under Sections 2.1 and/or 2.2 of this License shall, upon
-    60 days notice from Participant terminate prospectively and automatically
-    at the expiration of such 60 day notice period, unless if within such
-    60 day period You withdraw Your claim with respect to the Participant
-    Software against such Participant either unilaterally or pursuant to a
-    written agreement with Participant.
-
-    6.3. In the event of termination under Sections 6.1 or 6.2 above, all end
-    user licenses that have been validly granted by You or any distributor
-    hereunder prior to termination (excluding licenses granted to You by any
-    distributor) shall survive termination.
-
-    7. LIMITATION OF LIABILITY. UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL
-    THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL
-    YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF
-    COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY
-    PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF
-    ANY CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOST PROFITS,
-    LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY
-    AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE
-    BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF
-    LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY
-    RESULTING FROM SUCH PARTYS NEGLIGENCE TO THE EXTENT APPLICABLE LAW
-    PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION
-    OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION
-    AND LIMITATION MAY NOT APPLY TO YOU.
-
-    8. U.S. GOVERNMENT END USERS. The Covered Software is a commercial item,
-    as that term is defined in 48 C.F.R. 2.101 (Oct. 1995), consisting of
-    commercial computer software (as that term is defined at 48 C.F.R.
-    252.227-7014(a)(1)) and commercial computer software documentation as such
-    terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent with 48 C.F.R.
-    12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 (June 1995), all U.S.
-    Government End Users acquire Covered Software with only those rights set
-    forth herein. This U.S. Government Rights clause is in lieu of, and
-    supersedes, any other FAR, DFAR, or other clause or provision that
-    addresses Government rights in computer software under this License.
-
-    9. MISCELLANEOUS. This License represents the complete agreement
-    concerning subject matter hereof. If any provision of this License is
-    held to be unenforceable, such provision shall be reformed only to the
-    extent necessary to make it enforceable. This License shall be governed
-    by the law of the jurisdiction specified in a notice contained within
-    the Original Software (except to the extent applicable law, if any,
-    provides otherwise), excluding such jurisdictions conflict-of-law
-    provisions. Any litigation relating to this License shall be subject to
-    the jurisdiction of the courts located in the jurisdiction and venue
-    specified in a notice contained within the Original Software, with the
-    losing party responsible for costs, including, without limitation, court
-    costs and reasonable attorneys fees and expenses. The application of the
-    United Nations Convention on Contracts for the International Sale of
-    Goods is expressly excluded. Any law or regulation which provides that
-    the language of a contract shall be construed against the drafter shall
-    not apply to this License. You agree that You alone are responsible for
-    compliance with the United States export administration regulations (and
-    the export control laws and regulation of any other countries) when You
-    use, distribute or otherwise make available any Covered Software.
-
-    10. RESPONSIBILITY FOR CLAIMS. As between Initial Developer and the
-    Contributors, each party is responsible for claims and damages arising,
-    directly or indirectly, out of its utilization of rights under this
-    License and You agree to work with Initial Developer and Contributors
-    to distribute such responsibility on an equitable basis. Nothing herein
-    is intended or shall be deemed to constitute any admission of liability.
-
-    NOTICE PURSUANT TO SECTION 9 OF THE COMMON DEVELOPMENT AND DISTRIBUTION
-    LICENSE (CDDL) The code released under the CDDL shall be governed by the
-    laws of the State of California (excluding conflict-of-law provisions).
-    Any litigation relating to this License shall be subject to the
-    jurisdiction of the Federal Courts of the Northern District of California
-    and the state courts of the State of California, with venue lying in
-    Santa Clara County, California.
-
-
-==============================================================================
-
-For: jaxb-xjc-2.1.7.jar
-    Containing Project URL: 
-
-Copyright (c) 2004 Kohsuke Kawaguchi
-
-Permission is hereby granted, free of charge, to any person
-obtaining a copy of this software and associated documentation
-files (the "Software"), to deal in the Software without
-restriction, including without limitation the rights to use,
-copy, modify, merge, publish, distribute, sublicense, and/or
-sell copies of the Software, and to permit persons to whom
-the Software is furnished to do so, subject to the following
-conditions:
-
-The above copyright notice and this permission notice shall
-be included in all copies or substantial portions of the Software.
-
-THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY
-KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE
-WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR
-PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS
-OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR
-OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR
-OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE
-SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
-
-=============================================================================== 
-The following components are BSD Licensed 
-=============================================================================== 
-
-For jibx-bind-1.2.1.jar,jibx-run-1.2.1.jar, antlr-2.7.7.jar,hamcrest-all-1.1.jar,whirr-core-0.7.1.jar, whirr-hadoop-0.7.1.jar:
-    Containing Project URL: http://jibx.sourceforge.net, http://www.antlr.org/
-
-Copyright (c) 2003-2007, Dennis M. Sosnoski
-All rights reserved.
-
-Copyright (c) 2010 Terence Parr
-All rights reserved.
-
-[The BSD License]
-
-Redistribution and use in source and binary forms, with or without modification,
-are permitted provided that the following conditions are met:
-
- * Redistributions of source code must retain the above copyright notice, this
-   list of conditions and the following disclaimer.
- * Redistributions in binary form must reproduce the above copyright notice,
-   this list of conditions and the following disclaimer in the documentation
-   and/or other materials provided with the distribution.
- * Neither the name of JiBX nor the names of its contributors may be used
-   to endorse or promote products derived from this software without specific
-   prior written permission.
-
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
-ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
-WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
-DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
-ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
-(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
-LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
-ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
-SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
-==============================================================================
-
-For YFilter:
-    Containing Project URL: http://yfilter.cs.umass.edu/
-
-YFilter 1.0 COPYRIGHT, LICENSE and DISCLAIMER
-
-Copyright (c) 2002, 2004, Regents of the University of California All rights reserved.
-
-Redistribution and use in source and binary forms, with or without modification, are
-permitted provided that the following conditions are met:
-
-    * Redistributions of source code must retain the above copyright notice, this
-    list of conditions and the following disclaimer.
-    * Redistributions in binary form must reproduce the above copyright notice, this
-    list of conditions and the following disclaimer in the documentation and/or other
-    materials provided with the distribution.
-    * Neither the name of the University of California at Berkeley nor the names of
-    its contributors may be used to endorse or promote products derived from this
-    software without specific prior written permission.
-
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY
-EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
-OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT
-SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
-SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT
-OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION)
-HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY,
-OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
-SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
-==========================================================================================
-For jaxen-1.1.1.jar:
-    Containing Project URL: http://jaxen.codehaus.org/
-
- Copyright 2003-2006 The Werken Company. All Rights Reserved.
-
- Redistribution and use in source and binary forms, with or without
- modification, are permitted provided that the following conditions are
- met:
-
-  * Redistributions of source code must retain the above copyright
-    notice, this list of conditions and the following disclaimer.
-
-  * Redistributions in binary form must reproduce the above copyright
-    notice, this list of conditions and the following disclaimer in the
-    documentation and/or other materials provided with the distribution.
-
-  * Neither the name of the Jaxen Project nor the names of its
-    contributors may be used to endorse or promote products derived
-    from this software without specific prior written permission.
-
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS
-IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED
-TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A
-PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER
-OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
-EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
-PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
-PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
-LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
-NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
-SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
-=============================================================================== 
-The following components are CPL Licensed 
-=============================================================================== 
-
-For wsdl4j-1.6.2.jar:
-    Containing Project URL: http://sourceforge.net/projects/wsdl4j/
-
-Common Public License Version 1.0
-
-THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS COMMON PUBLIC\u2028LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION OF THE PROGRAM\u2028CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT.
-1. DEFINITIONS
-"Contribution" means:
-a) in the case of the initial Contributor, the initial code and\u2028documentation distributed under this Agreement, and
-b) in the case of each subsequent Contributor:
-i) changes to the Program, and
-ii) additions to the Program;
-where such changes and/or additions to the Program originate from and are\u2028distributed by that particular Contributor. A Contribution 'originates' from a\u2028Contributor if it was added to the Program by such Contributor itself or anyone\u2028acting on such Contributor's behalf. Contributions do not include additions to\u2028the Program which: (i) are separate modules of software distributed in\u2028conjunction with the Program under their own license agreement, and (ii) are not\u2028derivative works of the Program.
-"Contributor" means any person or entity that distributes the Program.
-"Licensed Patents " mean patent claims licensable by a Contributor which are\u2028necessarily infringed by the use or sale of its Contribution alone or when\u2028combined with the Program.
-"Program" means the Contributions distributed in accordance with this Agreement.
-"Recipient" means anyone who receives the Program under this Agreement,\u2028including all Contributors.
-2. GRANT OF RIGHTS
-a) Subject to the terms of this Agreement, each Contributor hereby grants\u2028Recipient a non-exclusive, worldwide, royalty-free copyright license to\u2028reproduce, prepare derivative works of, publicly display, publicly perform,\u2028distribute and sublicense the Contribution of such Contributor, if any, and such\u2028derivative works, in source code and object code form.
-b) Subject to the terms of this Agreement, each Contributor hereby grants\u2028Recipient a non-exclusive, worldwide, royalty-free patent license under Licensed\u2028Patents to make, use, sell, offer to sell, import and otherwise transfer the\u2028Contribution of such Contributor, if any, in source code and object code form.\u2028This patent license shall apply to the combination of the Contribution and the\u2028Program if, at the time the Contribution is added by the Contributor, such\u2028addition of the Contribution causes such combination to be covered by the\u2028Licensed Patents. The patent license shall not apply to any other combinations\u2028which include the Contribution. No hardware per se is licensed hereunder.
-c) Recipient understands that although each Contributor grants the licenses\u2028to its Contributions set forth herein, no assurances are provided by any\u2028Contributor that the Program does not infringe the patent or other intellectual\u2028property rights of any other entity. Each Contributor disclaims any liability to\u2028Recipient for claims brought by any other entity based on infringement of\u2028intellectual property rights or otherwise. As a condition to exercising the\u2028rights and licenses granted hereunder, each Recipient hereby assumes sole\u2028responsibility to secure any other intellectual property rights needed, if any.\u2028For example, if a third party patent license is required to allow Recipient to\u2028distribute the Program, it is Recipient's responsibility to acquire that license\u2028before distributing the Program.
-d) Each Contributor represents that to its knowledge it has sufficient\u2028copyright rights in its Contribution, if any, to grant the copyright license set\u2028forth in this Agreement.
-3. REQUIREMENTS
-A Contributor may choose to distribute the Program in object code form under its\u2028own license agreement, provided that:
-a) it complies with the terms and conditions of this Agreement; and
-b) its license agreement:
-i) effectively disclaims on behalf of all Contributors all warranties and\u2028conditions, express and implied, including warranties or conditions of title and\u2028non-infringement, and implied warranties or conditions of merchantability and\u2028fitness for a particular purpose;
-ii) effectively excludes on behalf of all Contributors all liability for\u2028damages, including direct, indirect, special, incidental and consequential\u2028damages, such as lost profits;
-iii) states that any provisions which differ from this Agreement are offered\u2028by that Contributor alone and not by any other party; and
-iv) states that source code for the Program is available from such\u2028Contributor, and informs licensees how to obtain it in a reasonable manner on or\u2028through a medium customarily used for software exchange.
-When the Program is made available in source code form:
-a) it must be made available under this Agreement; and
-b) a copy of this Agreement must be included with each copy of the Program.
-Contributors may not remove or alter any copyright notices contained within the\u2028Program.
-Each Contributor must identify itself as the originator of its Contribution, if\u2028any, in a manner that reasonably allows subsequent Recipients to identify the\u2028originator of the Contribution.
-4. COMMERCIAL DISTRIBUTION
-Commercial distributors of software may accept certain responsibilities with\u2028respect to end users, business partners and the like. While this license is\u2028intended to facilitate the commercial use of the Program, the Contributor who\u2028includes the Program in a commercial product offering should do so in a manner\u2028which does not create potential liability for other Contributors. Therefore, if\u2028a Contributor includes the Program in a commercial product offering, such\u2028Contributor ("Commercial Contributor") hereby agrees to defend and indemnify\u2028every other Contributor ("Indemnified Contributor") against any losses, damages\u2028and costs (collectively "Losses") arising from claims, lawsuits and other legal\u2028actions brought by a third party against the Indemnified Contributor to the\u2028extent caused by the acts or omissions of such Commercial Contributor in\u2028connection with its distribution of the Program in a commercial product\u2028offering. The obligations in this section do not ap
 ply to any claims or Losses\u2028relating to any actual or alleged intellectual property infringement. In order\u2028to qualify, an Indemnified Contributor must: a) promptly notify the Commercial\u2028Contributor in writing of such claim, and b) allow the Commercial Contributor to\u2028control, and cooperate with the Commercial Contributor in, the defense and any\u2028related settlement negotiations. The Indemnified Contributor may participate in\u2028any such claim at its own expense.
-For example, a Contributor might include the Program in a commercial product\u2028offering, Product X. That Contributor is then a Commercial Contributor. If that\u2028Commercial Contributor then makes performance claims, or offers warranties\u2028related to Product X, those performance claims and warranties are such\u2028Commercial Contributor's responsibility alone. Under this section, the\u2028Commercial Contributor would have to defend claims against the other\u2028Contributors related to those performance claims and warranties, and if a court\u2028requires any other Contributor to pay any damages as a result, the Commercial\u2028Contributor must pay those damages.
-5. NO WARRANTY
-EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED ON AN\u2028"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR\u2028IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF TITLE,\u2028NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR PURPOSE. Each\u2028Recipient is solely responsible for determining the appropriateness of using and\u2028distributing the Program and assumes all risks associated with its exercise of\u2028rights under this Agreement, including but not limited to the risks and costs of\u2028program errors, compliance with applicable laws, damage to or loss of data,\u2028programs or equipment, and unavailability or interruption of operations.
-6. DISCLAIMER OF LIABILITY
-EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR ANY\u2028CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL,\u2028SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST\u2028PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT,\u2028STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY\u2028OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS\u2028GRANTED HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
-7. GENERAL
-If any provision of this Agreement is invalid or unenforceable under applicable\u2028law, it shall not affect the validity or enforceability of the remainder of the\u2028terms of this Agreement, and without further action by the parties hereto, such\u2028provision shall be reformed to the minimum extent necessary to make such\u2028provision valid and enforceable.
-If Recipient institutes patent litigation against a Contributor with respect to\u2028a patent applicable to software (including a cross-claim or counterclaim in a\u2028lawsuit), then any patent licenses granted by that Contributor to such Recipient\u2028under this Agreement shall terminate as of the date such litigation is filed. In\u2028addition, if Recipient institutes patent litigation against any entity\u2028(including a cross-claim or counterclaim in a lawsuit) alleging that the Program\u2028itself (excluding combinations of the Program with other software or hardware)\u2028infringes such Recipient's patent(s), then such Recipient's rights granted under\u2028Section 2(b) shall terminate as of the date such litigation is filed.
-All Recipient's rights under this Agreement shall terminate if it fails to\u2028comply with any of the material terms or conditions of this Agreement and does\u2028not cure such failure in a reasonable period of time after becoming aware of\u2028such noncompliance. If all Recipient's rights under this Agreement terminate,\u2028Recipient agrees to cease use and distribution of the Program as soon as\u2028reasonably practicable. However, Recipient's obligations under this Agreement\u2028and any licenses granted by Recipient relating to the Program shall continue and\u2028survive.
-Everyone is permitted to copy and distribute copies of this Agreement, but in\u2028order to avoid inconsistency the Agreement is copyrighted and may only be\u2028modified in the following manner. The Agreement Steward reserves the right to\u2028publish new versions (including revisions) of this Agreement from time to time.\u2028No one other than the Agreement Steward has the right to modify this Agreement.\u2028IBM is the initial Agreement Steward. IBM may assign the responsibility to serve\u2028as the Agreement Steward to a suitable separate entity. Each new version of the\u2028Agreement will be given a distinguishing version number. The Program (including\u2028Contributions) may always be distributed subject to the version of the Agreement\u2028under which it was received. In addition, after a new version of the Agreement\u2028is published, Contributor may elect to distribute the Program (including its\u2028Contributions) under the new version. Except as expressly stated in Sections\u20282(a) and 2(b) above, Recipie
 nt receives no rights or licenses to the\u2028intellectual property of any Contributor under this Agreement, whether\u2028expressly, by implication, estoppel or otherwise. All rights in the Program not\u2028expressly granted under this Agreement are reserved.
-This Agreement is governed by the laws of the State of New York and the\u2028intellectual property laws of the United States of America. No party to this\u2028Agreement will bring a legal action under this Agreement more than one year\u2028after the cause of action arose. Each party waives its rights to a jury trial in\u2028any resulting litigation.
-
-==========================================================================================
-==========================================================================================
-
-For puretls:
-    Containing Project URL: 
-
-  This package is a SSLv3/TLS implementation written by Eric Rescorla
-   <ek...@rtfm.com> and licensed by Claymore Systems, Inc.
-
-   Redistribution and use in source and binary forms, with or without
-   modification, are permitted provided that the following conditions
-   are met:
-   1. Redistributions of source code must retain the above copyright
-      notice, this list of conditions and the following disclaimer.
-   2. Redistributions in binary form must reproduce the above copyright
-      notice, this list of conditions and the following disclaimer in the
-      documentation and/or other materials provided with the distribution.
-   3. Neither the name of Claymore Systems, Inc. nor the name of Eric
-      Rescorla may be used to endorse or promote products derived from this
-      software without specific prior written permission.
-   THIS SOFTWARE IS PROVIDED BY CLAYMORE SYSTEMS AND CONTRIBUTORS ``AS IS'' AND
-   ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
-   IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
-   ARE DISCLAIMED.  IN NO EVENT SHALL THE REGENTS OR CONTRIBUTORS BE LIABLE
-   FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
-   DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS
-   OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION)
-   HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT
-   LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY
-   OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF
-   SUCH DAMAGE.
-
-==============================================================================
-
-For xml-api,woden-api-1.0M8.jar,woden-impl-dom-1.0M8.jar:
-    Containing Project URL: 
-
-For the W3C schema and DTD files in the org.apache.woden.resolver package:
-
-W3C� DOCUMENT LICENSE
-http://www.w3.org/Consortium/Legal/2002/copyright-documents-20021231
-
-Public documents on the W3C site are provided by the copyright holders under
-the following license. By using and/or copying this document, or the W3C
-document from which this statement is linked, you (the licensee) agree that
-you have read, understood, and will comply with the following terms and
-conditions:
-
-Permission to copy, and distribute the contents of this document, or the W3C
-document from which this statement is linked, in any medium for any purpose
-and without fee or royalty is hereby granted, provided that you include the
-following on ALL copies of the document, or portions thereof, that you use:
-
-  1. A link or URL to the original W3C document.
-  2. The pre-existing copyright notice of the original author, or if it
-     doesn't exist, a notice (hypertext is preferred, but a textual
-     representation is permitted) of the form: "Copyright � [$date-of-document]
-     World Wide Web Consortium, (Massachusetts Institute of Technology,
-     European Research Consortium for Informatics and Mathematics, Keio
-     University). All Rights Reserved.
-     http://www.w3.org/Consortium/Legal/2002/copyright-documents-20021231"
-  3. If it exists, the STATUS of the W3C document.
-
-When space permits, inclusion of the full text of this NOTICE should be
-provided. We request that authorship attribution be provided in any software,
-documents, or other items or products that you create pursuant to the
-implementation of the contents of this document, or any portion thereof.
-
-No right to create modifications or derivatives of W3C documents is granted
-pursuant to this license. However, if additional requirements (documented in
-the Copyright FAQ) are satisfied, the right to create modifications or
-derivatives is sometimes granted by the W3C to individuals complying with
-those requirements.
-
-THIS DOCUMENT IS PROVIDED "AS IS," AND COPYRIGHT HOLDERS MAKE NO
-REPRESENTATIONS OR WARRANTIES, EXPRESS OR IMPLIED, INCLUDING, BUT NOT
-LIMITED TO, WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE,
-NON-INFRINGEMENT, OR TITLE; THAT THE CONTENTS OF THE DOCUMENT ARE SUITABLE
-FOR ANY PURPOSE; NOR THAT THE IMPLEMENTATION OF SUCH CONTENTS WILL NOT
-INFRINGE ANY THIRD PARTY PATENTS, COPYRIGHTS, TRADEMARKS OR OTHER RIGHTS.
-
-COPYRIGHT HOLDERS WILL NOT BE LIABLE FOR ANY DIRECT, INDIRECT, SPECIAL OR
-CONSEQUENTIAL DAMAGES ARISING OUT OF ANY USE OF THE DOCUMENT OR THE
-PERFORMANCE OR IMPLEMENTATION OF THE CONTENTS THEREOF.
-
-The name and trademarks of copyright holders may NOT be used in advertising
-or publicity pertaining to this document or its contents without specific,
-written prior permission. Title to copyright in this document will at all
-times remain with copyright holders.
-
-This formulation of W3C's notice and license became active on December 31 2002. 
-This version removes the copyright ownership notice such that this license can 
-be used with materials other than those owned by the W3C, reflects that ERCIM is 
-now a host of the W3C, includes references to this specific dated version of the 
-license, and removes the ambiguous grant of "use". Otherwise, this version is the 
-same as the previous version and is written so as to preserve the Free Software 
-Foundation's assessment of GPL compatibility and OSI's certification under the 
-Open Source Definition. Please see our Copyright FAQ for common questions about 
-using materials from our site, including specific terms and conditions for packages 
-like libwww, Amaya, and Jigsaw. Other questions about this notice can be directed 
-o site-policy@w3.org.
-
-Joseph Reagle <si...@w3.org>
- 
-Last revised $Id: copyright-software-20021231.html,v 1.11 2004/07/06 16:02:49 slesch Exp $ 
-
-==========================================================================================
-
-XML API library, org.w3c classes (xml-apis)
-    Containing Project URL: 
-
-    DOM Java Language Binding:
-    http://www.w3.org/TR/2000/REC-DOM-Level-2-Core-20001113/java-binding.html
-
-    W3C IPR SOFTWARE NOTICE
-    Copyright (C) 2000 World Wide Web Consortium, (Massachusetts Institute of
-    Technology, Institut National de Recherche en Informatique et en
-    Automatique, Keio University). All Rights Reserved.
-
-    The DOM bindings are published under the W3C Software Copyright Notice
-    and License. The software license requires "Notice of any changes or
-    modifications to the W3C files, including the date changes were made."
-    Consequently, modified versions of the DOM bindings must document that
-    they do not conform to the W3C standard; in the case of the IDL binding,
-    the pragma prefix can no longer be 'w3c.org'; in the case of the Java
-    binding, the package names can no longer be in the 'org.w3c' package.
-
-    Note: The original version of the W3C Software Copyright Notice and
-    License could be found at
-    http://www.w3.org/Consortium/Legal/copyright-software-19980720
-
-    Copyright (C) 1994-2000 World Wide Web Consortium, (Massachusetts
-    Institute of Technology, Institut National de Recherche en Informatique
-    et en Automatique, Keio University). All Rights Reserved.
-    http://www.w3.org/Consortium/Legal/
-
-    This W3C work (including software, documents, or other related items) is
-    being provided by the copyright holders under the following license. By
-    obtaining, using and/or copying this work, you (the licensee) agree that
-    you have read, understood, and will comply with the following terms and
-    conditions:
-
-    Permission to use, copy, and modify this software and its documentation,
-    with or without modification, for any purpose and without fee or royalty
-    is hereby granted, provided that you include the following on ALL copies
-    of the software and documentation or portions thereof, including
-    modifications, that you make:
-
-      1. The full text of this NOTICE in a location viewable to users of the
-         redistributed or derivative work.
-
-      2. Any pre-existing intellectual property disclaimers, notices, or
-         terms and conditions. If none exist, a short notice of the following
-         form (hypertext is preferred, text is permitted) should be used
-         within the body of any redistributed or derivative code:
-         "Copyright (C) [$date-of-software] World Wide Web Consortium,
-         (Massachusetts Institute of Technology, Institut National de
-         Recherche en Informatique et en Automatique, Keio University).
-         All Rights Reserved. http://www.w3.org/Consortium/Legal/"
-
-      3. Notice of any changes or modifications to the W3C files, including
-         the date changes were made. (We recommend you provide URIs to the
-         location from which the code is derived.)
-
-    THIS SOFTWARE AND DOCUMENTATION IS PROVIDED "AS IS," AND COPYRIGHT HOLDERS
-    MAKE NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR IMPLIED, INCLUDING BUT
-    NOT LIMITED TO, WARRANTIES OF MERCHANTABILITY OR FITNESS FOR ANY PARTICULAR
-    PURPOSE OR THAT THE USE OF THE SOFTWARE OR DOCUMENTATION WILL NOT INFRINGE
-    ANY THIRD PARTY PATENTS, COPYRIGHTS, TRADEMARKS OR OTHER RIGHTS.
-
-    COPYRIGHT HOLDERS WILL NOT BE LIABLE FOR ANY DIRECT, INDIRECT, SPECIAL
-    OR CONSEQUENTIAL DAMAGES ARISING OUT OF ANY USE OF THE SOFTWARE OR
-    DOCUMENTATION.
-
-    The name and trademarks of copyright holders may NOT be used in
-    advertising or publicity pertaining to the software without specific,
-    written prior permission. Title to copyright in this software and any
-    associated documentation will at all times remain with copyright holders.
-
-=============================================================================== 
-The following components come under the Eclipse Public 1.0 License 
-=============================================================================== 
-Eclipse JDT Core (core-3.1.1.jar)
-
--AspectJ runtime (http://www.aspectj.org) org.aspectj:aspectjrt:jar:1.6.12
-    License: Eclipse Public License - v 1.0  (http://www.eclipse.org/legal/epl-v10.html)
-
-  Eclipse Public License - v 1.0
-
-    THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE
-    PUBLIC LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION OF
-    THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT.
-
-    1. DEFINITIONS
-
-    "Contribution" means:
-
-    a) in the case of the initial Contributor, the initial code and
-       documentation distributed under this Agreement, and
-
-    b) in the case of each subsequent Contributor:
-
-       i) changes to the Program, and
-
-       ii) additions to the Program;
-
-       where such changes and/or additions to the Program originate from and
-       are distributed by that particular Contributor. A Contribution
-       'originates' from a Contributor if it was added to the Program by
-       such Contributor itself or anyone acting on such Contributor's behalf.
-       Contributions do not include additions to the Program which: (i) are
-       separate modules of software distributed in conjunction with the
-       Program under their own license agreement, and (ii) are not derivative
-       works of the Program.
-
-    "Contributor" means any person or entity that distributes the Program.
-
-    "Licensed Patents " mean patent claims licensable by a Contributor which
-    are necessarily infringed by the use or sale of its Contribution alone or
-    when combined with the Program.
-
-    "Program" means the Contributions distributed in accordance with this
-    Agreement.
-
-    "Recipient" means anyone who receives the Program under this Agreement,
-    including all Contributors.
-
-    2. GRANT OF RIGHTS
-
-    a) Subject to the terms of this Agreement, each Contributor hereby grants
-       Recipient a non-exclusive, worldwide, royalty-free copyright license to
-       reproduce, prepare derivative works of, publicly display, publicly
-       perform, distribute and sublicense the Contribution of such
-       Contributor, if any, and such derivative works, in source code and
-       object code form.
-
-    b) Subject to the terms of this Agreement, each Contributor hereby grants
-       Recipient a non-exclusive, worldwide, royalty-free patent license under
-       Licensed Patents to make, use, sell, offer to sell, import and
-       otherwise transfer the Contribution of such Contributor, if any, in
-       source code and object code form. This patent license shall apply to
-       the combination of the Contribution and the Program if, at the time
-       the Contribution is added by the Contributor, such addition of the
-       Contribution causes such combination to be covered by the Licensed
-       Patents. The patent license shall not apply to any other combinations
-       which include the Contribution. No hardware per se is licensed hereunder.
-
-    c) Recipient understands that although each Contributor grants the
-       licenses to its Contributions set forth herein, no assurances are
-       provided by any Contributor that the Program does not infringe the
-       patent or other intellectual property rights of any other entity. Each
-       Contributor disclaims any liability to Recipient for claims brought by
-       any other entity based on infringement of intellectual property rights
-       or otherwise. As a condition to exercising the rights and licenses
-       granted hereunder, each Recipient hereby assumes sole responsibility
-       to secure any other intellectual property rights needed, if any. For
-       example, if a third party patent license is required to allow Recipient
-       to distribute the Program, it is Recipient's responsibility to acquire
-       that license before distributing the Program.
-
-    d) Each Contributor represents that to its knowledge it has sufficient
-       copyright rights in its Contribution, if any, to grant the copyright
-       license set forth in this Agreement.
-
-    3. REQUIREMENTS
-
-    A Contributor may choose to distribute the Program in object code form
-    under its own license agreement, provided that:
-
-    a) it complies with the terms and conditions of this Agreement; and
-
-    b) its license agreement:
-
-       i)   effectively disclaims on behalf of all Contributors all warranties
-            and conditions, express and implied, including warranties or
-            conditions of title and non-infringement, and implied warranties
-            or conditions of merchantability and fitness for a particular
-            purpose;
-
-       ii)  effectively excludes on behalf of all Contributors all liability
-            for damages, including direct, indirect, special, incidental and
-            consequential damages, such as lost profits;
-
-       iii) states that any provisions which differ from this Agreement are
-            offered by that Contributor alone and not by any other party; and
-
-       iv)  states that source code for the Program is available from such
-            Contributor, and informs licensees how to obtain it in a
-            reasonable manner on or through a medium customarily used for
-            software exchange.
-
-    When the Program is made available in source code form:
-
-    a) it must be made available under this Agreement; and
-
-    b) a copy of this Agreement must be included with each copy of the
-       Program.
-
-    Contributors may not remove or alter any copyright notices contained
-    within the Program.
-
-    Each Contributor must identify itself as the originator of its
-    Contribution, if any, in a manner that reasonably allows subsequent
-    Recipients to identify the originator of the Contribution.
-
-    4. COMMERCIAL DISTRIBUTION
-
-    Commercial distributors of software may accept certain responsibilities
-    with respect to end users, business partners and the like. While this
-    license is intended to facilitate the commercial use of the Program,
-    the Contributor who includes the Program in a commercial product offering
-    should do so in a manner which does not create potential liability for
-    other Contributors. Therefore, if a Contributor includes the Program in
-    a commercial product offering, such Contributor ("Commercial Contributor")
-    hereby agrees to defend and indemnify every other Contributor
-    ("Indemnified Contributor") against any losses, damages and costs
-    (collectively "Losses") arising from claims, lawsuits and other legal
-    actions brought by a third party against the Indemnified Contributor to
-    the extent caused by the acts or omissions of such Commercial Contributor
-    in connection with its distribution of the Program in a commercial
-    product offering. The obligations in this section do not apply to any
-    claims or Losses relating to any actual or alleged intellectual property
-    infringement. In order to qualify, an Indemnified Contributor must:
-    a) promptly notify the Commercial Contributor in writing of such claim,
-    and b) allow the Commercial Contributor to control, and cooperate with
-    the Commercial Contributor in, the defense and any related settlement
-    negotiations. The Indemnified Contributor may participate in any such
-    claim at its own expense.
-
-    For example, a Contributor might include the Program in a commercial
-    product offering, Product X. That Contributor is then a Commercial
-    Contributor. If that Commercial Contributor then makes performance claims,
-    or offers warranties related to Product X, those performance claims and
-    warranties are such Commercial Contributor's responsibility alone. Under
-    this section, the Commercial Contributor would have to defend claims
-    against the other Contributors related to those performance claims and
-    warranties, and if a court requires any other Contributor to pay any
-    damages as a result, the Commercial Contributor must pay those damages.
-
-    5. NO WARRANTY
-
-    EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED
-    ON AN "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER
-    EXPRESS OR IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR
-    CONDITIONS OF TITLE, NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A
-    PARTICULAR PURPOSE. Each Recipient is solely responsible for determining
-    the appropriateness of using and distributing the Program and assumes all
-    risks associated with its exercise of rights under this Agreement ,
-    including but not limited to the risks and costs of program errors,
-    compliance with applicable laws, damage to or loss of data, programs or
-    equipment, and unavailability or interruption of operations.
-
-    6. DISCLAIMER OF LIABILITY
-
-    EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR
-    ANY CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT,
-    INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING
-    WITHOUT LIMITATION LOST PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF
-    LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
-    NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OR
-    DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS GRANTED
-    HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
-
-    7. GENERAL
-
-    If any provision of this Agreement is invalid or unenforceable under
-    applicable law, it shall not affect the validity or enforceability of
-    the remainder of the terms of this Agreement, and without further action
-    by the parties hereto, such provision shall be reformed to the minimum
-    extent necessary to make such provision valid and enforceable.
-
-    If Recipient institutes patent litigation against any entity (including
-    a cross-claim or counterclaim in a lawsuit) alleging that the Program
-    itself (excluding combinations of the Program with other software or
-    hardware) infringes such Recipient's patent(s), then such Recipient's
-    rights granted under Section 2(b) shall terminate as of the date such
-    litigation is filed.
-
-    All Recipient's rights under this Agreement shall terminate if it fails
-    to comply with any of the material terms or conditions of this Agreement
-    and does not cure such failure in a reasonable period of time after
-    becoming aware of such noncompliance. If all Recipient's rights under
-    this Agreement terminate, Recipient agrees to cease use and distribution
-    of the Program as soon as reasonably practicable. However, Recipient's
-    obligations under this Agreement and any licenses granted by Recipient
-    relating to the Program shall continue and survive.
-
-    Everyone is permitted to copy and distribute copies of this Agre

<TRUNCATED>

[36/50] [abbrv] airavata git commit: Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/airavata into develop

Posted by la...@apache.org.
Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/airavata into develop


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: 442771830e316a096204b02142a34793d8ee02c0
Parents: 0b29a5c d0c6957
Author: Shameera Rathnayaka <sh...@gmail.com>
Authored: Thu Aug 18 16:05:58 2016 -0400
Committer: Shameera Rathnayaka <sh...@gmail.com>
Committed: Thu Aug 18 16:05:58 2016 -0400

----------------------------------------------------------------------
 .../lib/airavata/workspace_model_types.cpp      |  30 ++++-
 .../lib/airavata/workspace_model_types.h        |  15 ++-
 .../lib/Airavata/Model/Workspace/Types.php      |  25 ++++
 .../apache/airavata/model/workspace/ttypes.py   |  18 ++-
 .../airavata/model/workspace/Gateway.java       | 114 ++++++++++++++++++-
 .../model/workspace/GatewayApprovalStatus.java  |   5 +-
 .../catalog/impl/GatewayRegistry.java           |   4 +-
 .../core/experiment/catalog/model/Gateway.java  |  10 ++
 .../catalog/resources/GatewayResource.java      |  11 ++
 .../experiment/catalog/resources/Utils.java     |   2 +-
 .../utils/ThriftDataModelConversion.java        |   1 +
 .../src/main/resources/expcatalog-derby.sql     |  14 +++
 .../src/main/resources/expcatalog-mysql.sql     |  14 +++
 .../workspace_model.thrift                      |   6 +-
 14 files changed, 252 insertions(+), 17 deletions(-)
----------------------------------------------------------------------



[04/50] [abbrv] airavata git commit: Removed PublisherFactory class and activity publisher properties from airavata server properties

Posted by la...@apache.org.
Removed PublisherFactory class and activity publisher properties from airavata server properties


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: 98cbb40733422c365309b83fff16df0df08b6548
Parents: e4cc54d
Author: Shameera Rathnayaka <sh...@gmail.com>
Authored: Wed Aug 10 15:41:39 2016 -0400
Committer: Shameera Rathnayaka <sh...@gmail.com>
Committed: Wed Aug 10 15:42:05 2016 -0400

----------------------------------------------------------------------
 .../server/handler/AiravataServerHandler.java   |   5 +-
 .../airavata/common/utils/ServerSettings.java   |  10 --
 .../main/resources/airavata-server.properties   |   1 -
 .../messaging/core/PublisherFactory.java        | 138 +++++++++----------
 .../core/impl/GFACPassiveJobSubmitter.java      |   9 +-
 .../server/OrchestratorServerHandler.java       |   3 +-
 6 files changed, 79 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/98cbb407/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java b/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
index 5ed43df..bf6b924 100644
--- a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
+++ b/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
@@ -41,8 +41,9 @@ import org.apache.airavata.grouper.group.Group;
 import org.apache.airavata.grouper.permission.PermissionAction;
 import org.apache.airavata.grouper.resource.Resource;
 import org.apache.airavata.messaging.core.MessageContext;
+import org.apache.airavata.messaging.core.MessagingFactory;
 import org.apache.airavata.messaging.core.Publisher;
-import org.apache.airavata.messaging.core.PublisherFactory;
+import org.apache.airavata.messaging.core.Type;
 import org.apache.airavata.model.WorkflowModel;
 import org.apache.airavata.model.appcatalog.appdeployment.ApplicationDeploymentDescription;
 import org.apache.airavata.model.appcatalog.appdeployment.ApplicationModule;
@@ -99,7 +100,7 @@ public class AiravataServerHandler implements Airavata.Iface {
 
     public AiravataServerHandler() {
         try {
-            publisher = PublisherFactory.createActivityPublisher();
+            publisher = MessagingFactory.getPublisher(Type.STATUS);
         } catch (ApplicationSettingsException e) {
             logger.error("Error occured while reading airavata-server properties..", e);
         } catch (AiravataException e) {

http://git-wip-us.apache.org/repos/asf/airavata/blob/98cbb407/modules/commons/src/main/java/org/apache/airavata/common/utils/ServerSettings.java
----------------------------------------------------------------------
diff --git a/modules/commons/src/main/java/org/apache/airavata/common/utils/ServerSettings.java b/modules/commons/src/main/java/org/apache/airavata/common/utils/ServerSettings.java
index 2459658..8d52a3d 100644
--- a/modules/commons/src/main/java/org/apache/airavata/common/utils/ServerSettings.java
+++ b/modules/commons/src/main/java/org/apache/airavata/common/utils/ServerSettings.java
@@ -79,8 +79,6 @@ public class ServerSettings extends ApplicationSettings {
     private static final String MY_PROXY_USER = "myproxy.user";
     private static final String MY_PROXY_PASSWORD = "myproxy.password";
     private static final String MY_PROXY_LIFETIME = "myproxy.life";
-    private static final String STATUS_PUBLISHER = "status.publisher";
-    private static final String TASK_LAUNCH_PUBLISHER = "task.launch.publisher";
     public static final String JOB_NOTIFICATION_ENABLE = "job.notification.enable";
     public static final String JOB_NOTIFICATION_EMAILIDS = "job.notification.emailids";
     public static final String JOB_NOTIFICATION_FLAGS = "job.notification.flags";
@@ -234,14 +232,6 @@ public class ServerSettings extends ApplicationSettings {
         return Integer.parseInt(getSetting(MY_PROXY_LIFETIME));
     }
 
-    public static String getStatusPublisher() throws ApplicationSettingsException {
-        return getSetting(STATUS_PUBLISHER);
-    }
-
-    public static String getTaskLaunchPublisher() throws ApplicationSettingsException {
-        return getSetting(TASK_LAUNCH_PUBLISHER);
-    }
-
     public static boolean isEmbeddedZK() {
         return Boolean.parseBoolean(getSetting(EMBEDDED_ZK, "true"));
     }

http://git-wip-us.apache.org/repos/asf/airavata/blob/98cbb407/modules/configuration/server/src/main/resources/airavata-server.properties
----------------------------------------------------------------------
diff --git a/modules/configuration/server/src/main/resources/airavata-server.properties b/modules/configuration/server/src/main/resources/airavata-server.properties
index 29b256f..0dad713 100644
--- a/modules/configuration/server/src/main/resources/airavata-server.properties
+++ b/modules/configuration/server/src/main/resources/airavata-server.properties
@@ -253,7 +253,6 @@ durable.queue=false
 prefetch.count=200
 process.launch.queue.name=process.launch.queue
 experiment.launch.queue.name=experiment.launch.queue
-activity.publisher=org.apache.airavata.messaging.core.impl.RabbitMQStatusPublisher
 
 ###########################################################################
 # Zookeeper Server Configuration

http://git-wip-us.apache.org/repos/asf/airavata/blob/98cbb407/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/PublisherFactory.java
----------------------------------------------------------------------
diff --git a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/PublisherFactory.java b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/PublisherFactory.java
index 2e560a3..9cab483 100644
--- a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/PublisherFactory.java
+++ b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/PublisherFactory.java
@@ -1,69 +1,69 @@
-/*
- *
- * 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.airavata.messaging.core;
-
-import org.apache.airavata.common.exception.AiravataException;
-import org.apache.airavata.common.utils.ServerSettings;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class PublisherFactory {
-    private static Logger log = LoggerFactory.getLogger(PublisherFactory.class);
-
-    public static Publisher createActivityPublisher() throws AiravataException {
-        String activityPublisher = ServerSettings.getStatusPublisher();
-
-        if (activityPublisher == null) {
-            String s = "Activity publisher is not specified";
-            log.error(s);
-            throw new AiravataException(s);
-        }
-
-        try {
-            Class<? extends Publisher> aPublisher = Class.forName(activityPublisher).asSubclass(Publisher.class);
-            return aPublisher.newInstance();
-        } catch (Exception e) {
-            String msg = "Failed to load the publisher from the publisher class property: " + activityPublisher;
-            log.error(msg, e);
-            throw new AiravataException(msg, e);
-        }
-    }
-
-    public static Publisher createTaskLaunchPublisher() throws AiravataException {
-        String taskLaunchPublisher = ServerSettings.getTaskLaunchPublisher();
-
-        if (taskLaunchPublisher == null) {
-            String s = "Task launch publisher is not specified";
-            log.error(s);
-            throw new AiravataException(s);
-        }
-
-        try {
-            Class<? extends Publisher> aPublisher = Class.forName(taskLaunchPublisher).asSubclass(Publisher.class);
-            return aPublisher.newInstance();
-        } catch (Exception e) {
-            String msg = "Failed to load the publisher from the publisher class property: " + taskLaunchPublisher;
-            log.error(msg, e);
-            throw new AiravataException(msg, e);
-        }
-    }
-}
+///*
+// *
+// * 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.airavata.messaging.core;
+//
+//import org.apache.airavata.common.exception.AiravataException;
+//import org.apache.airavata.common.utils.ServerSettings;
+//import org.slf4j.Logger;
+//import org.slf4j.LoggerFactory;
+//
+//public class PublisherFactory {
+//    private static Logger log = LoggerFactory.getLogger(PublisherFactory.class);
+//
+//    public static Publisher createActivityPublisher() throws AiravataException {
+//        String activityPublisher = ServerSettings.getStatusPublisher();
+//
+//        if (activityPublisher == null) {
+//            String s = "Activity publisher is not specified";
+//            log.error(s);
+//            throw new AiravataException(s);
+//        }
+//
+//        try {
+//            Class<? extends Publisher> aPublisher = Class.forName(activityPublisher).asSubclass(Publisher.class);
+//            return aPublisher.newInstance();
+//        } catch (Exception e) {
+//            String msg = "Failed to load the publisher from the publisher class property: " + activityPublisher;
+//            log.error(msg, e);
+//            throw new AiravataException(msg, e);
+//        }
+//    }
+//
+//    public static Publisher createTaskLaunchPublisher() throws AiravataException {
+//        String taskLaunchPublisher = ServerSettings.getTaskLaunchPublisher();
+//
+//        if (taskLaunchPublisher == null) {
+//            String s = "Task launch publisher is not specified";
+//            log.error(s);
+//            throw new AiravataException(s);
+//        }
+//
+//        try {
+//            Class<? extends Publisher> aPublisher = Class.forName(taskLaunchPublisher).asSubclass(Publisher.class);
+//            return aPublisher.newInstance();
+//        } catch (Exception e) {
+//            String msg = "Failed to load the publisher from the publisher class property: " + taskLaunchPublisher;
+//            log.error(msg, e);
+//            throw new AiravataException(msg, e);
+//        }
+//    }
+//}

http://git-wip-us.apache.org/repos/asf/airavata/blob/98cbb407/modules/orchestrator/orchestrator-core/src/main/java/org/apache/airavata/orchestrator/core/impl/GFACPassiveJobSubmitter.java
----------------------------------------------------------------------
diff --git a/modules/orchestrator/orchestrator-core/src/main/java/org/apache/airavata/orchestrator/core/impl/GFACPassiveJobSubmitter.java b/modules/orchestrator/orchestrator-core/src/main/java/org/apache/airavata/orchestrator/core/impl/GFACPassiveJobSubmitter.java
index 02ecfe1..856f9f3 100644
--- a/modules/orchestrator/orchestrator-core/src/main/java/org/apache/airavata/orchestrator/core/impl/GFACPassiveJobSubmitter.java
+++ b/modules/orchestrator/orchestrator-core/src/main/java/org/apache/airavata/orchestrator/core/impl/GFACPassiveJobSubmitter.java
@@ -27,9 +27,12 @@ import org.apache.airavata.credential.store.store.CredentialReader;
 import org.apache.airavata.gfac.client.GFACInstance;
 import org.apache.airavata.gfac.core.GFacUtils;
 import org.apache.airavata.messaging.core.MessageContext;
+import org.apache.airavata.messaging.core.MessagingFactory;
 import org.apache.airavata.messaging.core.Publisher;
-import org.apache.airavata.messaging.core.PublisherFactory;
-import org.apache.airavata.model.messaging.event.*;
+import org.apache.airavata.messaging.core.Type;
+import org.apache.airavata.model.messaging.event.MessageType;
+import org.apache.airavata.model.messaging.event.ProcessSubmitEvent;
+import org.apache.airavata.model.messaging.event.ProcessTerminateEvent;
 import org.apache.airavata.orchestrator.core.context.OrchestratorContext;
 import org.apache.airavata.orchestrator.core.exception.OrchestratorException;
 import org.apache.airavata.orchestrator.core.job.JobSubmitter;
@@ -53,7 +56,7 @@ public class GFACPassiveJobSubmitter implements JobSubmitter,Watcher {
             this.publisher = orchestratorContext.getPublisher();
         }else {
             try {
-                this.publisher = PublisherFactory.createTaskLaunchPublisher();
+                this.publisher = MessagingFactory.getPublisher(Type.PROCESS_LAUNCH);
             } catch (AiravataException e) {
                 logger.error(e.getMessage(), e);
                 throw new OrchestratorException("Cannot initialize " + GFACPassiveJobSubmitter.class + " need to start Rabbitmq server to use " + GFACPassiveJobSubmitter.class);

http://git-wip-us.apache.org/repos/asf/airavata/blob/98cbb407/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java
----------------------------------------------------------------------
diff --git a/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java b/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java
index 5d02100..03f6f8a 100644
--- a/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java
+++ b/modules/orchestrator/orchestrator-service/src/main/java/org/apache/airavata/orchestrator/server/OrchestratorServerHandler.java
@@ -33,7 +33,6 @@ import org.apache.airavata.messaging.core.MessageContext;
 import org.apache.airavata.messaging.core.MessageHandler;
 import org.apache.airavata.messaging.core.MessagingFactory;
 import org.apache.airavata.messaging.core.Publisher;
-import org.apache.airavata.messaging.core.PublisherFactory;
 import org.apache.airavata.messaging.core.Subscriber;
 import org.apache.airavata.messaging.core.Type;
 import org.apache.airavata.model.appcatalog.appdeployment.ApplicationDeploymentDescription;
@@ -112,7 +111,7 @@ public class OrchestratorServerHandler implements OrchestratorService.Iface {
 
 	public OrchestratorServerHandler() throws OrchestratorException{
 		try {
-	        publisher = PublisherFactory.createActivityPublisher();
+	        publisher = MessagingFactory.getPublisher(Type.STATUS);
             setAiravataUserName(ServerSettings.getDefaultUser());
 		} catch (AiravataException e) {
             log.error(e.getMessage(), e);


[05/50] [abbrv] airavata git commit: Merge branch 'messaging-refactor' of https://git-wip-us.apache.org/repos/asf/airavata into messaging-refactor

Posted by la...@apache.org.
Merge branch 'messaging-refactor' of https://git-wip-us.apache.org/repos/asf/airavata into messaging-refactor


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: 474ae23d51ca783df8082bb49ae151f07dad3ee3
Parents: 98cbb40 65c2a96
Author: Shameera Rathnayaka <sh...@gmail.com>
Authored: Wed Aug 10 15:42:58 2016 -0400
Committer: Shameera Rathnayaka <sh...@gmail.com>
Committed: Wed Aug 10 15:42:58 2016 -0400

----------------------------------------------------------------------

----------------------------------------------------------------------



[44/50] [abbrv] airavata git commit: making experiment status a list in ExperimentModel

Posted by la...@apache.org.
making experiment status a list in ExperimentModel


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: 415e9b701599b44999ca35306a53052012e44e63
Parents: 3536751
Author: scnakandala <su...@gmail.com>
Authored: Thu Aug 25 15:28:32 2016 -0400
Committer: scnakandala <su...@gmail.com>
Committed: Thu Aug 25 15:28:32 2016 -0400

----------------------------------------------------------------------
 .../server/handler/AiravataServerHandler.java   |  43 +-
 .../lib/airavata/experiment_model_types.cpp     | 406 ++++++++++---------
 .../lib/airavata/experiment_model_types.h       |   4 +-
 .../lib/Airavata/Model/Experiment/Types.php     | 205 +++++-----
 .../apache/airavata/model/experiment/ttypes.py  | 144 +++----
 .../client/samples/CreateLaunchBES.java         |  19 +-
 .../client/samples/CreateLaunchExperiment.java  |  14 +-
 .../model/experiment/ExperimentModel.java       | 219 ++++++----
 .../model/experiment/ExperimentStatistics.java  | 216 +++++-----
 .../impl/ExperimentStatusValidator.java         |   5 +-
 .../server/OrchestratorServerHandler.java       |  32 +-
 modules/registry/registry-core/pom.xml          |   2 +-
 .../catalog/impl/ExperimentRegistry.java        |   2 +-
 .../core/experiment/catalog/model/Gateway.java  |   2 +-
 .../utils/ThriftDataModelConversion.java        |   4 +-
 .../service/handler/RegistryServerHandler.java  |   2 +-
 .../experiment_model.thrift                     |   2 +-
 17 files changed, 685 insertions(+), 636 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/415e9b70/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java b/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
index b21be18..83eb771 100644
--- a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
+++ b/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
@@ -48,14 +48,7 @@ import org.apache.airavata.model.WorkflowModel;
 import org.apache.airavata.model.appcatalog.appdeployment.ApplicationDeploymentDescription;
 import org.apache.airavata.model.appcatalog.appdeployment.ApplicationModule;
 import org.apache.airavata.model.appcatalog.appinterface.ApplicationInterfaceDescription;
-import org.apache.airavata.model.appcatalog.computeresource.CloudJobSubmission;
-import org.apache.airavata.model.appcatalog.computeresource.ComputeResourceDescription;
-import org.apache.airavata.model.appcatalog.computeresource.JobSubmissionInterface;
-import org.apache.airavata.model.appcatalog.computeresource.JobSubmissionProtocol;
-import org.apache.airavata.model.appcatalog.computeresource.LOCALSubmission;
-import org.apache.airavata.model.appcatalog.computeresource.ResourceJobManager;
-import org.apache.airavata.model.appcatalog.computeresource.SSHJobSubmission;
-import org.apache.airavata.model.appcatalog.computeresource.UnicoreJobSubmission;
+import org.apache.airavata.model.appcatalog.computeresource.*;
 import org.apache.airavata.model.appcatalog.gatewayprofile.ComputeResourcePreference;
 import org.apache.airavata.model.appcatalog.gatewayprofile.GatewayResourceProfile;
 import org.apache.airavata.model.appcatalog.gatewayprofile.StoragePreference;
@@ -64,27 +57,11 @@ import org.apache.airavata.model.application.io.InputDataObjectType;
 import org.apache.airavata.model.application.io.OutputDataObjectType;
 import org.apache.airavata.model.commons.airavata_commonsConstants;
 import org.apache.airavata.model.data.movement.DMType;
-import org.apache.airavata.model.data.movement.DataMovementInterface;
-import org.apache.airavata.model.data.movement.DataMovementProtocol;
-import org.apache.airavata.model.data.movement.GridFTPDataMovement;
-import org.apache.airavata.model.data.movement.LOCALDataMovement;
-import org.apache.airavata.model.data.movement.SCPDataMovement;
-import org.apache.airavata.model.data.movement.UnicoreDataMovement;
+import org.apache.airavata.model.data.movement.*;
 import org.apache.airavata.model.data.replica.DataProductModel;
 import org.apache.airavata.model.data.replica.DataReplicaLocationModel;
-import org.apache.airavata.model.error.AiravataClientException;
-import org.apache.airavata.model.error.AiravataErrorType;
-import org.apache.airavata.model.error.AiravataSystemException;
-import org.apache.airavata.model.error.AuthorizationException;
-import org.apache.airavata.model.error.ExperimentNotFoundException;
-import org.apache.airavata.model.error.InvalidRequestException;
-import org.apache.airavata.model.error.ProjectNotFoundException;
-import org.apache.airavata.model.experiment.ExperimentModel;
-import org.apache.airavata.model.experiment.ExperimentSearchFields;
-import org.apache.airavata.model.experiment.ExperimentStatistics;
-import org.apache.airavata.model.experiment.ExperimentSummaryModel;
-import org.apache.airavata.model.experiment.ProjectSearchFields;
-import org.apache.airavata.model.experiment.UserConfigurationDataModel;
+import org.apache.airavata.model.error.*;
+import org.apache.airavata.model.experiment.*;
 import org.apache.airavata.model.group.GroupModel;
 import org.apache.airavata.model.group.ResourcePermissionType;
 import org.apache.airavata.model.group.ResourceType;
@@ -107,11 +84,7 @@ import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
+import java.util.*;
 
 public class AiravataServerHandler implements Airavata.Iface {
     private static final Logger logger = LoggerFactory.getLogger(AiravataServerHandler.class);
@@ -934,7 +907,7 @@ public class AiravataServerHandler implements Airavata.Iface {
                 }
             }
 
-            if(!(experimentModel.getExperimentStatus().getState() == ExperimentState.CREATED)){
+            if(!(experimentModel.getExperimentStatus().get(0).getState() == ExperimentState.CREATED)){
                 logger.error("Error while deleting the experiment");
                 throw new RegistryServiceException("Experiment is not in CREATED state. Hence cannot deleted. ID:"+ experimentId);
             }
@@ -1454,9 +1427,9 @@ public class AiravataServerHandler implements Airavata.Iface {
                 logger.error(airavataExperimentId, "Error while cancelling experiment {}, experiment doesn't exist.", airavataExperimentId);
                 throw new ExperimentNotFoundException("Requested experiment id " + airavataExperimentId + " does not exist in the system..");
             }
-            switch (existingExperiment.getExperimentStatus().getState()) {
+            switch (existingExperiment.getExperimentStatus().get(0).getState()) {
                 case COMPLETED: case CANCELED: case FAILED: case CANCELING:
-                    logger.warn("Can't terminate already {} experiment", existingExperiment.getExperimentStatus().getState().name());
+                    logger.warn("Can't terminate already {} experiment", existingExperiment.getExperimentStatus().get(0).getState().name());
                     break;
                 case CREATED:
                     logger.warn("Experiment termination is only allowed for launched experiments.");

http://git-wip-us.apache.org/repos/asf/airavata/blob/415e9b70/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/experiment_model_types.cpp
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/experiment_model_types.cpp b/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/experiment_model_types.cpp
index 0bf0ac2..1980b28 100644
--- a/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/experiment_model_types.cpp
+++ b/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/experiment_model_types.cpp
@@ -415,7 +415,7 @@ void ExperimentModel::__set_experimentOutputs(const std::vector< ::apache::airav
 __isset.experimentOutputs = true;
 }
 
-void ExperimentModel::__set_experimentStatus(const  ::apache::airavata::model::status::ExperimentStatus& val) {
+void ExperimentModel::__set_experimentStatus(const std::vector< ::apache::airavata::model::status::ExperimentStatus> & val) {
   this->experimentStatus = val;
 __isset.experimentStatus = true;
 }
@@ -624,45 +624,57 @@ uint32_t ExperimentModel::read(::apache::thrift::protocol::TProtocol* iprot) {
         }
         break;
       case 17:
-        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
-          xfer += this->experimentStatus.read(iprot);
-          this->__isset.experimentStatus = true;
-        } else {
-          xfer += iprot->skip(ftype);
-        }
-        break;
-      case 18:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
-            this->errors.clear();
+            this->experimentStatus.clear();
             uint32_t _size18;
             ::apache::thrift::protocol::TType _etype21;
             xfer += iprot->readListBegin(_etype21, _size18);
-            this->errors.resize(_size18);
+            this->experimentStatus.resize(_size18);
             uint32_t _i22;
             for (_i22 = 0; _i22 < _size18; ++_i22)
             {
-              xfer += this->errors[_i22].read(iprot);
+              xfer += this->experimentStatus[_i22].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
-          this->__isset.errors = true;
+          this->__isset.experimentStatus = true;
         } else {
           xfer += iprot->skip(ftype);
         }
         break;
-      case 19:
+      case 18:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
-            this->processes.clear();
+            this->errors.clear();
             uint32_t _size23;
             ::apache::thrift::protocol::TType _etype26;
             xfer += iprot->readListBegin(_etype26, _size23);
-            this->processes.resize(_size23);
+            this->errors.resize(_size23);
             uint32_t _i27;
             for (_i27 = 0; _i27 < _size23; ++_i27)
             {
-              xfer += this->processes[_i27].read(iprot);
+              xfer += this->errors[_i27].read(iprot);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.errors = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 19:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->processes.clear();
+            uint32_t _size28;
+            ::apache::thrift::protocol::TType _etype31;
+            xfer += iprot->readListBegin(_etype31, _size28);
+            this->processes.resize(_size28);
+            uint32_t _i32;
+            for (_i32 = 0; _i32 < _size28; ++_i32)
+            {
+              xfer += this->processes[_i32].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -758,10 +770,10 @@ uint32_t ExperimentModel::write(::apache::thrift::protocol::TProtocol* oprot) co
     xfer += oprot->writeFieldBegin("emailAddresses", ::apache::thrift::protocol::T_LIST, 13);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->emailAddresses.size()));
-      std::vector<std::string> ::const_iterator _iter28;
-      for (_iter28 = this->emailAddresses.begin(); _iter28 != this->emailAddresses.end(); ++_iter28)
+      std::vector<std::string> ::const_iterator _iter33;
+      for (_iter33 = this->emailAddresses.begin(); _iter33 != this->emailAddresses.end(); ++_iter33)
       {
-        xfer += oprot->writeString((*_iter28));
+        xfer += oprot->writeString((*_iter33));
       }
       xfer += oprot->writeListEnd();
     }
@@ -776,10 +788,10 @@ uint32_t ExperimentModel::write(::apache::thrift::protocol::TProtocol* oprot) co
     xfer += oprot->writeFieldBegin("experimentInputs", ::apache::thrift::protocol::T_LIST, 15);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->experimentInputs.size()));
-      std::vector< ::apache::airavata::model::application::io::InputDataObjectType> ::const_iterator _iter29;
-      for (_iter29 = this->experimentInputs.begin(); _iter29 != this->experimentInputs.end(); ++_iter29)
+      std::vector< ::apache::airavata::model::application::io::InputDataObjectType> ::const_iterator _iter34;
+      for (_iter34 = this->experimentInputs.begin(); _iter34 != this->experimentInputs.end(); ++_iter34)
       {
-        xfer += (*_iter29).write(oprot);
+        xfer += (*_iter34).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -789,28 +801,36 @@ uint32_t ExperimentModel::write(::apache::thrift::protocol::TProtocol* oprot) co
     xfer += oprot->writeFieldBegin("experimentOutputs", ::apache::thrift::protocol::T_LIST, 16);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->experimentOutputs.size()));
-      std::vector< ::apache::airavata::model::application::io::OutputDataObjectType> ::const_iterator _iter30;
-      for (_iter30 = this->experimentOutputs.begin(); _iter30 != this->experimentOutputs.end(); ++_iter30)
+      std::vector< ::apache::airavata::model::application::io::OutputDataObjectType> ::const_iterator _iter35;
+      for (_iter35 = this->experimentOutputs.begin(); _iter35 != this->experimentOutputs.end(); ++_iter35)
       {
-        xfer += (*_iter30).write(oprot);
+        xfer += (*_iter35).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
     xfer += oprot->writeFieldEnd();
   }
   if (this->__isset.experimentStatus) {
-    xfer += oprot->writeFieldBegin("experimentStatus", ::apache::thrift::protocol::T_STRUCT, 17);
-    xfer += this->experimentStatus.write(oprot);
+    xfer += oprot->writeFieldBegin("experimentStatus", ::apache::thrift::protocol::T_LIST, 17);
+    {
+      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->experimentStatus.size()));
+      std::vector< ::apache::airavata::model::status::ExperimentStatus> ::const_iterator _iter36;
+      for (_iter36 = this->experimentStatus.begin(); _iter36 != this->experimentStatus.end(); ++_iter36)
+      {
+        xfer += (*_iter36).write(oprot);
+      }
+      xfer += oprot->writeListEnd();
+    }
     xfer += oprot->writeFieldEnd();
   }
   if (this->__isset.errors) {
     xfer += oprot->writeFieldBegin("errors", ::apache::thrift::protocol::T_LIST, 18);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->errors.size()));
-      std::vector< ::apache::airavata::model::commons::ErrorModel> ::const_iterator _iter31;
-      for (_iter31 = this->errors.begin(); _iter31 != this->errors.end(); ++_iter31)
+      std::vector< ::apache::airavata::model::commons::ErrorModel> ::const_iterator _iter37;
+      for (_iter37 = this->errors.begin(); _iter37 != this->errors.end(); ++_iter37)
       {
-        xfer += (*_iter31).write(oprot);
+        xfer += (*_iter37).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -820,10 +840,10 @@ uint32_t ExperimentModel::write(::apache::thrift::protocol::TProtocol* oprot) co
     xfer += oprot->writeFieldBegin("processes", ::apache::thrift::protocol::T_LIST, 19);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->processes.size()));
-      std::vector< ::apache::airavata::model::process::ProcessModel> ::const_iterator _iter32;
-      for (_iter32 = this->processes.begin(); _iter32 != this->processes.end(); ++_iter32)
+      std::vector< ::apache::airavata::model::process::ProcessModel> ::const_iterator _iter38;
+      for (_iter38 = this->processes.begin(); _iter38 != this->processes.end(); ++_iter38)
       {
-        xfer += (*_iter32).write(oprot);
+        xfer += (*_iter38).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -858,49 +878,49 @@ void swap(ExperimentModel &a, ExperimentModel &b) {
   swap(a.__isset, b.__isset);
 }
 
-ExperimentModel::ExperimentModel(const ExperimentModel& other33) {
-  experimentId = other33.experimentId;
-  projectId = other33.projectId;
-  gatewayId = other33.gatewayId;
-  experimentType = other33.experimentType;
-  userName = other33.userName;
-  experimentName = other33.experimentName;
-  creationTime = other33.creationTime;
-  description = other33.description;
-  executionId = other33.executionId;
-  gatewayExecutionId = other33.gatewayExecutionId;
-  gatewayInstanceId = other33.gatewayInstanceId;
-  enableEmailNotification = other33.enableEmailNotification;
-  emailAddresses = other33.emailAddresses;
-  userConfigurationData = other33.userConfigurationData;
-  experimentInputs = other33.experimentInputs;
-  experimentOutputs = other33.experimentOutputs;
-  experimentStatus = other33.experimentStatus;
-  errors = other33.errors;
-  processes = other33.processes;
-  __isset = other33.__isset;
-}
-ExperimentModel& ExperimentModel::operator=(const ExperimentModel& other34) {
-  experimentId = other34.experimentId;
-  projectId = other34.projectId;
-  gatewayId = other34.gatewayId;
-  experimentType = other34.experimentType;
-  userName = other34.userName;
-  experimentName = other34.experimentName;
-  creationTime = other34.creationTime;
-  description = other34.description;
-  executionId = other34.executionId;
-  gatewayExecutionId = other34.gatewayExecutionId;
-  gatewayInstanceId = other34.gatewayInstanceId;
-  enableEmailNotification = other34.enableEmailNotification;
-  emailAddresses = other34.emailAddresses;
-  userConfigurationData = other34.userConfigurationData;
-  experimentInputs = other34.experimentInputs;
-  experimentOutputs = other34.experimentOutputs;
-  experimentStatus = other34.experimentStatus;
-  errors = other34.errors;
-  processes = other34.processes;
-  __isset = other34.__isset;
+ExperimentModel::ExperimentModel(const ExperimentModel& other39) {
+  experimentId = other39.experimentId;
+  projectId = other39.projectId;
+  gatewayId = other39.gatewayId;
+  experimentType = other39.experimentType;
+  userName = other39.userName;
+  experimentName = other39.experimentName;
+  creationTime = other39.creationTime;
+  description = other39.description;
+  executionId = other39.executionId;
+  gatewayExecutionId = other39.gatewayExecutionId;
+  gatewayInstanceId = other39.gatewayInstanceId;
+  enableEmailNotification = other39.enableEmailNotification;
+  emailAddresses = other39.emailAddresses;
+  userConfigurationData = other39.userConfigurationData;
+  experimentInputs = other39.experimentInputs;
+  experimentOutputs = other39.experimentOutputs;
+  experimentStatus = other39.experimentStatus;
+  errors = other39.errors;
+  processes = other39.processes;
+  __isset = other39.__isset;
+}
+ExperimentModel& ExperimentModel::operator=(const ExperimentModel& other40) {
+  experimentId = other40.experimentId;
+  projectId = other40.projectId;
+  gatewayId = other40.gatewayId;
+  experimentType = other40.experimentType;
+  userName = other40.userName;
+  experimentName = other40.experimentName;
+  creationTime = other40.creationTime;
+  description = other40.description;
+  executionId = other40.executionId;
+  gatewayExecutionId = other40.gatewayExecutionId;
+  gatewayInstanceId = other40.gatewayInstanceId;
+  enableEmailNotification = other40.enableEmailNotification;
+  emailAddresses = other40.emailAddresses;
+  userConfigurationData = other40.userConfigurationData;
+  experimentInputs = other40.experimentInputs;
+  experimentOutputs = other40.experimentOutputs;
+  experimentStatus = other40.experimentStatus;
+  errors = other40.errors;
+  processes = other40.processes;
+  __isset = other40.__isset;
   return *this;
 }
 void ExperimentModel::printTo(std::ostream& out) const {
@@ -1195,33 +1215,33 @@ void swap(ExperimentSummaryModel &a, ExperimentSummaryModel &b) {
   swap(a.__isset, b.__isset);
 }
 
-ExperimentSummaryModel::ExperimentSummaryModel(const ExperimentSummaryModel& other35) {
-  experimentId = other35.experimentId;
-  projectId = other35.projectId;
-  gatewayId = other35.gatewayId;
-  creationTime = other35.creationTime;
-  userName = other35.userName;
-  name = other35.name;
-  description = other35.description;
-  executionId = other35.executionId;
-  resourceHostId = other35.resourceHostId;
-  experimentStatus = other35.experimentStatus;
-  statusUpdateTime = other35.statusUpdateTime;
-  __isset = other35.__isset;
-}
-ExperimentSummaryModel& ExperimentSummaryModel::operator=(const ExperimentSummaryModel& other36) {
-  experimentId = other36.experimentId;
-  projectId = other36.projectId;
-  gatewayId = other36.gatewayId;
-  creationTime = other36.creationTime;
-  userName = other36.userName;
-  name = other36.name;
-  description = other36.description;
-  executionId = other36.executionId;
-  resourceHostId = other36.resourceHostId;
-  experimentStatus = other36.experimentStatus;
-  statusUpdateTime = other36.statusUpdateTime;
-  __isset = other36.__isset;
+ExperimentSummaryModel::ExperimentSummaryModel(const ExperimentSummaryModel& other41) {
+  experimentId = other41.experimentId;
+  projectId = other41.projectId;
+  gatewayId = other41.gatewayId;
+  creationTime = other41.creationTime;
+  userName = other41.userName;
+  name = other41.name;
+  description = other41.description;
+  executionId = other41.executionId;
+  resourceHostId = other41.resourceHostId;
+  experimentStatus = other41.experimentStatus;
+  statusUpdateTime = other41.statusUpdateTime;
+  __isset = other41.__isset;
+}
+ExperimentSummaryModel& ExperimentSummaryModel::operator=(const ExperimentSummaryModel& other42) {
+  experimentId = other42.experimentId;
+  projectId = other42.projectId;
+  gatewayId = other42.gatewayId;
+  creationTime = other42.creationTime;
+  userName = other42.userName;
+  name = other42.name;
+  description = other42.description;
+  executionId = other42.executionId;
+  resourceHostId = other42.resourceHostId;
+  experimentStatus = other42.experimentStatus;
+  statusUpdateTime = other42.statusUpdateTime;
+  __isset = other42.__isset;
   return *this;
 }
 void ExperimentSummaryModel::printTo(std::ostream& out) const {
@@ -1379,14 +1399,14 @@ uint32_t ExperimentStatistics::read(::apache::thrift::protocol::TProtocol* iprot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->allExperiments.clear();
-            uint32_t _size37;
-            ::apache::thrift::protocol::TType _etype40;
-            xfer += iprot->readListBegin(_etype40, _size37);
-            this->allExperiments.resize(_size37);
-            uint32_t _i41;
-            for (_i41 = 0; _i41 < _size37; ++_i41)
+            uint32_t _size43;
+            ::apache::thrift::protocol::TType _etype46;
+            xfer += iprot->readListBegin(_etype46, _size43);
+            this->allExperiments.resize(_size43);
+            uint32_t _i47;
+            for (_i47 = 0; _i47 < _size43; ++_i47)
             {
-              xfer += this->allExperiments[_i41].read(iprot);
+              xfer += this->allExperiments[_i47].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -1399,14 +1419,14 @@ uint32_t ExperimentStatistics::read(::apache::thrift::protocol::TProtocol* iprot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->completedExperiments.clear();
-            uint32_t _size42;
-            ::apache::thrift::protocol::TType _etype45;
-            xfer += iprot->readListBegin(_etype45, _size42);
-            this->completedExperiments.resize(_size42);
-            uint32_t _i46;
-            for (_i46 = 0; _i46 < _size42; ++_i46)
+            uint32_t _size48;
+            ::apache::thrift::protocol::TType _etype51;
+            xfer += iprot->readListBegin(_etype51, _size48);
+            this->completedExperiments.resize(_size48);
+            uint32_t _i52;
+            for (_i52 = 0; _i52 < _size48; ++_i52)
             {
-              xfer += this->completedExperiments[_i46].read(iprot);
+              xfer += this->completedExperiments[_i52].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -1419,14 +1439,14 @@ uint32_t ExperimentStatistics::read(::apache::thrift::protocol::TProtocol* iprot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->failedExperiments.clear();
-            uint32_t _size47;
-            ::apache::thrift::protocol::TType _etype50;
-            xfer += iprot->readListBegin(_etype50, _size47);
-            this->failedExperiments.resize(_size47);
-            uint32_t _i51;
-            for (_i51 = 0; _i51 < _size47; ++_i51)
+            uint32_t _size53;
+            ::apache::thrift::protocol::TType _etype56;
+            xfer += iprot->readListBegin(_etype56, _size53);
+            this->failedExperiments.resize(_size53);
+            uint32_t _i57;
+            for (_i57 = 0; _i57 < _size53; ++_i57)
             {
-              xfer += this->failedExperiments[_i51].read(iprot);
+              xfer += this->failedExperiments[_i57].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -1439,14 +1459,14 @@ uint32_t ExperimentStatistics::read(::apache::thrift::protocol::TProtocol* iprot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->cancelledExperiments.clear();
-            uint32_t _size52;
-            ::apache::thrift::protocol::TType _etype55;
-            xfer += iprot->readListBegin(_etype55, _size52);
-            this->cancelledExperiments.resize(_size52);
-            uint32_t _i56;
-            for (_i56 = 0; _i56 < _size52; ++_i56)
+            uint32_t _size58;
+            ::apache::thrift::protocol::TType _etype61;
+            xfer += iprot->readListBegin(_etype61, _size58);
+            this->cancelledExperiments.resize(_size58);
+            uint32_t _i62;
+            for (_i62 = 0; _i62 < _size58; ++_i62)
             {
-              xfer += this->cancelledExperiments[_i56].read(iprot);
+              xfer += this->cancelledExperiments[_i62].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -1459,14 +1479,14 @@ uint32_t ExperimentStatistics::read(::apache::thrift::protocol::TProtocol* iprot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->createdExperiments.clear();
-            uint32_t _size57;
-            ::apache::thrift::protocol::TType _etype60;
-            xfer += iprot->readListBegin(_etype60, _size57);
-            this->createdExperiments.resize(_size57);
-            uint32_t _i61;
-            for (_i61 = 0; _i61 < _size57; ++_i61)
+            uint32_t _size63;
+            ::apache::thrift::protocol::TType _etype66;
+            xfer += iprot->readListBegin(_etype66, _size63);
+            this->createdExperiments.resize(_size63);
+            uint32_t _i67;
+            for (_i67 = 0; _i67 < _size63; ++_i67)
             {
-              xfer += this->createdExperiments[_i61].read(iprot);
+              xfer += this->createdExperiments[_i67].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -1479,14 +1499,14 @@ uint32_t ExperimentStatistics::read(::apache::thrift::protocol::TProtocol* iprot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->runningExperiments.clear();
-            uint32_t _size62;
-            ::apache::thrift::protocol::TType _etype65;
-            xfer += iprot->readListBegin(_etype65, _size62);
-            this->runningExperiments.resize(_size62);
-            uint32_t _i66;
-            for (_i66 = 0; _i66 < _size62; ++_i66)
+            uint32_t _size68;
+            ::apache::thrift::protocol::TType _etype71;
+            xfer += iprot->readListBegin(_etype71, _size68);
+            this->runningExperiments.resize(_size68);
+            uint32_t _i72;
+            for (_i72 = 0; _i72 < _size68; ++_i72)
             {
-              xfer += this->runningExperiments[_i66].read(iprot);
+              xfer += this->runningExperiments[_i72].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -1552,10 +1572,10 @@ uint32_t ExperimentStatistics::write(::apache::thrift::protocol::TProtocol* opro
   xfer += oprot->writeFieldBegin("allExperiments", ::apache::thrift::protocol::T_LIST, 7);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->allExperiments.size()));
-    std::vector<ExperimentSummaryModel> ::const_iterator _iter67;
-    for (_iter67 = this->allExperiments.begin(); _iter67 != this->allExperiments.end(); ++_iter67)
+    std::vector<ExperimentSummaryModel> ::const_iterator _iter73;
+    for (_iter73 = this->allExperiments.begin(); _iter73 != this->allExperiments.end(); ++_iter73)
     {
-      xfer += (*_iter67).write(oprot);
+      xfer += (*_iter73).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -1565,10 +1585,10 @@ uint32_t ExperimentStatistics::write(::apache::thrift::protocol::TProtocol* opro
     xfer += oprot->writeFieldBegin("completedExperiments", ::apache::thrift::protocol::T_LIST, 8);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->completedExperiments.size()));
-      std::vector<ExperimentSummaryModel> ::const_iterator _iter68;
-      for (_iter68 = this->completedExperiments.begin(); _iter68 != this->completedExperiments.end(); ++_iter68)
+      std::vector<ExperimentSummaryModel> ::const_iterator _iter74;
+      for (_iter74 = this->completedExperiments.begin(); _iter74 != this->completedExperiments.end(); ++_iter74)
       {
-        xfer += (*_iter68).write(oprot);
+        xfer += (*_iter74).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -1578,10 +1598,10 @@ uint32_t ExperimentStatistics::write(::apache::thrift::protocol::TProtocol* opro
     xfer += oprot->writeFieldBegin("failedExperiments", ::apache::thrift::protocol::T_LIST, 9);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->failedExperiments.size()));
-      std::vector<ExperimentSummaryModel> ::const_iterator _iter69;
-      for (_iter69 = this->failedExperiments.begin(); _iter69 != this->failedExperiments.end(); ++_iter69)
+      std::vector<ExperimentSummaryModel> ::const_iterator _iter75;
+      for (_iter75 = this->failedExperiments.begin(); _iter75 != this->failedExperiments.end(); ++_iter75)
       {
-        xfer += (*_iter69).write(oprot);
+        xfer += (*_iter75).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -1591,10 +1611,10 @@ uint32_t ExperimentStatistics::write(::apache::thrift::protocol::TProtocol* opro
     xfer += oprot->writeFieldBegin("cancelledExperiments", ::apache::thrift::protocol::T_LIST, 10);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->cancelledExperiments.size()));
-      std::vector<ExperimentSummaryModel> ::const_iterator _iter70;
-      for (_iter70 = this->cancelledExperiments.begin(); _iter70 != this->cancelledExperiments.end(); ++_iter70)
+      std::vector<ExperimentSummaryModel> ::const_iterator _iter76;
+      for (_iter76 = this->cancelledExperiments.begin(); _iter76 != this->cancelledExperiments.end(); ++_iter76)
       {
-        xfer += (*_iter70).write(oprot);
+        xfer += (*_iter76).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -1604,10 +1624,10 @@ uint32_t ExperimentStatistics::write(::apache::thrift::protocol::TProtocol* opro
     xfer += oprot->writeFieldBegin("createdExperiments", ::apache::thrift::protocol::T_LIST, 11);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->createdExperiments.size()));
-      std::vector<ExperimentSummaryModel> ::const_iterator _iter71;
-      for (_iter71 = this->createdExperiments.begin(); _iter71 != this->createdExperiments.end(); ++_iter71)
+      std::vector<ExperimentSummaryModel> ::const_iterator _iter77;
+      for (_iter77 = this->createdExperiments.begin(); _iter77 != this->createdExperiments.end(); ++_iter77)
       {
-        xfer += (*_iter71).write(oprot);
+        xfer += (*_iter77).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -1617,10 +1637,10 @@ uint32_t ExperimentStatistics::write(::apache::thrift::protocol::TProtocol* opro
     xfer += oprot->writeFieldBegin("runningExperiments", ::apache::thrift::protocol::T_LIST, 12);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->runningExperiments.size()));
-      std::vector<ExperimentSummaryModel> ::const_iterator _iter72;
-      for (_iter72 = this->runningExperiments.begin(); _iter72 != this->runningExperiments.end(); ++_iter72)
+      std::vector<ExperimentSummaryModel> ::const_iterator _iter78;
+      for (_iter78 = this->runningExperiments.begin(); _iter78 != this->runningExperiments.end(); ++_iter78)
       {
-        xfer += (*_iter72).write(oprot);
+        xfer += (*_iter78).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -1648,35 +1668,35 @@ void swap(ExperimentStatistics &a, ExperimentStatistics &b) {
   swap(a.__isset, b.__isset);
 }
 
-ExperimentStatistics::ExperimentStatistics(const ExperimentStatistics& other73) {
-  allExperimentCount = other73.allExperimentCount;
-  completedExperimentCount = other73.completedExperimentCount;
-  cancelledExperimentCount = other73.cancelledExperimentCount;
-  failedExperimentCount = other73.failedExperimentCount;
-  createdExperimentCount = other73.createdExperimentCount;
-  runningExperimentCount = other73.runningExperimentCount;
-  allExperiments = other73.allExperiments;
-  completedExperiments = other73.completedExperiments;
-  failedExperiments = other73.failedExperiments;
-  cancelledExperiments = other73.cancelledExperiments;
-  createdExperiments = other73.createdExperiments;
-  runningExperiments = other73.runningExperiments;
-  __isset = other73.__isset;
-}
-ExperimentStatistics& ExperimentStatistics::operator=(const ExperimentStatistics& other74) {
-  allExperimentCount = other74.allExperimentCount;
-  completedExperimentCount = other74.completedExperimentCount;
-  cancelledExperimentCount = other74.cancelledExperimentCount;
-  failedExperimentCount = other74.failedExperimentCount;
-  createdExperimentCount = other74.createdExperimentCount;
-  runningExperimentCount = other74.runningExperimentCount;
-  allExperiments = other74.allExperiments;
-  completedExperiments = other74.completedExperiments;
-  failedExperiments = other74.failedExperiments;
-  cancelledExperiments = other74.cancelledExperiments;
-  createdExperiments = other74.createdExperiments;
-  runningExperiments = other74.runningExperiments;
-  __isset = other74.__isset;
+ExperimentStatistics::ExperimentStatistics(const ExperimentStatistics& other79) {
+  allExperimentCount = other79.allExperimentCount;
+  completedExperimentCount = other79.completedExperimentCount;
+  cancelledExperimentCount = other79.cancelledExperimentCount;
+  failedExperimentCount = other79.failedExperimentCount;
+  createdExperimentCount = other79.createdExperimentCount;
+  runningExperimentCount = other79.runningExperimentCount;
+  allExperiments = other79.allExperiments;
+  completedExperiments = other79.completedExperiments;
+  failedExperiments = other79.failedExperiments;
+  cancelledExperiments = other79.cancelledExperiments;
+  createdExperiments = other79.createdExperiments;
+  runningExperiments = other79.runningExperiments;
+  __isset = other79.__isset;
+}
+ExperimentStatistics& ExperimentStatistics::operator=(const ExperimentStatistics& other80) {
+  allExperimentCount = other80.allExperimentCount;
+  completedExperimentCount = other80.completedExperimentCount;
+  cancelledExperimentCount = other80.cancelledExperimentCount;
+  failedExperimentCount = other80.failedExperimentCount;
+  createdExperimentCount = other80.createdExperimentCount;
+  runningExperimentCount = other80.runningExperimentCount;
+  allExperiments = other80.allExperiments;
+  completedExperiments = other80.completedExperiments;
+  failedExperiments = other80.failedExperiments;
+  cancelledExperiments = other80.cancelledExperiments;
+  createdExperiments = other80.createdExperiments;
+  runningExperiments = other80.runningExperiments;
+  __isset = other80.__isset;
   return *this;
 }
 void ExperimentStatistics::printTo(std::ostream& out) const {

http://git-wip-us.apache.org/repos/asf/airavata/blob/415e9b70/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/experiment_model_types.h
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/experiment_model_types.h b/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/experiment_model_types.h
index c3e6fc0..c6368e3 100644
--- a/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/experiment_model_types.h
+++ b/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/experiment_model_types.h
@@ -231,7 +231,7 @@ class ExperimentModel {
   UserConfigurationDataModel userConfigurationData;
   std::vector< ::apache::airavata::model::application::io::InputDataObjectType>  experimentInputs;
   std::vector< ::apache::airavata::model::application::io::OutputDataObjectType>  experimentOutputs;
-   ::apache::airavata::model::status::ExperimentStatus experimentStatus;
+  std::vector< ::apache::airavata::model::status::ExperimentStatus>  experimentStatus;
   std::vector< ::apache::airavata::model::commons::ErrorModel>  errors;
   std::vector< ::apache::airavata::model::process::ProcessModel>  processes;
 
@@ -269,7 +269,7 @@ class ExperimentModel {
 
   void __set_experimentOutputs(const std::vector< ::apache::airavata::model::application::io::OutputDataObjectType> & val);
 
-  void __set_experimentStatus(const  ::apache::airavata::model::status::ExperimentStatus& val);
+  void __set_experimentStatus(const std::vector< ::apache::airavata::model::status::ExperimentStatus> & val);
 
   void __set_errors(const std::vector< ::apache::airavata::model::commons::ErrorModel> & val);
 

http://git-wip-us.apache.org/repos/asf/airavata/blob/415e9b70/airavata-api/airavata-client-sdks/airavata-php-sdk/src/main/resources/lib/Airavata/Model/Experiment/Types.php
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-client-sdks/airavata-php-sdk/src/main/resources/lib/Airavata/Model/Experiment/Types.php b/airavata-api/airavata-client-sdks/airavata-php-sdk/src/main/resources/lib/Airavata/Model/Experiment/Types.php
index 5018139..3e1dc12 100644
--- a/airavata-api/airavata-client-sdks/airavata-php-sdk/src/main/resources/lib/Airavata/Model/Experiment/Types.php
+++ b/airavata-api/airavata-client-sdks/airavata-php-sdk/src/main/resources/lib/Airavata/Model/Experiment/Types.php
@@ -407,7 +407,7 @@ class ExperimentModel {
    */
   public $experimentOutputs = null;
   /**
-   * @var \Airavata\Model\Status\ExperimentStatus
+   * @var \Airavata\Model\Status\ExperimentStatus[]
    */
   public $experimentStatus = null;
   /**
@@ -503,8 +503,12 @@ class ExperimentModel {
           ),
         17 => array(
           'var' => 'experimentStatus',
-          'type' => TType::STRUCT,
-          'class' => '\Airavata\Model\Status\ExperimentStatus',
+          'type' => TType::LST,
+          'etype' => TType::STRUCT,
+          'elem' => array(
+            'type' => TType::STRUCT,
+            'class' => '\Airavata\Model\Status\ExperimentStatus',
+            ),
           ),
         18 => array(
           'var' => 'errors',
@@ -752,43 +756,53 @@ class ExperimentModel {
           }
           break;
         case 17:
-          if ($ftype == TType::STRUCT) {
-            $this->experimentStatus = new \Airavata\Model\Status\ExperimentStatus();
-            $xfer += $this->experimentStatus->read($input);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
-        case 18:
           if ($ftype == TType::LST) {
-            $this->errors = array();
+            $this->experimentStatus = array();
             $_size18 = 0;
             $_etype21 = 0;
             $xfer += $input->readListBegin($_etype21, $_size18);
             for ($_i22 = 0; $_i22 < $_size18; ++$_i22)
             {
               $elem23 = null;
-              $elem23 = new \Airavata\Model\Commons\ErrorModel();
+              $elem23 = new \Airavata\Model\Status\ExperimentStatus();
               $xfer += $elem23->read($input);
-              $this->errors []= $elem23;
+              $this->experimentStatus []= $elem23;
             }
             $xfer += $input->readListEnd();
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
-        case 19:
+        case 18:
           if ($ftype == TType::LST) {
-            $this->processes = array();
+            $this->errors = array();
             $_size24 = 0;
             $_etype27 = 0;
             $xfer += $input->readListBegin($_etype27, $_size24);
             for ($_i28 = 0; $_i28 < $_size24; ++$_i28)
             {
               $elem29 = null;
-              $elem29 = new \Airavata\Model\Process\ProcessModel();
+              $elem29 = new \Airavata\Model\Commons\ErrorModel();
               $xfer += $elem29->read($input);
-              $this->processes []= $elem29;
+              $this->errors []= $elem29;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 19:
+          if ($ftype == TType::LST) {
+            $this->processes = array();
+            $_size30 = 0;
+            $_etype33 = 0;
+            $xfer += $input->readListBegin($_etype33, $_size30);
+            for ($_i34 = 0; $_i34 < $_size30; ++$_i34)
+            {
+              $elem35 = null;
+              $elem35 = new \Airavata\Model\Process\ProcessModel();
+              $xfer += $elem35->read($input);
+              $this->processes []= $elem35;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -876,9 +890,9 @@ class ExperimentModel {
       {
         $output->writeListBegin(TType::STRING, count($this->emailAddresses));
         {
-          foreach ($this->emailAddresses as $iter30)
+          foreach ($this->emailAddresses as $iter36)
           {
-            $xfer += $output->writeString($iter30);
+            $xfer += $output->writeString($iter36);
           }
         }
         $output->writeListEnd();
@@ -901,9 +915,9 @@ class ExperimentModel {
       {
         $output->writeListBegin(TType::STRUCT, count($this->experimentInputs));
         {
-          foreach ($this->experimentInputs as $iter31)
+          foreach ($this->experimentInputs as $iter37)
           {
-            $xfer += $iter31->write($output);
+            $xfer += $iter37->write($output);
           }
         }
         $output->writeListEnd();
@@ -918,9 +932,9 @@ class ExperimentModel {
       {
         $output->writeListBegin(TType::STRUCT, count($this->experimentOutputs));
         {
-          foreach ($this->experimentOutputs as $iter32)
+          foreach ($this->experimentOutputs as $iter38)
           {
-            $xfer += $iter32->write($output);
+            $xfer += $iter38->write($output);
           }
         }
         $output->writeListEnd();
@@ -928,11 +942,20 @@ class ExperimentModel {
       $xfer += $output->writeFieldEnd();
     }
     if ($this->experimentStatus !== null) {
-      if (!is_object($this->experimentStatus)) {
+      if (!is_array($this->experimentStatus)) {
         throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
       }
-      $xfer += $output->writeFieldBegin('experimentStatus', TType::STRUCT, 17);
-      $xfer += $this->experimentStatus->write($output);
+      $xfer += $output->writeFieldBegin('experimentStatus', TType::LST, 17);
+      {
+        $output->writeListBegin(TType::STRUCT, count($this->experimentStatus));
+        {
+          foreach ($this->experimentStatus as $iter39)
+          {
+            $xfer += $iter39->write($output);
+          }
+        }
+        $output->writeListEnd();
+      }
       $xfer += $output->writeFieldEnd();
     }
     if ($this->errors !== null) {
@@ -943,9 +966,9 @@ class ExperimentModel {
       {
         $output->writeListBegin(TType::STRUCT, count($this->errors));
         {
-          foreach ($this->errors as $iter33)
+          foreach ($this->errors as $iter40)
           {
-            $xfer += $iter33->write($output);
+            $xfer += $iter40->write($output);
           }
         }
         $output->writeListEnd();
@@ -960,9 +983,9 @@ class ExperimentModel {
       {
         $output->writeListBegin(TType::STRUCT, count($this->processes));
         {
-          foreach ($this->processes as $iter34)
+          foreach ($this->processes as $iter41)
           {
-            $xfer += $iter34->write($output);
+            $xfer += $iter41->write($output);
           }
         }
         $output->writeListEnd();
@@ -1520,15 +1543,15 @@ class ExperimentStatistics {
         case 7:
           if ($ftype == TType::LST) {
             $this->allExperiments = array();
-            $_size35 = 0;
-            $_etype38 = 0;
-            $xfer += $input->readListBegin($_etype38, $_size35);
-            for ($_i39 = 0; $_i39 < $_size35; ++$_i39)
+            $_size42 = 0;
+            $_etype45 = 0;
+            $xfer += $input->readListBegin($_etype45, $_size42);
+            for ($_i46 = 0; $_i46 < $_size42; ++$_i46)
             {
-              $elem40 = null;
-              $elem40 = new \Airavata\Model\Experiment\ExperimentSummaryModel();
-              $xfer += $elem40->read($input);
-              $this->allExperiments []= $elem40;
+              $elem47 = null;
+              $elem47 = new \Airavata\Model\Experiment\ExperimentSummaryModel();
+              $xfer += $elem47->read($input);
+              $this->allExperiments []= $elem47;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -1538,15 +1561,15 @@ class ExperimentStatistics {
         case 8:
           if ($ftype == TType::LST) {
             $this->completedExperiments = array();
-            $_size41 = 0;
-            $_etype44 = 0;
-            $xfer += $input->readListBegin($_etype44, $_size41);
-            for ($_i45 = 0; $_i45 < $_size41; ++$_i45)
+            $_size48 = 0;
+            $_etype51 = 0;
+            $xfer += $input->readListBegin($_etype51, $_size48);
+            for ($_i52 = 0; $_i52 < $_size48; ++$_i52)
             {
-              $elem46 = null;
-              $elem46 = new \Airavata\Model\Experiment\ExperimentSummaryModel();
-              $xfer += $elem46->read($input);
-              $this->completedExperiments []= $elem46;
+              $elem53 = null;
+              $elem53 = new \Airavata\Model\Experiment\ExperimentSummaryModel();
+              $xfer += $elem53->read($input);
+              $this->completedExperiments []= $elem53;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -1556,15 +1579,15 @@ class ExperimentStatistics {
         case 9:
           if ($ftype == TType::LST) {
             $this->failedExperiments = array();
-            $_size47 = 0;
-            $_etype50 = 0;
-            $xfer += $input->readListBegin($_etype50, $_size47);
-            for ($_i51 = 0; $_i51 < $_size47; ++$_i51)
+            $_size54 = 0;
+            $_etype57 = 0;
+            $xfer += $input->readListBegin($_etype57, $_size54);
+            for ($_i58 = 0; $_i58 < $_size54; ++$_i58)
             {
-              $elem52 = null;
-              $elem52 = new \Airavata\Model\Experiment\ExperimentSummaryModel();
-              $xfer += $elem52->read($input);
-              $this->failedExperiments []= $elem52;
+              $elem59 = null;
+              $elem59 = new \Airavata\Model\Experiment\ExperimentSummaryModel();
+              $xfer += $elem59->read($input);
+              $this->failedExperiments []= $elem59;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -1574,15 +1597,15 @@ class ExperimentStatistics {
         case 10:
           if ($ftype == TType::LST) {
             $this->cancelledExperiments = array();
-            $_size53 = 0;
-            $_etype56 = 0;
-            $xfer += $input->readListBegin($_etype56, $_size53);
-            for ($_i57 = 0; $_i57 < $_size53; ++$_i57)
+            $_size60 = 0;
+            $_etype63 = 0;
+            $xfer += $input->readListBegin($_etype63, $_size60);
+            for ($_i64 = 0; $_i64 < $_size60; ++$_i64)
             {
-              $elem58 = null;
-              $elem58 = new \Airavata\Model\Experiment\ExperimentSummaryModel();
-              $xfer += $elem58->read($input);
-              $this->cancelledExperiments []= $elem58;
+              $elem65 = null;
+              $elem65 = new \Airavata\Model\Experiment\ExperimentSummaryModel();
+              $xfer += $elem65->read($input);
+              $this->cancelledExperiments []= $elem65;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -1592,15 +1615,15 @@ class ExperimentStatistics {
         case 11:
           if ($ftype == TType::LST) {
             $this->createdExperiments = array();
-            $_size59 = 0;
-            $_etype62 = 0;
-            $xfer += $input->readListBegin($_etype62, $_size59);
-            for ($_i63 = 0; $_i63 < $_size59; ++$_i63)
+            $_size66 = 0;
+            $_etype69 = 0;
+            $xfer += $input->readListBegin($_etype69, $_size66);
+            for ($_i70 = 0; $_i70 < $_size66; ++$_i70)
             {
-              $elem64 = null;
-              $elem64 = new \Airavata\Model\Experiment\ExperimentSummaryModel();
-              $xfer += $elem64->read($input);
-              $this->createdExperiments []= $elem64;
+              $elem71 = null;
+              $elem71 = new \Airavata\Model\Experiment\ExperimentSummaryModel();
+              $xfer += $elem71->read($input);
+              $this->createdExperiments []= $elem71;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -1610,15 +1633,15 @@ class ExperimentStatistics {
         case 12:
           if ($ftype == TType::LST) {
             $this->runningExperiments = array();
-            $_size65 = 0;
-            $_etype68 = 0;
-            $xfer += $input->readListBegin($_etype68, $_size65);
-            for ($_i69 = 0; $_i69 < $_size65; ++$_i69)
+            $_size72 = 0;
+            $_etype75 = 0;
+            $xfer += $input->readListBegin($_etype75, $_size72);
+            for ($_i76 = 0; $_i76 < $_size72; ++$_i76)
             {
-              $elem70 = null;
-              $elem70 = new \Airavata\Model\Experiment\ExperimentSummaryModel();
-              $xfer += $elem70->read($input);
-              $this->runningExperiments []= $elem70;
+              $elem77 = null;
+              $elem77 = new \Airavata\Model\Experiment\ExperimentSummaryModel();
+              $xfer += $elem77->read($input);
+              $this->runningExperiments []= $elem77;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -1676,9 +1699,9 @@ class ExperimentStatistics {
       {
         $output->writeListBegin(TType::STRUCT, count($this->allExperiments));
         {
-          foreach ($this->allExperiments as $iter71)
+          foreach ($this->allExperiments as $iter78)
           {
-            $xfer += $iter71->write($output);
+            $xfer += $iter78->write($output);
           }
         }
         $output->writeListEnd();
@@ -1693,9 +1716,9 @@ class ExperimentStatistics {
       {
         $output->writeListBegin(TType::STRUCT, count($this->completedExperiments));
         {
-          foreach ($this->completedExperiments as $iter72)
+          foreach ($this->completedExperiments as $iter79)
           {
-            $xfer += $iter72->write($output);
+            $xfer += $iter79->write($output);
           }
         }
         $output->writeListEnd();
@@ -1710,9 +1733,9 @@ class ExperimentStatistics {
       {
         $output->writeListBegin(TType::STRUCT, count($this->failedExperiments));
         {
-          foreach ($this->failedExperiments as $iter73)
+          foreach ($this->failedExperiments as $iter80)
           {
-            $xfer += $iter73->write($output);
+            $xfer += $iter80->write($output);
           }
         }
         $output->writeListEnd();
@@ -1727,9 +1750,9 @@ class ExperimentStatistics {
       {
         $output->writeListBegin(TType::STRUCT, count($this->cancelledExperiments));
         {
-          foreach ($this->cancelledExperiments as $iter74)
+          foreach ($this->cancelledExperiments as $iter81)
           {
-            $xfer += $iter74->write($output);
+            $xfer += $iter81->write($output);
           }
         }
         $output->writeListEnd();
@@ -1744,9 +1767,9 @@ class ExperimentStatistics {
       {
         $output->writeListBegin(TType::STRUCT, count($this->createdExperiments));
         {
-          foreach ($this->createdExperiments as $iter75)
+          foreach ($this->createdExperiments as $iter82)
           {
-            $xfer += $iter75->write($output);
+            $xfer += $iter82->write($output);
           }
         }
         $output->writeListEnd();
@@ -1761,9 +1784,9 @@ class ExperimentStatistics {
       {
         $output->writeListBegin(TType::STRUCT, count($this->runningExperiments));
         {
-          foreach ($this->runningExperiments as $iter76)
+          foreach ($this->runningExperiments as $iter83)
           {
-            $xfer += $iter76->write($output);
+            $xfer += $iter83->write($output);
           }
         }
         $output->writeListEnd();

http://git-wip-us.apache.org/repos/asf/airavata/blob/415e9b70/airavata-api/airavata-client-sdks/airavata-python-sdk/src/main/resources/lib/apache/airavata/model/experiment/ttypes.py
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-client-sdks/airavata-python-sdk/src/main/resources/lib/apache/airavata/model/experiment/ttypes.py b/airavata-api/airavata-client-sdks/airavata-python-sdk/src/main/resources/lib/apache/airavata/model/experiment/ttypes.py
index 75f6271..80ff4b1 100644
--- a/airavata-api/airavata-client-sdks/airavata-python-sdk/src/main/resources/lib/apache/airavata/model/experiment/ttypes.py
+++ b/airavata-api/airavata-client-sdks/airavata-python-sdk/src/main/resources/lib/apache/airavata/model/experiment/ttypes.py
@@ -315,7 +315,7 @@ class ExperimentModel:
     (14, TType.STRUCT, 'userConfigurationData', (UserConfigurationDataModel, UserConfigurationDataModel.thrift_spec), None, ), # 14
     (15, TType.LIST, 'experimentInputs', (TType.STRUCT,(apache.airavata.model.application.io.ttypes.InputDataObjectType, apache.airavata.model.application.io.ttypes.InputDataObjectType.thrift_spec)), None, ), # 15
     (16, TType.LIST, 'experimentOutputs', (TType.STRUCT,(apache.airavata.model.application.io.ttypes.OutputDataObjectType, apache.airavata.model.application.io.ttypes.OutputDataObjectType.thrift_spec)), None, ), # 16
-    (17, TType.STRUCT, 'experimentStatus', (apache.airavata.model.status.ttypes.ExperimentStatus, apache.airavata.model.status.ttypes.ExperimentStatus.thrift_spec), None, ), # 17
+    (17, TType.LIST, 'experimentStatus', (TType.STRUCT,(apache.airavata.model.status.ttypes.ExperimentStatus, apache.airavata.model.status.ttypes.ExperimentStatus.thrift_spec)), None, ), # 17
     (18, TType.LIST, 'errors', (TType.STRUCT,(apache.airavata.model.commons.ttypes.ErrorModel, apache.airavata.model.commons.ttypes.ErrorModel.thrift_spec)), None, ), # 18
     (19, TType.LIST, 'processes', (TType.STRUCT,(apache.airavata.model.process.ttypes.ProcessModel, apache.airavata.model.process.ttypes.ProcessModel.thrift_spec)), None, ), # 19
   )
@@ -449,30 +449,35 @@ class ExperimentModel:
         else:
           iprot.skip(ftype)
       elif fid == 17:
-        if ftype == TType.STRUCT:
-          self.experimentStatus = apache.airavata.model.status.ttypes.ExperimentStatus()
-          self.experimentStatus.read(iprot)
-        else:
-          iprot.skip(ftype)
-      elif fid == 18:
         if ftype == TType.LIST:
-          self.errors = []
+          self.experimentStatus = []
           (_etype21, _size18) = iprot.readListBegin()
           for _i22 in xrange(_size18):
-            _elem23 = apache.airavata.model.commons.ttypes.ErrorModel()
+            _elem23 = apache.airavata.model.status.ttypes.ExperimentStatus()
             _elem23.read(iprot)
-            self.errors.append(_elem23)
+            self.experimentStatus.append(_elem23)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
-      elif fid == 19:
+      elif fid == 18:
         if ftype == TType.LIST:
-          self.processes = []
+          self.errors = []
           (_etype27, _size24) = iprot.readListBegin()
           for _i28 in xrange(_size24):
-            _elem29 = apache.airavata.model.process.ttypes.ProcessModel()
+            _elem29 = apache.airavata.model.commons.ttypes.ErrorModel()
             _elem29.read(iprot)
-            self.processes.append(_elem29)
+            self.errors.append(_elem29)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 19:
+        if ftype == TType.LIST:
+          self.processes = []
+          (_etype33, _size30) = iprot.readListBegin()
+          for _i34 in xrange(_size30):
+            _elem35 = apache.airavata.model.process.ttypes.ProcessModel()
+            _elem35.read(iprot)
+            self.processes.append(_elem35)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -537,8 +542,8 @@ class ExperimentModel:
     if self.emailAddresses is not None:
       oprot.writeFieldBegin('emailAddresses', TType.LIST, 13)
       oprot.writeListBegin(TType.STRING, len(self.emailAddresses))
-      for iter30 in self.emailAddresses:
-        oprot.writeString(iter30)
+      for iter36 in self.emailAddresses:
+        oprot.writeString(iter36)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.userConfigurationData is not None:
@@ -548,33 +553,36 @@ class ExperimentModel:
     if self.experimentInputs is not None:
       oprot.writeFieldBegin('experimentInputs', TType.LIST, 15)
       oprot.writeListBegin(TType.STRUCT, len(self.experimentInputs))
-      for iter31 in self.experimentInputs:
-        iter31.write(oprot)
+      for iter37 in self.experimentInputs:
+        iter37.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.experimentOutputs is not None:
       oprot.writeFieldBegin('experimentOutputs', TType.LIST, 16)
       oprot.writeListBegin(TType.STRUCT, len(self.experimentOutputs))
-      for iter32 in self.experimentOutputs:
-        iter32.write(oprot)
+      for iter38 in self.experimentOutputs:
+        iter38.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.experimentStatus is not None:
-      oprot.writeFieldBegin('experimentStatus', TType.STRUCT, 17)
-      self.experimentStatus.write(oprot)
+      oprot.writeFieldBegin('experimentStatus', TType.LIST, 17)
+      oprot.writeListBegin(TType.STRUCT, len(self.experimentStatus))
+      for iter39 in self.experimentStatus:
+        iter39.write(oprot)
+      oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.errors is not None:
       oprot.writeFieldBegin('errors', TType.LIST, 18)
       oprot.writeListBegin(TType.STRUCT, len(self.errors))
-      for iter33 in self.errors:
-        iter33.write(oprot)
+      for iter40 in self.errors:
+        iter40.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.processes is not None:
       oprot.writeFieldBegin('processes', TType.LIST, 19)
       oprot.writeListBegin(TType.STRUCT, len(self.processes))
-      for iter34 in self.processes:
-        iter34.write(oprot)
+      for iter41 in self.processes:
+        iter41.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -925,66 +933,66 @@ class ExperimentStatistics:
       elif fid == 7:
         if ftype == TType.LIST:
           self.allExperiments = []
-          (_etype38, _size35) = iprot.readListBegin()
-          for _i39 in xrange(_size35):
-            _elem40 = ExperimentSummaryModel()
-            _elem40.read(iprot)
-            self.allExperiments.append(_elem40)
+          (_etype45, _size42) = iprot.readListBegin()
+          for _i46 in xrange(_size42):
+            _elem47 = ExperimentSummaryModel()
+            _elem47.read(iprot)
+            self.allExperiments.append(_elem47)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 8:
         if ftype == TType.LIST:
           self.completedExperiments = []
-          (_etype44, _size41) = iprot.readListBegin()
-          for _i45 in xrange(_size41):
-            _elem46 = ExperimentSummaryModel()
-            _elem46.read(iprot)
-            self.completedExperiments.append(_elem46)
+          (_etype51, _size48) = iprot.readListBegin()
+          for _i52 in xrange(_size48):
+            _elem53 = ExperimentSummaryModel()
+            _elem53.read(iprot)
+            self.completedExperiments.append(_elem53)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 9:
         if ftype == TType.LIST:
           self.failedExperiments = []
-          (_etype50, _size47) = iprot.readListBegin()
-          for _i51 in xrange(_size47):
-            _elem52 = ExperimentSummaryModel()
-            _elem52.read(iprot)
-            self.failedExperiments.append(_elem52)
+          (_etype57, _size54) = iprot.readListBegin()
+          for _i58 in xrange(_size54):
+            _elem59 = ExperimentSummaryModel()
+            _elem59.read(iprot)
+            self.failedExperiments.append(_elem59)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 10:
         if ftype == TType.LIST:
           self.cancelledExperiments = []
-          (_etype56, _size53) = iprot.readListBegin()
-          for _i57 in xrange(_size53):
-            _elem58 = ExperimentSummaryModel()
-            _elem58.read(iprot)
-            self.cancelledExperiments.append(_elem58)
+          (_etype63, _size60) = iprot.readListBegin()
+          for _i64 in xrange(_size60):
+            _elem65 = ExperimentSummaryModel()
+            _elem65.read(iprot)
+            self.cancelledExperiments.append(_elem65)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 11:
         if ftype == TType.LIST:
           self.createdExperiments = []
-          (_etype62, _size59) = iprot.readListBegin()
-          for _i63 in xrange(_size59):
-            _elem64 = ExperimentSummaryModel()
-            _elem64.read(iprot)
-            self.createdExperiments.append(_elem64)
+          (_etype69, _size66) = iprot.readListBegin()
+          for _i70 in xrange(_size66):
+            _elem71 = ExperimentSummaryModel()
+            _elem71.read(iprot)
+            self.createdExperiments.append(_elem71)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 12:
         if ftype == TType.LIST:
           self.runningExperiments = []
-          (_etype68, _size65) = iprot.readListBegin()
-          for _i69 in xrange(_size65):
-            _elem70 = ExperimentSummaryModel()
-            _elem70.read(iprot)
-            self.runningExperiments.append(_elem70)
+          (_etype75, _size72) = iprot.readListBegin()
+          for _i76 in xrange(_size72):
+            _elem77 = ExperimentSummaryModel()
+            _elem77.read(iprot)
+            self.runningExperiments.append(_elem77)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -1025,43 +1033,43 @@ class ExperimentStatistics:
     if self.allExperiments is not None:
       oprot.writeFieldBegin('allExperiments', TType.LIST, 7)
       oprot.writeListBegin(TType.STRUCT, len(self.allExperiments))
-      for iter71 in self.allExperiments:
-        iter71.write(oprot)
+      for iter78 in self.allExperiments:
+        iter78.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.completedExperiments is not None:
       oprot.writeFieldBegin('completedExperiments', TType.LIST, 8)
       oprot.writeListBegin(TType.STRUCT, len(self.completedExperiments))
-      for iter72 in self.completedExperiments:
-        iter72.write(oprot)
+      for iter79 in self.completedExperiments:
+        iter79.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.failedExperiments is not None:
       oprot.writeFieldBegin('failedExperiments', TType.LIST, 9)
       oprot.writeListBegin(TType.STRUCT, len(self.failedExperiments))
-      for iter73 in self.failedExperiments:
-        iter73.write(oprot)
+      for iter80 in self.failedExperiments:
+        iter80.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.cancelledExperiments is not None:
       oprot.writeFieldBegin('cancelledExperiments', TType.LIST, 10)
       oprot.writeListBegin(TType.STRUCT, len(self.cancelledExperiments))
-      for iter74 in self.cancelledExperiments:
-        iter74.write(oprot)
+      for iter81 in self.cancelledExperiments:
+        iter81.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.createdExperiments is not None:
       oprot.writeFieldBegin('createdExperiments', TType.LIST, 11)
       oprot.writeListBegin(TType.STRUCT, len(self.createdExperiments))
-      for iter75 in self.createdExperiments:
-        iter75.write(oprot)
+      for iter82 in self.createdExperiments:
+        iter82.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.runningExperiments is not None:
       oprot.writeFieldBegin('runningExperiments', TType.LIST, 12)
       oprot.writeListBegin(TType.STRUCT, len(self.runningExperiments))
-      for iter76 in self.runningExperiments:
-        iter76.write(oprot)
+      for iter83 in self.runningExperiments:
+        iter83.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()

http://git-wip-us.apache.org/repos/asf/airavata/blob/415e9b70/airavata-api/airavata-client-sdks/java-client-samples/src/main/java/org/apache/airavata/client/samples/CreateLaunchBES.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-client-sdks/java-client-samples/src/main/java/org/apache/airavata/client/samples/CreateLaunchBES.java b/airavata-api/airavata-client-sdks/java-client-samples/src/main/java/org/apache/airavata/client/samples/CreateLaunchBES.java
index dc869c2..b37bd29 100644
--- a/airavata-api/airavata-client-sdks/java-client-samples/src/main/java/org/apache/airavata/client/samples/CreateLaunchBES.java
+++ b/airavata-api/airavata-client-sdks/java-client-samples/src/main/java/org/apache/airavata/client/samples/CreateLaunchBES.java
@@ -20,10 +20,6 @@
  */
 package org.apache.airavata.client.samples;
 
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
 import org.apache.airavata.api.Airavata;
 import org.apache.airavata.api.client.AiravataClientFactory;
 import org.apache.airavata.client.tools.RegisterSampleApplications;
@@ -36,23 +32,22 @@ import org.apache.airavata.model.application.io.InputDataObjectType;
 import org.apache.airavata.model.application.io.OutputDataObjectType;
 import org.apache.airavata.model.commons.ErrorModel;
 import org.apache.airavata.model.data.movement.SecurityProtocol;
-import org.apache.airavata.model.error.AiravataClientException;
-import org.apache.airavata.model.error.AiravataErrorType;
-import org.apache.airavata.model.error.AiravataSystemException;
-import org.apache.airavata.model.error.ExperimentNotFoundException;
-import org.apache.airavata.model.error.InvalidRequestException;
-import org.apache.airavata.model.experiment.ExperimentSummaryModel;
+import org.apache.airavata.model.error.*;
+import org.apache.airavata.model.experiment.ExperimentModel;
 import org.apache.airavata.model.experiment.UserConfigurationDataModel;
 import org.apache.airavata.model.scheduling.ComputationalResourceSchedulingModel;
 import org.apache.airavata.model.security.AuthzToken;
 import org.apache.airavata.model.util.ExperimentModelUtil;
 import org.apache.airavata.model.workspace.Gateway;
 import org.apache.airavata.model.workspace.Project;
-import org.apache.airavata.model.experiment.ExperimentModel;
 import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
 public class CreateLaunchBES {
  
     public static final String THRIFT_SERVER_HOST = "localhost";
@@ -167,7 +162,7 @@ public class CreateLaunchBES {
             Thread.sleep(10000);
             for (String exId : experimentIds) {
                 ExperimentModel experiment = airavataClient.getExperiment(new AuthzToken(""), exId);
-                System.out.println(experiment.getExperimentId() + " " + experiment.getExperimentStatus().getState().name());
+                System.out.println(experiment.getExperimentId() + " " + experiment.getExperimentStatus().get(0).getState().name());
             }
 
 

http://git-wip-us.apache.org/repos/asf/airavata/blob/415e9b70/airavata-api/airavata-client-sdks/java-client-samples/src/main/java/org/apache/airavata/client/samples/CreateLaunchExperiment.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-client-sdks/java-client-samples/src/main/java/org/apache/airavata/client/samples/CreateLaunchExperiment.java b/airavata-api/airavata-client-sdks/java-client-samples/src/main/java/org/apache/airavata/client/samples/CreateLaunchExperiment.java
index 11c84a4..aff662b 100644
--- a/airavata-api/airavata-client-sdks/java-client-samples/src/main/java/org/apache/airavata/client/samples/CreateLaunchExperiment.java
+++ b/airavata-api/airavata-client-sdks/java-client-samples/src/main/java/org/apache/airavata/client/samples/CreateLaunchExperiment.java
@@ -25,7 +25,10 @@ import org.apache.airavata.api.Airavata;
 import org.apache.airavata.api.client.AiravataClientFactory;
 import org.apache.airavata.client.tools.RegisterSampleApplications;
 import org.apache.airavata.client.tools.RegisterSampleApplicationsUtils;
-import org.apache.airavata.model.appcatalog.computeresource.*;
+import org.apache.airavata.model.appcatalog.computeresource.ComputeResourceDescription;
+import org.apache.airavata.model.appcatalog.computeresource.JobSubmissionInterface;
+import org.apache.airavata.model.appcatalog.computeresource.JobSubmissionProtocol;
+import org.apache.airavata.model.appcatalog.computeresource.UnicoreJobSubmission;
 import org.apache.airavata.model.application.io.DataType;
 import org.apache.airavata.model.application.io.InputDataObjectType;
 import org.apache.airavata.model.application.io.OutputDataObjectType;
@@ -33,7 +36,6 @@ import org.apache.airavata.model.commons.ErrorModel;
 import org.apache.airavata.model.data.movement.SecurityProtocol;
 import org.apache.airavata.model.error.*;
 import org.apache.airavata.model.experiment.ExperimentModel;
-import org.apache.airavata.model.experiment.ExperimentSummaryModel;
 import org.apache.airavata.model.experiment.UserConfigurationDataModel;
 import org.apache.airavata.model.scheduling.ComputationalResourceSchedulingModel;
 import org.apache.airavata.model.security.AuthzToken;
@@ -211,12 +213,12 @@ public class CreateLaunchExperiment {
                 allNotFinished = false;
                 for (String exId : experimentIds) {
                     ExperimentModel experiment = airavataClient.getExperiment(new AuthzToken(""), exId);
-                    if(!experiment.getExperimentStatus().getState().equals(ExperimentState.COMPLETED)&&
-                            !experiment.getExperimentStatus().getState().equals(ExperimentState.FAILED)
-                            &&!experiment.getExperimentStatus().getState().equals(ExperimentState.CANCELED)){
+                    if(!experiment.getExperimentStatus().get(0).getState().equals(ExperimentState.COMPLETED)&&
+                            !experiment.getExperimentStatus().get(0).getState().equals(ExperimentState.FAILED)
+                            &&!experiment.getExperimentStatus().get(0).getState().equals(ExperimentState.CANCELED)){
                         allNotFinished = true;
                     }
-                    System.out.println(experiment.getExperimentId() + " " + experiment.getExperimentStatus().getState().name());
+                    System.out.println(experiment.getExperimentId() + " " + experiment.getExperimentStatus().get(0).getState().name());
                 }
                 System.out.println("----------------------------------------------------");
                 Thread.sleep(10000);


[33/50] [abbrv] airavata git commit: merging develop

Posted by la...@apache.org.
merging develop


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: 779cc376ef7c25a60586214cd7bb06056fdaaf79
Parents: 67102da b284167
Author: scnakandala <su...@gmail.com>
Authored: Wed Aug 17 11:14:20 2016 -0400
Committer: scnakandala <su...@gmail.com>
Committed: Wed Aug 17 11:14:20 2016 -0400

----------------------------------------------------------------------
 .../server/handler/AiravataServerHandler.java   | 13 ++---
 .../org/apache/airavata/gfac/impl/Factory.java  | 14 +++--
 .../airavata/gfac/server/GfacServerHandler.java |  5 +-
 .../messaging/core/MessagingFactory.java        |  6 +--
 .../messaging/core/impl/RabbitMQPublisher.java  |  2 +-
 .../messaging/core/impl/RabbitMQSubscriber.java |  4 +-
 .../server/OrchestratorServerHandler.java       | 56 +++++++++++++++-----
 7 files changed, 70 insertions(+), 30 deletions(-)
----------------------------------------------------------------------



[13/50] [abbrv] airavata git commit: Fixed NPE with Gateway resources

Posted by la...@apache.org.
Fixed NPE with Gateway resources


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: 79aa48650ec211b29ba2dcebe429b64a2a6f7eb5
Parents: 01716e5
Author: Shameera Rathnayaka <sh...@gmail.com>
Authored: Fri Aug 12 10:52:52 2016 -0400
Committer: Shameera Rathnayaka <sh...@gmail.com>
Committed: Fri Aug 12 10:52:52 2016 -0400

----------------------------------------------------------------------
 .../core/experiment/catalog/utils/ThriftDataModelConversion.java | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/79aa4865/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/utils/ThriftDataModelConversion.java
----------------------------------------------------------------------
diff --git a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/utils/ThriftDataModelConversion.java b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/utils/ThriftDataModelConversion.java
index 72897ee..97a83cb 100644
--- a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/utils/ThriftDataModelConversion.java
+++ b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/utils/ThriftDataModelConversion.java
@@ -93,7 +93,9 @@ public class ThriftDataModelConversion {
         gateway.setIdentityServerPasswordToken(resource.getIdentityServerPasswordToken());
         gateway.setOauthClientId(resource.getOauthClientId());
         gateway.setOauthClientSecret(resource.getOauthClientSecret());
-        gateway.setRequestCreationTime(resource.getRequestCreationTime().getTime());
+        if (resource.getRequestCreationTime() != null) {
+            gateway.setRequestCreationTime(resource.getRequestCreationTime().getTime());
+        }
         return gateway;
     }
 


[38/50] [abbrv] airavata git commit: Added qos and reservataion entry to configuration resources

Posted by la...@apache.org.
Added qos and reservataion entry to configuration resources


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: ef310d3ca9820a3aefabd59a6c5e6d0b52ecde37
Parents: 4427718
Author: Shameera Rathnayaka <sh...@gmail.com>
Authored: Mon Aug 22 14:45:53 2016 -0400
Committer: Shameera Rathnayaka <sh...@gmail.com>
Committed: Mon Aug 22 14:45:53 2016 -0400

----------------------------------------------------------------------
 .../server/src/main/resources/SLURMTemplate.xslt          | 10 ++++++++++
 1 file changed, 10 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/ef310d3c/modules/configuration/server/src/main/resources/SLURMTemplate.xslt
----------------------------------------------------------------------
diff --git a/modules/configuration/server/src/main/resources/SLURMTemplate.xslt b/modules/configuration/server/src/main/resources/SLURMTemplate.xslt
index 43dd552..a8eb84b 100644
--- a/modules/configuration/server/src/main/resources/SLURMTemplate.xslt
+++ b/modules/configuration/server/src/main/resources/SLURMTemplate.xslt
@@ -59,6 +59,16 @@
 #SBATCH -e <xsl:value-of select="ns:standardErrorFile"/>
     </xsl:when>
     </xsl:choose>
+    <xsl:choose>
+    <xsl:when test="ns:qualityOfService">
+#SBATCH --qos=<xsl:value-of select="ns:qualityOfService"/>
+    </xsl:when>
+    </xsl:choose>
+    <xsl:choose>
+    <xsl:when test="ns:reservation">
+#SBATCH --reservation=<xsl:value-of select="ns:reservation"/>
+    </xsl:when>
+    </xsl:choose>
     <xsl:text>&#xa;</xsl:text>
 <xsl:for-each select="ns:exports/ns:name">
    <xsl:value-of select="."/>=<xsl:value-of select="./@value"/><xsl:text>&#xa;</xsl:text>


[23/50] [abbrv] airavata git commit: Fixed messaging publishing issues

Posted by la...@apache.org.
Fixed messaging publishing issues


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: 9f979b50b972db48b0d80bcdbbe3dd932c9a0bc4
Parents: 3fcde52
Author: Shameera Rathnayaka <sh...@gmail.com>
Authored: Mon Aug 15 15:45:12 2016 -0400
Committer: Shameera Rathnayaka <sh...@gmail.com>
Committed: Mon Aug 15 15:45:12 2016 -0400

----------------------------------------------------------------------
 .../apache/airavata/gfac/server/GfacServerHandler.java    |  2 +-
 .../apache/airavata/messaging/core/MessagingFactory.java  |  3 ++-
 .../airavata/messaging/core/impl/ExperimentConsumer.java  |  2 +-
 .../airavata/messaging/core/impl/ProcessConsumer.java     | 10 ++++------
 .../airavata/messaging/core/impl/RabbitMQPublisher.java   |  3 ++-
 .../airavata/messaging/core/impl/RabbitMQSubscriber.java  |  7 ++++++-
 6 files changed, 16 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/9f979b50/modules/gfac/gfac-service/src/main/java/org/apache/airavata/gfac/server/GfacServerHandler.java
----------------------------------------------------------------------
diff --git a/modules/gfac/gfac-service/src/main/java/org/apache/airavata/gfac/server/GfacServerHandler.java b/modules/gfac/gfac-service/src/main/java/org/apache/airavata/gfac/server/GfacServerHandler.java
index 44073dc..a7b0714 100644
--- a/modules/gfac/gfac-service/src/main/java/org/apache/airavata/gfac/server/GfacServerHandler.java
+++ b/modules/gfac/gfac-service/src/main/java/org/apache/airavata/gfac/server/GfacServerHandler.java
@@ -97,7 +97,7 @@ public class GfacServerHandler implements GfacService.Iface {
     private void initAMQPClient() throws AiravataException {
 	    // init process consumer
         List<String> routingKeys = new ArrayList<>();
-        routingKeys.add(ServerSettings.getRabbitmqProcessLaunchQueueName());
+        routingKeys.add(ServerSettings.getRabbitmqProcessExchangeName());
         processLaunchSubscriber = MessagingFactory.getSubscriber(new ProcessLaunchMessageHandler(),routingKeys, Type.PROCESS_LAUNCH);
         // init status publisher
 	    statusPublisher = Factory.getStatusPublisher();

http://git-wip-us.apache.org/repos/asf/airavata/blob/9f979b50/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessagingFactory.java
----------------------------------------------------------------------
diff --git a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessagingFactory.java b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessagingFactory.java
index b3e6d35..802ea5a 100644
--- a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessagingFactory.java
+++ b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessagingFactory.java
@@ -123,7 +123,7 @@ public class MessagingFactory {
 
     private static RabbitMQSubscriber getProcessSubscriber(RabbitMQProperties sp) throws AiravataException {
         sp.setExchangeName(ServerSettings.getRabbitmqProcessExchangeName())
-                .setQueueName(ServerSettings.getRabbitmqProcessLaunchQueueName())
+                .setQueueName("process_launch")
                 .setAutoAck(false);
         return new RabbitMQSubscriber(sp);
     }
@@ -131,6 +131,7 @@ public class MessagingFactory {
 
     private static Subscriber getExperimentSubscriber(RabbitMQProperties sp) throws AiravataException {
         sp.setExchangeName(ServerSettings.getRabbitmqExperimentExchangeName())
+                .setQueueName("experiment_launch")
                 .setAutoAck(false);
         return new RabbitMQSubscriber(sp);
 

http://git-wip-us.apache.org/repos/asf/airavata/blob/9f979b50/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/ExperimentConsumer.java
----------------------------------------------------------------------
diff --git a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/ExperimentConsumer.java b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/ExperimentConsumer.java
index 6e4c46a..5010358 100644
--- a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/ExperimentConsumer.java
+++ b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/ExperimentConsumer.java
@@ -72,7 +72,7 @@ public class ExperimentConsumer extends QueueingConsumer {
                 String gatewayId = null;
                 ExperimentSubmitEvent experimentEvent = new ExperimentSubmitEvent();
                 ThriftUtils.createThriftFromBytes(message.getEvent(), experimentEvent);
-                log.debug(" Message Received with message id '" + message.getMessageId()
+                log.info(" Message Received with message id '" + message.getMessageId()
                         + "' and with message type '" + message.getMessageType() + "'  for experimentId:" +
                         " " +
                         experimentEvent.getExperimentId());

http://git-wip-us.apache.org/repos/asf/airavata/blob/9f979b50/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/ProcessConsumer.java
----------------------------------------------------------------------
diff --git a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/ProcessConsumer.java b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/ProcessConsumer.java
index e95a7ca..69910bd 100644
--- a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/ProcessConsumer.java
+++ b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/ProcessConsumer.java
@@ -55,8 +55,7 @@ public class ProcessConsumer extends QueueingConsumer{
     }
 
 
-    @Override
-    public void handleDelivery(String consumerTag,
+    @Override public void handleDelivery(String consumerTag,
                                Envelope envelope,
                                AMQP.BasicProperties basicProperties,
                                byte[] body) throws IOException {
@@ -71,10 +70,9 @@ public class ProcessConsumer extends QueueingConsumer{
             if (message.getMessageType().equals(MessageType.LAUNCHPROCESS)) {
                 ProcessSubmitEvent processSubmitEvent = new ProcessSubmitEvent();
                 ThriftUtils.createThriftFromBytes(message.getEvent(), processSubmitEvent);
-                log.debug(" Message Received with message id '" + message.getMessageId()
-                        + "' and with message type '" + message.getMessageType() + "'  for experimentId:" +
-                        " " +
-                        processSubmitEvent.getProcessId());
+                log.info(" Message Received with message id '" + message.getMessageId()
+                        + " and with message type:" + message.getMessageType() + ", for processId:" +
+                        processSubmitEvent.getProcessId() + ", expId:" + processSubmitEvent.getExperimentId());
                 event = processSubmitEvent;
                 gatewayId = processSubmitEvent.getGatewayId();
                 MessageContext messageContext = new MessageContext(event, message.getMessageType(),

http://git-wip-us.apache.org/repos/asf/airavata/blob/9f979b50/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQPublisher.java
----------------------------------------------------------------------
diff --git a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQPublisher.java b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQPublisher.java
index 3fdb3a1..6f1d1d8 100644
--- a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQPublisher.java
+++ b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQPublisher.java
@@ -71,7 +71,7 @@ public class RabbitMQPublisher implements Publisher {
             if (properties.getExchangeName() != null) {
                 channel.exchangeDeclare(properties.getExchangeName(),
                                         properties.getExchangeType(),
-                                        false);
+                                        true); //durable
             }
         } catch (Exception e) {
             String msg = "RabbitMQ connection issue for exchange : " + properties.getExchangeName();
@@ -93,6 +93,7 @@ public class RabbitMQPublisher implements Publisher {
             message.setMessageType(messageContext.getType());
             message.setUpdatedTime(messageContext.getUpdatedTime().getTime());
             String routingKey = routingKeySupplier.apply(messageContext);
+            log.info("publish messageId:" + messageContext.getMessageId() + ", messageType:" + messageContext.getType() + ", to routingKey:" + routingKey);
             byte[] messageBody = ThriftUtils.serializeThriftObject(message);
             send(messageBody, routingKey);
         } catch (TException e) {

http://git-wip-us.apache.org/repos/asf/airavata/blob/9f979b50/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQSubscriber.java
----------------------------------------------------------------------
diff --git a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQSubscriber.java b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQSubscriber.java
index 441281d..6b28723 100644
--- a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQSubscriber.java
+++ b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/RabbitMQSubscriber.java
@@ -85,7 +85,11 @@ public class RabbitMQSubscriber implements Subscriber {
             if (queueName == null) {
                 queueName = channel.queueDeclare().getQueue();
             } else {
-                channel.queueDeclare(queueName, true, false, false, null);
+                channel.queueDeclare(queueName,
+                                     true, // durable
+                                     false, // exclusive
+                                     false, // autoDelete
+                                     null);// arguments
             }
             final String id = getId(routingKeys, queueName);
             if (queueDetailMap.containsKey(id)) {
@@ -94,6 +98,7 @@ public class RabbitMQSubscriber implements Subscriber {
             }
             // bind all the routing keys
             for (String key : routingKeys) {
+                log.info("Binding key:" + key + " to queue:" + queueName);
                 channel.queueBind(queueName, properties.getExchangeName(), key);
             }
 


[07/50] [abbrv] airavata git commit: ApiServer publish experiment submit events

Posted by la...@apache.org.
ApiServer publish experiment submit events


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: 4e3dc9a9e90e6043173a98d6f546b25ff7f2896a
Parents: 63696ff
Author: Shameera Rathnayaka <sh...@gmail.com>
Authored: Wed Aug 10 18:38:23 2016 -0400
Committer: Shameera Rathnayaka <sh...@gmail.com>
Committed: Wed Aug 10 18:38:23 2016 -0400

----------------------------------------------------------------------
 .../server/handler/AiravataServerHandler.java   | 154 +++---
 .../lib/airavata/messaging_events_types.cpp     | 227 ++++++---
 .../lib/airavata/messaging_events_types.h       |  60 ++-
 .../Airavata/Model/Messaging/Event/Types.php    | 124 ++++-
 .../airavata/model/messaging/event/ttypes.py    | 121 ++++-
 .../messaging/event/ExperimentSubmitEvent.java  | 507 +++++++++++++++++++
 .../model/messaging/event/MessageType.java      |  25 +-
 .../messaging/core/MessagingFactory.java        |  20 +-
 .../messaging/core/impl/ExperimentConsumer.java |  74 ++-
 .../messaging/core/impl/ProcessConsumer.java    |   6 +-
 .../core/impl/GFACPassiveJobSubmitter.java      |   2 +-
 .../airavata-apis/messaging_events.thrift       |   6 +
 12 files changed, 1153 insertions(+), 173 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/4e3dc9a9/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java b/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
index ff0149e..cd21124 100644
--- a/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
+++ b/airavata-api/airavata-api-server/src/main/java/org/apache/airavata/api/server/handler/AiravataServerHandler.java
@@ -48,7 +48,14 @@ import org.apache.airavata.model.WorkflowModel;
 import org.apache.airavata.model.appcatalog.appdeployment.ApplicationDeploymentDescription;
 import org.apache.airavata.model.appcatalog.appdeployment.ApplicationModule;
 import org.apache.airavata.model.appcatalog.appinterface.ApplicationInterfaceDescription;
-import org.apache.airavata.model.appcatalog.computeresource.*;
+import org.apache.airavata.model.appcatalog.computeresource.CloudJobSubmission;
+import org.apache.airavata.model.appcatalog.computeresource.ComputeResourceDescription;
+import org.apache.airavata.model.appcatalog.computeresource.JobSubmissionInterface;
+import org.apache.airavata.model.appcatalog.computeresource.JobSubmissionProtocol;
+import org.apache.airavata.model.appcatalog.computeresource.LOCALSubmission;
+import org.apache.airavata.model.appcatalog.computeresource.ResourceJobManager;
+import org.apache.airavata.model.appcatalog.computeresource.SSHJobSubmission;
+import org.apache.airavata.model.appcatalog.computeresource.UnicoreJobSubmission;
 import org.apache.airavata.model.appcatalog.gatewayprofile.ComputeResourcePreference;
 import org.apache.airavata.model.appcatalog.gatewayprofile.GatewayResourceProfile;
 import org.apache.airavata.model.appcatalog.gatewayprofile.StoragePreference;
@@ -57,16 +64,33 @@ import org.apache.airavata.model.application.io.InputDataObjectType;
 import org.apache.airavata.model.application.io.OutputDataObjectType;
 import org.apache.airavata.model.commons.airavata_commonsConstants;
 import org.apache.airavata.model.data.movement.DMType;
-import org.apache.airavata.model.data.movement.*;
+import org.apache.airavata.model.data.movement.DataMovementInterface;
+import org.apache.airavata.model.data.movement.DataMovementProtocol;
+import org.apache.airavata.model.data.movement.GridFTPDataMovement;
+import org.apache.airavata.model.data.movement.LOCALDataMovement;
+import org.apache.airavata.model.data.movement.SCPDataMovement;
+import org.apache.airavata.model.data.movement.UnicoreDataMovement;
 import org.apache.airavata.model.data.replica.DataProductModel;
 import org.apache.airavata.model.data.replica.DataReplicaLocationModel;
-import org.apache.airavata.model.error.*;
-import org.apache.airavata.model.experiment.*;
+import org.apache.airavata.model.error.AiravataClientException;
+import org.apache.airavata.model.error.AiravataErrorType;
+import org.apache.airavata.model.error.AiravataSystemException;
+import org.apache.airavata.model.error.AuthorizationException;
+import org.apache.airavata.model.error.ExperimentNotFoundException;
+import org.apache.airavata.model.error.InvalidRequestException;
+import org.apache.airavata.model.error.ProjectNotFoundException;
+import org.apache.airavata.model.experiment.ExperimentModel;
+import org.apache.airavata.model.experiment.ExperimentSearchFields;
+import org.apache.airavata.model.experiment.ExperimentStatistics;
+import org.apache.airavata.model.experiment.ExperimentSummaryModel;
+import org.apache.airavata.model.experiment.ProjectSearchFields;
+import org.apache.airavata.model.experiment.UserConfigurationDataModel;
 import org.apache.airavata.model.group.GroupModel;
 import org.apache.airavata.model.group.ResourcePermissionType;
 import org.apache.airavata.model.group.ResourceType;
 import org.apache.airavata.model.job.JobModel;
 import org.apache.airavata.model.messaging.event.ExperimentStatusChangeEvent;
+import org.apache.airavata.model.messaging.event.ExperimentSubmitEvent;
 import org.apache.airavata.model.messaging.event.MessageType;
 import org.apache.airavata.model.scheduling.ComputationalResourceSchedulingModel;
 import org.apache.airavata.model.security.AuthzToken;
@@ -76,14 +100,16 @@ import org.apache.airavata.model.status.JobStatus;
 import org.apache.airavata.model.workspace.Gateway;
 import org.apache.airavata.model.workspace.Notification;
 import org.apache.airavata.model.workspace.Project;
-import org.apache.airavata.orchestrator.client.OrchestratorClientFactory;
-import org.apache.airavata.orchestrator.cpi.OrchestratorService;
-import org.apache.airavata.orchestrator.cpi.OrchestratorService.Client;
 import org.apache.airavata.registry.api.RegistryService;
 import org.apache.airavata.registry.api.client.RegistryServiceClientFactory;
 import org.apache.airavata.registry.api.exception.RegistryServiceException;
 import org.apache.airavata.registry.core.experiment.catalog.impl.RegistryFactory;
-import org.apache.airavata.registry.cpi.*;
+import org.apache.airavata.registry.cpi.AppCatalogException;
+import org.apache.airavata.registry.cpi.ComputeResource;
+import org.apache.airavata.registry.cpi.ExperimentCatalog;
+import org.apache.airavata.registry.cpi.ExperimentCatalogException;
+import org.apache.airavata.registry.cpi.ExperimentCatalogModelType;
+import org.apache.airavata.registry.cpi.RegistryException;
 import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -92,15 +118,18 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.UUID;
 
 public class AiravataServerHandler implements Airavata.Iface {
     private static final Logger logger = LoggerFactory.getLogger(AiravataServerHandler.class);
-    private Publisher publisher;
+    private Publisher statusPublisher;
+    private Publisher experimentPublisher;
     private CredentialStoreService.Client csClient;
 
     public AiravataServerHandler() {
         try {
-            publisher = MessagingFactory.getPublisher(Type.STATUS);
+            statusPublisher = MessagingFactory.getPublisher(Type.STATUS);
+            experimentPublisher = MessagingFactory.getPublisher(Type.EXPERIMENT_LAUNCH);
         } catch (ApplicationSettingsException e) {
             logger.error("Error occured while reading airavata-server properties..", e);
         } catch (AiravataException e) {
@@ -365,7 +394,7 @@ public class AiravataServerHandler implements Airavata.Iface {
      *
      * @param authzToken
      * @param gatewayId  The identifier for the requested Gateway.
-     * @param userName   The User for which the credential should be registered. For community accounts, this user is the name of the
+     * @param portalUserName The User for which the credential should be registered. For community accounts, this user is the name of the
      *                   community user name. For computational resources, this user name need not be the same user name on resoruces.
      * @param password
      * @return airavataCredStoreToken
@@ -867,8 +896,8 @@ public class AiravataServerHandler implements Airavata.Iface {
             String messageId = AiravataUtils.getId("EXPERIMENT");
             MessageContext messageContext = new MessageContext(event, MessageType.EXPERIMENT, messageId, gatewayId);
             messageContext.setUpdatedTime(AiravataUtils.getCurrentTimestamp());
-            if(publisher!=null) {
-                publisher.publish(messageContext);
+            if(statusPublisher !=null) {
+                statusPublisher.publish(messageContext);
             }
             logger.debug(experimentId, "Created new experiment with experiment name {}", experiment.getExperimentName());
             return experimentId;
@@ -1122,9 +1151,9 @@ public class AiravataServerHandler implements Airavata.Iface {
      */
     @Override
     @SecurityCheck
-    public boolean validateExperiment(AuthzToken authzToken, String airavataExperimentId) throws InvalidRequestException,
-            ExperimentNotFoundException, AiravataClientException, AiravataSystemException, AuthorizationException, TException {
-     	try {
+    public boolean validateExperiment(AuthzToken authzToken, String airavataExperimentId) throws TException {
+        // TODO - call validation module and validate experiment
+/*     	try {
             ExperimentModel experimentModel = getRegistryServiceClient().getExperiment(airavataExperimentId);
  			if (experimentModel == null) {
                 logger.error(airavataExperimentId, "Experiment validation failed , experiment {} doesn't exist.", airavataExperimentId);
@@ -1151,9 +1180,9 @@ public class AiravataServerHandler implements Airavata.Iface {
         }finally {
             orchestratorClient.getOutputProtocol().getTransport().close();
             orchestratorClient.getInputProtocol().getTransport().close();
-        }
-
+        }*/
 
+        return true;
     }
 
     /**
@@ -1180,8 +1209,7 @@ public class AiravataServerHandler implements Airavata.Iface {
      */
     @Override
     @SecurityCheck
-    public ExperimentStatus getExperimentStatus(AuthzToken authzToken, String airavataExperimentId) throws InvalidRequestException,
-            ExperimentNotFoundException, AiravataClientException, AiravataSystemException, AuthorizationException, TException {
+    public ExperimentStatus getExperimentStatus(AuthzToken authzToken, String airavataExperimentId) throws TException {
         try {
             return getRegistryServiceClient().getExperimentStatus(airavataExperimentId);
         } catch (ApplicationSettingsException e) {
@@ -1274,50 +1302,37 @@ public class AiravataServerHandler implements Airavata.Iface {
     @Override
     @SecurityCheck
     public void launchExperiment(AuthzToken authzToken, final String airavataExperimentId, String gatewayId)
-            throws AuthorizationException, TException {
-    	try {
+            throws TException {
+        try {
             ExperimentModel experiment = getRegistryServiceClient().getExperiment(airavataExperimentId);
             if (experiment == null) {
                 logger.error(airavataExperimentId, "Error while launching experiment, experiment {} doesn't exist.", airavataExperimentId);
                 throw new ExperimentNotFoundException("Requested experiment id " + airavataExperimentId + " does not exist in the system..");
             }
-//            FIXME
-//            String applicationID = experiment.getExecutionId();
-//            if (!appCatalog.getApplicationInterface().isApplicationInterfaceExists(applicationID)){
-//                logger.error(airavataExperimentId, "Error while launching experiment, application id {} for experiment {} doesn't exist.", applicationID, airavataExperimentId);
-//                AiravataSystemException exception = new AiravataSystemException();
-//                exception.setAiravataErrorType(AiravataErrorType.INTERNAL_ERROR);
-//                exception.setMessage("Error while launching experiment, application id : " + applicationID  + " for experiment : " + airavataExperimentId +
-//                        " doesn't exist..");
-//                throw exception;
-//            }
-            OrchestratorService.Client orchestratorClient = getOrchestratorClient();
-            if (orchestratorClient.validateExperiment(airavataExperimentId)) {
-                orchestratorClient.launchExperiment(airavataExperimentId, gatewayId);
-                logger.debug("Airavata launched experiment with experiment id : " + airavataExperimentId);
-            }else {
-                logger.error(airavataExperimentId, "Couldn't identify experiment type, experiment {} is neither single application nor workflow.", airavataExperimentId);
-                throw new InvalidRequestException("Experiment '" + airavataExperimentId + "' launch failed. Unable to figureout execution type for application " + experiment.getExecutionId());
-            }
+            submitExperiment(gatewayId, airavataExperimentId);
         } catch (RegistryServiceException | ApplicationSettingsException e1) {
             logger.error(airavataExperimentId, "Error while instantiate the registry instance", e1);
             AiravataSystemException exception = new AiravataSystemException();
             exception.setAiravataErrorType(AiravataErrorType.INTERNAL_ERROR);
             exception.setMessage("Error while instantiate the registry instance. More info : " + e1.getMessage());
             throw exception;
+        } catch (AiravataException ex) {
+            logger.error("Experiment publish event fails", ex);
+
         }
     }
 
 
-    private OrchestratorService.Client getOrchestratorClient() throws TException {
-	    try {
-		    final String serverHost = ServerSettings.getOrchestratorServerHost();
-		    final int serverPort = ServerSettings.getOrchestratorServerPort();
-		    return OrchestratorClientFactory.createOrchestratorClient(serverHost, serverPort);
-	    } catch (AiravataException e) {
-		    throw new TException(e);
-	    }
-    }
+
+//    private OrchestratorService.Client getOrchestratorClient() throws TException {
+//	    try {
+//		    final String serverHost = ServerSettings.getOrchestratorServerHost();
+//		    final int serverPort = ServerSettings.getOrchestratorServerPort();
+//		    return OrchestratorClientFactory.createOrchestratorClient(serverHost, serverPort);
+//	    } catch (AiravataException e) {
+//		    throw new TException(e);
+//	    }
+//    }
 
     /**
      * Clone an specified experiment with a new name. A copy of the experiment configuration is made and is persisted with new metadata.
@@ -1437,26 +1452,34 @@ public class AiravataServerHandler implements Airavata.Iface {
      */
     @Override
     @SecurityCheck
-    public void terminateExperiment(AuthzToken authzToken, String airavataExperimentId, String gatewayId) throws InvalidRequestException,
-            ExperimentNotFoundException, AiravataClientException, AiravataSystemException, AuthorizationException, TException {
+    public void terminateExperiment(AuthzToken authzToken, String airavataExperimentId, String gatewayId)
+            throws TException {
         try {
             RegistryService.Client regClient = getRegistryServiceClient();
             ExperimentModel existingExperiment = regClient.getExperiment(airavataExperimentId);
             if (existingExperiment == null){
-                logger.error(airavataExperimentId, "Error while cloning experiment {}, experiment doesn't exist.", airavataExperimentId);
+                logger.error(airavataExperimentId, "Error while cancelling experiment {}, experiment doesn't exist.", airavataExperimentId);
                 throw new ExperimentNotFoundException("Requested experiment id " + airavataExperimentId + " does not exist in the system..");
             }
+            ExperimentStatus experimentStatus = null;
+            switch (experimentStatus.getState()) {
+                case COMPLETED: case CANCELED: case FAILED: case CANCELING:
+                    logger.warn("Can't terminate already {} experiment", experimentStatus.getState().name());
+                case CREATED:
+                    logger.warn("Experiment termination is only allowed for launched experiments.");
+                default:
+                    submitCancelExperiment(airavataExperimentId, gatewayId);
 
-            Client client = getOrchestratorClient();
-            client.terminateExperiment(airavataExperimentId, gatewayId);
+            }
             logger.debug("Airavata cancelled experiment with experiment id : " + airavataExperimentId);
-        } catch (RegistryServiceException | ApplicationSettingsException e) {
+        } catch (RegistryServiceException | AiravataException e) {
             logger.error(airavataExperimentId, "Error while cancelling the experiment...", e);
             AiravataSystemException exception = new AiravataSystemException();
             exception.setAiravataErrorType(AiravataErrorType.INTERNAL_ERROR);
             exception.setMessage("Error while cancelling the experiment. More info : " + e.getMessage());
             throw exception;
         }
+
     }
 
     /**
@@ -2902,8 +2925,9 @@ public class AiravataServerHandler implements Airavata.Iface {
      */
     @Override
     @SecurityCheck
-    public boolean updateGatewayResourceProfile(AuthzToken authzToken, String gatewayID, GatewayResourceProfile gatewayResourceProfile)
-            throws InvalidRequestException, AiravataClientException, AiravataSystemException, AuthorizationException, TException {
+    public boolean updateGatewayResourceProfile(AuthzToken authzToken,
+                                                String gatewayID,
+                                                GatewayResourceProfile gatewayResourceProfile) throws TException {
         try {
             return getRegistryServiceClient().updateGatewayResourceProfile(gatewayID, gatewayResourceProfile);
         } catch (ApplicationSettingsException | RegistryServiceException e) {
@@ -2924,8 +2948,7 @@ public class AiravataServerHandler implements Airavata.Iface {
      */
     @Override
     @SecurityCheck
-    public boolean deleteGatewayResourceProfile(AuthzToken authzToken, String gatewayID) throws InvalidRequestException,
-            AiravataClientException, AiravataSystemException, AuthorizationException, TException {
+    public boolean deleteGatewayResourceProfile(AuthzToken authzToken, String gatewayID) throws TException {
         try {
             return getRegistryServiceClient().deleteGatewayResourceProfile(gatewayID);
         } catch (ApplicationSettingsException | RegistryServiceException e) {
@@ -3652,6 +3675,19 @@ public class AiravataServerHandler implements Airavata.Iface {
         return allAccessibleResources;
     }
 
+
+    private void submitExperiment(String gatewayId,String experimentId) throws AiravataException {
+        ExperimentSubmitEvent event = new ExperimentSubmitEvent(experimentId, gatewayId);
+        MessageContext messageContext = new MessageContext(event, MessageType.EXPERIMENT, "LAUNCH.EXP-" + UUID.randomUUID().toString(), gatewayId);
+        experimentPublisher.publish(messageContext);
+    }
+
+    private void submitCancelExperiment(String gatewayId, String experimentId) throws AiravataException {
+        ExperimentSubmitEvent event = new ExperimentSubmitEvent(experimentId, gatewayId);
+        MessageContext messageContext = new MessageContext(event, MessageType.EXPERIMENT_CANCEL, "CANCEL.EXP-" + UUID.randomUUID().toString(), gatewayId);
+        experimentPublisher.publish(messageContext);
+    }
+
     private CredentialStoreService.Client getCredentialStoreServiceClient() throws TException, ApplicationSettingsException {
         final int serverPort = Integer.parseInt(ServerSettings.getCredentialStoreServerPort());
         final String serverHost = ServerSettings.getCredentialStoreServerHost();

http://git-wip-us.apache.org/repos/asf/airavata/blob/4e3dc9a9/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/messaging_events_types.cpp
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/messaging_events_types.cpp b/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/messaging_events_types.cpp
index 8317b71..f7d5d87 100644
--- a/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/messaging_events_types.cpp
+++ b/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/messaging_events_types.cpp
@@ -46,6 +46,7 @@ const std::map<int, const char*> _MessageLevel_VALUES_TO_NAMES(::apache::thrift:
 
 int _kMessageTypeValues[] = {
   MessageType::EXPERIMENT,
+  MessageType::EXPERIMENT_CANCEL,
   MessageType::TASK,
   MessageType::PROCESS,
   MessageType::JOB,
@@ -55,6 +56,7 @@ int _kMessageTypeValues[] = {
 };
 const char* _kMessageTypeNames[] = {
   "EXPERIMENT",
+  "EXPERIMENT_CANCEL",
   "TASK",
   "PROCESS",
   "JOB",
@@ -62,7 +64,7 @@ const char* _kMessageTypeNames[] = {
   "TERMINATEPROCESS",
   "PROCESSOUTPUT"
 };
-const std::map<int, const char*> _MessageType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(7, _kMessageTypeValues, _kMessageTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL));
+const std::map<int, const char*> _MessageType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(8, _kMessageTypeValues, _kMessageTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL));
 
 
 ExperimentStatusChangeEvent::~ExperimentStatusChangeEvent() throw() {
@@ -1237,6 +1239,115 @@ void JobIdentifier::printTo(std::ostream& out) const {
 }
 
 
+ExperimentSubmitEvent::~ExperimentSubmitEvent() throw() {
+}
+
+
+void ExperimentSubmitEvent::__set_experimentId(const std::string& val) {
+  this->experimentId = val;
+}
+
+void ExperimentSubmitEvent::__set_gatewayId(const std::string& val) {
+  this->gatewayId = val;
+}
+
+uint32_t ExperimentSubmitEvent::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+  bool isset_experimentId = false;
+  bool isset_gatewayId = false;
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->experimentId);
+          isset_experimentId = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->gatewayId);
+          isset_gatewayId = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  if (!isset_experimentId)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_gatewayId)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t ExperimentSubmitEvent::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("ExperimentSubmitEvent");
+
+  xfer += oprot->writeFieldBegin("experimentId", ::apache::thrift::protocol::T_STRING, 1);
+  xfer += oprot->writeString(this->experimentId);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("gatewayId", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString(this->gatewayId);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(ExperimentSubmitEvent &a, ExperimentSubmitEvent &b) {
+  using ::std::swap;
+  swap(a.experimentId, b.experimentId);
+  swap(a.gatewayId, b.gatewayId);
+}
+
+ExperimentSubmitEvent::ExperimentSubmitEvent(const ExperimentSubmitEvent& other29) {
+  experimentId = other29.experimentId;
+  gatewayId = other29.gatewayId;
+}
+ExperimentSubmitEvent& ExperimentSubmitEvent::operator=(const ExperimentSubmitEvent& other30) {
+  experimentId = other30.experimentId;
+  gatewayId = other30.gatewayId;
+  return *this;
+}
+void ExperimentSubmitEvent::printTo(std::ostream& out) const {
+  using ::apache::thrift::to_string;
+  out << "ExperimentSubmitEvent(";
+  out << "experimentId=" << to_string(experimentId);
+  out << ", " << "gatewayId=" << to_string(gatewayId);
+  out << ")";
+}
+
+
 ProcessSubmitEvent::~ProcessSubmitEvent() throw() {
 }
 
@@ -1368,17 +1479,17 @@ void swap(ProcessSubmitEvent &a, ProcessSubmitEvent &b) {
   swap(a.tokenId, b.tokenId);
 }
 
-ProcessSubmitEvent::ProcessSubmitEvent(const ProcessSubmitEvent& other29) {
-  processId = other29.processId;
-  gatewayId = other29.gatewayId;
-  experimentId = other29.experimentId;
-  tokenId = other29.tokenId;
+ProcessSubmitEvent::ProcessSubmitEvent(const ProcessSubmitEvent& other31) {
+  processId = other31.processId;
+  gatewayId = other31.gatewayId;
+  experimentId = other31.experimentId;
+  tokenId = other31.tokenId;
 }
-ProcessSubmitEvent& ProcessSubmitEvent::operator=(const ProcessSubmitEvent& other30) {
-  processId = other30.processId;
-  gatewayId = other30.gatewayId;
-  experimentId = other30.experimentId;
-  tokenId = other30.tokenId;
+ProcessSubmitEvent& ProcessSubmitEvent::operator=(const ProcessSubmitEvent& other32) {
+  processId = other32.processId;
+  gatewayId = other32.gatewayId;
+  experimentId = other32.experimentId;
+  tokenId = other32.tokenId;
   return *this;
 }
 void ProcessSubmitEvent::printTo(std::ostream& out) const {
@@ -1503,15 +1614,15 @@ void swap(ProcessTerminateEvent &a, ProcessTerminateEvent &b) {
   swap(a.tokenId, b.tokenId);
 }
 
-ProcessTerminateEvent::ProcessTerminateEvent(const ProcessTerminateEvent& other31) {
-  processId = other31.processId;
-  gatewayId = other31.gatewayId;
-  tokenId = other31.tokenId;
+ProcessTerminateEvent::ProcessTerminateEvent(const ProcessTerminateEvent& other33) {
+  processId = other33.processId;
+  gatewayId = other33.gatewayId;
+  tokenId = other33.tokenId;
 }
-ProcessTerminateEvent& ProcessTerminateEvent::operator=(const ProcessTerminateEvent& other32) {
-  processId = other32.processId;
-  gatewayId = other32.gatewayId;
-  tokenId = other32.tokenId;
+ProcessTerminateEvent& ProcessTerminateEvent::operator=(const ProcessTerminateEvent& other34) {
+  processId = other34.processId;
+  gatewayId = other34.gatewayId;
+  tokenId = other34.tokenId;
   return *this;
 }
 void ProcessTerminateEvent::printTo(std::ostream& out) const {
@@ -1561,9 +1672,9 @@ uint32_t JobStatusChangeEvent::read(::apache::thrift::protocol::TProtocol* iprot
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast33;
-          xfer += iprot->readI32(ecast33);
-          this->state = ( ::apache::airavata::model::status::JobState::type)ecast33;
+          int32_t ecast35;
+          xfer += iprot->readI32(ecast35);
+          this->state = ( ::apache::airavata::model::status::JobState::type)ecast35;
           isset_state = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -1617,13 +1728,13 @@ void swap(JobStatusChangeEvent &a, JobStatusChangeEvent &b) {
   swap(a.jobIdentity, b.jobIdentity);
 }
 
-JobStatusChangeEvent::JobStatusChangeEvent(const JobStatusChangeEvent& other34) {
-  state = other34.state;
-  jobIdentity = other34.jobIdentity;
+JobStatusChangeEvent::JobStatusChangeEvent(const JobStatusChangeEvent& other36) {
+  state = other36.state;
+  jobIdentity = other36.jobIdentity;
 }
-JobStatusChangeEvent& JobStatusChangeEvent::operator=(const JobStatusChangeEvent& other35) {
-  state = other35.state;
-  jobIdentity = other35.jobIdentity;
+JobStatusChangeEvent& JobStatusChangeEvent::operator=(const JobStatusChangeEvent& other37) {
+  state = other37.state;
+  jobIdentity = other37.jobIdentity;
   return *this;
 }
 void JobStatusChangeEvent::printTo(std::ostream& out) const {
@@ -1672,9 +1783,9 @@ uint32_t JobStatusChangeRequestEvent::read(::apache::thrift::protocol::TProtocol
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast36;
-          xfer += iprot->readI32(ecast36);
-          this->state = ( ::apache::airavata::model::status::JobState::type)ecast36;
+          int32_t ecast38;
+          xfer += iprot->readI32(ecast38);
+          this->state = ( ::apache::airavata::model::status::JobState::type)ecast38;
           isset_state = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -1728,13 +1839,13 @@ void swap(JobStatusChangeRequestEvent &a, JobStatusChangeRequestEvent &b) {
   swap(a.jobIdentity, b.jobIdentity);
 }
 
-JobStatusChangeRequestEvent::JobStatusChangeRequestEvent(const JobStatusChangeRequestEvent& other37) {
-  state = other37.state;
-  jobIdentity = other37.jobIdentity;
+JobStatusChangeRequestEvent::JobStatusChangeRequestEvent(const JobStatusChangeRequestEvent& other39) {
+  state = other39.state;
+  jobIdentity = other39.jobIdentity;
 }
-JobStatusChangeRequestEvent& JobStatusChangeRequestEvent::operator=(const JobStatusChangeRequestEvent& other38) {
-  state = other38.state;
-  jobIdentity = other38.jobIdentity;
+JobStatusChangeRequestEvent& JobStatusChangeRequestEvent::operator=(const JobStatusChangeRequestEvent& other40) {
+  state = other40.state;
+  jobIdentity = other40.jobIdentity;
   return *this;
 }
 void JobStatusChangeRequestEvent::printTo(std::ostream& out) const {
@@ -1814,9 +1925,9 @@ uint32_t Message::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 3:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast39;
-          xfer += iprot->readI32(ecast39);
-          this->messageType = (MessageType::type)ecast39;
+          int32_t ecast41;
+          xfer += iprot->readI32(ecast41);
+          this->messageType = (MessageType::type)ecast41;
           isset_messageType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -1832,9 +1943,9 @@ uint32_t Message::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 5:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast40;
-          xfer += iprot->readI32(ecast40);
-          this->messageLevel = (MessageLevel::type)ecast40;
+          int32_t ecast42;
+          xfer += iprot->readI32(ecast42);
+          this->messageLevel = (MessageLevel::type)ecast42;
           this->__isset.messageLevel = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -1900,21 +2011,21 @@ void swap(Message &a, Message &b) {
   swap(a.__isset, b.__isset);
 }
 
-Message::Message(const Message& other41) {
-  event = other41.event;
-  messageId = other41.messageId;
-  messageType = other41.messageType;
-  updatedTime = other41.updatedTime;
-  messageLevel = other41.messageLevel;
-  __isset = other41.__isset;
-}
-Message& Message::operator=(const Message& other42) {
-  event = other42.event;
-  messageId = other42.messageId;
-  messageType = other42.messageType;
-  updatedTime = other42.updatedTime;
-  messageLevel = other42.messageLevel;
-  __isset = other42.__isset;
+Message::Message(const Message& other43) {
+  event = other43.event;
+  messageId = other43.messageId;
+  messageType = other43.messageType;
+  updatedTime = other43.updatedTime;
+  messageLevel = other43.messageLevel;
+  __isset = other43.__isset;
+}
+Message& Message::operator=(const Message& other44) {
+  event = other44.event;
+  messageId = other44.messageId;
+  messageType = other44.messageType;
+  updatedTime = other44.updatedTime;
+  messageLevel = other44.messageLevel;
+  __isset = other44.__isset;
   return *this;
 }
 void Message::printTo(std::ostream& out) const {

http://git-wip-us.apache.org/repos/asf/airavata/blob/4e3dc9a9/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/messaging_events_types.h
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/messaging_events_types.h b/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/messaging_events_types.h
index d5a2411..15caed1 100644
--- a/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/messaging_events_types.h
+++ b/airavata-api/airavata-client-sdks/airavata-cpp-sdk/src/main/resources/lib/airavata/messaging_events_types.h
@@ -53,12 +53,13 @@ extern const std::map<int, const char*> _MessageLevel_VALUES_TO_NAMES;
 struct MessageType {
   enum type {
     EXPERIMENT = 0,
-    TASK = 1,
-    PROCESS = 2,
-    JOB = 3,
-    LAUNCHPROCESS = 4,
-    TERMINATEPROCESS = 5,
-    PROCESSOUTPUT = 6
+    EXPERIMENT_CANCEL = 1,
+    TASK = 2,
+    PROCESS = 3,
+    JOB = 4,
+    LAUNCHPROCESS = 5,
+    TERMINATEPROCESS = 6,
+    PROCESSOUTPUT = 7
   };
 };
 
@@ -82,6 +83,8 @@ class TaskOutputChangeEvent;
 
 class JobIdentifier;
 
+class ExperimentSubmitEvent;
+
 class ProcessSubmitEvent;
 
 class ProcessTerminateEvent;
@@ -533,6 +536,51 @@ inline std::ostream& operator<<(std::ostream& out, const JobIdentifier& obj)
 }
 
 
+class ExperimentSubmitEvent {
+ public:
+
+  ExperimentSubmitEvent(const ExperimentSubmitEvent&);
+  ExperimentSubmitEvent& operator=(const ExperimentSubmitEvent&);
+  ExperimentSubmitEvent() : experimentId(), gatewayId() {
+  }
+
+  virtual ~ExperimentSubmitEvent() throw();
+  std::string experimentId;
+  std::string gatewayId;
+
+  void __set_experimentId(const std::string& val);
+
+  void __set_gatewayId(const std::string& val);
+
+  bool operator == (const ExperimentSubmitEvent & rhs) const
+  {
+    if (!(experimentId == rhs.experimentId))
+      return false;
+    if (!(gatewayId == rhs.gatewayId))
+      return false;
+    return true;
+  }
+  bool operator != (const ExperimentSubmitEvent &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ExperimentSubmitEvent & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(ExperimentSubmitEvent &a, ExperimentSubmitEvent &b);
+
+inline std::ostream& operator<<(std::ostream& out, const ExperimentSubmitEvent& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+
 class ProcessSubmitEvent {
  public:
 

http://git-wip-us.apache.org/repos/asf/airavata/blob/4e3dc9a9/airavata-api/airavata-client-sdks/airavata-php-sdk/src/main/resources/lib/Airavata/Model/Messaging/Event/Types.php
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-client-sdks/airavata-php-sdk/src/main/resources/lib/Airavata/Model/Messaging/Event/Types.php b/airavata-api/airavata-client-sdks/airavata-php-sdk/src/main/resources/lib/Airavata/Model/Messaging/Event/Types.php
index 9c6a1e8..96a015e 100644
--- a/airavata-api/airavata-client-sdks/airavata-php-sdk/src/main/resources/lib/Airavata/Model/Messaging/Event/Types.php
+++ b/airavata-api/airavata-client-sdks/airavata-php-sdk/src/main/resources/lib/Airavata/Model/Messaging/Event/Types.php
@@ -32,20 +32,22 @@ final class MessageLevel {
 
 final class MessageType {
   const EXPERIMENT = 0;
-  const TASK = 1;
-  const PROCESS = 2;
-  const JOB = 3;
-  const LAUNCHPROCESS = 4;
-  const TERMINATEPROCESS = 5;
-  const PROCESSOUTPUT = 6;
+  const EXPERIMENT_CANCEL = 1;
+  const TASK = 2;
+  const PROCESS = 3;
+  const JOB = 4;
+  const LAUNCHPROCESS = 5;
+  const TERMINATEPROCESS = 6;
+  const PROCESSOUTPUT = 7;
   static public $__names = array(
     0 => 'EXPERIMENT',
-    1 => 'TASK',
-    2 => 'PROCESS',
-    3 => 'JOB',
-    4 => 'LAUNCHPROCESS',
-    5 => 'TERMINATEPROCESS',
-    6 => 'PROCESSOUTPUT',
+    1 => 'EXPERIMENT_CANCEL',
+    2 => 'TASK',
+    3 => 'PROCESS',
+    4 => 'JOB',
+    5 => 'LAUNCHPROCESS',
+    6 => 'TERMINATEPROCESS',
+    7 => 'PROCESSOUTPUT',
   );
 }
 
@@ -1145,6 +1147,104 @@ class JobIdentifier {
 
 }
 
+class ExperimentSubmitEvent {
+  static $_TSPEC;
+
+  /**
+   * @var string
+   */
+  public $experimentId = null;
+  /**
+   * @var string
+   */
+  public $gatewayId = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'experimentId',
+          'type' => TType::STRING,
+          ),
+        2 => array(
+          'var' => 'gatewayId',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['experimentId'])) {
+        $this->experimentId = $vals['experimentId'];
+      }
+      if (isset($vals['gatewayId'])) {
+        $this->gatewayId = $vals['gatewayId'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'ExperimentSubmitEvent';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->experimentId);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->gatewayId);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('ExperimentSubmitEvent');
+    if ($this->experimentId !== null) {
+      $xfer += $output->writeFieldBegin('experimentId', TType::STRING, 1);
+      $xfer += $output->writeString($this->experimentId);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->gatewayId !== null) {
+      $xfer += $output->writeFieldBegin('gatewayId', TType::STRING, 2);
+      $xfer += $output->writeString($this->gatewayId);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
 class ProcessSubmitEvent {
   static $_TSPEC;
 

http://git-wip-us.apache.org/repos/asf/airavata/blob/4e3dc9a9/airavata-api/airavata-client-sdks/airavata-python-sdk/src/main/resources/lib/apache/airavata/model/messaging/event/ttypes.py
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-client-sdks/airavata-python-sdk/src/main/resources/lib/apache/airavata/model/messaging/event/ttypes.py b/airavata-api/airavata-client-sdks/airavata-python-sdk/src/main/resources/lib/apache/airavata/model/messaging/event/ttypes.py
index 312e07a..818841d 100644
--- a/airavata-api/airavata-client-sdks/airavata-python-sdk/src/main/resources/lib/apache/airavata/model/messaging/event/ttypes.py
+++ b/airavata-api/airavata-client-sdks/airavata-python-sdk/src/main/resources/lib/apache/airavata/model/messaging/event/ttypes.py
@@ -42,31 +42,34 @@ class MessageLevel:
 
 class MessageType:
   EXPERIMENT = 0
-  TASK = 1
-  PROCESS = 2
-  JOB = 3
-  LAUNCHPROCESS = 4
-  TERMINATEPROCESS = 5
-  PROCESSOUTPUT = 6
+  EXPERIMENT_CANCEL = 1
+  TASK = 2
+  PROCESS = 3
+  JOB = 4
+  LAUNCHPROCESS = 5
+  TERMINATEPROCESS = 6
+  PROCESSOUTPUT = 7
 
   _VALUES_TO_NAMES = {
     0: "EXPERIMENT",
-    1: "TASK",
-    2: "PROCESS",
-    3: "JOB",
-    4: "LAUNCHPROCESS",
-    5: "TERMINATEPROCESS",
-    6: "PROCESSOUTPUT",
+    1: "EXPERIMENT_CANCEL",
+    2: "TASK",
+    3: "PROCESS",
+    4: "JOB",
+    5: "LAUNCHPROCESS",
+    6: "TERMINATEPROCESS",
+    7: "PROCESSOUTPUT",
   }
 
   _NAMES_TO_VALUES = {
     "EXPERIMENT": 0,
-    "TASK": 1,
-    "PROCESS": 2,
-    "JOB": 3,
-    "LAUNCHPROCESS": 4,
-    "TERMINATEPROCESS": 5,
-    "PROCESSOUTPUT": 6,
+    "EXPERIMENT_CANCEL": 1,
+    "TASK": 2,
+    "PROCESS": 3,
+    "JOB": 4,
+    "LAUNCHPROCESS": 5,
+    "TERMINATEPROCESS": 6,
+    "PROCESSOUTPUT": 7,
   }
 
 
@@ -927,6 +930,88 @@ class JobIdentifier:
   def __ne__(self, other):
     return not (self == other)
 
+class ExperimentSubmitEvent:
+  """
+  Attributes:
+   - experimentId
+   - gatewayId
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.STRING, 'experimentId', None, None, ), # 1
+    (2, TType.STRING, 'gatewayId', None, None, ), # 2
+  )
+
+  def __init__(self, experimentId=None, gatewayId=None,):
+    self.experimentId = experimentId
+    self.gatewayId = gatewayId
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.STRING:
+          self.experimentId = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.gatewayId = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('ExperimentSubmitEvent')
+    if self.experimentId is not None:
+      oprot.writeFieldBegin('experimentId', TType.STRING, 1)
+      oprot.writeString(self.experimentId)
+      oprot.writeFieldEnd()
+    if self.gatewayId is not None:
+      oprot.writeFieldBegin('gatewayId', TType.STRING, 2)
+      oprot.writeString(self.gatewayId)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.experimentId is None:
+      raise TProtocol.TProtocolException(message='Required field experimentId is unset!')
+    if self.gatewayId is None:
+      raise TProtocol.TProtocolException(message='Required field gatewayId is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.experimentId)
+    value = (value * 31) ^ hash(self.gatewayId)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class ProcessSubmitEvent:
   """
   Attributes:

http://git-wip-us.apache.org/repos/asf/airavata/blob/4e3dc9a9/airavata-api/airavata-data-models/src/main/java/org/apache/airavata/model/messaging/event/ExperimentSubmitEvent.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-data-models/src/main/java/org/apache/airavata/model/messaging/event/ExperimentSubmitEvent.java b/airavata-api/airavata-data-models/src/main/java/org/apache/airavata/model/messaging/event/ExperimentSubmitEvent.java
new file mode 100644
index 0000000..7ec7315
--- /dev/null
+++ b/airavata-api/airavata-data-models/src/main/java/org/apache/airavata/model/messaging/event/ExperimentSubmitEvent.java
@@ -0,0 +1,507 @@
+/**
+ * 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.
+ */
+
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.airavata.model.messaging.event;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+public class ExperimentSubmitEvent implements org.apache.thrift.TBase<ExperimentSubmitEvent, ExperimentSubmitEvent._Fields>, java.io.Serializable, Cloneable, Comparable<ExperimentSubmitEvent> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ExperimentSubmitEvent");
+
+  private static final org.apache.thrift.protocol.TField EXPERIMENT_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("experimentId", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField GATEWAY_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("gatewayId", org.apache.thrift.protocol.TType.STRING, (short)2);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new ExperimentSubmitEventStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new ExperimentSubmitEventTupleSchemeFactory());
+  }
+
+  private String experimentId; // required
+  private String gatewayId; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    EXPERIMENT_ID((short)1, "experimentId"),
+    GATEWAY_ID((short)2, "gatewayId");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // EXPERIMENT_ID
+          return EXPERIMENT_ID;
+        case 2: // GATEWAY_ID
+          return GATEWAY_ID;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.EXPERIMENT_ID, new org.apache.thrift.meta_data.FieldMetaData("experimentId", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.GATEWAY_ID, new org.apache.thrift.meta_data.FieldMetaData("gatewayId", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ExperimentSubmitEvent.class, metaDataMap);
+  }
+
+  public ExperimentSubmitEvent() {
+  }
+
+  public ExperimentSubmitEvent(
+    String experimentId,
+    String gatewayId)
+  {
+    this();
+    this.experimentId = experimentId;
+    this.gatewayId = gatewayId;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ExperimentSubmitEvent(ExperimentSubmitEvent other) {
+    if (other.isSetExperimentId()) {
+      this.experimentId = other.experimentId;
+    }
+    if (other.isSetGatewayId()) {
+      this.gatewayId = other.gatewayId;
+    }
+  }
+
+  public ExperimentSubmitEvent deepCopy() {
+    return new ExperimentSubmitEvent(this);
+  }
+
+  @Override
+  public void clear() {
+    this.experimentId = null;
+    this.gatewayId = null;
+  }
+
+  public String getExperimentId() {
+    return this.experimentId;
+  }
+
+  public void setExperimentId(String experimentId) {
+    this.experimentId = experimentId;
+  }
+
+  public void unsetExperimentId() {
+    this.experimentId = null;
+  }
+
+  /** Returns true if field experimentId is set (has been assigned a value) and false otherwise */
+  public boolean isSetExperimentId() {
+    return this.experimentId != null;
+  }
+
+  public void setExperimentIdIsSet(boolean value) {
+    if (!value) {
+      this.experimentId = null;
+    }
+  }
+
+  public String getGatewayId() {
+    return this.gatewayId;
+  }
+
+  public void setGatewayId(String gatewayId) {
+    this.gatewayId = gatewayId;
+  }
+
+  public void unsetGatewayId() {
+    this.gatewayId = null;
+  }
+
+  /** Returns true if field gatewayId is set (has been assigned a value) and false otherwise */
+  public boolean isSetGatewayId() {
+    return this.gatewayId != null;
+  }
+
+  public void setGatewayIdIsSet(boolean value) {
+    if (!value) {
+      this.gatewayId = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case EXPERIMENT_ID:
+      if (value == null) {
+        unsetExperimentId();
+      } else {
+        setExperimentId((String)value);
+      }
+      break;
+
+    case GATEWAY_ID:
+      if (value == null) {
+        unsetGatewayId();
+      } else {
+        setGatewayId((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case EXPERIMENT_ID:
+      return getExperimentId();
+
+    case GATEWAY_ID:
+      return getGatewayId();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case EXPERIMENT_ID:
+      return isSetExperimentId();
+    case GATEWAY_ID:
+      return isSetGatewayId();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof ExperimentSubmitEvent)
+      return this.equals((ExperimentSubmitEvent)that);
+    return false;
+  }
+
+  public boolean equals(ExperimentSubmitEvent that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_experimentId = true && this.isSetExperimentId();
+    boolean that_present_experimentId = true && that.isSetExperimentId();
+    if (this_present_experimentId || that_present_experimentId) {
+      if (!(this_present_experimentId && that_present_experimentId))
+        return false;
+      if (!this.experimentId.equals(that.experimentId))
+        return false;
+    }
+
+    boolean this_present_gatewayId = true && this.isSetGatewayId();
+    boolean that_present_gatewayId = true && that.isSetGatewayId();
+    if (this_present_gatewayId || that_present_gatewayId) {
+      if (!(this_present_gatewayId && that_present_gatewayId))
+        return false;
+      if (!this.gatewayId.equals(that.gatewayId))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_experimentId = true && (isSetExperimentId());
+    list.add(present_experimentId);
+    if (present_experimentId)
+      list.add(experimentId);
+
+    boolean present_gatewayId = true && (isSetGatewayId());
+    list.add(present_gatewayId);
+    if (present_gatewayId)
+      list.add(gatewayId);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(ExperimentSubmitEvent other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetExperimentId()).compareTo(other.isSetExperimentId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetExperimentId()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.experimentId, other.experimentId);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetGatewayId()).compareTo(other.isSetGatewayId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetGatewayId()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.gatewayId, other.gatewayId);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ExperimentSubmitEvent(");
+    boolean first = true;
+
+    sb.append("experimentId:");
+    if (this.experimentId == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.experimentId);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("gatewayId:");
+    if (this.gatewayId == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.gatewayId);
+    }
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetExperimentId()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'experimentId' is unset! Struct:" + toString());
+    }
+
+    if (!isSetGatewayId()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'gatewayId' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class ExperimentSubmitEventStandardSchemeFactory implements SchemeFactory {
+    public ExperimentSubmitEventStandardScheme getScheme() {
+      return new ExperimentSubmitEventStandardScheme();
+    }
+  }
+
+  private static class ExperimentSubmitEventStandardScheme extends StandardScheme<ExperimentSubmitEvent> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, ExperimentSubmitEvent struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // EXPERIMENT_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.experimentId = iprot.readString();
+              struct.setExperimentIdIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // GATEWAY_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.gatewayId = iprot.readString();
+              struct.setGatewayIdIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, ExperimentSubmitEvent struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.experimentId != null) {
+        oprot.writeFieldBegin(EXPERIMENT_ID_FIELD_DESC);
+        oprot.writeString(struct.experimentId);
+        oprot.writeFieldEnd();
+      }
+      if (struct.gatewayId != null) {
+        oprot.writeFieldBegin(GATEWAY_ID_FIELD_DESC);
+        oprot.writeString(struct.gatewayId);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class ExperimentSubmitEventTupleSchemeFactory implements SchemeFactory {
+    public ExperimentSubmitEventTupleScheme getScheme() {
+      return new ExperimentSubmitEventTupleScheme();
+    }
+  }
+
+  private static class ExperimentSubmitEventTupleScheme extends TupleScheme<ExperimentSubmitEvent> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, ExperimentSubmitEvent struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.experimentId);
+      oprot.writeString(struct.gatewayId);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, ExperimentSubmitEvent struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.experimentId = iprot.readString();
+      struct.setExperimentIdIsSet(true);
+      struct.gatewayId = iprot.readString();
+      struct.setGatewayIdIsSet(true);
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/airavata/blob/4e3dc9a9/airavata-api/airavata-data-models/src/main/java/org/apache/airavata/model/messaging/event/MessageType.java
----------------------------------------------------------------------
diff --git a/airavata-api/airavata-data-models/src/main/java/org/apache/airavata/model/messaging/event/MessageType.java b/airavata-api/airavata-data-models/src/main/java/org/apache/airavata/model/messaging/event/MessageType.java
index 48df9b4..59b6f33 100644
--- a/airavata-api/airavata-data-models/src/main/java/org/apache/airavata/model/messaging/event/MessageType.java
+++ b/airavata-api/airavata-data-models/src/main/java/org/apache/airavata/model/messaging/event/MessageType.java
@@ -30,12 +30,13 @@ import org.apache.thrift.TEnum;
 
 public enum MessageType implements org.apache.thrift.TEnum {
   EXPERIMENT(0),
-  TASK(1),
-  PROCESS(2),
-  JOB(3),
-  LAUNCHPROCESS(4),
-  TERMINATEPROCESS(5),
-  PROCESSOUTPUT(6);
+  EXPERIMENT_CANCEL(1),
+  TASK(2),
+  PROCESS(3),
+  JOB(4),
+  LAUNCHPROCESS(5),
+  TERMINATEPROCESS(6),
+  PROCESSOUTPUT(7);
 
   private final int value;
 
@@ -59,16 +60,18 @@ public enum MessageType implements org.apache.thrift.TEnum {
       case 0:
         return EXPERIMENT;
       case 1:
-        return TASK;
+        return EXPERIMENT_CANCEL;
       case 2:
-        return PROCESS;
+        return TASK;
       case 3:
-        return JOB;
+        return PROCESS;
       case 4:
-        return LAUNCHPROCESS;
+        return JOB;
       case 5:
-        return TERMINATEPROCESS;
+        return LAUNCHPROCESS;
       case 6:
+        return TERMINATEPROCESS;
+      case 7:
         return PROCESSOUTPUT;
       default:
         return null;

http://git-wip-us.apache.org/repos/asf/airavata/blob/4e3dc9a9/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessagingFactory.java
----------------------------------------------------------------------
diff --git a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessagingFactory.java b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessagingFactory.java
index 99c11b8..573304a 100644
--- a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessagingFactory.java
+++ b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/MessagingFactory.java
@@ -22,6 +22,7 @@ package org.apache.airavata.messaging.core;
 
 import org.apache.airavata.common.exception.AiravataException;
 import org.apache.airavata.common.utils.ServerSettings;
+import org.apache.airavata.messaging.core.impl.ExperimentConsumer;
 import org.apache.airavata.messaging.core.impl.ProcessConsumer;
 import org.apache.airavata.messaging.core.impl.RabbitMQPublisher;
 import org.apache.airavata.messaging.core.impl.RabbitMQSubscriber;
@@ -45,6 +46,10 @@ public class MessagingFactory {
 
         switch (type) {
             case EXPERIMENT_LAUNCH:
+                subscriber = getExperimentSubscriber(rProperties);
+                subscriber.listen(((connection, channel) -> new ExperimentConsumer(messageHandler, connection, channel)),
+                        null,
+                        routingKeys);
                 break;
             case PROCESS_LAUNCH:
                 subscriber = getProcessSubscriber(rProperties);
@@ -70,6 +75,7 @@ public class MessagingFactory {
         Publisher publiser = null;
         switch (type) {
             case EXPERIMENT_LAUNCH:
+                publiser = getExperimentPublisher(rProperties);
                 break;
             case PROCESS_LAUNCH:
                 publiser = gerProcessPublisher(rProperties);
@@ -84,6 +90,11 @@ public class MessagingFactory {
         return publiser;
     }
 
+    private static Publisher getExperimentPublisher(RabbitMQProperties rProperties) throws AiravataException {
+        rProperties.setExchangeName(ServerSettings.getRabbitmqExperimentExchangeName());
+        return new RabbitMQPublisher(rProperties, messageContext -> rProperties.getExchangeName());
+    }
+
     private static Publisher getStatusPublisher(RabbitMQProperties rProperties) throws AiravataException {
         rProperties.setExchangeName(ServerSettings.getRabbitmqStatusExchangeName());
         return new RabbitMQPublisher(rProperties, MessagingFactory::statusRoutingkey);
@@ -110,7 +121,6 @@ public class MessagingFactory {
         return new RabbitMQSubscriber(sp);
     }
 
-
     private static RabbitMQSubscriber getProcessSubscriber(RabbitMQProperties sp) throws AiravataException {
         sp.setExchangeName(ServerSettings.getRabbitmqProcessExchangeName())
                 .setQueueName(ServerSettings.getRabbitmqProcessLaunchQueueName())
@@ -119,6 +129,14 @@ public class MessagingFactory {
     }
 
 
+    private static Subscriber getExperimentSubscriber(RabbitMQProperties sp) throws AiravataException {
+        sp.setExchangeName(ServerSettings.getRabbitmqExperimentExchangeName())
+                .setAutoAck(false);
+        return new RabbitMQSubscriber(sp);
+
+    }
+
+
     private static String statusRoutingkey(MessageContext msgCtx) {
         String gatewayId = msgCtx.getGatewayId();
         String routingKey = null;

http://git-wip-us.apache.org/repos/asf/airavata/blob/4e3dc9a9/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/ExperimentConsumer.java
----------------------------------------------------------------------
diff --git a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/ExperimentConsumer.java b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/ExperimentConsumer.java
index 058b99e..6e4c46a 100644
--- a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/ExperimentConsumer.java
+++ b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/ExperimentConsumer.java
@@ -22,14 +22,37 @@ package org.apache.airavata.messaging.core.impl;
 
 import com.rabbitmq.client.AMQP;
 import com.rabbitmq.client.Channel;
+import com.rabbitmq.client.Connection;
 import com.rabbitmq.client.Envelope;
 import com.rabbitmq.client.QueueingConsumer;
+import org.apache.airavata.common.utils.AiravataUtils;
+import org.apache.airavata.common.utils.ServerSettings;
+import org.apache.airavata.common.utils.ThriftUtils;
+import org.apache.airavata.messaging.core.MessageContext;
+import org.apache.airavata.messaging.core.MessageHandler;
+import org.apache.airavata.model.messaging.event.ExperimentSubmitEvent;
+import org.apache.airavata.model.messaging.event.Message;
+import org.apache.airavata.model.messaging.event.MessageType;
+import org.apache.airavata.model.messaging.event.ProcessSubmitEvent;
+import org.apache.thrift.TBase;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 
 public class ExperimentConsumer extends QueueingConsumer {
-    public ExperimentConsumer(Channel ch) {
-        super(ch);
+    private static final Logger log = LoggerFactory.getLogger(ExperimentConsumer.class);
+
+    private MessageHandler handler;
+    private Channel channel;
+    private Connection connection;
+
+    public ExperimentConsumer(MessageHandler messageHandler, Connection connection, Channel channel) {
+        super(channel);
+        this.handler = messageHandler;
+        this.connection = connection;
+        this.channel = channel;
     }
 
 
@@ -38,5 +61,52 @@ public class ExperimentConsumer extends QueueingConsumer {
                                Envelope envelope,
                                AMQP.BasicProperties properties,
                                byte[] body) throws IOException {
+
+        Message message = new Message();
+
+        try {
+            ThriftUtils.createThriftFromBytes(body, message);
+            long deliveryTag = envelope.getDeliveryTag();
+            if (message.getMessageType() == MessageType.EXPERIMENT || message.getMessageType() == MessageType.EXPERIMENT_CANCEL) {
+                TBase event = null;
+                String gatewayId = null;
+                ExperimentSubmitEvent experimentEvent = new ExperimentSubmitEvent();
+                ThriftUtils.createThriftFromBytes(message.getEvent(), experimentEvent);
+                log.debug(" Message Received with message id '" + message.getMessageId()
+                        + "' and with message type '" + message.getMessageType() + "'  for experimentId:" +
+                        " " +
+                        experimentEvent.getExperimentId());
+                event = experimentEvent;
+                gatewayId = experimentEvent.getGatewayId();
+                MessageContext messageContext = new MessageContext(event, message.getMessageType(),
+                        message.getMessageId(), gatewayId, deliveryTag);
+                messageContext.setUpdatedTime(AiravataUtils.getTime(message.getUpdatedTime()));
+                messageContext.setIsRedeliver(envelope.isRedeliver());
+                handler.onMessage(messageContext);
+            } else {
+                log.error("{} message type is not handle in ProcessLaunch Subscriber. Sending ack for " +
+                        "delivery tag {} ", message.getMessageType().name(), deliveryTag);
+                sendAck(deliveryTag);
+            }
+        } catch (TException e) {
+            String msg = "Failed to de-serialize the thrift message, from routing keys:" + envelope.getRoutingKey();
+            log.warn(msg, e);
+        }
+
+    }
+
+
+    private void sendAck(long deliveryTag){
+        try {
+            if (channel.isOpen()){
+                channel.basicAck(deliveryTag,false);
+            }else {
+                channel = connection.createChannel();
+                channel.basicQos(ServerSettings.getRabbitmqPrefetchCount());
+                channel.basicAck(deliveryTag, false);
+            }
+        } catch (IOException e) {
+            log.error(e.getMessage(), e);
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/airavata/blob/4e3dc9a9/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/ProcessConsumer.java
----------------------------------------------------------------------
diff --git a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/ProcessConsumer.java b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/ProcessConsumer.java
index 368c100..e95a7ca 100644
--- a/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/ProcessConsumer.java
+++ b/modules/messaging/core/src/main/java/org/apache/airavata/messaging/core/impl/ProcessConsumer.java
@@ -48,17 +48,13 @@ public class ProcessConsumer extends QueueingConsumer{
     private Connection connection;
 
     public ProcessConsumer(MessageHandler messageHandler, Connection connection, Channel channel){
-        this(channel);
+        super(channel);
         this.handler = messageHandler;
         this.connection = connection;
         this.channel = channel;
     }
 
 
-    private ProcessConsumer(Channel ch) {
-        super(ch);
-    }
-
     @Override
     public void handleDelivery(String consumerTag,
                                Envelope envelope,

http://git-wip-us.apache.org/repos/asf/airavata/blob/4e3dc9a9/modules/orchestrator/orchestrator-core/src/main/java/org/apache/airavata/orchestrator/core/impl/GFACPassiveJobSubmitter.java
----------------------------------------------------------------------
diff --git a/modules/orchestrator/orchestrator-core/src/main/java/org/apache/airavata/orchestrator/core/impl/GFACPassiveJobSubmitter.java b/modules/orchestrator/orchestrator-core/src/main/java/org/apache/airavata/orchestrator/core/impl/GFACPassiveJobSubmitter.java
index 856f9f3..3438475 100644
--- a/modules/orchestrator/orchestrator-core/src/main/java/org/apache/airavata/orchestrator/core/impl/GFACPassiveJobSubmitter.java
+++ b/modules/orchestrator/orchestrator-core/src/main/java/org/apache/airavata/orchestrator/core/impl/GFACPassiveJobSubmitter.java
@@ -96,7 +96,7 @@ public class GFACPassiveJobSubmitter implements JobSubmitter,Watcher {
 	        ProcessSubmitEvent processSubmitEvent = new ProcessSubmitEvent(processId, gatewayId, experimentId,
 			        tokenId);
 	        MessageContext messageContext = new MessageContext(processSubmitEvent, MessageType.LAUNCHPROCESS, "LAUNCH" +
-			        ".TASK-" + UUID.randomUUID().toString(), gatewayId);
+			        ".PROCESS-" + UUID.randomUUID().toString(), gatewayId);
 	        messageContext.setUpdatedTime(AiravataUtils.getCurrentTimestamp());
             publisher.publish(messageContext);
         } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/airavata/blob/4e3dc9a9/thrift-interface-descriptions/airavata-apis/messaging_events.thrift
----------------------------------------------------------------------
diff --git a/thrift-interface-descriptions/airavata-apis/messaging_events.thrift b/thrift-interface-descriptions/airavata-apis/messaging_events.thrift
index 7ffc88d..2533752 100644
--- a/thrift-interface-descriptions/airavata-apis/messaging_events.thrift
+++ b/thrift-interface-descriptions/airavata-apis/messaging_events.thrift
@@ -36,6 +36,7 @@ enum MessageLevel {
 
 enum MessageType {
     EXPERIMENT,
+    EXPERIMENT_CANCEL,
     TASK,
     PROCESS,
     JOB,
@@ -108,6 +109,11 @@ struct JobIdentifier {
 //    //8:
 // }
 
+struct ExperimentSubmitEvent{
+    1: required string experimentId,
+    2: required string gatewayId,
+}
+
 struct ProcessSubmitEvent{
     1: required string processId,
     2: required string gatewayId,


[27/50] [abbrv] airavata git commit: Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/airavata into develop

Posted by la...@apache.org.
Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/airavata into develop


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: b168933a9443e29af21b4c3d280892ab5c71758c
Parents: 3f29cfd c6a82a0
Author: Shameera Rathnayaka <sh...@gmail.com>
Authored: Mon Aug 15 16:39:56 2016 -0400
Committer: Shameera Rathnayaka <sh...@gmail.com>
Committed: Mon Aug 15 16:39:56 2016 -0400

----------------------------------------------------------------------
 .../airavata/registry/core/Committer.java       |  29 ----
 .../apache/airavata/registry/core/JPAUtil.java  |  57 -------
 .../catalog/resources/ExperimentResource.java   | 148 +++++++++++++------
 3 files changed, 100 insertions(+), 134 deletions(-)
----------------------------------------------------------------------



[32/50] [abbrv] airavata git commit: setting default gateway request cretion time

Posted by la...@apache.org.
setting default gateway request cretion time


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

Branch: refs/heads/lahiru/AIRAVATA-2057
Commit: 67102da4443e92d99d5750c1fa204e87fe6e8a30
Parents: c6a82a0
Author: scnakandala <su...@gmail.com>
Authored: Tue Aug 16 21:14:08 2016 -0400
Committer: scnakandala <su...@gmail.com>
Committed: Tue Aug 16 21:14:08 2016 -0400

----------------------------------------------------------------------
 .../registry/core/experiment/catalog/impl/GatewayRegistry.java    | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/airavata/blob/67102da4/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/impl/GatewayRegistry.java
----------------------------------------------------------------------
diff --git a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/impl/GatewayRegistry.java b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/impl/GatewayRegistry.java
index 37d8820..515b83b 100644
--- a/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/impl/GatewayRegistry.java
+++ b/modules/registry/registry-core/src/main/java/org/apache/airavata/registry/core/experiment/catalog/impl/GatewayRegistry.java
@@ -66,7 +66,7 @@ public class GatewayRegistry {
             resource.setDeclinedReason(gateway.getDeclinedReason());
             resource.setOauthClientId(gateway.getOauthClientId());
             resource.setOauthClientSecret(gateway.getOauthClientSecret());
-            resource.setRequestCreationTime(new Timestamp(gateway.getRequestCreationTime()));
+            resource.setRequestCreationTime(new Timestamp(System.currentTimeMillis()));
             resource.save();
             return gateway.getGatewayId();
         }catch (RegistryException e){
@@ -94,7 +94,6 @@ public class GatewayRegistry {
             existingGateway.setDeclinedReason(updatedGateway.getDeclinedReason());
             existingGateway.setOauthClientId(updatedGateway.getOauthClientId());
             existingGateway.setOauthClientSecret(updatedGateway.getOauthClientSecret());
-            existingGateway.setRequestCreationTime(new Timestamp(updatedGateway.getRequestCreationTime()));
             existingGateway.save();
         }catch (RegistryException e){
             logger.error("Error while updating gateway to registry", e);