You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@activemq.apache.org by cl...@apache.org on 2015/07/29 23:22:29 UTC

[23/24] activemq-artemis git commit: ARTEMIS-178 Refactor examples to use CLI

ARTEMIS-178 Refactor examples to use CLI

This is changing the examples to run the real servers.
The maven plugin should be an automation of the CLI
Where we run real servers instead of any embedded framework

Going forward we will need to remove the common examples.
Failover examples will be able to use Process Builders


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

Branch: refs/heads/master
Commit: b3af4bb77751996419868be9784ba73f1c7e72fe
Parents: c8d053e
Author: Clebert Suconic <cl...@apache.org>
Authored: Tue May 12 17:59:02 2015 -0400
Committer: jbertram <jb...@apache.org>
Committed: Wed Jul 29 16:00:41 2015 -0500

----------------------------------------------------------------------
 .../apache/activemq/artemis/cli/Artemis.java    |   6 +
 .../activemq/artemis/cli/commands/Create.java   |  29 +-
 .../activemq/artemis/cli/commands/Run.java      |   2 +
 .../artemis/cli/process/ProcessBuilder.java     | 192 +++++++++++
 .../artemis/cli/process/package-info.java       |  19 ++
 .../cli/commands/etc/replicated-settings.txt    |   2 +-
 .../cli/commands/etc/shared-store-settings.txt  |   2 +-
 .../src/main/resources/bin/artemis              |   2 +-
 artemis-maven-plugin/pom.xml                    |  20 ++
 .../activemq/artemis/maven/ActiveMQClient.java  |  22 --
 .../artemis/maven/ActiveMQClientPlugin.java     |  19 +-
 .../artemis/maven/ActiveMQCreatePlugin.java     | 331 +++++++++++++++++++
 .../artemis/maven/ActiveMQStartPlugin.java      | 230 -------------
 .../artemis/maven/ActiveMQStopPlugin.java       |  76 -----
 .../artemis/maven/ArtemisCLIPlugin.java         | 172 ++++++++++
 .../artemis/maven/InVMNodeManagerServer.java    |  75 -----
 .../artemis/maven/LibInstallPlugin.java         | 147 ++++++++
 .../activemq/artemis/maven/PluginUtil.java      |  50 ---
 .../artemis/maven/TestClusterManager.java       |  52 ---
 .../artemis/maven/TestClusterManagerMBean.java  |  29 --
 .../apache/activemq/artemis/maven/TestNode.java |  49 ---
 .../artemis/server/ActiveMQBootstrap.java       | 270 ---------------
 .../server/SpawnedActiveMQBootstrap.java        |  38 ---
 .../artemis/server/SpawnedVMSupport.java        | 252 --------------
 examples/jms/aerogear/pom.xml                   | 137 ++++----
 .../artemis/jms/example/AerogearExample.java    |  16 +-
 .../activemq/server0/artemis-roles.properties   |  17 -
 .../activemq/server0/artemis-users.properties   |  17 -
 .../main/resources/activemq/server0/broker.xml  |  10 +-
 examples/jms/application-layer-failover/pom.xml |  77 +----
 .../ApplicationLayerFailoverExample.java        |   6 +-
 .../activemq/server0/artemis-roles.properties   |  17 -
 .../activemq/server0/artemis-users.properties   |  17 -
 .../main/resources/activemq/server0/broker.xml  |  53 ---
 .../activemq/server1/artemis-roles.properties   |  17 -
 .../activemq/server1/artemis-users.properties   |  17 -
 .../main/resources/activemq/server1/broker.xml  |  56 ----
 examples/jms/bridge/pom.xml                     |  65 +---
 .../artemis/jms/example/BridgeExample.java      |   3 +-
 .../activemq/server0/artemis-roles.properties   |  17 -
 .../activemq/server0/artemis-users.properties   |  17 -
 .../activemq/server1/artemis-roles.properties   |  17 -
 .../activemq/server1/artemis-users.properties   |  17 -
 examples/jms/browser/pom.xml                    |  53 +--
 .../jms/example/QueueBrowserExample.java        |   3 +-
 .../activemq/server0/artemis-roles.properties   |  17 -
 .../activemq/server0/artemis-users.properties   |  17 -
 .../main/resources/activemq/server0/broker.xml  |  61 ----
 examples/jms/client-kickoff/pom.xml             |  72 +---
 .../jms/example/ClientKickoffExample.java       |   5 +-
 .../activemq/server0/artemis-roles.properties   |  17 -
 .../activemq/server0/artemis-users.properties   |  17 -
 .../jms/client-side-failoverlistener/pom.xml    | 107 +-----
 .../ClientSideFailoverListerExample.java        |  14 +-
 .../activemq/server0/artemis-roles.properties   |  17 -
 .../activemq/server0/artemis-users.properties   |  17 -
 .../main/resources/activemq/server0/broker.xml  |  92 ------
 .../activemq/server1/artemis-roles.properties   |  17 -
 .../activemq/server1/artemis-users.properties   |  17 -
 .../main/resources/activemq/server1/broker.xml  |  93 ------
 examples/jms/client-side-load-balancing/pom.xml | 128 ++-----
 .../activemq/server0/artemis-roles.properties   |  17 -
 .../activemq/server0/artemis-users.properties   |  17 -
 .../main/resources/activemq/server0/broker.xml  |  92 ------
 .../activemq/server1/artemis-roles.properties   |  17 -
 .../activemq/server1/artemis-users.properties   |  17 -
 .../main/resources/activemq/server1/broker.xml  |  89 -----
 .../activemq/server2/artemis-roles.properties   |  17 -
 .../activemq/server2/artemis-users.properties   |  17 -
 .../main/resources/activemq/server2/broker.xml  |  87 -----
 .../jms/clustered-durable-subscription/pom.xml  |  95 +-----
 .../ClusteredDurableSubscriptionExample.java    |   7 +-
 .../activemq/server0/artemis-roles.properties   |  17 -
 .../activemq/server0/artemis-users.properties   |  17 -
 .../activemq/server1/artemis-roles.properties   |  17 -
 .../activemq/server1/artemis-users.properties   |  17 -
 examples/jms/clustered-grouping/pom.xml         | 133 +-------
 .../jms/example/ClusteredGroupingExample.java   |   9 +-
 .../activemq/server0/artemis-roles.properties   |  17 -
 .../activemq/server0/artemis-users.properties   |  17 -
 .../main/resources/activemq/server0/broker.xml  |   9 +-
 .../activemq/server1/artemis-roles.properties   |  17 -
 .../activemq/server1/artemis-users.properties   |  17 -
 .../main/resources/activemq/server1/broker.xml  |   9 +-
 .../activemq/server2/artemis-roles.properties   |  17 -
 .../activemq/server2/artemis-users.properties   |  17 -
 .../main/resources/activemq/server2/broker.xml  |   9 +-
 examples/jms/clustered-jgroups/pom.xml          |  95 ++----
 .../jms/example/ClusteredJgroupsExample.java    |   7 +-
 .../activemq/server0/artemis-roles.properties   |  17 -
 .../activemq/server0/artemis-users.properties   |  17 -
 .../activemq/server0/test-jgroups-file_ping.xml |   3 +-
 .../activemq/server1/artemis-roles.properties   |  17 -
 .../activemq/server1/artemis-users.properties   |  17 -
 .../activemq/server1/test-jgroups-file_ping.xml |   3 +-
 examples/jms/clustered-queue/pom.xml            |  83 +----
 .../jms/example/ClusteredQueueExample.java      |   7 +-
 examples/jms/clustered-standalone/pom.xml       | 112 +------
 .../jms/example/ClusteredStandaloneExample.java |  11 +-
 examples/jms/clustered-static-discovery/pom.xml | 123 ++-----
 .../example/StaticClusteredQueueExample.java    |   5 +-
 examples/jms/clustered-static-oneway/pom.xml    | 101 +-----
 .../jms/example/ClusterStaticOnewayExample.java |   5 +-
 examples/jms/clustered-topic/pom.xml            | 102 +-----
 .../jms/example/ClusteredTopicExample.java      |   7 +-
 .../jms/colocated-failover-scale-down/pom.xml   |  88 +----
 .../ColocatedFailoverScaleDownExample.java      |  11 +-
 .../main/resources/activemq/server0/broker.xml  |   8 +-
 .../main/resources/activemq/server1/broker.xml  |   8 +-
 examples/jms/colocated-failover/pom.xml         |  88 +----
 .../jms/example/ColocatedFailoverExample.java   |  11 +-
 .../main/resources/activemq/server0/broker.xml  |   8 +-
 .../main/resources/activemq/server1/broker.xml  |   8 +-
 examples/jms/common/pom.xml                     |   5 +
 .../artemis/common/example/ActiveMQExample.java | 222 ++++++++-----
 .../artemis/common/example/ExampleUtil.java     | 122 +++++++
 examples/jms/consumer-rate-limit/pom.xml        |  54 +--
 examples/jms/dead-letter/pom.xml                |  54 +--
 examples/jms/delayed-redelivery/pom.xml         |  56 +---
 examples/jms/divert/pom.xml                     |  82 +----
 .../artemis/jms/example/DivertExample.java      |   7 +-
 examples/jms/durable-subscription/pom.xml       |  54 +--
 examples/jms/embedded-simple/pom.xml            |  44 ---
 .../artemis/jms/example/EmbeddedExample.java    |  11 +-
 examples/jms/embedded/pom.xml                   |  44 ---
 .../artemis/jms/example/EmbeddedExample.java    |   7 +-
 examples/jms/expiry/pom.xml                     |  54 +--
 examples/jms/ha-policy-autobackup/pom.xml       |  88 +----
 .../jms/example/HAPolicyAutoBackupExample.java  |  28 +-
 .../main/resources/activemq/server0/broker.xml  |   8 +-
 .../main/resources/activemq/server1/broker.xml  |   8 +-
 examples/jms/http-transport/pom.xml             |  60 +---
 .../main/resources/activemq/server0/broker.xml  |   2 +
 .../jms/instantiate-connection-factory/pom.xml  |  54 +--
 .../InstantiateConnectionFactoryExample.java    |   7 +-
 .../main/resources/activemq/server0/broker.xml  |   2 +-
 examples/jms/interceptor/pom.xml                |  56 +---
 examples/jms/jms-auto-closeable/pom.xml         |  55 +--
 examples/jms/jms-bridge/pom.xml                 |  82 +----
 .../artemis/jms/example/JMSBridgeExample.java   |  34 +-
 .../main/resources/activemq/server1/broker.xml  |   3 +-
 examples/jms/jms-completion-listener/pom.xml    |  57 +---
 examples/jms/jms-context/pom.xml                |  55 +--
 examples/jms/jms-shared-consumer/pom.xml        |  54 +--
 .../jms/example/JMSSharedConsumerExample.java   |   5 +-
 examples/jms/jmx/pom.xml                        |  77 +----
 examples/jms/large-message/pom.xml              |  63 +---
 .../jms/example/LargeMessageExample.java        |  15 +-
 examples/jms/last-value-queue/pom.xml           |  57 +---
 examples/jms/management-notifications/pom.xml   |  57 +---
 examples/jms/management/pom.xml                 |  55 +--
 examples/jms/message-counters/pom.xml           |  77 +----
 examples/jms/message-group/pom.xml              |  57 +---
 examples/jms/message-group2/pom.xml             |  55 +--
 examples/jms/message-priority/pom.xml           |  57 +---
 examples/jms/multiple-failover-failback/pom.xml | 132 +-------
 .../MultipleFailoverFailbackExample.java        |  16 +-
 .../main/resources/activemq/server0/broker.xml  |   8 +-
 .../main/resources/activemq/server1/broker.xml  |   8 +-
 .../main/resources/activemq/server2/broker.xml  |   8 +-
 examples/jms/multiple-failover/pom.xml          | 130 +-------
 .../jms/example/MultipleFailoverExample.java    |  19 +-
 .../main/resources/activemq/server0/broker.xml  |   8 +-
 .../main/resources/activemq/server1/broker.xml  |   8 +-
 .../main/resources/activemq/server2/broker.xml  |   8 +-
 examples/jms/no-consumer-buffering/pom.xml      |  55 +--
 examples/jms/non-transaction-failover/pom.xml   |  98 +-----
 .../example/NonTransactionFailoverExample.java  |  13 +-
 .../main/resources/activemq/server0/broker.xml  |   8 +-
 .../main/resources/activemq/server1/broker.xml  |   8 +-
 examples/jms/openwire/pom.xml                   |  60 +---
 .../main/resources/activemq/server0/broker.xml  |   4 +-
 examples/jms/paging/pom.xml                     |  55 +--
 examples/jms/perf/pom.xml                       |  90 ++---
 .../activemq/artemis/jms/example/Server.java    |  28 ++
 .../jms/perf/src/main/resources/perf.properties |   6 +-
 examples/jms/pre-acknowledge/pom.xml            |  54 +--
 examples/jms/producer-rate-limit/pom.xml        |  55 +--
 examples/jms/proton-cpp/pom.xml                 |  61 +---
 .../artemis/jms/example/ProtonCPPExample.java   |   3 -
 examples/jms/proton-j/pom.xml                   |  59 +---
 .../main/resources/activemq/server0/broker.xml  |   1 +
 examples/jms/proton-ruby/pom.xml                |  52 +--
 examples/jms/proton-ruby/readme.html            |   3 +-
 .../jms/queue-message-redistribution/pom.xml    | 102 +-----
 .../QueueMessageRedistributionExample.java      |   7 +-
 .../activemq/server0/artemis-roles.properties   |  17 -
 .../activemq/server0/artemis-users.properties   |  17 -
 .../activemq/server1/artemis-roles.properties   |  17 -
 .../activemq/server1/artemis-users.properties   |  17 -
 examples/jms/queue-requestor/pom.xml            |  55 +--
 examples/jms/queue-selector/pom.xml             |  55 +--
 examples/jms/queue/pom.xml                      |  56 +---
 examples/jms/queue/readme.html                  |   1 +
 .../activemq/server0/artemis-roles.properties   |  17 -
 .../activemq/server0/artemis-users.properties   |  17 -
 .../main/resources/activemq/server0/broker.xml  |  60 ----
 examples/jms/reattach-node/pom.xml              |  64 +---
 examples/jms/replicated-failback-static/pom.xml | 104 +-----
 examples/jms/replicated-failback/pom.xml        | 104 +-----
 .../jms/example/ReplicatedFailbackExample.java  |   6 +
 .../main/resources/activemq/server0/broker.xml  |  10 +-
 .../main/resources/activemq/server1/broker.xml  |  10 +-
 .../jms/replicated-multiple-failover/pom.xml    | 133 +-------
 .../jms/replicated-transaction-failover/pom.xml | 104 +-----
 examples/jms/request-reply/pom.xml              |  55 +--
 examples/jms/scale-down/pom.xml                 |  90 +----
 .../artemis/jms/example/ScaleDownExample.java   |   8 +-
 .../main/resources/activemq/server0/broker.xml  |   8 +-
 .../main/resources/activemq/server1/broker.xml  |   8 +-
 examples/jms/scheduled-message/pom.xml          |  55 +--
 examples/jms/security/pom.xml                   |  55 +--
 examples/jms/send-acknowledgements/pom.xml      |  55 +--
 examples/jms/spring-integration/pom.xml         |  37 +--
 examples/jms/ssl-enabled/pom.xml                |  55 +--
 examples/jms/static-selector-jms/pom.xml        |  54 +--
 examples/jms/static-selector/pom.xml            |  55 +--
 examples/jms/stomp-websockets/pom.xml           |  68 +---
 examples/jms/stomp/pom.xml                      |  60 +---
 examples/jms/stomp1.1/pom.xml                   |  60 +---
 examples/jms/stomp1.2/pom.xml                   |  60 +---
 examples/jms/stop-server-failover/pom.xml       | 104 +-----
 examples/jms/symmetric-cluster/pom.xml          | 225 +++----------
 .../jms/example/SymmetricClusterExample.java    |  12 +-
 examples/jms/temp-queue/pom.xml                 |  55 +--
 examples/jms/topic-hierarchies/pom.xml          |  55 +--
 examples/jms/topic-selector-example1/pom.xml    |  55 +--
 examples/jms/topic-selector-example2/pom.xml    |  55 +--
 examples/jms/topic/pom.xml                      |  54 +--
 examples/jms/transaction-failover/pom.xml       | 104 +-----
 .../main/resources/activemq/server0/broker.xml  |   8 +-
 .../main/resources/activemq/server1/broker.xml  |   8 +-
 examples/jms/transactional/pom.xml              |  55 +--
 examples/jms/xa-heuristic/pom.xml               |  76 +----
 .../artemis/jms/example/XAHeuristicExample.java |  12 +-
 examples/jms/xa-receive/pom.xml                 |  58 +---
 .../artemis/jms/example/XAReceiveExample.java   |  10 +-
 .../artemis/jms/example/XASendExample.java      |  10 +-
 examples/soak/normal/README                     |  21 +-
 examples/soak/normal/pom.xml                    | 125 +++----
 examples/soak/normal/server0/broker.xml         |   5 +-
 .../artemis/jms/soak/example/SoakReceiver.java  |  31 +-
 .../artemis/jms/soak/example/SoakSender.java    |  27 +-
 .../normal/src/main/resources/jndi.properties   |  20 ++
 244 files changed, 2501 insertions(+), 8758 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/b3af4bb7/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/Artemis.java
----------------------------------------------------------------------
diff --git a/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/Artemis.java b/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/Artemis.java
index 1b09ea4..edcdce0 100644
--- a/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/Artemis.java
+++ b/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/Artemis.java
@@ -19,6 +19,7 @@ package org.apache.activemq.artemis.cli;
 import java.io.File;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.util.List;
 
 import io.airlift.airline.Cli;
 import org.apache.activemq.artemis.cli.commands.Action;
@@ -70,6 +71,11 @@ public class Artemis
       return execute(null, null, args);
    }
 
+   public static Object execute(File artemisHome, File artemisInstance, List<String> args) throws Exception
+   {
+      return execute(artemisHome, artemisInstance, (String[]) args.toArray(new String[args.size()]));
+   }
+
    public static Object execute(File artemisHome, File artemisInstance, String... args) throws Exception
    {
       Action action = builder(artemisInstance).build().parse(args);

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/b3af4bb7/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/Create.java
----------------------------------------------------------------------
diff --git a/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/Create.java b/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/Create.java
index b349ad1..580b411 100644
--- a/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/Create.java
+++ b/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/Create.java
@@ -118,6 +118,9 @@ public class Create extends InputAbstract
    @Option(name = "--shared-store", description = "Enable broker shared store")
    boolean sharedStore = false;
 
+   @Option(name = "--slave", description = "Valid for shared store or replication: this is a slave server?")
+   boolean slave;
+
    @Option(name = "--cluster-user", description = "The cluster user to use for clustering. (Default: input)")
    String clusterUser = null;
 
@@ -411,6 +414,26 @@ public class Create extends InputAbstract
       this.role = role;
    }
 
+   public boolean isSlave()
+   {
+      return slave;
+   }
+
+   public void setSlave(boolean slave)
+   {
+      this.slave = slave;
+   }
+
+   public Boolean getAllowAnonymous()
+   {
+      return allowAnonymous;
+   }
+
+   public void setAllowAnonymous(Boolean allowAnonymous)
+   {
+      this.allowAnonymous = allowAnonymous;
+   }
+
    @Override
    public Object execute(ActionContext context) throws Exception
    {
@@ -469,10 +492,12 @@ public class Create extends InputAbstract
 
       HashMap<String, String> filters = new HashMap<String, String>();
 
+      filters.put("${master-slave}", isSlave() ? "slave" : "master");
+
       if (replicated)
       {
          clustered = true;
-         filters.put("${replicated.settings}", readTextFile(ETC_REPLICATED_SETTINGS_TXT));
+         filters.put("${replicated.settings}", applyFilters(readTextFile(ETC_REPLICATED_SETTINGS_TXT), filters));
       }
       else
       {
@@ -482,7 +507,7 @@ public class Create extends InputAbstract
       if (sharedStore)
       {
          clustered = true;
-         filters.put("${shared-store.settings}", readTextFile(ETC_SHARED_STORE_SETTINGS_TXT));
+         filters.put("${shared-store.settings}", applyFilters(readTextFile(ETC_SHARED_STORE_SETTINGS_TXT), filters));
       }
       else
       {

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/b3af4bb7/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/Run.java
----------------------------------------------------------------------
diff --git a/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/Run.java b/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/Run.java
index e9b7705..67824ce 100644
--- a/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/Run.java
+++ b/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/commands/Run.java
@@ -164,6 +164,8 @@ public class Run extends Configurable
                }
                finally
                {
+                  System.out.println("Server stopped!");
+                  System.out.flush();
                   latchRunning.countDown();
                   if (!embedded)
                   {

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/b3af4bb7/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/process/ProcessBuilder.java
----------------------------------------------------------------------
diff --git a/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/process/ProcessBuilder.java b/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/process/ProcessBuilder.java
new file mode 100644
index 0000000..05d09d5
--- /dev/null
+++ b/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/process/ProcessBuilder.java
@@ -0,0 +1,192 @@
+/*
+ * 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.activemq.artemis.cli.process;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+
+import org.apache.activemq.artemis.utils.ConcurrentHashSet;
+
+public class ProcessBuilder
+{
+   static ConcurrentHashSet<Process> processes = new ConcurrentHashSet<>();
+
+   static
+   {
+      Runtime.getRuntime().addShutdownHook(new Thread()
+      {
+         public void run()
+         {
+            for (Process p : processes)
+            {
+               if (p.isAlive())
+               {
+                  p.destroy();
+               }
+            }
+         }
+      });
+   }
+
+
+   /**
+    * it will lookup for process that are dead already, eliminating leaks.
+    */
+   public static void cleanupProcess()
+   {
+      for (Process p: processes)
+      {
+         if (!p.isAlive())
+         {
+            processes.remove(p);
+         }
+      }
+   }
+
+
+
+   /**
+    * *
+    * @param logname the prefix for log output
+    * @param location The location where this command is being executed from
+    * @param hook it will finish the process upon shutdown of the VM
+    * @param args The arguments being passwed to the the CLI tool
+    * @return
+    * @throws Exception
+    */
+   public static Process build(String logname, File location, boolean hook, String... args) throws Exception
+   {
+      boolean IS_WINDOWS = System.getProperty("os.name").toLowerCase().trim().startsWith("win");
+
+      String[] newArgs;
+      if (IS_WINDOWS)
+      {
+         newArgs = rebuildArgs(args, "cmd", "/c", "artemis.cmd");
+      }
+      else
+      {
+         newArgs = rebuildArgs(args, "./artemis");
+      }
+
+      java.lang.ProcessBuilder builder = new java.lang.ProcessBuilder(newArgs);
+
+      builder.directory(new File(location, "bin"));
+
+      Process process = builder.start();
+
+      ProcessLogger outputLogger = new ProcessLogger(true,
+                                                     process.getInputStream(),
+                                                     logname + "::Out",
+                                                     false);
+      outputLogger.start();
+
+      // Adding a reader to System.err, so the VM won't hang on a System.err.println as identified on this forum thread:
+      ProcessLogger errorLogger = new ProcessLogger(true,
+                                                    process.getErrorStream(),
+                                                    logname + "::Err",
+                                                    true);
+      errorLogger.start();
+
+      processes.add(process);
+
+      cleanupProcess();
+
+      return process;
+   }
+
+   public static String[] rebuildArgs(String[] args, String ... prefixArgs)
+   {
+      String[] resultArgs = new String[args.length + prefixArgs.length];
+
+      int i = 0;
+
+      for (String arg: prefixArgs)
+      {
+         resultArgs[i++] = arg;
+      }
+
+      for (String arg: args)
+      {
+         resultArgs[i++] = arg;
+      }
+
+      return resultArgs;
+   }
+
+
+   /**
+    * Redirect the input stream to a logger (as debug logs)
+    */
+   static class ProcessLogger extends Thread
+   {
+      private final InputStream is;
+
+      private final String logName;
+
+      private final boolean print;
+
+      private final boolean sendToErr;
+
+      boolean failed = false;
+
+      ProcessLogger(final boolean print,
+                    final InputStream is,
+                    final String logName,
+                    final boolean sendToErr) throws ClassNotFoundException
+      {
+         this.is = is;
+         this.print = print;
+         this.logName = logName;
+         this.sendToErr = sendToErr;
+         setDaemon(false);
+      }
+
+      @Override
+      public void run()
+      {
+         try
+         {
+            InputStreamReader isr = new InputStreamReader(is);
+            BufferedReader br = new BufferedReader(isr);
+            String line;
+            while ((line = br.readLine()) != null)
+            {
+               if (print)
+               {
+                  if (sendToErr)
+                  {
+                     System.err.println(logName + " err:" + line);
+                  }
+                  else
+                  {
+                     System.out.println(logName + " out:" + line);
+                  }
+               }
+            }
+         }
+         catch (IOException e)
+         {
+            // ok, stream closed
+         }
+
+      }
+   }
+
+}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/b3af4bb7/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/process/package-info.java
----------------------------------------------------------------------
diff --git a/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/process/package-info.java b/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/process/package-info.java
new file mode 100644
index 0000000..d902bbd
--- /dev/null
+++ b/artemis-cli/src/main/java/org/apache/activemq/artemis/cli/process/package-info.java
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+
+/** Contains useful classes for spawning process from client classes */
+package org.apache.activemq.artemis.cli.process;

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/b3af4bb7/artemis-cli/src/main/resources/org/apache/activemq/artemis/cli/commands/etc/replicated-settings.txt
----------------------------------------------------------------------
diff --git a/artemis-cli/src/main/resources/org/apache/activemq/artemis/cli/commands/etc/replicated-settings.txt b/artemis-cli/src/main/resources/org/apache/activemq/artemis/cli/commands/etc/replicated-settings.txt
index db9faa0..17a2bc3 100644
--- a/artemis-cli/src/main/resources/org/apache/activemq/artemis/cli/commands/etc/replicated-settings.txt
+++ b/artemis-cli/src/main/resources/org/apache/activemq/artemis/cli/commands/etc/replicated-settings.txt
@@ -1,6 +1,6 @@
 
       <ha-policy>
          <replication>
-            <master/>
+            <${master-slave}/>
          </replication>
       </ha-policy>

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/b3af4bb7/artemis-cli/src/main/resources/org/apache/activemq/artemis/cli/commands/etc/shared-store-settings.txt
----------------------------------------------------------------------
diff --git a/artemis-cli/src/main/resources/org/apache/activemq/artemis/cli/commands/etc/shared-store-settings.txt b/artemis-cli/src/main/resources/org/apache/activemq/artemis/cli/commands/etc/shared-store-settings.txt
index 5901938..750dbc1 100644
--- a/artemis-cli/src/main/resources/org/apache/activemq/artemis/cli/commands/etc/shared-store-settings.txt
+++ b/artemis-cli/src/main/resources/org/apache/activemq/artemis/cli/commands/etc/shared-store-settings.txt
@@ -1,6 +1,6 @@
 
       <ha-policy>
          <shared-store>
-            <master/>
+            <${master-slave}/>
          </shared-store>
       </ha-policy>

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/b3af4bb7/artemis-distribution/src/main/resources/bin/artemis
----------------------------------------------------------------------
diff --git a/artemis-distribution/src/main/resources/bin/artemis b/artemis-distribution/src/main/resources/bin/artemis
index a91cf5c..296fb29 100755
--- a/artemis-distribution/src/main/resources/bin/artemis
+++ b/artemis-distribution/src/main/resources/bin/artemis
@@ -105,4 +105,4 @@ exec "$JAVACMD" $JAVA_ARGS $ARTEMIS_CLUSTER_PROPS \
     -Dartemis.home="$ARTEMIS_HOME" \
     -Djava.library.path="$ARTEMIS_HOME/bin/lib/linux-i686:$ARTEMIS_HOME/bin/lib/linux-x86_64" \
     $DEBUG_ARGS \
-    org.apache.activemq.artemis.boot.Artemis $@
+    org.apache.activemq.artemis.boot.Artemis "$@"

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/b3af4bb7/artemis-maven-plugin/pom.xml
----------------------------------------------------------------------
diff --git a/artemis-maven-plugin/pom.xml b/artemis-maven-plugin/pom.xml
index 9d086ad..960a46e 100644
--- a/artemis-maven-plugin/pom.xml
+++ b/artemis-maven-plugin/pom.xml
@@ -43,6 +43,16 @@
          <version>2.0</version>
       </dependency>
       <dependency>
+         <groupId>org.eclipse.aether</groupId>
+         <artifactId>aether-api</artifactId>
+         <version>1.0.2.v20150114</version>
+      </dependency>
+      <dependency>
+         <groupId>org.eclipse.aether</groupId>
+         <artifactId>aether-util</artifactId>
+         <version>1.0.2.v20150114</version>
+      </dependency>
+      <dependency>
          <groupId>org.apache.activemq</groupId>
          <artifactId>artemis-server</artifactId>
          <version>${project.version}</version>
@@ -53,6 +63,16 @@
          <version>${project.version}</version>
       </dependency>
       <dependency>
+         <groupId>org.apache.activemq</groupId>
+         <artifactId>artemis-boot</artifactId>
+         <version>${project.version}</version>
+      </dependency>
+      <dependency>
+         <groupId>org.apache.activemq</groupId>
+         <artifactId>artemis-cli</artifactId>
+         <version>${project.version}</version>
+      </dependency>
+      <dependency>
          <groupId>io.netty</groupId>
          <artifactId>netty-all</artifactId>
       </dependency>

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/b3af4bb7/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ActiveMQClient.java
----------------------------------------------------------------------
diff --git a/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ActiveMQClient.java b/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ActiveMQClient.java
deleted file mode 100644
index 4b7de95..0000000
--- a/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ActiveMQClient.java
+++ /dev/null
@@ -1,22 +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.activemq.artemis.maven;
-
-public interface ActiveMQClient
-{
-   void run();
-}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/b3af4bb7/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ActiveMQClientPlugin.java
----------------------------------------------------------------------
diff --git a/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ActiveMQClientPlugin.java b/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ActiveMQClientPlugin.java
index 90a6112..c269e4f 100644
--- a/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ActiveMQClientPlugin.java
+++ b/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ActiveMQClientPlugin.java
@@ -16,30 +16,27 @@
  */
 package org.apache.activemq.artemis.maven;
 
-import java.lang.reflect.Method;
-import java.util.Properties;
-
 import org.apache.maven.plugin.AbstractMojo;
 import org.apache.maven.plugin.MojoExecutionException;
 import org.apache.maven.plugin.MojoFailureException;
 import org.apache.maven.plugins.annotations.LifecyclePhase;
 import org.apache.maven.plugins.annotations.Mojo;
+import org.apache.maven.plugins.annotations.Parameter;
+
+import java.lang.reflect.Method;
+import java.util.Properties;
 
 /**
- *         Allows a Java Client to be run which must hve a static main(String[] args) method
+ * Allows a Java Client to be run which must hve a static main(String[] args) method
  */
 @Mojo(name = "runClient", defaultPhase = LifecyclePhase.VERIFY)
 public class ActiveMQClientPlugin extends AbstractMojo
 {
 
-   /**
-    * @parameter
-    */
+   @Parameter
    String clientClass;
 
-   /**
-    * @parameter
-    */
+   @Parameter
    String[] args;
 
    /**
@@ -61,7 +58,7 @@ public class ActiveMQClientPlugin extends AbstractMojo
       }
       catch (Exception e)
       {
-         e.printStackTrace();
+         getLog().error(e);
          throw new MojoFailureException(e.getMessage());
       }
    }

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/b3af4bb7/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ActiveMQCreatePlugin.java
----------------------------------------------------------------------
diff --git a/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ActiveMQCreatePlugin.java b/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ActiveMQCreatePlugin.java
new file mode 100644
index 0000000..5fca4a3
--- /dev/null
+++ b/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ActiveMQCreatePlugin.java
@@ -0,0 +1,331 @@
+/**
+ * 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.activemq.artemis.maven;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.LinkOption;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.activemq.artemis.cli.Artemis;
+import org.apache.maven.plugin.AbstractMojo;
+import org.apache.maven.plugin.MojoExecutionException;
+import org.apache.maven.plugin.MojoFailureException;
+import org.apache.maven.plugin.descriptor.PluginDescriptor;
+import org.apache.maven.plugins.annotations.Component;
+import org.apache.maven.plugins.annotations.LifecyclePhase;
+import org.apache.maven.plugins.annotations.Mojo;
+import org.apache.maven.plugins.annotations.Parameter;
+import org.apache.maven.project.MavenProject;
+import org.eclipse.aether.RepositorySystem;
+import org.eclipse.aether.RepositorySystemSession;
+import org.eclipse.aether.artifact.Artifact;
+import org.eclipse.aether.artifact.DefaultArtifact;
+import org.eclipse.aether.repository.RemoteRepository;
+import org.eclipse.aether.resolution.ArtifactRequest;
+import org.eclipse.aether.resolution.ArtifactResolutionException;
+import org.eclipse.aether.resolution.ArtifactResult;
+
+@Mojo(name = "create", defaultPhase = LifecyclePhase.VERIFY)
+public class ActiveMQCreatePlugin extends AbstractMojo
+
+{
+
+   @Parameter
+   String name;
+
+   /**
+    * The plugin descriptor
+    */
+   private PluginDescriptor descriptor;
+
+   @Parameter(defaultValue = "${basedir}/target/classes/activemq/server0", required = true)
+   private File configuration;
+
+   @Parameter(defaultValue = "${activemq.basedir}", required = true)
+   private File home;
+
+   @Parameter(defaultValue = "${activemq.basedir}/artemis-distribution/target/apache-artemis-${project.version}-bin/apache-artemis-${project.version}/", required = true)
+   private File alternateHome;
+
+   @Parameter(defaultValue = "${basedir}/target/server0", required = true)
+   private File instance;
+
+   @Parameter(defaultValue = "true")
+   private boolean noWeb;
+
+   @Parameter(defaultValue = "guest")
+   private String user;
+
+   @Parameter(defaultValue = "guest")
+   private String password;
+
+   @Parameter(defaultValue = "guest")
+   private String role;
+
+   @Parameter(defaultValue = "")
+   private String javaOptions = "";
+
+   @Parameter(defaultValue = "0")
+   private int portOffset = 0;
+
+   @Parameter(defaultValue = "true")
+   private boolean allowAnonymous;
+
+   @Parameter(defaultValue = "false")
+   private boolean replicated;
+
+   @Parameter(defaultValue = "false")
+   private boolean sharedStore;
+
+   @Parameter(defaultValue = "true")
+   private boolean clustered;
+
+   @Parameter(defaultValue = "false")
+   private boolean slave;
+
+   @Parameter(defaultValue = "../data")
+   String dataFolder;
+
+   @Component
+   private RepositorySystem repositorySystem;
+
+   @Parameter(defaultValue = "${repositorySystemSession}")
+   private RepositorySystemSession repoSession;
+
+   @Parameter(defaultValue = "${project.remoteProjectRepositories}")
+   private List<RemoteRepository> remoteRepos;
+
+
+   @Parameter
+   private String[] libList;
+
+   @Parameter(defaultValue = "${localRepository}")
+   private org.apache.maven.artifact.repository.ArtifactRepository localRepository;
+
+   /**
+    * Validate if the directory is a artemis.home *
+    *
+    * @param path
+    * @return
+    */
+   private boolean lookupHome(Path path)
+   {
+
+      if (path == null)
+      {
+         return false;
+      }
+
+      Path binFolder = path.resolve("bin");
+
+      if (binFolder == null && Files.exists(binFolder, LinkOption.NOFOLLOW_LINKS))
+      {
+         return false;
+      }
+
+      Path artemisScript = binFolder.resolve("artemis");
+
+
+      return artemisScript != null && Files.exists(artemisScript, LinkOption.NOFOLLOW_LINKS);
+
+
+   }
+
+   private void add(List<String> list,  String ... str)
+   {
+      for (String s: str)
+      {
+         list.add(s);
+      }
+   }
+
+   public void execute() throws MojoExecutionException, MojoFailureException
+   {
+      getLog().info("Local " + localRepository);
+      MavenProject project = (MavenProject) getPluginContext().get("project");
+
+
+      if (!lookupHome(home.toPath()))
+      {
+         if (lookupHome(alternateHome.toPath()))
+         {
+            home = alternateHome;
+         }
+         else
+         {
+            getLog().error("********************************************************************************************");
+            getLog().error("Could not locate suitable Artemis.home on either " + home + " or " + alternateHome);
+            getLog().error("Use the binary distribution or build the distribution before running the examples");
+            getLog().error("********************************************************************************************");
+
+            throw new MojoExecutionException("Couldn't find artemis.home");
+         }
+      }
+
+
+      Map properties = getPluginContext();
+
+      Set<Map.Entry> entries = properties.entrySet();
+
+      getLog().info("Entries.size " + entries.size());
+      for (Map.Entry entry : entries)
+      {
+         getLog().info("... key=" + entry.getKey() + " = " + entry.getValue());
+      }
+
+      ArrayList<String> listCommands = new ArrayList<>();
+      add(listCommands, "create", "--allow-anonymous", "--silent-input", "--force", "--no-web", "--user", user, "--password", password,
+                        "--role", role,
+                        "--port-offset", "" + portOffset,
+                        "--data", dataFolder);
+
+      if (allowAnonymous)
+      {
+         add(listCommands, "--allow-anonymous");
+      }
+      else
+      {
+         add(listCommands, "--require-login");
+      }
+
+      if (!javaOptions.isEmpty())
+      {
+         add(listCommands, "--java-options", javaOptions);
+      }
+
+      if (slave)
+      {
+         add(listCommands, "--slave");
+      }
+
+      if (replicated)
+      {
+         add(listCommands, "--replicated");
+      }
+
+      if (replicated)
+      {
+         add(listCommands, "--shared-store");
+      }
+
+      if (replicated)
+      {
+         add(listCommands, "--shared-store");
+      }
+
+      if (clustered)
+      {
+         add(listCommands, "--clustered");
+      }
+
+      add(listCommands, "--verbose");
+
+      add(listCommands, instance.getAbsolutePath());
+
+      getLog().info("************************************************");
+      getLog().info("Calling create server at " + instance + " home= " + home);
+
+      try
+      {
+         Artemis.execute(home, null, listCommands);
+
+         String[] list = configuration.list();
+
+         if (list != null)
+         {
+            getLog().debug("************************************************");
+            getLog().debug("Replacing configuration files:");
+
+            for (String file : configuration.list())
+            {
+               Path target = instance.toPath().resolve("etc").resolve(file);
+               getLog().debug("Replacing " + file + " into " + target);
+
+
+               Files.copy(configuration.toPath().resolve(file), target, StandardCopyOption.REPLACE_EXISTING);
+            }
+         }
+
+         File projectLib = project.getArtifact().getFile();
+         copyToLib(projectLib);
+
+         if (libList != null)
+         {
+            for (int i = 0; i < libList.length; i++)
+            {
+               String[] splitString = libList[i].split(":");
+
+               getLog().debug("********************" + splitString[0] + "/" + splitString[1] + "/" + splitString[2]);
+
+               Artifact artifact;
+               try
+               {
+                  artifact = new DefaultArtifact( libList[i] );
+               }
+               catch ( IllegalArgumentException e )
+               {
+                  throw new MojoFailureException( e.getMessage(), e );
+               }
+
+               ArtifactRequest request = new ArtifactRequest();
+               request.setArtifact( artifact );
+               request.setRepositories( remoteRepos );
+
+               getLog().debug("Resolving artifact " + artifact + " from " + remoteRepos);
+
+               ArtifactResult result;
+               try
+               {
+                  result = repositorySystem.resolveArtifact( repoSession, request );
+               }
+               catch ( ArtifactResolutionException e )
+               {
+                  throw new MojoExecutionException( e.getMessage(), e );
+               }
+
+               File artifactFile = result.getArtifact().getFile();
+
+               getLog().debug("Artifact:: " + artifact + " file = " + artifactFile);
+
+               copyToLib(artifactFile);
+
+            }
+         }
+
+      }
+      catch (Exception e)
+      {
+         getLog().error(e);
+         throw new MojoFailureException(e.getMessage());
+      }
+   }
+
+   private void copyToLib(File projectLib) throws IOException
+   {
+      Path target = instance.toPath().resolve("lib").resolve(projectLib.getName());
+      target.toFile().mkdirs();
+      getLog().debug("Copying " + projectLib.getName() + " as " + target.toFile().getAbsolutePath());
+      Files.copy(projectLib.toPath(), target, StandardCopyOption.REPLACE_EXISTING);
+   }
+}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/b3af4bb7/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ActiveMQStartPlugin.java
----------------------------------------------------------------------
diff --git a/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ActiveMQStartPlugin.java b/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ActiveMQStartPlugin.java
deleted file mode 100644
index 6a7ca64..0000000
--- a/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ActiveMQStartPlugin.java
+++ /dev/null
@@ -1,230 +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.activemq.artemis.maven;
-
-import javax.management.MBeanServer;
-import javax.management.ObjectName;
-import java.io.File;
-import java.lang.management.ManagementFactory;
-import java.util.Arrays;
-import java.util.Properties;
-
-import org.apache.activemq.artemis.server.ActiveMQBootstrap;
-import org.apache.activemq.artemis.server.SpawnedActiveMQBootstrap;
-import org.apache.activemq.artemis.server.SpawnedVMSupport;
-import org.apache.activemq.artemis.spi.core.security.ActiveMQSecurityManager;
-import org.apache.maven.plugin.AbstractMojo;
-import org.apache.maven.plugin.MojoExecutionException;
-import org.apache.maven.plugin.MojoFailureException;
-import org.apache.maven.plugin.descriptor.PluginDescriptor;
-import org.apache.maven.plugins.annotations.LifecyclePhase;
-import org.apache.maven.plugins.annotations.Mojo;
-import org.codehaus.classworlds.ClassRealm;
-import org.codehaus.classworlds.ClassWorld;
-
-@Mojo(name = "start", defaultPhase = LifecyclePhase.VERIFY)
-public class ActiveMQStartPlugin extends AbstractMojo
-
-{
-   static final String SKIPBROKERSTART = "skipBrokerStart";
-
-   /**
-    * The plugin descriptor
-    */
-   private PluginDescriptor descriptor;
-
-
-   /**
-    * @parameter default-value=false
-    */
-   private Boolean waitOnStart;
-
-   /**
-    * @parameter
-    */
-   private String configurationDir;
-
-   /**
-    * @parameter
-    */
-   private String nodeId;
-
-   /**
-    * @parameter default-value=false;
-    */
-   private Boolean fork;
-
-   /**
-    * @parameter default-value=false
-    */
-   private Boolean debug;
-
-   /**
-    * @parameter
-    */
-   private Properties systemProperties;
-
-   /**
-    * @parameter default-value=STARTED::
-    */
-   private String serverStartString;
-
-   /**
-    * @parameter
-    */
-   private ActiveMQSecurityManager securityManager;
-
-   /**
-    * registers a TestClusterMBean for test clients to use.
-    */
-   private boolean testClusterManager;
-
-   public void execute() throws MojoExecutionException, MojoFailureException
-   {
-      String property = System.getProperty(SKIPBROKERSTART);
-      if (property != null)
-      {
-         getLog().info("skipping Broker Start");
-         return;
-      }
-      if (testClusterManager)
-      {
-         try
-         {
-            createClusterManagerMBean();
-         }
-         catch (Exception e)
-         {
-            throw new MojoExecutionException("Failed to create cluster manager mbean", e);
-         }
-      }
-
-      if (systemProperties != null && !systemProperties.isEmpty())
-      {
-         System.getProperties()
-            .putAll(systemProperties);
-      }
-
-      String workingPath = new File(".").getAbsolutePath();
-
-      try
-      {
-         registerNode(nodeId, workingPath, configurationDir);
-      }
-      catch (Exception e1)
-      {
-         throw new MojoExecutionException("Failed to create cluster manager mbean", e1);
-      }
-
-      if (fork)
-      {
-         try
-         {
-            PluginDescriptor pd = (PluginDescriptor) getPluginContext().get("pluginDescriptor");
-            final Process p = SpawnedVMSupport.spawnVM(pd.getArtifacts(),
-                                                       "ActiveMQServer_" + (nodeId != null ? nodeId : ""),
-                                                       SpawnedActiveMQBootstrap.class.getName(),
-                                                       systemProperties,
-                                                       true,
-                                                       serverStartString,
-                                                       "FAILED::",
-                                                       ".",
-                                                       configurationDir,
-                                                       debug,
-                                                       configurationDir,
-                                                       "" + waitOnStart,
-                                                       nodeId);
-            Runtime.getRuntime().addShutdownHook(new Thread()
-            {
-               @Override
-               public void run()
-               {
-                  //just to be on the safe side
-                  p.destroy();
-               }
-            });
-            if (waitOnStart)
-            {
-               p.waitFor();
-            }
-         }
-         catch (Throwable e)
-         {
-            e.printStackTrace();
-            throw new MojoExecutionException(e.getMessage());
-         }
-      }
-      else
-      {
-         ActiveMQBootstrap bootstrap = new ActiveMQBootstrap(configurationDir, waitOnStart, nodeId, securityManager);
-         if (configurationDir != null)
-         {
-            extendPluginClasspath(configurationDir);
-         }
-         try
-         {
-            bootstrap.execute();
-         }
-         catch (Exception e)
-         {
-            throw new MojoExecutionException(e.getMessage(), e);
-         }
-      }
-   }
-
-   private void registerNode(String nodeId, String workingPath,
-                             String hornetqConfigurationDir) throws Exception
-   {
-      TestClusterManagerMBean control = PluginUtil.getTestClusterManager();
-      if (control != null)
-      {
-         control.registerNode(nodeId, workingPath, hornetqConfigurationDir);
-      }
-   }
-
-   private void createClusterManagerMBean() throws Exception
-   {
-      MBeanServer mbeanServer = ManagementFactory.getPlatformMBeanServer();
-      ObjectName name = ObjectName.getInstance("hornetq:module=test,type=TestClusterManager");
-      mbeanServer.registerMBean(new TestClusterManager(), name);
-   }
-
-   public void extendPluginClasspath(String element) throws MojoExecutionException
-   {
-      ClassWorld world = new ClassWorld();
-      ClassRealm realm;
-      try
-      {
-         realm = world.newRealm(
-            "maven.plugin." + getClass().getSimpleName() + ((nodeId == null) ? "" : nodeId),
-            Thread.currentThread()
-               .getContextClassLoader()
-         );
-         File elementFile = new File(element);
-         getLog().debug("Adding element to plugin classpath" + elementFile.getPath());
-         realm.addConstituent(elementFile.toURI()
-                                 .toURL());
-      }
-      catch (Exception ex)
-      {
-         throw new MojoExecutionException(ex.toString(), ex);
-      }
-      System.out.println(Arrays.toString(realm.getConstituents()));
-      Thread.currentThread()
-         .setContextClassLoader(realm.getClassLoader());
-   }
-}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/b3af4bb7/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ActiveMQStopPlugin.java
----------------------------------------------------------------------
diff --git a/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ActiveMQStopPlugin.java b/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ActiveMQStopPlugin.java
deleted file mode 100644
index 98f7b24..0000000
--- a/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ActiveMQStopPlugin.java
+++ /dev/null
@@ -1,76 +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.activemq.artemis.maven;
-
-import java.io.File;
-import java.io.IOException;
-
-import org.apache.maven.plugin.AbstractMojo;
-import org.apache.maven.plugin.MojoExecutionException;
-import org.apache.maven.plugin.MojoFailureException;
-import org.apache.maven.plugins.annotations.LifecyclePhase;
-import org.apache.maven.plugins.annotations.Mojo;
-
-@Mojo(name = "stop", defaultPhase = LifecyclePhase.VERIFY)
-public class ActiveMQStopPlugin extends AbstractMojo
-{
-
-   /**
-    * @parameter
-    */
-   private String configurationDir;
-
-   public void execute() throws MojoExecutionException, MojoFailureException
-   {
-      String property = System.getProperty(ActiveMQStartPlugin.SKIPBROKERSTART);
-      if (property != null)
-      {
-         return;
-      }
-      try
-      {
-         String dirName = configurationDir != null ? configurationDir : ".";
-         final File file = new File(dirName + "/" + "/STOP_ME");
-         file.createNewFile();
-         long time = System.currentTimeMillis();
-         while (System.currentTimeMillis() < time + 60000)
-         {
-            if (!file.exists())
-            {
-               break;
-            }
-            try
-            {
-               Thread.sleep(200);
-            }
-            catch (InterruptedException e)
-            {
-               //ignore
-            }
-         }
-         if (file.exists())
-         {
-            throw new MojoExecutionException("looks like the server hasn't been stopped");
-         }
-      }
-      catch (IOException e)
-      {
-         e.printStackTrace();
-         throw new MojoExecutionException(e.getMessage());
-      }
-   }
-}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/b3af4bb7/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ArtemisCLIPlugin.java
----------------------------------------------------------------------
diff --git a/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ArtemisCLIPlugin.java b/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ArtemisCLIPlugin.java
new file mode 100644
index 0000000..79aad50
--- /dev/null
+++ b/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/ArtemisCLIPlugin.java
@@ -0,0 +1,172 @@
+/**
+ * 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.activemq.artemis.maven;
+
+import java.io.File;
+import java.nio.file.Files;
+import java.nio.file.LinkOption;
+import java.nio.file.Path;
+import java.util.Map;
+
+import org.apache.activemq.artemis.cli.Artemis;
+import org.apache.activemq.artemis.cli.commands.Run;
+import org.apache.activemq.artemis.jms.client.ActiveMQConnectionFactory;
+import org.apache.maven.plugin.AbstractMojo;
+import org.apache.maven.plugin.MojoExecutionException;
+import org.apache.maven.plugin.MojoFailureException;
+import org.apache.maven.plugin.descriptor.PluginDescriptor;
+import org.apache.maven.plugins.annotations.LifecyclePhase;
+import org.apache.maven.plugins.annotations.Mojo;
+import org.apache.maven.plugins.annotations.Parameter;
+import org.apache.maven.project.MavenProject;
+
+@Mojo(name = "cli", defaultPhase = LifecyclePhase.VERIFY)
+public class ArtemisCLIPlugin extends AbstractMojo
+
+{
+
+   @Parameter
+   String name;
+
+   /**
+    * The plugin descriptor
+    */
+   private PluginDescriptor descriptor;
+
+   @Parameter(defaultValue = "${activemq.basedir}", required = true)
+   private File home;
+
+   @Parameter(defaultValue = "${activemq.basedir}/artemis-distribution/target/apache-artemis-${project.version}-bin/apache-artemis-${project.version}/", required = true)
+   private File alternateHome;
+
+   @Parameter(defaultValue = "${basedir}/target/server0", required = true)
+   private File location;
+
+   @Parameter
+   private String[] args;
+
+   @Parameter
+   private boolean spawn = false;
+
+   @Parameter
+   private boolean testServer;
+
+
+   /**
+    * Validate if the directory is a artemis.home *
+    *
+    * @param path
+    * @return
+    */
+   private boolean lookupHome(Path path)
+   {
+
+      if (path == null)
+      {
+         return false;
+      }
+
+      Path binFolder = path.resolve("bin");
+
+      if (binFolder == null && Files.exists(binFolder, LinkOption.NOFOLLOW_LINKS))
+      {
+         return false;
+      }
+
+      Path artemisScript = binFolder.resolve("artemis");
+
+
+      return artemisScript != null && Files.exists(artemisScript, LinkOption.NOFOLLOW_LINKS);
+
+
+   }
+
+   public void execute() throws MojoExecutionException, MojoFailureException
+   {
+      // This is to avoid the Run issuing a kill at any point
+      Run.setEmbedded(true);
+
+      MavenProject project = (MavenProject) getPluginContext().get("project");
+
+
+      if (!lookupHome(home.toPath()))
+      {
+         if (lookupHome(alternateHome.toPath()))
+         {
+            home = alternateHome;
+         }
+         else
+         {
+            getLog().error("********************************************************************************************");
+            getLog().error("Could not locate suitable Artemis.home on either " + home + " or " + alternateHome);
+            getLog().error("Use the binary distribution or build the distribution before running the examples");
+            getLog().error("********************************************************************************************");
+
+            throw new MojoExecutionException("Couldn't find artemis.home");
+         }
+      }
+
+      Map properties = getPluginContext();
+
+      try
+      {
+         if (spawn)
+         {
+            final Process process = org.apache.activemq.artemis.cli.process.ProcessBuilder.build("server", location, true, args);
+            Runtime.getRuntime().addShutdownHook(new Thread()
+            {
+               public void run()
+               {
+                  process.destroy();
+               }
+            });
+
+            if (testServer)
+            {
+               for (int tryNr = 0; tryNr < 20; tryNr++)
+               {
+                  try
+                  {
+                     ActiveMQConnectionFactory cf = new ActiveMQConnectionFactory();
+                     cf.createConnection().close();
+                     getLog().info("Server started");
+                  }
+                  catch (Exception e)
+                  {
+                     getLog().info("awaiting server to start");
+                     Thread.sleep(500);
+                     continue;
+                  }
+                  break;
+               }
+            }
+         }
+         else
+         {
+            Artemis.execute(home, location, args);
+         }
+
+         Thread.sleep(600);
+
+         org.apache.activemq.artemis.cli.process.ProcessBuilder.cleanupProcess();
+      }
+      catch (Exception e)
+      {
+         throw new MojoExecutionException(e.getMessage(), e);
+      }
+   }
+}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/b3af4bb7/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/InVMNodeManagerServer.java
----------------------------------------------------------------------
diff --git a/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/InVMNodeManagerServer.java b/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/InVMNodeManagerServer.java
deleted file mode 100644
index b8b1a67..0000000
--- a/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/InVMNodeManagerServer.java
+++ /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.
- */
-package org.apache.activemq.artemis.maven;
-
-import javax.management.MBeanServer;
-
-import java.io.File;
-
-import org.apache.activemq.artemis.core.config.Configuration;
-import org.apache.activemq.artemis.core.server.NodeManager;
-import org.apache.activemq.artemis.core.server.impl.ActiveMQServerImpl;
-import org.apache.activemq.artemis.spi.core.security.ActiveMQSecurityManager;
-
-public final class InVMNodeManagerServer extends ActiveMQServerImpl
-{
-   final NodeManager nodeManager;
-
-   public InVMNodeManagerServer(final NodeManager nodeManager)
-   {
-      super();
-      this.nodeManager = nodeManager;
-   }
-
-   public InVMNodeManagerServer(final Configuration configuration, final NodeManager nodeManager)
-   {
-      super(configuration);
-      this.nodeManager = nodeManager;
-   }
-
-   public InVMNodeManagerServer(final Configuration configuration,
-                                final MBeanServer mbeanServer,
-                                final NodeManager nodeManager)
-   {
-      super(configuration, mbeanServer);
-      this.nodeManager = nodeManager;
-   }
-
-   public InVMNodeManagerServer(final Configuration configuration,
-                                final ActiveMQSecurityManager securityManager,
-                                final NodeManager nodeManager)
-   {
-      super(configuration, securityManager);
-      this.nodeManager = nodeManager;
-   }
-
-   public InVMNodeManagerServer(final Configuration configuration,
-                                final MBeanServer mbeanServer,
-                                final ActiveMQSecurityManager securityManager,
-                                final NodeManager nodeManager)
-   {
-      super(configuration, mbeanServer, securityManager);
-      this.nodeManager = nodeManager;
-   }
-
-   @Override
-   protected NodeManager createNodeManager(final File directory, boolean replicatingBackup)
-   {
-      return nodeManager;
-   }
-
-}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/b3af4bb7/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/LibInstallPlugin.java
----------------------------------------------------------------------
diff --git a/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/LibInstallPlugin.java b/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/LibInstallPlugin.java
new file mode 100644
index 0000000..e65108d
--- /dev/null
+++ b/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/LibInstallPlugin.java
@@ -0,0 +1,147 @@
+/**
+ * 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.activemq.artemis.maven;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.maven.plugin.AbstractMojo;
+import org.apache.maven.plugin.MojoExecutionException;
+import org.apache.maven.plugin.MojoFailureException;
+import org.apache.maven.plugin.descriptor.PluginDescriptor;
+import org.apache.maven.plugins.annotations.Component;
+import org.apache.maven.plugins.annotations.LifecyclePhase;
+import org.apache.maven.plugins.annotations.Mojo;
+import org.apache.maven.plugins.annotations.Parameter;
+import org.apache.maven.project.MavenProject;
+import org.eclipse.aether.RepositorySystem;
+import org.eclipse.aether.RepositorySystemSession;
+import org.eclipse.aether.artifact.Artifact;
+import org.eclipse.aether.artifact.DefaultArtifact;
+import org.eclipse.aether.repository.RemoteRepository;
+import org.eclipse.aether.resolution.ArtifactRequest;
+import org.eclipse.aether.resolution.ArtifactResolutionException;
+import org.eclipse.aether.resolution.ArtifactResult;
+
+@Mojo(name = "lib-install", defaultPhase = LifecyclePhase.VERIFY)
+public class LibInstallPlugin extends AbstractMojo
+
+{
+
+   @Parameter
+   String name;
+
+   /**
+    * The plugin descriptor
+    */
+   private PluginDescriptor descriptor;
+
+   @Parameter(defaultValue = "${basedir}/target/server0", required = true)
+   private File instance;
+
+   @Component
+   private RepositorySystem repositorySystem;
+
+   @Parameter(defaultValue = "${repositorySystemSession}")
+   private RepositorySystemSession repoSession;
+
+   @Parameter(defaultValue = "${project.remoteProjectRepositories}")
+   private List<RemoteRepository> remoteRepos;
+
+
+   @Parameter
+   private String[] libList;
+
+   @Parameter(defaultValue = "${localRepository}")
+   private org.apache.maven.artifact.repository.ArtifactRepository localRepository;
+
+   public void execute() throws MojoExecutionException, MojoFailureException
+   {
+      MavenProject project = (MavenProject) getPluginContext().get("project");
+
+      Map properties = getPluginContext();
+
+      try
+      {
+
+         File projectLib = project.getArtifact().getFile();
+         copyToLib(projectLib);
+
+         if (libList != null)
+         {
+            for (int i = 0; i < libList.length; i++)
+            {
+               String[] splitString = libList[i].split(":");
+
+               getLog().info("********************" + splitString[0] + "/" + splitString[1] + "/" + splitString[2]);
+
+               Artifact artifact;
+               try
+               {
+                  artifact = new DefaultArtifact(libList[i]);
+               }
+               catch (IllegalArgumentException e)
+               {
+                  throw new MojoFailureException(e.getMessage(), e);
+               }
+
+               ArtifactRequest request = new ArtifactRequest();
+               request.setArtifact(artifact);
+               request.setRepositories(remoteRepos);
+
+               getLog().info("Resolving artifact " + artifact + " from " + remoteRepos);
+
+               ArtifactResult result;
+               try
+               {
+                  result = repositorySystem.resolveArtifact(repoSession, request);
+               }
+               catch (ArtifactResolutionException e)
+               {
+                  throw new MojoExecutionException(e.getMessage(), e);
+               }
+
+               File artifactFile = result.getArtifact().getFile();
+
+               getLog().info("Artifact:: " + artifact + " file = " + artifactFile);
+
+               copyToLib(artifactFile);
+
+            }
+         }
+
+      }
+      catch (Exception e)
+      {
+         getLog().error(e);
+         throw new MojoFailureException(e.getMessage());
+      }
+   }
+
+   private void copyToLib(File projectLib) throws IOException
+   {
+      Path target = instance.toPath().resolve("lib").resolve(projectLib.getName());
+      target.toFile().mkdirs();
+      getLog().info("Copying " + projectLib.getName() + " as " + target.toFile().getAbsolutePath());
+      Files.copy(projectLib.toPath(), target, StandardCopyOption.REPLACE_EXISTING);
+   }
+}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/b3af4bb7/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/PluginUtil.java
----------------------------------------------------------------------
diff --git a/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/PluginUtil.java b/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/PluginUtil.java
deleted file mode 100644
index a3f3505..0000000
--- a/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/PluginUtil.java
+++ /dev/null
@@ -1,50 +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.activemq.artemis.maven;
-
-import javax.management.MBeanServerConnection;
-import javax.management.MBeanServerInvocationHandler;
-import javax.management.ObjectName;
-import javax.management.remote.JMXConnector;
-import javax.management.remote.JMXConnectorFactory;
-import javax.management.remote.JMXServiceURL;
-import java.util.HashMap;
-
-public class PluginUtil
-{
-   public static TestClusterManagerMBean getTestClusterManager()
-   {
-      final String JMX_URL = "service:jmx:rmi:///jndi/rmi://localhost:3000/jmxrmi";
-      try
-      {
-         JMXConnector connector = JMXConnectorFactory.connect(new JMXServiceURL(JMX_URL), new HashMap<String, String>());
-         ObjectName name = ObjectName.getInstance("activemq:module=test,type=TestClusterManager");
-         MBeanServerConnection mbsc = connector.getMBeanServerConnection();
-         TestClusterManagerMBean clusterControl = MBeanServerInvocationHandler.newProxyInstance(mbsc,
-                                                                                                name,
-                                                                                                TestClusterManagerMBean.class,
-                                                                                                false);
-         clusterControl.getNumNodes();//serves as a validation.
-         return clusterControl;
-      }
-      catch (Exception e)
-      {
-         return null;
-      }
-   }
-
-}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/b3af4bb7/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/TestClusterManager.java
----------------------------------------------------------------------
diff --git a/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/TestClusterManager.java b/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/TestClusterManager.java
deleted file mode 100644
index 7f7aced..0000000
--- a/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/TestClusterManager.java
+++ /dev/null
@@ -1,52 +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.activemq.artemis.maven;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-public class TestClusterManager implements TestClusterManagerMBean
-{
-   private final List<TestNode> testNodes = new ArrayList<TestNode>();
-
-   @Override
-   public int getNumNodes()
-   {
-      synchronized (testNodes)
-      {
-         return testNodes.size();
-      }
-   }
-
-   @Override
-   public void registerNode(String nodeId, String workingDir,
-                            String hornetqConfigurationDir)
-   {
-      synchronized (testNodes)
-      {
-         testNodes.add(new TestNode(nodeId, workingDir, hornetqConfigurationDir));
-      }
-   }
-
-   @Override
-   public void killNode(int i) throws IOException
-   {
-      TestNode node = testNodes.get(i);
-      node.kill();
-   }
-}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/b3af4bb7/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/TestClusterManagerMBean.java
----------------------------------------------------------------------
diff --git a/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/TestClusterManagerMBean.java b/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/TestClusterManagerMBean.java
deleted file mode 100644
index 50b3e8f..0000000
--- a/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/TestClusterManagerMBean.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.activemq.artemis.maven;
-
-import java.io.IOException;
-
-public interface TestClusterManagerMBean
-{
-   int getNumNodes();
-
-   void registerNode(String nodeId, String workingDir,
-                     String hornetqConfigurationDirt);
-
-   void killNode(int i) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/activemq-artemis/blob/b3af4bb7/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/TestNode.java
----------------------------------------------------------------------
diff --git a/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/TestNode.java b/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/TestNode.java
deleted file mode 100644
index 1de6a0c..0000000
--- a/artemis-maven-plugin/src/main/java/org/apache/activemq/artemis/maven/TestNode.java
+++ /dev/null
@@ -1,49 +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.activemq.artemis.maven;
-
-import java.io.File;
-import java.io.IOException;
-
-public class TestNode
-{
-   String nodeId;
-   String workingDir;
-   String configDir;
-
-   public TestNode(String nodeId, String workingDir,
-                   String configDir)
-   {
-      this.nodeId = nodeId;
-      this.workingDir = workingDir;
-      this.configDir = configDir;
-   }
-
-   public void kill() throws IOException
-   {
-      File file = new File(configDir, "KILL_ME");
-      file.createNewFile();
-      try
-      {
-         Thread.sleep(3000);
-      }
-      catch (InterruptedException e)
-      {
-      }
-   }
-
-}