You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by ka...@apache.org on 2017/04/12 08:29:09 UTC

[01/10] storm git commit: STORM-2447: add in storm local to avoid having server on worker classpath

Repository: storm
Updated Branches:
  refs/heads/master 65c8b7bd7 -> 78b074caa


http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/storm-core/src/jvm/org/apache/storm/command/KillTopology.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/command/KillTopology.java b/storm-core/src/jvm/org/apache/storm/command/KillTopology.java
index bad77be..b8f9a91 100644
--- a/storm-core/src/jvm/org/apache/storm/command/KillTopology.java
+++ b/storm-core/src/jvm/org/apache/storm/command/KillTopology.java
@@ -43,7 +43,7 @@ public class KillTopology {
         }
         NimbusClient.withConfiguredClient(new NimbusClient.WithNimbus() {
           @Override
-          public void run(Nimbus.Client nimbus) throws Exception {
+          public void run(Nimbus.Iface nimbus) throws Exception {
             for (String name: names) {
               nimbus.killTopologyWithOpts(name, opts);
               LOG.info("Killed topology: {}", name);

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/storm-core/src/jvm/org/apache/storm/command/ListTopologies.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/command/ListTopologies.java b/storm-core/src/jvm/org/apache/storm/command/ListTopologies.java
index 97d18d5..231ca0f 100644
--- a/storm-core/src/jvm/org/apache/storm/command/ListTopologies.java
+++ b/storm-core/src/jvm/org/apache/storm/command/ListTopologies.java
@@ -33,7 +33,7 @@ public class ListTopologies {
     public static void main(String [] args) throws Exception {
         NimbusClient.withConfiguredClient(new NimbusClient.WithNimbus() {
           @Override
-          public void run(Nimbus.Client nimbus) throws Exception {
+          public void run(Nimbus.Iface nimbus) throws Exception {
               List<TopologySummary> topologies = nimbus.getClusterInfo().get_topologies();
               if (topologies == null || topologies.isEmpty()) {
                   System.out.println("No topologies running.");

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/storm-core/src/jvm/org/apache/storm/command/Monitor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/command/Monitor.java b/storm-core/src/jvm/org/apache/storm/command/Monitor.java
index 68a65ea..c4da6c8 100644
--- a/storm-core/src/jvm/org/apache/storm/command/Monitor.java
+++ b/storm-core/src/jvm/org/apache/storm/command/Monitor.java
@@ -57,7 +57,7 @@ public class Monitor {
 
         NimbusClient.withConfiguredClient(new NimbusClient.WithNimbus() {
             @Override
-            public void run(Nimbus.Client nimbus) throws Exception {
+            public void run(Nimbus.Iface nimbus) throws Exception {
                 monitor.metrics(nimbus);
             }
         });

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/storm-core/src/jvm/org/apache/storm/command/Rebalance.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/command/Rebalance.java b/storm-core/src/jvm/org/apache/storm/command/Rebalance.java
index ed65950..343658c 100644
--- a/storm-core/src/jvm/org/apache/storm/command/Rebalance.java
+++ b/storm-core/src/jvm/org/apache/storm/command/Rebalance.java
@@ -57,7 +57,7 @@ public class Rebalance {
 
         NimbusClient.withConfiguredClient(new NimbusClient.WithNimbus() {
             @Override
-            public void run(Nimbus.Client nimbus) throws Exception {
+            public void run(Nimbus.Iface nimbus) throws Exception {
                 nimbus.rebalance(name, rebalanceOptions);
                 LOG.info("Topology {} is rebalancing", name);
             }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/storm-core/src/jvm/org/apache/storm/command/SetLogLevel.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/command/SetLogLevel.java b/storm-core/src/jvm/org/apache/storm/command/SetLogLevel.java
index 956f99b..75708a8 100644
--- a/storm-core/src/jvm/org/apache/storm/command/SetLogLevel.java
+++ b/storm-core/src/jvm/org/apache/storm/command/SetLogLevel.java
@@ -60,7 +60,7 @@ public class SetLogLevel {
 
         NimbusClient.withConfiguredClient(new NimbusClient.WithNimbus() {
             @Override
-            public void run(Nimbus.Client nimbus) throws Exception {
+            public void run(Nimbus.Iface nimbus) throws Exception {
                 String topologyId = Utils.getTopologyId(topologyName, nimbus);
                 if (null == topologyId) {
                     throw new IllegalArgumentException(topologyName + " is not a running topology");

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/storm-core/src/jvm/org/apache/storm/utils/Monitor.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/utils/Monitor.java b/storm-core/src/jvm/org/apache/storm/utils/Monitor.java
index cd7244c..15569d0 100644
--- a/storm-core/src/jvm/org/apache/storm/utils/Monitor.java
+++ b/storm-core/src/jvm/org/apache/storm/utils/Monitor.java
@@ -96,7 +96,7 @@ public class Monitor {
         }
     }
 
-    private HashSet<String> getComponents(Nimbus.Client client, String topology) throws Exception{
+    private HashSet<String> getComponents(Nimbus.Iface client, String topology) throws Exception{
         HashSet<String> components = new HashSet<>();
         ClusterSummary clusterSummary = client.getClusterInfo();
         TopologySummary topologySummary = null;
@@ -120,7 +120,7 @@ public class Monitor {
         return components;
     }
 
-    public void metrics(Nimbus.Client client) throws Exception {
+    public void metrics(Nimbus.Iface client) throws Exception {
         if (_interval <= 0) {
             throw new IllegalArgumentException("poll interval must be positive");
         }
@@ -166,7 +166,7 @@ public class Monitor {
         } while (true);
     }
 
-    public void metrics(Nimbus.Client client, long now, MetricsState state) throws Exception {
+    public void metrics(Nimbus.Iface client, long now, MetricsState state) throws Exception {
         long totalStatted = 0;
 
         int componentParallelism = 0;

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/storm-server/src/main/java/org/apache/storm/LocalCluster.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/LocalCluster.java b/storm-server/src/main/java/org/apache/storm/LocalCluster.java
index c188b69..4cd24eb 100644
--- a/storm-server/src/main/java/org/apache/storm/LocalCluster.java
+++ b/storm-server/src/main/java/org/apache/storm/LocalCluster.java
@@ -17,6 +17,8 @@
  */
 package org.apache.storm;
 
+import java.lang.reflect.Method;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -24,6 +26,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
+import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -44,16 +47,34 @@ import org.apache.storm.daemon.supervisor.ReadClusterState;
 import org.apache.storm.daemon.supervisor.StandaloneSupervisor;
 import org.apache.storm.daemon.supervisor.Supervisor;
 import org.apache.storm.executor.LocalExecutor;
+import org.apache.storm.generated.AlreadyAliveException;
 import org.apache.storm.generated.AuthorizationException;
+import org.apache.storm.generated.BeginDownloadResult;
 import org.apache.storm.generated.ClusterSummary;
+import org.apache.storm.generated.ComponentPageInfo;
 import org.apache.storm.generated.Credentials;
+import org.apache.storm.generated.GetInfoOptions;
+import org.apache.storm.generated.InvalidTopologyException;
+import org.apache.storm.generated.KeyAlreadyExistsException;
+import org.apache.storm.generated.KeyNotFoundException;
 import org.apache.storm.generated.KillOptions;
+import org.apache.storm.generated.ListBlobsResult;
+import org.apache.storm.generated.LogConfig;
+import org.apache.storm.generated.Nimbus.Iface;
 import org.apache.storm.generated.Nimbus.Processor;
 import org.apache.storm.generated.NimbusSummary;
+import org.apache.storm.generated.NotAliveException;
+import org.apache.storm.generated.ProfileAction;
+import org.apache.storm.generated.ProfileRequest;
+import org.apache.storm.generated.ReadableBlobMeta;
 import org.apache.storm.generated.RebalanceOptions;
+import org.apache.storm.generated.SettableBlobMeta;
 import org.apache.storm.generated.StormTopology;
 import org.apache.storm.generated.SubmitOptions;
+import org.apache.storm.generated.SupervisorPageInfo;
+import org.apache.storm.generated.TopologyHistoryInfo;
 import org.apache.storm.generated.TopologyInfo;
+import org.apache.storm.generated.TopologyPageInfo;
 import org.apache.storm.messaging.IContext;
 import org.apache.storm.messaging.local.Context;
 import org.apache.storm.nimbus.ILeaderElector;
@@ -68,6 +89,8 @@ import org.apache.storm.testing.NonRichBoltTracker;
 import org.apache.storm.testing.TmpPath;
 import org.apache.storm.testing.TrackedTopology;
 import org.apache.storm.utils.ConfigUtils;
+import org.apache.storm.utils.DRPCClient;
+import org.apache.storm.utils.NimbusClient;
 import org.apache.storm.utils.Utils;
 import org.apache.storm.utils.ObjectReader;
 import org.apache.storm.utils.RegisteredGlobalState;
@@ -76,6 +99,7 @@ import org.apache.storm.utils.Time;
 import org.apache.storm.utils.Time.SimulatedTime;
 import org.apache.thrift.TException;
 import org.json.simple.JSONValue;
+import org.json.simple.parser.ParseException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -92,7 +116,7 @@ import org.slf4j.LoggerFactory;
  * }
  * // The cluster has been shut down.
  */
-public class LocalCluster implements ILocalClusterTrackedTopologyAware {
+public class LocalCluster implements ILocalClusterTrackedTopologyAware, Iface {
     private static final Logger LOG = LoggerFactory.getLogger(LocalCluster.class);
     
     private static ThriftServer startNimbusDaemon(Map<String, Object> conf, Nimbus nimbus) {
@@ -831,4 +855,274 @@ public class LocalCluster implements ILocalClusterTrackedTopologyAware {
     public String getTrackedId() {
         return trackId;
     }
+
+    //Nimbus Compatibility
+    
+    @Override
+    public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology)
+            throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, TException {
+        try {
+            @SuppressWarnings("unchecked")
+            Map<String, Object> conf = (Map<String, Object>) JSONValue.parseWithException(jsonConf);
+            submitTopology(name, conf, topology);
+        } catch (ParseException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public void submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology,
+            SubmitOptions options)
+            throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, TException {
+        try {
+            @SuppressWarnings("unchecked")
+            Map<String, Object> conf = (Map<String, Object>) JSONValue.parseWithException(jsonConf);
+            submitTopologyWithOpts(name, conf, topology, options);
+        } catch (ParseException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public void setLogConfig(String name, LogConfig config) throws TException {
+        // TODO Auto-generated method stub
+        throw new RuntimeException("NOT IMPLMENETED YET");
+    }
+
+    @Override
+    public LogConfig getLogConfig(String name) throws TException {
+        // TODO Auto-generated method stub
+        throw new RuntimeException("NOT IMPLMENETED YET");
+    }
+
+    @Override
+    public void debug(String name, String component, boolean enable, double samplingPercentage)
+            throws NotAliveException, AuthorizationException, TException {
+        // TODO Auto-generated method stub
+        throw new RuntimeException("NOT IMPLMENETED YET");
+    }
+
+    @Override
+    public void setWorkerProfiler(String id, ProfileRequest profileRequest) throws TException {
+        // TODO Auto-generated method stub
+        throw new RuntimeException("NOT IMPLMENETED YET");
+    }
+
+    @Override
+    public List<ProfileRequest> getComponentPendingProfileActions(String id, String component_id, ProfileAction action)
+            throws TException {
+        // TODO Auto-generated method stub
+        throw new RuntimeException("NOT IMPLMENETED YET");
+    }
+
+    @Override
+    public String beginCreateBlob(String key, SettableBlobMeta meta)
+            throws AuthorizationException, KeyAlreadyExistsException, TException {
+        throw new RuntimeException("BLOBS NOT SUPPORTED IN LOCAL MODE");
+    }
+
+    @Override
+    public String beginUpdateBlob(String key) throws AuthorizationException, KeyNotFoundException, TException {
+        throw new KeyNotFoundException("BLOBS NOT SUPPORTED IN LOCAL MODE");
+    }
+
+    @Override
+    public void uploadBlobChunk(String session, ByteBuffer chunk) throws AuthorizationException, TException {
+        throw new RuntimeException("BLOBS NOT SUPPORTED IN LOCAL MODE");
+    }
+
+    @Override
+    public void finishBlobUpload(String session) throws AuthorizationException, TException {
+        throw new RuntimeException("BLOBS NOT SUPPORTED IN LOCAL MODE");
+    }
+
+    @Override
+    public void cancelBlobUpload(String session) throws AuthorizationException, TException {
+        throw new RuntimeException("BLOBS NOT SUPPORTED IN LOCAL MODE");
+    }
+
+    @Override
+    public ReadableBlobMeta getBlobMeta(String key) throws AuthorizationException, KeyNotFoundException, TException {
+        throw new KeyNotFoundException("BLOBS NOT SUPPORTED IN LOCAL MODE");
+    }
+
+    @Override
+    public void setBlobMeta(String key, SettableBlobMeta meta)
+            throws AuthorizationException, KeyNotFoundException, TException {
+        throw new KeyNotFoundException("BLOBS NOT SUPPORTED IN LOCAL MODE");
+    }
+
+    @Override
+    public BeginDownloadResult beginBlobDownload(String key)
+            throws AuthorizationException, KeyNotFoundException, TException {
+        throw new KeyNotFoundException("BLOBS NOT SUPPORTED IN LOCAL MODE");
+    }
+
+    @Override
+    public ByteBuffer downloadBlobChunk(String session) throws AuthorizationException, TException {
+        throw new RuntimeException("BLOBS NOT SUPPORTED IN LOCAL MODE");
+    }
+
+    @Override
+    public void deleteBlob(String key) throws AuthorizationException, KeyNotFoundException, TException {
+        throw new KeyNotFoundException("BLOBS NOT SUPPORTED IN LOCAL MODE");
+    }
+
+    @Override
+    public ListBlobsResult listBlobs(String session) throws TException {
+        //Blobs are not supported in local mode.  Return nothing
+        ListBlobsResult ret = new ListBlobsResult();
+        ret.set_keys(new ArrayList<>());
+        return ret;
+    }
+
+    @Override
+    public int getBlobReplication(String key) throws AuthorizationException, KeyNotFoundException, TException {
+        throw new KeyNotFoundException("BLOBS NOT SUPPORTED IN LOCAL MODE");
+    }
+
+    @Override
+    public int updateBlobReplication(String key, int replication)
+            throws AuthorizationException, KeyNotFoundException, TException {
+        throw new KeyNotFoundException("BLOBS NOT SUPPORTED IN LOCAL MODE");
+    }
+
+    @Override
+    public void createStateInZookeeper(String key) throws TException {
+        // TODO Auto-generated method stub
+        throw new RuntimeException("NOT IMPLMENETED YET");
+    }
+
+    @Override
+    public String beginFileUpload() throws AuthorizationException, TException {
+        //Just ignore these for now.  We are going to throw it away anyways
+        return Utils.uuid();
+    }
+
+    @Override
+    public void uploadChunk(String location, ByteBuffer chunk) throws AuthorizationException, TException {
+        //Just throw it away in local mode
+    }
+
+    @Override
+    public void finishFileUpload(String location) throws AuthorizationException, TException {
+        //Just throw it away in local mode
+    }
+
+    @Override
+    public String beginFileDownload(String file) throws AuthorizationException, TException {
+        throw new AuthorizationException("FILE DOWNLOAD NOT SUPPORTED IN LOCAL MODE");
+    }
+
+    @Override
+    public ByteBuffer downloadChunk(String id) throws AuthorizationException, TException {
+        throw new AuthorizationException("FILE DOWNLOAD NOT SUPPORTED IN LOCAL MODE");
+    }
+
+    @Override
+    public String getNimbusConf() throws AuthorizationException, TException {
+        // TODO Auto-generated method stub
+        throw new RuntimeException("NOT IMPLMENETED YET");
+    }
+
+    @Override
+    public NimbusSummary getLeader() throws AuthorizationException, TException {
+        return nimbus.getLeader();
+    }
+
+    @Override
+    public boolean isTopologyNameAllowed(String name) throws AuthorizationException, TException {
+        return nimbus.isTopologyNameAllowed(name);
+    }
+
+    @Override
+    public TopologyInfo getTopologyInfoWithOpts(String id, GetInfoOptions options)
+            throws NotAliveException, AuthorizationException, TException {
+        // TODO Auto-generated method stub
+        throw new RuntimeException("NOT IMPLMENETED YET");
+    }
+
+    @Override
+    public TopologyPageInfo getTopologyPageInfo(String id, String window, boolean is_include_sys)
+            throws NotAliveException, AuthorizationException, TException {
+        // TODO Auto-generated method stub
+        throw new RuntimeException("NOT IMPLMENETED YET");
+    }
+
+    @Override
+    public SupervisorPageInfo getSupervisorPageInfo(String id, String host, boolean is_include_sys)
+            throws NotAliveException, AuthorizationException, TException {
+        // TODO Auto-generated method stub
+        throw new RuntimeException("NOT IMPLMENETED YET");
+    }
+
+    @Override
+    public ComponentPageInfo getComponentPageInfo(String topology_id, String component_id, String window,
+            boolean is_include_sys) throws NotAliveException, AuthorizationException, TException {
+        // TODO Auto-generated method stub
+        throw new RuntimeException("NOT IMPLMENETED YET");
+    }
+
+    @Override
+    public StormTopology getUserTopology(String id) throws NotAliveException, AuthorizationException, TException {
+        // TODO Auto-generated method stub
+        throw new RuntimeException("NOT IMPLMENETED YET");
+    }
+
+    @Override
+    public TopologyHistoryInfo getTopologyHistory(String user) throws AuthorizationException, TException {
+        // TODO Auto-generated method stub
+        throw new RuntimeException("NOT IMPLMENETED YET");
+    }
+    
+    /**
+     * Run c with a local mode cluster overriding the NimbusClient and DRPCClient calls.
+     * @param c the callable to run in this mode
+     * @param ttlSec the number of seconds to let the cluster run after c has completed
+     * @return the result of calling C
+     * @throws Exception on any Exception.
+     */
+    public static <T> T withLocalModeOverride(Callable<T> c, long ttlSec) throws Exception {
+        LOG.info("\n\n\t\tSTARTING LOCAL MODE CLUSTER\n\n");
+        try (LocalCluster local = new LocalCluster();
+                NimbusClient.LocalOverride nimbusOverride = new NimbusClient.LocalOverride(local);
+                LocalDRPC drpc = new LocalDRPC();
+                DRPCClient.LocalOverride drpcOverride = new DRPCClient.LocalOverride(drpc)) {
+
+            T ret = c.call();
+            LOG.info("\n\n\t\tRUNNING LOCAL CLUSTER for {} seconds.\n\n", ttlSec);
+            Thread.sleep(ttlSec * 1000);
+            
+            LOG.info("\n\n\t\tSTOPPING LOCAL MODE CLUSTER\n\n");
+            return ret;
+        }
+    }
+    
+    public static void main(final String [] args) throws Exception {
+        if (args.length < 1) {
+            throw new IllegalArgumentException("No class was specified to run");
+        }
+        
+        long ttl = 20;
+        String ttlString = System.getProperty("storm.local.sleeptime", "20");
+        try {
+            ttl = Long.valueOf(ttlString);
+        } catch (NumberFormatException e) {
+            LOG.warn("could not parse the sleep time defaulting to {} seconds", ttl);
+        }
+        
+        withLocalModeOverride(() -> {
+            String klass = args[0];
+            String [] newArgs = Arrays.copyOfRange(args, 1, args.length); 
+            Class<?> c = Class.forName(klass);
+            Method main = c.getDeclaredMethod("main", String[].class);
+            
+            LOG.info("\n\n\t\tRUNNING {} with args {}\n\n", main, Arrays.toString(newArgs));
+            main.invoke(null, (Object)newArgs);
+            return (Void)null;
+        }, ttl);
+        
+        //Sometimes external things used with testing don't shut down all the way
+        System.exit(0);
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/storm-server/src/main/java/org/apache/storm/topology/ConfigurableTopology.java
----------------------------------------------------------------------
diff --git a/storm-server/src/main/java/org/apache/storm/topology/ConfigurableTopology.java b/storm-server/src/main/java/org/apache/storm/topology/ConfigurableTopology.java
deleted file mode 100644
index adc969c..0000000
--- a/storm-server/src/main/java/org/apache/storm/topology/ConfigurableTopology.java
+++ /dev/null
@@ -1,184 +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.storm.topology;
-
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.InputStreamReader;
-import java.nio.charset.Charset;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.lang.StringUtils;
-import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
-import org.apache.storm.StormSubmitter;
-import org.apache.storm.utils.Utils;
-import org.yaml.snakeyaml.Yaml;
-
-/**
- * Extensions of this class use command line arguments to determine whether the
- * topology should run locally or remote, with a time to live and takes a
- * reference to one or more configuration files. The main() method should call
- * ConfigurableTopology.start() and it must instantiate a TopologyBuilder in the
- * run() method.
- * 
- * <pre>
- * {
- *    public class MyTopology extends ConfigurableTopology {
- *
-  *   public static void main(String[] args) throws Exception {
-  *       ConfigurableTopology.start(new MyTopology(), args);
-  *   }
-  *
-  *   &#64;Override
-  *   protected int run(String[] args) {
-  *       TopologyBuilder builder = new TopologyBuilder();
-  *
-  *       // build topology as usual
-  *    
-  *       return submit("crawl", conf, builder);
-  *   }
- * }
- * </pre>
- **/
-public abstract class ConfigurableTopology {
-
-    protected Config conf = new Config();
-    protected boolean isLocal = false;
-    protected int ttl = -1;
-
-    public static void start(ConfigurableTopology topology, String args[]) {
-        String[] remainingArgs = topology.parse(args);
-        topology.run(remainingArgs);
-    }
-
-    protected Config getConf() {
-        return conf;
-    }
-
-    protected abstract int run(String args[]);
-
-    /** Submits the topology with the name taken from the configuration **/
-    protected int submit(Config conf, TopologyBuilder builder) {
-        String name = (String) Utils.get(conf, Config.TOPOLOGY_NAME, null);
-        if (StringUtils.isBlank(name))
-            throw new RuntimeException(
-                    "No value found for " + Config.TOPOLOGY_NAME);
-        return submit(name, conf, builder);
-    }
-
-    /** Submits the topology under a specific name **/
-    protected int submit(String name, Config conf, TopologyBuilder builder) {
-
-        if (isLocal) {
-            try (LocalCluster cluster = new LocalCluster();
-                    LocalTopology topo = cluster.submitTopology(name, conf,
-                            builder.createTopology());) {
-                if (ttl != -1) {
-                    Utils.sleep(ttl * 1000);
-                    cluster.shutdown();
-                }
-            } catch (Exception e) {
-                e.printStackTrace();
-                return -1;
-            }
-        }
-
-        else {
-            try {
-                StormSubmitter.submitTopology(name, conf,
-                        builder.createTopology());
-            } catch (Exception e) {
-                e.printStackTrace();
-                return -1;
-            }
-        }
-        return 0;
-    }
-
-    private String[] parse(String args[]) {
-
-        List<String> newArgs = new ArrayList<>();
-        Collections.addAll(newArgs, args);
-
-        Iterator<String> iter = newArgs.iterator();
-        while (iter.hasNext()) {
-            String param = iter.next();
-            if (param.equals("-conf")) {
-                if (!iter.hasNext()) {
-                    throw new RuntimeException("Conf file not specified");
-                }
-                iter.remove();
-                String resource = iter.next();
-                try {
-                    loadConf(resource, conf);
-                } catch (FileNotFoundException e) {
-                    throw new RuntimeException("File not found : " + resource);
-                }
-                iter.remove();
-            } else if (param.equals("-local")) {
-                isLocal = true;
-                iter.remove();
-            } else if (param.equals("-ttl")) {
-                if (!iter.hasNext()) {
-                    throw new RuntimeException("ttl value not specified");
-                }
-                iter.remove();
-                String ttlValue = iter.next();
-                try {
-                    ttl = Integer.parseInt(ttlValue);
-                } catch (NumberFormatException nfe) {
-                    throw new RuntimeException("ttl value incorrect");
-                }
-                iter.remove();
-            }
-        }
-
-        return newArgs.toArray(new String[newArgs.size()]);
-    }
-
-    public static Config loadConf(String resource, Config conf)
-            throws FileNotFoundException {
-        Yaml yaml = new Yaml();
-        Map ret = (Map) yaml.load(new InputStreamReader(
-                new FileInputStream(resource), Charset.defaultCharset()));
-        if (ret == null) {
-            ret = new HashMap();
-        }
-        // If the config consists of a single key 'config', its values are used
-        // instead. This means that the same config files can be used with Flux
-        // and the ConfigurableTopology.
-        else {
-            if (ret.size() == 1) {
-                Object confNode = ret.get("config");
-                if (confNode != null && confNode instanceof Map) {
-                    ret = (Map) ret;
-                }
-            }
-        }
-        conf.putAll(ret);
-        return conf;
-    }
-}


[09/10] storm git commit: Merge branch 'STORM-2447' of https://github.com/revans2/incubator-storm into STORM-2447-merge

Posted by ka...@apache.org.
Merge branch 'STORM-2447' of https://github.com/revans2/incubator-storm into STORM-2447-merge


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

Branch: refs/heads/master
Commit: a011de4e8b489f5ee09b9079a1271bd22d78b8f2
Parents: 65c8b7b 71dc2e0
Author: Jungtaek Lim <ka...@gmail.com>
Authored: Wed Apr 12 16:33:26 2017 +0900
Committer: Jungtaek Lim <ka...@gmail.com>
Committed: Wed Apr 12 16:33:26 2017 +0900

----------------------------------------------------------------------
 bin/storm.py                                    | 143 ++++-
 docs/Clojure-DSL.md                             |   2 +-
 docs/Distributed-RPC.md                         |  27 +-
 docs/Local-mode.md                              |  65 +-
 docs/Tutorial.md                                |  31 +-
 docs/flux.md                                    |  33 +-
 docs/storm-hbase.md                             |  24 +-
 examples/storm-elasticsearch-examples/pom.xml   |   2 +-
 .../elasticsearch/bolt/EsIndexTopology.java     |  26 +-
 .../trident/TridentEsTopology.java              |  23 +-
 examples/storm-hbase-examples/pom.xml           |   2 +-
 .../storm/hbase/topology/LookupWordCount.java   |  27 +-
 .../hbase/topology/PersistentWordCount.java     |  33 +-
 .../storm/hbase/trident/WordCountTrident.java   |  27 +-
 examples/storm-hdfs-examples/pom.xml            |   2 +-
 .../storm/hdfs/bolt/HdfsFileTopology.java       |  52 +-
 .../storm/hdfs/bolt/SequenceFileTopology.java   |  51 +-
 .../storm/hdfs/spout/HdfsSpoutTopology.java     |   6 +-
 .../storm/hdfs/trident/TridentFileTopology.java |  37 +-
 .../hdfs/trident/TridentSequenceTopology.java   |  38 +-
 examples/storm-hive-examples/pom.xml            |   2 +-
 .../storm/hive/bolt/BucketTestHiveTopology.java |  44 +-
 .../apache/storm/hive/bolt/HiveTopology.java    |  30 +-
 .../hive/bolt/HiveTopologyPartitioned.java      |  29 +-
 .../storm/hive/trident/TridentHiveTopology.java |  70 +-
 examples/storm-jdbc-examples/pom.xml            |   2 +-
 .../jdbc/topology/AbstractUserTopology.java     |  31 +-
 examples/storm-jms-examples/pom.xml             |  37 +-
 .../storm/jms/example/ExampleJmsTopology.java   |  23 +-
 examples/storm-kafka-client-examples/pom.xml    |   4 +-
 .../TridentKafkaClientWordCountNamedTopics.java |  70 +-
 examples/storm-kafka-examples/pom.xml           |   2 +-
 .../storm/kafka/trident/DrpcResultsPrinter.java |  18 +-
 .../storm/kafka/trident/LocalSubmitter.java     | 106 ---
 .../trident/TridentKafkaConsumerTopology.java   |  17 +-
 .../kafka/trident/TridentKafkaTopology.java     |  13 +-
 .../kafka/trident/TridentKafkaWordCount.java    |  48 +-
 examples/storm-mongodb-examples/pom.xml         |   2 +-
 .../storm/mongodb/topology/InsertWordCount.java |  25 +-
 .../storm/mongodb/topology/LookupWordCount.java |  18 +-
 .../storm/mongodb/topology/UpdateWordCount.java |  24 +-
 .../storm/mongodb/trident/WordCountTrident.java |  20 +-
 .../mongodb/trident/WordCountTridentMap.java    |  22 +-
 examples/storm-mqtt-examples/pom.xml            |   2 +-
 examples/storm-opentsdb-examples/pom.xml        |   2 +-
 .../opentsdb/SampleOpenTsdbBoltTopology.java    |  27 +-
 .../opentsdb/SampleOpenTsdbTridentTopology.java |  26 +-
 examples/storm-perf/pom.xml                     |   2 +-
 .../perf/ConstSpoutIdBoltNullBoltTopo.java      |  37 +-
 .../storm/perf/ConstSpoutNullBoltTopo.java      |  42 +-
 .../apache/storm/perf/ConstSpoutOnlyTopo.java   |  32 +-
 .../storm/perf/FileReadWordCountTopo.java       |  42 +-
 .../storm/perf/StrGenSpoutHdfsBoltTopo.java     |  45 +-
 .../storm/perf/utils/BasicMetricsCollector.java |  49 +-
 .../org/apache/storm/perf/utils/Helper.java     | 134 ++--
 .../apache/storm/perf/utils/MetricsSample.java  |  15 +-
 examples/storm-pmml-examples/pom.xml            |   4 +-
 .../storm/pmml/JpmmlRunnerTestTopology.java     |  44 +-
 examples/storm-redis-examples/pom.xml           |   2 +-
 .../storm/redis/topology/LookupWordCount.java   |  35 +-
 .../redis/topology/PersistentWordCount.java     |  32 +-
 .../redis/topology/WhitelistWordCount.java      |  25 +-
 .../redis/trident/WordCountTridentRedis.java    |  31 +-
 .../trident/WordCountTridentRedisCluster.java   |  37 +-
 .../WordCountTridentRedisClusterMap.java        |  38 +-
 .../redis/trident/WordCountTridentRedisMap.java |  31 +-
 examples/storm-solr-examples/pom.xml            |   2 +-
 .../storm/solr/topology/SolrTopology.java       |  23 +-
 examples/storm-starter/pom.xml                  |  27 +-
 .../apache/storm/starter/clj/exclamation.clj    |   9 +-
 .../storm/starter/clj/rolling_top_words.clj     |   9 +-
 .../org/apache/storm/starter/clj/word_count.clj |   9 +-
 .../apache/storm/starter/BasicDRPCTopology.java |  61 +-
 .../storm/starter/ExclamationTopology.java      |   6 +-
 .../storm/starter/FastWordCountTopology.java    |   6 +-
 .../storm/starter/InOrderDeliveryTest.java      |   6 +-
 .../apache/storm/starter/JoinBoltExample.java   |  20 +-
 .../org/apache/storm/starter/ManualDRPC.java    |  25 +-
 .../storm/starter/MultipleLoggerTopology.java   |  20 +-
 .../org/apache/storm/starter/ReachTopology.java | 239 ++++---
 .../starter/ResourceAwareExampleTopology.java   |  20 +-
 .../apache/storm/starter/RollingTopWords.java   |   9 -
 .../apache/storm/starter/SingleJoinExample.java |  60 +-
 .../storm/starter/SkewedRollingTopWords.java    |   9 -
 .../storm/starter/SlidingTupleTsTopology.java   |  28 +-
 .../storm/starter/SlidingWindowTopology.java    |  29 +-
 .../apache/storm/starter/StatefulTopology.java  |  19 +-
 .../starter/StatefulWindowingTopology.java      |  22 +-
 .../storm/starter/ThroughputVsLatency.java      | 638 +++++++++----------
 .../storm/starter/TransactionalGlobalCount.java | 235 +++----
 .../storm/starter/TransactionalWords.java       | 355 ++++++-----
 .../apache/storm/starter/WordCountTopology.java |  10 +-
 .../storm/starter/WordCountTopologyNode.java    | 107 ++--
 .../storm/starter/streams/AggregateExample.java |  13 +-
 .../storm/starter/streams/BranchExample.java    |  14 +-
 .../streams/GroupByKeyAndWindowExample.java     |  20 +-
 .../storm/starter/streams/JoinExample.java      |  20 +-
 .../starter/streams/StateQueryExample.java      |  17 +-
 .../starter/streams/StatefulWordCount.java      |  14 +-
 .../starter/streams/TypedTupleExample.java      |  19 +-
 .../starter/streams/WindowedWordCount.java      |  20 +-
 .../storm/starter/streams/WordCountToBolt.java  |  13 +-
 .../TridentHBaseWindowingStoreTopology.java     |  22 +-
 .../starter/trident/TridentMapExample.java      |  35 +-
 .../trident/TridentMinMaxOfDevicesTopology.java |  25 +-
 .../TridentMinMaxOfVehiclesTopology.java        |  25 +-
 .../storm/starter/trident/TridentReach.java     | 190 +++---
 .../TridentWindowingInmemoryStoreTopology.java  |  36 +-
 .../storm/starter/trident/TridentWordCount.java |  82 ++-
 external/storm-eventhubs/pom.xml                |   7 -
 .../storm/eventhubs/samples/EventCount.java     |  39 +-
 .../main/java/org/apache/storm/flux/Flux.java   |  72 +--
 flux/pom.xml                                    |   9 +-
 .../org/apache/storm/ExclamationTopology.java   |  16 +-
 .../apache/storm/st/wrapper/StormCluster.java   |   4 +-
 pom.xml                                         |   3 +-
 .../jvm/org/apache/storm/drpc/DRPCSpout.java    |  10 +-
 .../storm/security/auth/ThriftClient.java       |   6 +-
 .../security/auth/ThriftConnectionType.java     |  29 +-
 .../storm/topology/ConfigurableTopology.java    | 151 +++++
 .../jvm/org/apache/storm/utils/DRPCClient.java  |  78 ++-
 .../org/apache/storm/utils/NimbusClient.java    |  45 +-
 .../src/jvm/org/apache/storm/utils/Utils.java   |   2 +-
 storm-clojure-test/pom.xml                      |  66 ++
 .../src/clj/org/apache/storm/testing.clj        | 270 ++++++++
 storm-clojure/pom.xml                           |  15 +-
 .../src/clj/org/apache/storm/config.clj         |  28 +
 storm-clojure/src/clj/org/apache/storm/log.clj  |  34 +
 .../src/clj/org/apache/storm/testing.clj        | 270 --------
 storm-clojure/src/clj/org/apache/storm/util.clj | 134 ++++
 storm-core/pom.xml                              |   2 +-
 .../jvm/org/apache/storm/command/Activate.java  |   2 +-
 .../org/apache/storm/command/Deactivate.java    |   2 +-
 .../jvm/org/apache/storm/command/GetErrors.java |   2 +-
 .../org/apache/storm/command/KillTopology.java  |   2 +-
 .../apache/storm/command/ListTopologies.java    |   2 +-
 .../jvm/org/apache/storm/command/Monitor.java   |   2 +-
 .../jvm/org/apache/storm/command/Rebalance.java |   2 +-
 .../org/apache/storm/command/SetLogLevel.java   |   2 +-
 .../src/jvm/org/apache/storm/utils/Monitor.java |   6 +-
 .../java/org/apache/storm/LocalCluster.java     | 296 ++++++++-
 .../storm/topology/ConfigurableTopology.java    | 184 ------
 142 files changed, 3102 insertions(+), 3300 deletions(-)
----------------------------------------------------------------------



[02/10] storm git commit: STORM-2447: add in storm local to avoid having server on worker classpath

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentMinMaxOfVehiclesTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentMinMaxOfVehiclesTopology.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentMinMaxOfVehiclesTopology.java
index 947b64b..0d77914 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentMinMaxOfVehiclesTopology.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentMinMaxOfVehiclesTopology.java
@@ -17,9 +17,12 @@
  */
 package org.apache.storm.starter.trident;
 
+import java.io.Serializable;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.generated.StormTopology;
 import org.apache.storm.trident.Stream;
@@ -29,12 +32,6 @@ import org.apache.storm.trident.testing.FixedBatchSpout;
 import org.apache.storm.trident.tuple.TridentTuple;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Values;
-import org.apache.storm.utils.Utils;
-
-import java.io.Serializable;
-import java.util.Comparator;
-import java.util.List;
-import java.util.concurrent.ThreadLocalRandom;
 
 /**
  * This class demonstrates different usages of
@@ -94,16 +91,8 @@ public class TridentMinMaxOfVehiclesTopology {
         StormTopology topology = buildVehiclesTopology();
         Config conf = new Config();
         conf.setMaxSpoutPending(20);
-        if (args.length == 0) {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology("vehicles-topology", conf, topology);) {
-                Utils.sleep(60 * 1000);
-            }
-            System.exit(0);
-        } else {
-            conf.setNumWorkers(3);
-            StormSubmitter.submitTopologyWithProgressBar("vehicles-topology", conf, topology);
-        }
+        conf.setNumWorkers(3);
+        StormSubmitter.submitTopologyWithProgressBar("vehicles-topology", conf, topology);
     }
 
     static class SpeedComparator implements Comparator<TridentTuple>, Serializable {

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentReach.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentReach.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentReach.java
index 6533a4e..7294091 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentReach.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentReach.java
@@ -17,14 +17,16 @@
  */
 package org.apache.storm.starter.trident;
 
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
-import org.apache.storm.LocalDRPC;
+import org.apache.storm.StormSubmitter;
 import org.apache.storm.generated.StormTopology;
 import org.apache.storm.task.IMetricsContext;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Values;
 import org.apache.storm.trident.TridentState;
 import org.apache.storm.trident.TridentTopology;
 import org.apache.storm.trident.operation.BaseFunction;
@@ -37,117 +39,115 @@ import org.apache.storm.trident.state.State;
 import org.apache.storm.trident.state.StateFactory;
 import org.apache.storm.trident.state.map.ReadOnlyMapState;
 import org.apache.storm.trident.tuple.TridentTuple;
-
-import java.util.*;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.DRPCClient;
 
 public class TridentReach {
-  public static Map<String, List<String>> TWEETERS_DB = new HashMap<String, List<String>>() {{
-    put("foo.com/blog/1", Arrays.asList("sally", "bob", "tim", "george", "nathan"));
-    put("engineering.twitter.com/blog/5", Arrays.asList("adam", "david", "sally", "nathan"));
-    put("tech.backtype.com/blog/123", Arrays.asList("tim", "mike", "john"));
-  }};
-
-  public static Map<String, List<String>> FOLLOWERS_DB = new HashMap<String, List<String>>() {{
-    put("sally", Arrays.asList("bob", "tim", "alice", "adam", "jim", "chris", "jai"));
-    put("bob", Arrays.asList("sally", "nathan", "jim", "mary", "david", "vivian"));
-    put("tim", Arrays.asList("alex"));
-    put("nathan", Arrays.asList("sally", "bob", "adam", "harry", "chris", "vivian", "emily", "jordan"));
-    put("adam", Arrays.asList("david", "carissa"));
-    put("mike", Arrays.asList("john", "bob"));
-    put("john", Arrays.asList("alice", "nathan", "jim", "mike", "bob"));
-  }};
-
-  public static class StaticSingleKeyMapState extends ReadOnlyState implements ReadOnlyMapState<Object> {
-    public static class Factory implements StateFactory {
-      Map _map;
-
-      public Factory(Map map) {
-        _map = map;
-      }
-
-      @Override
-      public State makeState(Map conf, IMetricsContext metrics, int partitionIndex, int numPartitions) {
-        return new StaticSingleKeyMapState(_map);
-      }
+    public static Map<String, List<String>> TWEETERS_DB = new HashMap<String, List<String>>() {{
+        put("foo.com/blog/1", Arrays.asList("sally", "bob", "tim", "george", "nathan"));
+        put("engineering.twitter.com/blog/5", Arrays.asList("adam", "david", "sally", "nathan"));
+        put("tech.backtype.com/blog/123", Arrays.asList("tim", "mike", "john"));
+    }};
+
+    public static Map<String, List<String>> FOLLOWERS_DB = new HashMap<String, List<String>>() {{
+        put("sally", Arrays.asList("bob", "tim", "alice", "adam", "jim", "chris", "jai"));
+        put("bob", Arrays.asList("sally", "nathan", "jim", "mary", "david", "vivian"));
+        put("tim", Arrays.asList("alex"));
+        put("nathan", Arrays.asList("sally", "bob", "adam", "harry", "chris", "vivian", "emily", "jordan"));
+        put("adam", Arrays.asList("david", "carissa"));
+        put("mike", Arrays.asList("john", "bob"));
+        put("john", Arrays.asList("alice", "nathan", "jim", "mike", "bob"));
+    }};
+
+    public static class StaticSingleKeyMapState extends ReadOnlyState implements ReadOnlyMapState<Object> {
+        public static class Factory implements StateFactory {
+            Map _map;
+
+            public Factory(Map map) {
+                _map = map;
+            }
+
+            @Override
+            public State makeState(Map conf, IMetricsContext metrics, int partitionIndex, int numPartitions) {
+                return new StaticSingleKeyMapState(_map);
+            }
 
-    }
-
-    Map _map;
-
-    public StaticSingleKeyMapState(Map map) {
-      _map = map;
-    }
+        }
 
+        Map _map;
 
-    @Override
-    public List<Object> multiGet(List<List<Object>> keys) {
-      List<Object> ret = new ArrayList();
-      for (List<Object> key : keys) {
-        Object singleKey = key.get(0);
-        ret.add(_map.get(singleKey));
-      }
-      return ret;
-    }
+        public StaticSingleKeyMapState(Map map) {
+            _map = map;
+        }
 
-  }
 
-  public static class One implements CombinerAggregator<Integer> {
-    @Override
-    public Integer init(TridentTuple tuple) {
-      return 1;
-    }
+        @Override
+        public List<Object> multiGet(List<List<Object>> keys) {
+            List<Object> ret = new ArrayList();
+            for (List<Object> key : keys) {
+                Object singleKey = key.get(0);
+                ret.add(_map.get(singleKey));
+            }
+            return ret;
+        }
 
-    @Override
-    public Integer combine(Integer val1, Integer val2) {
-      return 1;
     }
 
-    @Override
-    public Integer zero() {
-      return 1;
-    }
-  }
+    public static class One implements CombinerAggregator<Integer> {
+        @Override
+        public Integer init(TridentTuple tuple) {
+            return 1;
+        }
 
-  public static class ExpandList extends BaseFunction {
+        @Override
+        public Integer combine(Integer val1, Integer val2) {
+            return 1;
+        }
 
-    @Override
-    public void execute(TridentTuple tuple, TridentCollector collector) {
-      List l = (List) tuple.getValue(0);
-      if (l != null) {
-        for (Object o : l) {
-          collector.emit(new Values(o));
+        @Override
+        public Integer zero() {
+            return 1;
         }
-      }
     }
 
-  }
+    public static class ExpandList extends BaseFunction {
 
-  public static StormTopology buildTopology(LocalDRPC drpc) {
-    TridentTopology topology = new TridentTopology();
-    TridentState urlToTweeters = topology.newStaticState(new StaticSingleKeyMapState.Factory(TWEETERS_DB));
-    TridentState tweetersToFollowers = topology.newStaticState(new StaticSingleKeyMapState.Factory(FOLLOWERS_DB));
+        @Override
+        public void execute(TridentTuple tuple, TridentCollector collector) {
+            List l = (List) tuple.getValue(0);
+            if (l != null) {
+                for (Object o : l) {
+                    collector.emit(new Values(o));
+                }
+            }
+        }
 
+    }
 
-    topology.newDRPCStream("reach", drpc).stateQuery(urlToTweeters, new Fields("args"), new MapGet(), new Fields(
-        "tweeters")).each(new Fields("tweeters"), new ExpandList(), new Fields("tweeter")).shuffle().stateQuery(
-        tweetersToFollowers, new Fields("tweeter"), new MapGet(), new Fields("followers")).each(new Fields("followers"),
-        new ExpandList(), new Fields("follower")).groupBy(new Fields("follower")).aggregate(new One(), new Fields(
-        "one")).aggregate(new Fields("one"), new Sum(), new Fields("reach"));
-    return topology.build();
-  }
+    public static StormTopology buildTopology() {
+        TridentTopology topology = new TridentTopology();
+        TridentState urlToTweeters = topology.newStaticState(new StaticSingleKeyMapState.Factory(TWEETERS_DB));
+        TridentState tweetersToFollowers = topology.newStaticState(new StaticSingleKeyMapState.Factory(FOLLOWERS_DB));
 
-  public static void main(String[] args) throws Exception {
 
-    Config conf = new Config();
-    try (LocalDRPC drpc = new LocalDRPC();
-         LocalCluster cluster = new LocalCluster();
-         LocalTopology topo = cluster.submitTopology("reach", conf, buildTopology(drpc));) {
+        topology.newDRPCStream("reach").stateQuery(urlToTweeters, new Fields("args"), new MapGet(), new Fields(
+                "tweeters")).each(new Fields("tweeters"), new ExpandList(), new Fields("tweeter")).shuffle().stateQuery(
+                        tweetersToFollowers, new Fields("tweeter"), new MapGet(), new Fields("followers")).each(new Fields("followers"),
+                                new ExpandList(), new Fields("follower")).groupBy(new Fields("follower")).aggregate(new One(), new Fields(
+                                        "one")).aggregate(new Fields("one"), new Sum(), new Fields("reach"));
+        return topology.build();
+    }
 
-        Thread.sleep(2000);
+    public static void main(String[] args) throws Exception {
+        Config conf = new Config();
+        StormSubmitter.submitTopology("reach", conf, buildTopology());
+        try (DRPCClient drpc = DRPCClient.getConfiguredClient(conf)) {
+            Thread.sleep(2000);
 
-        System.out.println("REACH: " + drpc.execute("reach", "aaa"));
-        System.out.println("REACH: " + drpc.execute("reach", "foo.com/blog/1"));
-        System.out.println("REACH: " + drpc.execute("reach", "engineering.twitter.com/blog/5"));
+            System.out.println("REACH: " + drpc.execute("reach", "aaa"));
+            System.out.println("REACH: " + drpc.execute("reach", "foo.com/blog/1"));
+            System.out.println("REACH: " + drpc.execute("reach", "engineering.twitter.com/blog/5"));
+        }
     }
-  }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentWindowingInmemoryStoreTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentWindowingInmemoryStoreTopology.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentWindowingInmemoryStoreTopology.java
index c43b4b0..2da29bf 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentWindowingInmemoryStoreTopology.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentWindowingInmemoryStoreTopology.java
@@ -19,11 +19,8 @@
 package org.apache.storm.starter.trident;
 
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.generated.StormTopology;
-import org.apache.storm.topology.base.BaseWindowedBolt;
 import org.apache.storm.trident.Stream;
 import org.apache.storm.trident.TridentTopology;
 import org.apache.storm.trident.operation.Consumer;
@@ -33,17 +30,13 @@ import org.apache.storm.trident.testing.Split;
 import org.apache.storm.trident.tuple.TridentTuple;
 import org.apache.storm.trident.windowing.InMemoryWindowsStoreFactory;
 import org.apache.storm.trident.windowing.WindowsStoreFactory;
-import org.apache.storm.trident.windowing.config.*;
+import org.apache.storm.trident.windowing.config.SlidingCountWindow;
+import org.apache.storm.trident.windowing.config.WindowConfig;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Values;
-import org.apache.storm.utils.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.Arrays;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
 /**
  * Sample application of trident windowing which uses inmemory store for storing tuples in window.
  */
@@ -74,25 +67,12 @@ public class TridentWindowingInmemoryStoreTopology {
     public static void main(String[] args) throws Exception {
         Config conf = new Config();
         WindowsStoreFactory mapState = new InMemoryWindowsStoreFactory();
-
-        if (args.length == 0) {
-            List<? extends WindowConfig> list = Arrays.asList(
-                    SlidingCountWindow.of(1000, 100)
-                    ,TumblingCountWindow.of(1000)
-                    ,SlidingDurationWindow.of(new BaseWindowedBolt.Duration(6, TimeUnit.SECONDS), new BaseWindowedBolt.Duration(3, TimeUnit.SECONDS))
-                    ,TumblingDurationWindow.of(new BaseWindowedBolt.Duration(3, TimeUnit.SECONDS))
-            );
-
-            for (WindowConfig windowConfig : list) {
-                try (LocalCluster cluster = new LocalCluster();
-                     LocalTopology topo = cluster.submitTopology("wordCounter", conf, buildTopology(mapState, windowConfig));) {
-                    Utils.sleep(60 * 1000);
-                }
-            }
-            System.exit(0);
-        } else {
-            conf.setNumWorkers(3);
-            StormSubmitter.submitTopologyWithProgressBar(args[0], conf, buildTopology(mapState, SlidingCountWindow.of(1000, 100)));
+        String topoName = "wordCounter";
+        if (args.length > 0) {
+            topoName = args[0];
         }
+        
+        conf.setNumWorkers(3);
+        StormSubmitter.submitTopologyWithProgressBar(topoName, conf, buildTopology(mapState, SlidingCountWindow.of(1000, 100)));
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentWordCount.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentWordCount.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentWordCount.java
index 553c26f..0f86e1f 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentWordCount.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentWordCount.java
@@ -18,9 +18,6 @@
 package org.apache.storm.starter.trident;
 
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
-import org.apache.storm.LocalDRPC;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.generated.StormTopology;
 import org.apache.storm.trident.TridentState;
@@ -35,54 +32,53 @@ import org.apache.storm.trident.testing.MemoryMapState;
 import org.apache.storm.trident.tuple.TridentTuple;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.DRPCClient;
 
 
 public class TridentWordCount {
-  public static class Split extends BaseFunction {
-    @Override
-    public void execute(TridentTuple tuple, TridentCollector collector) {
-      String sentence = tuple.getString(0);
-      for (String word : sentence.split(" ")) {
-        collector.emit(new Values(word));
-      }
+    public static class Split extends BaseFunction {
+        @Override
+        public void execute(TridentTuple tuple, TridentCollector collector) {
+            String sentence = tuple.getString(0);
+            for (String word : sentence.split(" ")) {
+                collector.emit(new Values(word));
+            }
+        }
     }
-  }
 
-  public static StormTopology buildTopology(LocalDRPC drpc) {
-    FixedBatchSpout spout = new FixedBatchSpout(new Fields("sentence"), 3, new Values("the cow jumped over the moon"),
-        new Values("the man went to the store and bought some candy"), new Values("four score and seven years ago"),
-        new Values("how many apples can you eat"), new Values("to be or not to be the person"));
-    spout.setCycle(true);
+    public static StormTopology buildTopology() {
+        FixedBatchSpout spout = new FixedBatchSpout(new Fields("sentence"), 3, new Values("the cow jumped over the moon"),
+                new Values("the man went to the store and bought some candy"), new Values("four score and seven years ago"),
+                new Values("how many apples can you eat"), new Values("to be or not to be the person"));
+        spout.setCycle(true);
 
-    TridentTopology topology = new TridentTopology();
-    TridentState wordCounts = topology.newStream("spout1", spout).parallelismHint(16).each(new Fields("sentence"),
-        new Split(), new Fields("word")).groupBy(new Fields("word")).persistentAggregate(new MemoryMapState.Factory(),
-        new Count(), new Fields("count")).parallelismHint(16);
+        TridentTopology topology = new TridentTopology();
+        TridentState wordCounts = topology.newStream("spout1", spout).parallelismHint(16).each(new Fields("sentence"),
+                new Split(), new Fields("word")).groupBy(new Fields("word")).persistentAggregate(new MemoryMapState.Factory(),
+                        new Count(), new Fields("count")).parallelismHint(16);
 
-    topology.newDRPCStream("words", drpc).each(new Fields("args"), new Split(), new Fields("word"))
-            .groupBy(new Fields("word"))
-            .stateQuery(wordCounts, new Fields("word"), new MapGet(), new Fields("count"))
-            .each(new Fields("count"), new FilterNull())
-            .project(new Fields("word", "count"));
-    return topology.build();
-  }
+        topology.newDRPCStream("words").each(new Fields("args"), new Split(), new Fields("word"))
+        .groupBy(new Fields("word"))
+        .stateQuery(wordCounts, new Fields("word"), new MapGet(), new Fields("count"))
+        .each(new Fields("count"), new FilterNull())
+        .project(new Fields("word", "count"));
+        return topology.build();
+    }
 
-  public static void main(String[] args) throws Exception {
-    Config conf = new Config();
-    conf.setMaxSpoutPending(20);
-    if (args.length == 0) {
-      try (LocalDRPC drpc = new LocalDRPC();
-           LocalCluster cluster = new LocalCluster();
-           LocalTopology topo = cluster.submitTopology("wordCounter", conf, buildTopology(drpc));) {
-        for (int i = 0; i < 100; i++) {
-          System.out.println("DRPC RESULT: " + drpc.execute("words", "cat the dog jumped"));
-          Thread.sleep(1000);
+    public static void main(String[] args) throws Exception {
+        Config conf = new Config();
+        conf.setMaxSpoutPending(20);
+        String topoName = "wordCounter";
+        if (args.length > 0) {
+            topoName = args[0];
+        }
+        conf.setNumWorkers(3);
+        StormSubmitter.submitTopologyWithProgressBar(topoName, conf, buildTopology());
+        try (DRPCClient drpc = DRPCClient.getConfiguredClient(conf)) {
+            for (int i = 0; i < 10; i++) {
+                System.out.println("DRPC RESULT: " + drpc.execute("words", "cat the dog jumped"));
+                Thread.sleep(1000);
+            }
         }
-      }
-    }
-    else {
-      conf.setNumWorkers(3);
-      StormSubmitter.submitTopologyWithProgressBar(args[0], conf, buildTopology(null));
     }
-  }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/external/storm-eventhubs/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-eventhubs/pom.xml b/external/storm-eventhubs/pom.xml
index 7fe6071..deaefb8 100755
--- a/external/storm-eventhubs/pom.xml
+++ b/external/storm-eventhubs/pom.xml
@@ -65,13 +65,6 @@
             <scope>${provided.scope}</scope>
         </dependency>
         <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>storm-server</artifactId>
-            <version>${project.version}</version>
-            <type>jar</type>
-            <scope>test</scope>
-        </dependency>
-        <dependency>
             <groupId>org.apache.curator</groupId>
             <artifactId>curator-framework</artifactId>
             <version>${curator.version}</version>

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/EventCount.java
----------------------------------------------------------------------
diff --git a/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/EventCount.java b/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/EventCount.java
index ae15634..4709b5e 100755
--- a/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/EventCount.java
+++ b/external/storm-eventhubs/src/test/java/org/apache/storm/eventhubs/samples/EventCount.java
@@ -17,20 +17,17 @@
  *******************************************************************************/
 package org.apache.storm.eventhubs.samples;
 
-import org.apache.storm.StormSubmitter;
-import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
-import org.apache.storm.generated.StormTopology;
-import org.apache.storm.topology.TopologyBuilder;
+import java.io.FileReader;
+import java.util.Properties;
 
+import org.apache.storm.Config;
+import org.apache.storm.StormSubmitter;
 import org.apache.storm.eventhubs.samples.bolt.GlobalCountBolt;
 import org.apache.storm.eventhubs.samples.bolt.PartialCountBolt;
 import org.apache.storm.eventhubs.spout.EventHubSpout;
 import org.apache.storm.eventhubs.spout.EventHubSpoutConfig;
-
-import java.io.FileReader;
-import java.util.Properties;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.topology.TopologyBuilder;
 
 /**
  * The basic scenario topology that uses EventHubSpout with PartialCountBolt
@@ -124,23 +121,17 @@ public class EventCount {
   }
 	
   protected void submitTopology(String[] args, StormTopology topology) throws Exception {
-	  Config config = new Config();
-    config.setDebug(false);
-    //Enable metrics
-    config.registerMetricsConsumer(org.apache.storm.metric.LoggingMetricsConsumer.class, 1);
+      Config config = new Config();
+      config.setDebug(false);
+      //Enable metrics
+      config.registerMetricsConsumer(org.apache.storm.metric.LoggingMetricsConsumer.class, 1);
 
-    
-	  if (args != null && args.length > 0) {
-      config.setNumWorkers(numWorkers);
-      StormSubmitter.submitTopology(args[0], config, topology);
-    } else {
-      config.setMaxTaskParallelism(2);
-
-      try (LocalCluster localCluster = new LocalCluster();
-           LocalTopology topo = localCluster.submitTopology("test", config, topology);) {
-        Thread.sleep(5000000);
+      String topoName = "test";
+      if (args.length > 0) {
+          topoName = args[0];
       }
-    }
+      config.setNumWorkers(numWorkers);
+      StormSubmitter.submitTopology(topoName, config, topology);
   }
   
   protected void runScenario(String[] args) throws Exception{

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/flux/flux-core/src/main/java/org/apache/storm/flux/Flux.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/Flux.java b/flux/flux-core/src/main/java/org/apache/storm/flux/Flux.java
index 222bf2d..1d38ada 100644
--- a/flux/flux-core/src/main/java/org/apache/storm/flux/Flux.java
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/Flux.java
@@ -17,22 +17,33 @@
  */
 package org.apache.storm.flux;
 
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+
+import org.apache.commons.cli.BasicParser;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
+import org.apache.storm.flux.model.BoltDef;
+import org.apache.storm.flux.model.ExecutionContext;
+import org.apache.storm.flux.model.SpoutDef;
+import org.apache.storm.flux.model.StreamDef;
+import org.apache.storm.flux.model.TopologyDef;
+import org.apache.storm.flux.parser.FluxParser;
 import org.apache.storm.generated.StormTopology;
 import org.apache.storm.generated.SubmitOptions;
 import org.apache.storm.generated.TopologyInitialStatus;
-import org.apache.storm.utils.Utils;
-import org.apache.commons.cli.*;
-import org.apache.storm.flux.model.*;
-import org.apache.storm.flux.parser.FluxParser;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.*;
-
 /**
  * Flux entry point.
  *
@@ -40,10 +51,6 @@ import java.io.*;
 public class Flux {
     private static final Logger LOG = LoggerFactory.getLogger(Flux.class);
 
-    private static final Long DEFAULT_LOCAL_SLEEP_TIME = 60000l;
-
-    private static final Long DEFAULT_ZK_PORT = 2181l;
-
     private static final String OPTION_LOCAL = "local";
     private static final String OPTION_REMOTE = "remote";
     private static final String OPTION_RESOURCE = "resource";
@@ -169,45 +176,8 @@ public class Flux {
                 }
                 StormSubmitter.submitTopology(topologyName, conf, topology, submitOptions, null);
             } else {
-                LOG.info("Running in local mode...");
-
-                String sleepStr = cmd.getOptionValue(OPTION_SLEEP);
-                Long sleepTime = DEFAULT_LOCAL_SLEEP_TIME;
-                if (sleepStr != null) {
-                    sleepTime = Long.parseLong(sleepStr);
-                }
-                LOG.debug("Sleep time: {}", sleepTime);
-                LocalCluster cluster = null;
-
-                // in-process or external zookeeper
-                if(cmd.hasOption(OPTION_ZOOKEEPER)){
-                    String zkStr = cmd.getOptionValue(OPTION_ZOOKEEPER);
-                    LOG.info("Using ZooKeeper at '{}' instead of in-process one.", zkStr);
-                    long zkPort = DEFAULT_ZK_PORT;
-                    String zkHost = null;
-                    if(zkStr.contains(":")){
-                        String[] hostPort = zkStr.split(":");
-                        zkHost = hostPort[0];
-                        zkPort = hostPort.length > 1 ? Long.parseLong(hostPort[1]) : DEFAULT_ZK_PORT;
-
-                    } else {
-                        zkHost = zkStr;
-                    }
-                    // the following constructor is only available in 0.9.3 and later
-                    try {
-                        cluster = new LocalCluster(zkHost, zkPort);
-                    } catch (NoSuchMethodError e){
-                        LOG.error("The --zookeeper option can only be used with Apache Storm 0.9.3 and later.");
-                        System.exit(1);
-                    }
-                } else {
-                    cluster = new LocalCluster();
-                }
-                try (LocalTopology topo = cluster.submitTopology(topologyName, conf, topology)) {
-                    Utils.sleep(sleepTime);
-                } finally {
-                    cluster.shutdown();
-                }
+                LOG.error("To run in local mode run with 'storm local' instead of 'storm jar'");
+                return;
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/flux/pom.xml
----------------------------------------------------------------------
diff --git a/flux/pom.xml b/flux/pom.xml
index d1f330f..4e04fa8 100644
--- a/flux/pom.xml
+++ b/flux/pom.xml
@@ -44,16 +44,9 @@
     </modules>
 
     <dependencies>
-        <!--
-        Since Flux uses LocalCluster to provide the feature: running topology locally...
-        User should notice that configured topology will be run with 'storm-client' dependencies
-        when adding Flux into topology dependency.
-        If user want to run topology with 'storm-server' dependencies, user can just include
-        'storm-server' as 'compile' scope, and exclude 'storm-client' from 'storm-server'.
-        -->
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-server</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1281171..91017f9 100644
--- a/pom.xml
+++ b/pom.xml
@@ -320,11 +320,13 @@
         <module>storm-buildtools/maven-shade-clojure-transformer</module>
         <module>storm-buildtools/storm-maven-plugins</module>
         <module>storm-client</module>
+        <module>storm-client-misc</module>
         <module>storm-server</module>
         <module>storm-core</module>
         <module>storm-webapp</module>
         <module>storm-rename-hack</module>
         <module>storm-clojure</module>
+        <module>storm-clojure-test</module>
         <module>storm-submit-tools</module>
         <module>flux</module>
         <module>sql</module>
@@ -368,7 +370,6 @@
         <module>examples/storm-pmml-examples</module>
         <module>examples/storm-jms-examples</module>
         <module>examples/storm-perf</module>
-        <module>storm-client-misc</module>
     </modules>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/storm-client/src/jvm/org/apache/storm/drpc/DRPCSpout.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/drpc/DRPCSpout.java b/storm-client/src/jvm/org/apache/storm/drpc/DRPCSpout.java
index 5abb04a..f359722 100644
--- a/storm-client/src/jvm/org/apache/storm/drpc/DRPCSpout.java
+++ b/storm-client/src/jvm/org/apache/storm/drpc/DRPCSpout.java
@@ -30,6 +30,7 @@ import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Values;
 import org.apache.storm.utils.Utils;
 import org.apache.storm.utils.ObjectReader;
+import org.apache.storm.utils.DRPCClient;
 import org.apache.storm.utils.ExtendedThreadPoolExecutor;
 import org.apache.storm.utils.ServiceRegistry;
 import java.util.ArrayList;
@@ -59,8 +60,8 @@ public class DRPCSpout extends BaseRichSpout {
     List<DRPCInvocationsClient> _clients = new ArrayList<>();
     transient LinkedList<Future<Void>> _futures = null;
     transient ExecutorService _backround = null;
-    String _function;
-    String _local_drpc_id = null;
+    final String _function;
+    final String _local_drpc_id;
     
     private static class DRPCMessageId {
         String id;
@@ -75,6 +76,11 @@ public class DRPCSpout extends BaseRichSpout {
     
     public DRPCSpout(String function) {
         _function = function;
+        if (DRPCClient.isLocalOverride()) {
+            _local_drpc_id = DRPCClient.getOverrideServiceId();
+        } else {
+            _local_drpc_id = null; 
+        }
     }
 
     public DRPCSpout(String function, ILocalDRPC drpc) {

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/storm-client/src/jvm/org/apache/storm/security/auth/ThriftClient.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/ThriftClient.java b/storm-client/src/jvm/org/apache/storm/security/auth/ThriftClient.java
index 9a207b4..d84c5fa 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/ThriftClient.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/ThriftClient.java
@@ -56,7 +56,7 @@ public class ThriftClient implements AutoCloseable {
             port = type.getPort(storm_conf);
         }
 
-        if (port<=0) {
+        if (port<=0 && !type.isFake()) {
             throw new IllegalArgumentException("invalid port: "+port);
         }          
 
@@ -66,7 +66,9 @@ public class ThriftClient implements AutoCloseable {
         _conf = storm_conf;
         _type = type;
         _asUser = asUser;
-        reconnect();
+        if (!type.isFake()) {
+            reconnect();
+        }
     }
 
     public synchronized TTransport transport() {

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/storm-client/src/jvm/org/apache/storm/security/auth/ThriftConnectionType.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/security/auth/ThriftConnectionType.java b/storm-client/src/jvm/org/apache/storm/security/auth/ThriftConnectionType.java
index 0108d73..19d8dd6 100644
--- a/storm-client/src/jvm/org/apache/storm/security/auth/ThriftConnectionType.java
+++ b/storm-client/src/jvm/org/apache/storm/security/auth/ThriftConnectionType.java
@@ -31,7 +31,8 @@ public enum ThriftConnectionType {
     DRPC(Config.DRPC_THRIFT_TRANSPORT_PLUGIN, Config.DRPC_PORT, Config.DRPC_QUEUE_SIZE,
          Config.DRPC_WORKER_THREADS, Config.DRPC_MAX_BUFFER_SIZE, null),
     DRPC_INVOCATIONS(Config.DRPC_INVOCATIONS_THRIFT_TRANSPORT_PLUGIN, Config.DRPC_INVOCATIONS_PORT, null,
-         Config.DRPC_INVOCATIONS_THREADS, Config.DRPC_MAX_BUFFER_SIZE, null);
+         Config.DRPC_INVOCATIONS_THREADS, Config.DRPC_MAX_BUFFER_SIZE, null),
+    LOCAL_FAKE;
 
     private final String _transConf;
     private final String _portConf;
@@ -39,17 +40,32 @@ public enum ThriftConnectionType {
     private final String _threadsConf;
     private final String _buffConf;
     private final String _socketTimeoutConf;
+    private final boolean _isFake;
 
+    ThriftConnectionType() {
+        this(null, null, null, null, null, null, true);
+    }
+    
+    ThriftConnectionType(String transConf, String portConf, String qConf,
+            String threadsConf, String buffConf, String socketTimeoutConf) {
+        this(transConf, portConf, qConf, threadsConf, buffConf, socketTimeoutConf, false);
+    }
+    
     ThriftConnectionType(String transConf, String portConf, String qConf,
-                         String threadsConf, String buffConf, String socketTimeoutConf) {
+                         String threadsConf, String buffConf, String socketTimeoutConf, boolean isFake) {
         _transConf = transConf;
         _portConf = portConf;
         _qConf = qConf;
         _threadsConf = threadsConf;
         _buffConf = buffConf;
         _socketTimeoutConf = socketTimeoutConf;
+        _isFake = isFake;
     }
 
+    public boolean isFake() {
+        return _isFake;
+    }
+    
     public String getTransportPlugin(Map conf) {
         String ret = (String)conf.get(_transConf);
         if (ret == null) {
@@ -59,6 +75,9 @@ public enum ThriftConnectionType {
     }
 
     public int getPort(Map conf) {
+        if (_isFake) {
+            return -1;
+        }
         return ObjectReader.getInt(conf.get(_portConf));
     }
 
@@ -70,10 +89,16 @@ public enum ThriftConnectionType {
     }
 
     public int getNumThreads(Map conf) {
+        if (_isFake) {
+            return 1;
+        }
         return ObjectReader.getInt(conf.get(_threadsConf));
     }
 
     public int getMaxBufferSize(Map conf) {
+        if (_isFake) {
+            return 1;
+        }
         return ObjectReader.getInt(conf.get(_buffConf));
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/storm-client/src/jvm/org/apache/storm/topology/ConfigurableTopology.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/topology/ConfigurableTopology.java b/storm-client/src/jvm/org/apache/storm/topology/ConfigurableTopology.java
new file mode 100644
index 0000000..3641e07
--- /dev/null
+++ b/storm-client/src/jvm/org/apache/storm/topology/ConfigurableTopology.java
@@ -0,0 +1,151 @@
+/*
+ * 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.storm.topology;
+
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.InputStreamReader;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.storm.Config;
+import org.apache.storm.StormSubmitter;
+import org.apache.storm.utils.Utils;
+import org.yaml.snakeyaml.Yaml;
+
+/**
+ * Extensions of this class takes a
+ * reference to one or more configuration files. The main() method should call
+ * ConfigurableTopology.start() and it must instantiate a TopologyBuilder in the
+ * run() method.
+ * 
+ * <pre>
+ * {
+ *    public class MyTopology extends ConfigurableTopology {
+ *
+  *   public static void main(String[] args) throws Exception {
+  *       ConfigurableTopology.start(new MyTopology(), args);
+  *   }
+  *
+  *   &#64;Override
+  *   protected int run(String[] args) {
+  *       TopologyBuilder builder = new TopologyBuilder();
+  *
+  *       // build topology as usual
+  *    
+  *       return submit("crawl", conf, builder);
+  *   }
+ * }
+ * </pre>
+ **/
+public abstract class ConfigurableTopology {
+
+    protected Config conf = new Config();
+
+    public static void start(ConfigurableTopology topology, String args[]) {
+        String[] remainingArgs = topology.parse(args);
+        try {
+            topology.run(remainingArgs);
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    protected Config getConf() {
+        return conf;
+    }
+
+    protected abstract int run(String args[]) throws Exception;
+
+    /** Submits the topology with the name taken from the configuration **/
+    protected int submit(Config conf, TopologyBuilder builder) {
+        String name = (String) Utils.get(conf, Config.TOPOLOGY_NAME, null);
+        if (StringUtils.isBlank(name))
+            throw new RuntimeException(
+                    "No value found for " + Config.TOPOLOGY_NAME);
+        return submit(name, conf, builder);
+    }
+
+    /** Submits the topology under a specific name **/
+    protected int submit(String name, Config conf, TopologyBuilder builder) {
+        try {
+            StormSubmitter.submitTopology(name, conf,
+                    builder.createTopology());
+        } catch (Exception e) {
+            e.printStackTrace();
+            return -1;
+        }
+        return 0;
+    }
+
+    private String[] parse(String args[]) {
+
+        List<String> newArgs = new ArrayList<>();
+        Collections.addAll(newArgs, args);
+
+        Iterator<String> iter = newArgs.iterator();
+        while (iter.hasNext()) {
+            String param = iter.next();
+            if (param.equals("-conf")) {
+                if (!iter.hasNext()) {
+                    throw new RuntimeException("Conf file not specified");
+                }
+                iter.remove();
+                String resource = iter.next();
+                try {
+                    loadConf(resource, conf);
+                } catch (FileNotFoundException e) {
+                    throw new RuntimeException("File not found : " + resource);
+                }
+                iter.remove();
+            }
+        }
+
+        return newArgs.toArray(new String[newArgs.size()]);
+    }
+
+    public static Config loadConf(String resource, Config conf)
+            throws FileNotFoundException {
+        Yaml yaml = new Yaml();
+        Map<String, Object> ret = (Map<String, Object>) yaml.load(new InputStreamReader(
+                new FileInputStream(resource), Charset.defaultCharset()));
+        if (ret == null) {
+            ret = new HashMap<>();
+        }
+        // If the config consists of a single key 'config', its values are used
+        // instead. This means that the same config files can be used with Flux
+        // and the ConfigurableTopology.
+        else {
+            if (ret.size() == 1) {
+                Object confNode = ret.get("config");
+                if (confNode != null && confNode instanceof Map) {
+                    ret = (Map<String, Object>) ret;
+                }
+            }
+        }
+        conf.putAll(ret);
+        return conf;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/storm-client/src/jvm/org/apache/storm/utils/DRPCClient.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/utils/DRPCClient.java b/storm-client/src/jvm/org/apache/storm/utils/DRPCClient.java
index 7f83789..1338231 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/DRPCClient.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/DRPCClient.java
@@ -17,34 +17,90 @@
  */
 package org.apache.storm.utils;
 
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.storm.Config;
+import org.apache.storm.ILocalDRPC;
+import org.apache.storm.generated.AuthorizationException;
 import org.apache.storm.generated.DRPCExecutionException;
 import org.apache.storm.generated.DistributedRPC;
-import org.apache.storm.generated.AuthorizationException;
-import org.apache.thrift.TException;
-import org.apache.thrift.transport.TTransport;
 import org.apache.storm.security.auth.ThriftClient;
 import org.apache.storm.security.auth.ThriftConnectionType;
+import org.apache.thrift.TException;
 import org.apache.thrift.transport.TTransportException;
 
-import java.util.Map;
-
 public class DRPCClient extends ThriftClient implements DistributedRPC.Iface {
-    private TTransport conn;
-    private DistributedRPC.Client client;
+    private static volatile ILocalDRPC _localOverrideClient = null;
+    
+    public static class LocalOverride implements AutoCloseable {
+        public LocalOverride(ILocalDRPC client) {
+            _localOverrideClient = client;
+        }
+        
+        @Override
+        public void close() throws Exception {
+            _localOverrideClient = null;
+        }
+    }
+    
+    /**
+     * @return true of new clients will be overridden to connect to a local cluster
+     * and not the configured remote cluster.
+     */
+    public static boolean isLocalOverride() {
+        return _localOverrideClient != null;
+    }
+    
+    /**
+     * @return the service ID of the local override DRPC instance
+     */
+    public static String getOverrideServiceId() {
+        return _localOverrideClient.getServiceId();
+    }
+
+    public static DRPCClient getConfiguredClient(Map conf) throws TTransportException {
+        DistributedRPC.Iface override = _localOverrideClient;
+        if (override != null) {
+            return new DRPCClient(override);
+        }
+
+        List<String> servers = (List<String>) conf.get(Config.DRPC_SERVERS);
+        Collections.shuffle(servers);
+        String host = servers.get(0);
+        int port = Integer.parseInt(conf.get(Config.DRPC_PORT).toString());
+        return new DRPCClient(conf, host, port);
+    }
+    
+    private DistributedRPC.Iface client;
     private String host;
     private int port;
-    private Integer timeout;
 
+    private DRPCClient(DistributedRPC.Iface override) {
+        super(new HashMap<>(), ThriftConnectionType.LOCAL_FAKE,
+                "localhost", 1234, null, null);
+        this.host = "localhost";
+        this.port = 1234;
+        this.client = override;
+    }
+    
     public DRPCClient(Map conf, String host, int port) throws TTransportException {
         this(conf, host, port, null);
         _retryForever = true;
     }
 
     public DRPCClient(Map conf, String host, int port, Integer timeout) throws TTransportException {
-        super(conf, ThriftConnectionType.DRPC, host, port, timeout, null);
+        super(conf, _localOverrideClient != null ? ThriftConnectionType.LOCAL_FAKE : ThriftConnectionType.DRPC,
+                host, port, timeout, null);
         this.host = host;
         this.port = port;
-        this.client = new DistributedRPC.Client(_protocol);
+        if (_localOverrideClient != null) {
+            this.client = _localOverrideClient;
+        } else {
+            this.client = new DistributedRPC.Client(_protocol);
+        }
         _retryForever = true;
     }
         
@@ -60,7 +116,7 @@ public class DRPCClient extends ThriftClient implements DistributedRPC.Iface {
         return client.execute(func, args);
     }
 
-    public DistributedRPC.Client getClient() {
+    public DistributedRPC.Iface getClient() {
         return client;
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/storm-client/src/jvm/org/apache/storm/utils/NimbusClient.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/utils/NimbusClient.java b/storm-client/src/jvm/org/apache/storm/utils/NimbusClient.java
index 6699ba8..cb2e1af 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/NimbusClient.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/NimbusClient.java
@@ -29,15 +29,41 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.security.Principal;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
 public class NimbusClient extends ThriftClient {
-    private Nimbus.Client _client;
+    private static volatile Nimbus.Iface _localOverrideClient = null;
+
+    public static final class LocalOverride implements AutoCloseable {
+        public LocalOverride(Nimbus.Iface client) {
+            _localOverrideClient = client;
+        }
+        
+        @Override
+        public void close() throws Exception {
+            _localOverrideClient = null;
+        }
+    }
+    
+    /**
+     * @return true of new clients will be overridden to connect to a local cluster
+     * and not the configured remote cluster.
+     */
+    public static boolean isLocalOverride() {
+        return _localOverrideClient != null;
+    }
+    
+    private Nimbus.Iface _client;
+    /**
+     * Indicates if this is a special client that is overwritten for local mode.
+     */
+    public final boolean _isLocal;
     private static final Logger LOG = LoggerFactory.getLogger(NimbusClient.class);
 
     public interface WithNimbus {
-        public void run(Nimbus.Client client) throws Exception;
+        public void run(Nimbus.Iface client) throws Exception;
     }
 
     public static void withConfiguredClient(WithNimbus cb) throws Exception {
@@ -58,6 +84,10 @@ public class NimbusClient extends ThriftClient {
     }
 
     public static NimbusClient getConfiguredClientAs(Map conf, String asUser) {
+        Nimbus.Iface override = _localOverrideClient;
+        if (override != null) {
+            return new NimbusClient(override);
+        }
         if (conf.containsKey(Config.STORM_DO_AS_USER)) {
             if (asUser != null && !asUser.isEmpty()) {
                 LOG.warn("You have specified a doAsUser as param {} and a doAsParam as config, config will take precedence."
@@ -121,19 +151,28 @@ public class NimbusClient extends ThriftClient {
     public NimbusClient(Map conf, String host, int port, Integer timeout) throws TTransportException {
         super(conf, ThriftConnectionType.NIMBUS, host, port, timeout, null);
         _client = new Nimbus.Client(_protocol);
+        _isLocal = false;
     }
 
     public NimbusClient(Map conf, String host, Integer port, Integer timeout, String asUser) throws TTransportException {
         super(conf, ThriftConnectionType.NIMBUS, host, port, timeout, asUser);
         _client = new Nimbus.Client(_protocol);
+        _isLocal = false;
     }
 
     public NimbusClient(Map conf, String host) throws TTransportException {
         super(conf, ThriftConnectionType.NIMBUS, host, null, null, null);
         _client = new Nimbus.Client(_protocol);
+        _isLocal = false;
+    }
+    
+    private NimbusClient(Nimbus.Iface client) {
+        super(new HashMap<>(), ThriftConnectionType.LOCAL_FAKE, "localhost", null, null, null);
+        _client = client;
+        _isLocal = true;
     }
 
-    public Nimbus.Client getClient() {
+    public Nimbus.Iface getClient() {
         return _client;
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/storm-client/src/jvm/org/apache/storm/utils/Utils.java
----------------------------------------------------------------------
diff --git a/storm-client/src/jvm/org/apache/storm/utils/Utils.java b/storm-client/src/jvm/org/apache/storm/utils/Utils.java
index 57101dc..771dc70 100644
--- a/storm-client/src/jvm/org/apache/storm/utils/Utils.java
+++ b/storm-client/src/jvm/org/apache/storm/utils/Utils.java
@@ -982,7 +982,7 @@ public class Utils {
         }
     }
 
-    public static String getTopologyId(String name, Nimbus.Client client) {
+    public static String getTopologyId(String name, Nimbus.Iface client) {
         try {
             ClusterSummary summary = client.getClusterInfo();
             for(TopologySummary s : summary.get_topologies()) {

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/storm-clojure-test/pom.xml
----------------------------------------------------------------------
diff --git a/storm-clojure-test/pom.xml b/storm-clojure-test/pom.xml
new file mode 100644
index 0000000..743b72b
--- /dev/null
+++ b/storm-clojure-test/pom.xml
@@ -0,0 +1,66 @@
+<?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/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <artifactId>storm</artifactId>
+        <groupId>org.apache.storm</groupId>
+        <version>2.0.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>storm-clojure-test</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-clojure</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-core</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+    </dependencies>
+
+    <build> 
+        <plugins>
+            <plugin>
+                <groupId>com.theoryinpractise</groupId>
+                <artifactId>clojure-maven-plugin</artifactId>
+                <extensions>true</extensions>
+                <configuration>
+                    <sourceDirectories>
+                        <sourceDirectory>src/clj</sourceDirectory>
+                    </sourceDirectories>
+                </configuration>
+                <executions>
+                    <execution>
+                        <id>compile</id>
+                        <phase>compile</phase>
+                        <goals>
+                            <goal>compile</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+</project>

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/storm-clojure-test/src/clj/org/apache/storm/testing.clj
----------------------------------------------------------------------
diff --git a/storm-clojure-test/src/clj/org/apache/storm/testing.clj b/storm-clojure-test/src/clj/org/apache/storm/testing.clj
new file mode 100644
index 0000000..1e374ed
--- /dev/null
+++ b/storm-clojure-test/src/clj/org/apache/storm/testing.clj
@@ -0,0 +1,270 @@
+;; 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.
+
+(ns org.apache.storm.testing
+  (:import [org.apache.storm LocalCluster$Builder])
+  (:import [java.util.function UnaryOperator])
+  (:import [org.apache.storm.utils Time Time$SimulatedTime RegisteredGlobalState Utils])
+  (:import [org.apache.storm.testing InProcessZookeeper MkTupleParam TestJob MkClusterParam 
+            TrackedTopology CompleteTopologyParam MockedSources])
+  (:import [org.apache.storm Thrift Testing Testing$Condition])
+  (:import [org.apache.storm.testing MockLeaderElector])
+  (:import [org.json.simple JSONValue])
+  (:use [org.apache.storm util config log])
+  (:use [org.apache.storm thrift]))
+
+(defnk add-supervisor
+  [cluster-map :ports 2 :conf {} :id nil]
+  (let [local-cluster (:local-cluster cluster-map)]
+    (.addSupervisor local-cluster ports conf id)))
+
+(defnk mock-leader-elector [:is-leader true :leader-name "test-host" :leader-port 9999]
+  (MockLeaderElector. is-leader leader-name leader-port))
+
+(defn local-cluster-state [local-cluster]
+    {:nimbus (.getNimbus local-cluster)
+     :daemon-conf (.getDaemonConf local-cluster)
+     :storm-cluster-state (.getClusterState local-cluster)
+     :local-cluster local-cluster})
+
+(defnk mk-mocked-nimbus 
+  [:daemon-conf {} :inimbus nil :blob-store nil :cluster-state nil 
+   :leader-elector nil :group-mapper nil :nimbus-daemon false :nimbus-wrapper nil]
+  (let [builder (doto (LocalCluster$Builder.)
+                  (.withDaemonConf daemon-conf)
+                  (.withINimbus inimbus)
+                  (.withBlobStore blob-store)
+                  (.withClusterState cluster-state)
+                  (.withLeaderElector leader-elector)
+                  (.withGroupMapper group-mapper)
+                  (.withNimbusDaemon nimbus-daemon)
+                  (.withNimbusWrapper (when nimbus-wrapper (reify UnaryOperator (apply [this nimbus] (nimbus-wrapper nimbus))))))
+        local-cluster (.build builder)]
+    (local-cluster-state local-cluster)))
+
+(defnk mk-local-storm-cluster [:supervisors 2 :ports-per-supervisor 3 :daemon-conf {} :inimbus nil :group-mapper nil :supervisor-slot-port-min 1024 :nimbus-daemon false]
+  (let [builder (doto (LocalCluster$Builder.)
+                  (.withSupervisors supervisors)
+                  (.withPortsPerSupervisor ports-per-supervisor)
+                  (.withDaemonConf daemon-conf)
+                  (.withINimbus inimbus)
+                  (.withGroupMapper group-mapper)
+                  (.withSupervisorSlotPortMin supervisor-slot-port-min)
+                  (.withNimbusDaemon nimbus-daemon))
+        local-cluster (.build builder)]
+    (local-cluster-state local-cluster)))
+
+(defn get-supervisor [cluster-map supervisor-id]
+ (let [local-cluster (:local-cluster cluster-map)]
+  (.getSupervisor local-cluster supervisor-id))) 
+
+(defn kill-supervisor [cluster-map supervisor-id]
+ (let [local-cluster (:local-cluster cluster-map)]
+  (.killSupervisor local-cluster supervisor-id))) 
+
+(defn kill-local-storm-cluster [cluster-map]
+ (let [local-cluster (:local-cluster cluster-map)]
+  (.close local-cluster))) 
+
+(defmacro while-timeout [timeout-ms condition & body]
+  `(Testing/whileTimeout ~timeout-ms
+     (reify Testing$Condition (exec [this] ~condition))
+     (fn [] ~@body)))
+
+(defn wait-for-condition
+  ([apredicate]
+    (wait-for-condition Testing/TEST_TIMEOUT_MS apredicate))
+  ([timeout-ms apredicate]
+    (while-timeout timeout-ms (not (apredicate))
+      (Time/sleep 100))))
+
+(defn wait-until-cluster-waiting
+  "Wait until the cluster is idle. Should be used with time simulation."
+  ([cluster-map]
+    (let [local-cluster (:local-cluster cluster-map)]
+      (.waitForIdle local-cluster)))
+  ([cluster-map timeout-ms]
+    (let [local-cluster (:local-cluster cluster-map)]
+      (.waitForIdle local-cluster timeout-ms))))
+
+(defn advance-cluster-time
+  ([cluster-map secs increment-secs]
+    (let [local-cluster (:local-cluster cluster-map)]
+      (.advanceClusterTime local-cluster secs increment-secs))) 
+  ([cluster-map secs]
+    (let [local-cluster (:local-cluster cluster-map)]
+      (.advanceClusterTime local-cluster secs))))
+
+(defmacro with-mocked-nimbus
+  [[nimbus-sym & args] & body]
+  `(let [~nimbus-sym (mk-mocked-nimbus ~@args)]
+     (try
+       ~@body
+       (catch Throwable t#
+         (log-error t# "Error in cluster")
+         (throw t#))
+       (finally
+         (let [keep-waiting?# (atom true)
+               f# (future (while @keep-waiting?# (simulate-wait ~nimbus-sym)))]
+           (kill-local-storm-cluster ~nimbus-sym)
+           (reset! keep-waiting?# false)
+            @f#)))))
+
+(defmacro with-local-cluster
+  [[cluster-sym & args] & body]
+  `(let [~cluster-sym (mk-local-storm-cluster ~@args)]
+     (try
+       ~@body
+       (catch Throwable t#
+         (log-error t# "Error in cluster")
+         (throw t#))
+       (finally
+         (let [keep-waiting?# (atom true)
+               f# (future (while @keep-waiting?# (simulate-wait ~cluster-sym)))]
+           (kill-local-storm-cluster ~cluster-sym)
+           (reset! keep-waiting?# false)
+            @f#)))))
+
+(defmacro with-simulated-time-local-cluster
+  [& args]
+  `(with-open [_# (Time$SimulatedTime.)]
+     (with-local-cluster ~@args)))
+
+(defmacro with-inprocess-zookeeper
+  [port-sym & body]
+  `(with-open [zks# (InProcessZookeeper. )]
+     (let [~port-sym (.getPort zks#)]
+       ~@body)))
+
+(defn submit-local-topology
+  [nimbus storm-name conf topology]
+  (when-not (Utils/isValidConf conf)
+    (throw (IllegalArgumentException. "Topology conf is not json-serializable")))
+  (.submitTopology nimbus storm-name nil (JSONValue/toJSONString conf) topology))
+
+(defn submit-local-topology-with-opts
+  [nimbus storm-name conf topology submit-opts]
+  (when-not (Utils/isValidConf conf)
+    (throw (IllegalArgumentException. "Topology conf is not json-serializable")))
+  (.submitTopologyWithOpts nimbus storm-name nil (JSONValue/toJSONString conf) topology submit-opts))
+
+(defn simulate-wait
+  [cluster-map]
+  (Testing/simulateWait (:local-cluster cluster-map)))
+
+(defn spout-objects [spec-map]
+  (for [[_ spout-spec] spec-map]
+    (-> spout-spec
+        .get_spout_object
+        (Thrift/deserializeComponentObject))))
+
+(defn capture-topology
+  [topology]
+  (let [cap-topo (Testing/captureTopology topology)]
+    {:topology (.topology cap-topo)
+     :capturer (.capturer cap-topo)}))
+
+(defnk complete-topology
+  [cluster-map topology
+   :mock-sources {}
+   :storm-conf {}
+   :cleanup-state true
+   :topology-name nil
+   :timeout-ms Testing/TEST_TIMEOUT_MS]
+  (Testing/completeTopology (:local-cluster cluster-map) topology, 
+      (doto (CompleteTopologyParam.)
+            (.setStormConf storm-conf)
+            (.setTopologyName topology-name)
+            (.setTimeoutMs timeout-ms)
+            (.setMockedSources (MockedSources. mock-sources))
+            (.setCleanupState cleanup-state))))
+
+(defn read-tuples
+  ([results component-id stream-id]
+   (Testing/readTuples results component-id stream-id))
+  ([results component-id]
+   (Testing/readTuples results component-id )))
+
+(defn ms=
+  [a b]
+  (Testing/multiseteq a b))
+
+(def TRACKER-BOLT-ID "+++tracker-bolt")
+
+;; TODO: should override system-topology! and wrap everything there
+(defn mk-tracked-topology
+  ([tracked-cluster topology]
+   (let [tt (TrackedTopology. topology (:local-cluster tracked-cluster))]
+     {:topology (.getTopology tt)
+      :tracked-topo tt})))
+
+(defn increment-global!
+  [id key amt]
+  (-> (RegisteredGlobalState/getState id)
+      (get key)
+      (.addAndGet amt)))
+
+(defn global-amt
+  [id key]
+  (-> (RegisteredGlobalState/getState id)
+      (get key)
+      .get))
+
+(defnk mkClusterParam 
+  [:supervisors 2 :ports-per-supervisor 3 :daemon-conf {} :nimbus-daemon false]
+  ;;TODO do we need to support inimbus?, group-mapper, or supervisor-slot-port-min
+  (doto (MkClusterParam. )
+    (.setDaemonConf daemon-conf)
+    (.setNimbusDaemon nimbus-daemon)
+    (.setPortsPerSupervisor (int ports-per-supervisor))
+    (.setSupervisors (int supervisors))))
+
+(defmacro with-tracked-cluster
+  [[cluster-sym & cluster-args] & body]
+  `(Testing/withTrackedCluster
+       (mkClusterParam ~@cluster-args)
+       (reify TestJob
+         (run [this# lc#]
+           (let [~cluster-sym (local-cluster-state lc#)]
+             ~@body)))))
+
+(defn tracked-wait
+  "Waits until topology is idle and 'amt' more tuples have been emitted by spouts."
+  ([tracked-map]
+     (Testing/trackedWait (:tracked-topo tracked-map)))
+  ([tracked-map amt]
+     (Testing/trackedWait (:tracked-topo tracked-map) (int amt)))
+  ([tracked-map amt timeout-ms]
+     (Testing/trackedWait (:tracked-topo tracked-map) (int amt) (int timeout-ms))))
+
+(defnk test-tuple
+  [values
+   :stream Utils/DEFAULT_STREAM_ID
+   :component "component"
+   :fields nil]
+  (Testing/testTuple
+    values
+    (doto (MkTupleParam. )
+      (.setStream stream)
+      (.setComponent component)
+      (.setFieldsList fields))))
+
+(defmacro with-timeout
+  [millis unit & body]
+  `(let [f# (future ~@body)]
+     (try
+       (.get f# ~millis ~unit)
+       (finally (future-cancel f#)))))

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/storm-clojure/pom.xml
----------------------------------------------------------------------
diff --git a/storm-clojure/pom.xml b/storm-clojure/pom.xml
index 1d50df0..cef8fd0 100644
--- a/storm-clojure/pom.xml
+++ b/storm-clojure/pom.xml
@@ -35,17 +35,8 @@
             <scope>provided</scope>
         </dependency>
         <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
-            <version>${project.version}</version>
-            <scope>provided</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
-            <version>${project.version}</version>
-            <type>test-jar</type>
-            <scope>test</scope>
+            <groupId>org.clojure</groupId>
+            <artifactId>tools.logging</artifactId>
         </dependency>
         <dependency>
             <groupId>com.googlecode.json-simple</groupId>
@@ -54,7 +45,7 @@
         </dependency>
     </dependencies>
 
-    <build>
+    <build> 
         <plugins>
             <plugin>
                 <groupId>com.theoryinpractise</groupId>

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/storm-clojure/src/clj/org/apache/storm/config.clj
----------------------------------------------------------------------
diff --git a/storm-clojure/src/clj/org/apache/storm/config.clj b/storm-clojure/src/clj/org/apache/storm/config.clj
new file mode 100644
index 0000000..bfe47ed
--- /dev/null
+++ b/storm-clojure/src/clj/org/apache/storm/config.clj
@@ -0,0 +1,28 @@
+;; 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.
+
+(ns org.apache.storm.config
+  (:import [org.apache.storm Config]))
+
+(defn- clojure-config-name [name]
+  (.replace (.toUpperCase name) "_" "-"))
+
+; define clojure constants for every configuration parameter
+(doseq [f (seq (.getFields Config))]
+  (let [name (.getName f)
+        new-name (clojure-config-name name)]
+    (eval
+      `(def ~(symbol new-name) (. Config ~(symbol name))))))

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/storm-clojure/src/clj/org/apache/storm/log.clj
----------------------------------------------------------------------
diff --git a/storm-clojure/src/clj/org/apache/storm/log.clj b/storm-clojure/src/clj/org/apache/storm/log.clj
new file mode 100644
index 0000000..7a006ef
--- /dev/null
+++ b/storm-clojure/src/clj/org/apache/storm/log.clj
@@ -0,0 +1,34 @@
+;; 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.
+
+(ns org.apache.storm.log
+  (:require [clojure.tools.logging :as log]))
+
+(defmacro log-message
+  [& args]
+  `(log/info (str ~@args)))
+
+(defmacro log-error
+  [e & args]
+  `(log/log :error ~e (str ~@args)))
+
+(defmacro log-debug
+  [& args]
+  `(log/debug (str ~@args)))
+
+(defmacro log-warn
+  [& args]
+  `(log/warn (str ~@args)))

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/storm-clojure/src/clj/org/apache/storm/testing.clj
----------------------------------------------------------------------
diff --git a/storm-clojure/src/clj/org/apache/storm/testing.clj b/storm-clojure/src/clj/org/apache/storm/testing.clj
deleted file mode 100644
index 1e374ed..0000000
--- a/storm-clojure/src/clj/org/apache/storm/testing.clj
+++ /dev/null
@@ -1,270 +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.
-
-(ns org.apache.storm.testing
-  (:import [org.apache.storm LocalCluster$Builder])
-  (:import [java.util.function UnaryOperator])
-  (:import [org.apache.storm.utils Time Time$SimulatedTime RegisteredGlobalState Utils])
-  (:import [org.apache.storm.testing InProcessZookeeper MkTupleParam TestJob MkClusterParam 
-            TrackedTopology CompleteTopologyParam MockedSources])
-  (:import [org.apache.storm Thrift Testing Testing$Condition])
-  (:import [org.apache.storm.testing MockLeaderElector])
-  (:import [org.json.simple JSONValue])
-  (:use [org.apache.storm util config log])
-  (:use [org.apache.storm thrift]))
-
-(defnk add-supervisor
-  [cluster-map :ports 2 :conf {} :id nil]
-  (let [local-cluster (:local-cluster cluster-map)]
-    (.addSupervisor local-cluster ports conf id)))
-
-(defnk mock-leader-elector [:is-leader true :leader-name "test-host" :leader-port 9999]
-  (MockLeaderElector. is-leader leader-name leader-port))
-
-(defn local-cluster-state [local-cluster]
-    {:nimbus (.getNimbus local-cluster)
-     :daemon-conf (.getDaemonConf local-cluster)
-     :storm-cluster-state (.getClusterState local-cluster)
-     :local-cluster local-cluster})
-
-(defnk mk-mocked-nimbus 
-  [:daemon-conf {} :inimbus nil :blob-store nil :cluster-state nil 
-   :leader-elector nil :group-mapper nil :nimbus-daemon false :nimbus-wrapper nil]
-  (let [builder (doto (LocalCluster$Builder.)
-                  (.withDaemonConf daemon-conf)
-                  (.withINimbus inimbus)
-                  (.withBlobStore blob-store)
-                  (.withClusterState cluster-state)
-                  (.withLeaderElector leader-elector)
-                  (.withGroupMapper group-mapper)
-                  (.withNimbusDaemon nimbus-daemon)
-                  (.withNimbusWrapper (when nimbus-wrapper (reify UnaryOperator (apply [this nimbus] (nimbus-wrapper nimbus))))))
-        local-cluster (.build builder)]
-    (local-cluster-state local-cluster)))
-
-(defnk mk-local-storm-cluster [:supervisors 2 :ports-per-supervisor 3 :daemon-conf {} :inimbus nil :group-mapper nil :supervisor-slot-port-min 1024 :nimbus-daemon false]
-  (let [builder (doto (LocalCluster$Builder.)
-                  (.withSupervisors supervisors)
-                  (.withPortsPerSupervisor ports-per-supervisor)
-                  (.withDaemonConf daemon-conf)
-                  (.withINimbus inimbus)
-                  (.withGroupMapper group-mapper)
-                  (.withSupervisorSlotPortMin supervisor-slot-port-min)
-                  (.withNimbusDaemon nimbus-daemon))
-        local-cluster (.build builder)]
-    (local-cluster-state local-cluster)))
-
-(defn get-supervisor [cluster-map supervisor-id]
- (let [local-cluster (:local-cluster cluster-map)]
-  (.getSupervisor local-cluster supervisor-id))) 
-
-(defn kill-supervisor [cluster-map supervisor-id]
- (let [local-cluster (:local-cluster cluster-map)]
-  (.killSupervisor local-cluster supervisor-id))) 
-
-(defn kill-local-storm-cluster [cluster-map]
- (let [local-cluster (:local-cluster cluster-map)]
-  (.close local-cluster))) 
-
-(defmacro while-timeout [timeout-ms condition & body]
-  `(Testing/whileTimeout ~timeout-ms
-     (reify Testing$Condition (exec [this] ~condition))
-     (fn [] ~@body)))
-
-(defn wait-for-condition
-  ([apredicate]
-    (wait-for-condition Testing/TEST_TIMEOUT_MS apredicate))
-  ([timeout-ms apredicate]
-    (while-timeout timeout-ms (not (apredicate))
-      (Time/sleep 100))))
-
-(defn wait-until-cluster-waiting
-  "Wait until the cluster is idle. Should be used with time simulation."
-  ([cluster-map]
-    (let [local-cluster (:local-cluster cluster-map)]
-      (.waitForIdle local-cluster)))
-  ([cluster-map timeout-ms]
-    (let [local-cluster (:local-cluster cluster-map)]
-      (.waitForIdle local-cluster timeout-ms))))
-
-(defn advance-cluster-time
-  ([cluster-map secs increment-secs]
-    (let [local-cluster (:local-cluster cluster-map)]
-      (.advanceClusterTime local-cluster secs increment-secs))) 
-  ([cluster-map secs]
-    (let [local-cluster (:local-cluster cluster-map)]
-      (.advanceClusterTime local-cluster secs))))
-
-(defmacro with-mocked-nimbus
-  [[nimbus-sym & args] & body]
-  `(let [~nimbus-sym (mk-mocked-nimbus ~@args)]
-     (try
-       ~@body
-       (catch Throwable t#
-         (log-error t# "Error in cluster")
-         (throw t#))
-       (finally
-         (let [keep-waiting?# (atom true)
-               f# (future (while @keep-waiting?# (simulate-wait ~nimbus-sym)))]
-           (kill-local-storm-cluster ~nimbus-sym)
-           (reset! keep-waiting?# false)
-            @f#)))))
-
-(defmacro with-local-cluster
-  [[cluster-sym & args] & body]
-  `(let [~cluster-sym (mk-local-storm-cluster ~@args)]
-     (try
-       ~@body
-       (catch Throwable t#
-         (log-error t# "Error in cluster")
-         (throw t#))
-       (finally
-         (let [keep-waiting?# (atom true)
-               f# (future (while @keep-waiting?# (simulate-wait ~cluster-sym)))]
-           (kill-local-storm-cluster ~cluster-sym)
-           (reset! keep-waiting?# false)
-            @f#)))))
-
-(defmacro with-simulated-time-local-cluster
-  [& args]
-  `(with-open [_# (Time$SimulatedTime.)]
-     (with-local-cluster ~@args)))
-
-(defmacro with-inprocess-zookeeper
-  [port-sym & body]
-  `(with-open [zks# (InProcessZookeeper. )]
-     (let [~port-sym (.getPort zks#)]
-       ~@body)))
-
-(defn submit-local-topology
-  [nimbus storm-name conf topology]
-  (when-not (Utils/isValidConf conf)
-    (throw (IllegalArgumentException. "Topology conf is not json-serializable")))
-  (.submitTopology nimbus storm-name nil (JSONValue/toJSONString conf) topology))
-
-(defn submit-local-topology-with-opts
-  [nimbus storm-name conf topology submit-opts]
-  (when-not (Utils/isValidConf conf)
-    (throw (IllegalArgumentException. "Topology conf is not json-serializable")))
-  (.submitTopologyWithOpts nimbus storm-name nil (JSONValue/toJSONString conf) topology submit-opts))
-
-(defn simulate-wait
-  [cluster-map]
-  (Testing/simulateWait (:local-cluster cluster-map)))
-
-(defn spout-objects [spec-map]
-  (for [[_ spout-spec] spec-map]
-    (-> spout-spec
-        .get_spout_object
-        (Thrift/deserializeComponentObject))))
-
-(defn capture-topology
-  [topology]
-  (let [cap-topo (Testing/captureTopology topology)]
-    {:topology (.topology cap-topo)
-     :capturer (.capturer cap-topo)}))
-
-(defnk complete-topology
-  [cluster-map topology
-   :mock-sources {}
-   :storm-conf {}
-   :cleanup-state true
-   :topology-name nil
-   :timeout-ms Testing/TEST_TIMEOUT_MS]
-  (Testing/completeTopology (:local-cluster cluster-map) topology, 
-      (doto (CompleteTopologyParam.)
-            (.setStormConf storm-conf)
-            (.setTopologyName topology-name)
-            (.setTimeoutMs timeout-ms)
-            (.setMockedSources (MockedSources. mock-sources))
-            (.setCleanupState cleanup-state))))
-
-(defn read-tuples
-  ([results component-id stream-id]
-   (Testing/readTuples results component-id stream-id))
-  ([results component-id]
-   (Testing/readTuples results component-id )))
-
-(defn ms=
-  [a b]
-  (Testing/multiseteq a b))
-
-(def TRACKER-BOLT-ID "+++tracker-bolt")
-
-;; TODO: should override system-topology! and wrap everything there
-(defn mk-tracked-topology
-  ([tracked-cluster topology]
-   (let [tt (TrackedTopology. topology (:local-cluster tracked-cluster))]
-     {:topology (.getTopology tt)
-      :tracked-topo tt})))
-
-(defn increment-global!
-  [id key amt]
-  (-> (RegisteredGlobalState/getState id)
-      (get key)
-      (.addAndGet amt)))
-
-(defn global-amt
-  [id key]
-  (-> (RegisteredGlobalState/getState id)
-      (get key)
-      .get))
-
-(defnk mkClusterParam 
-  [:supervisors 2 :ports-per-supervisor 3 :daemon-conf {} :nimbus-daemon false]
-  ;;TODO do we need to support inimbus?, group-mapper, or supervisor-slot-port-min
-  (doto (MkClusterParam. )
-    (.setDaemonConf daemon-conf)
-    (.setNimbusDaemon nimbus-daemon)
-    (.setPortsPerSupervisor (int ports-per-supervisor))
-    (.setSupervisors (int supervisors))))
-
-(defmacro with-tracked-cluster
-  [[cluster-sym & cluster-args] & body]
-  `(Testing/withTrackedCluster
-       (mkClusterParam ~@cluster-args)
-       (reify TestJob
-         (run [this# lc#]
-           (let [~cluster-sym (local-cluster-state lc#)]
-             ~@body)))))
-
-(defn tracked-wait
-  "Waits until topology is idle and 'amt' more tuples have been emitted by spouts."
-  ([tracked-map]
-     (Testing/trackedWait (:tracked-topo tracked-map)))
-  ([tracked-map amt]
-     (Testing/trackedWait (:tracked-topo tracked-map) (int amt)))
-  ([tracked-map amt timeout-ms]
-     (Testing/trackedWait (:tracked-topo tracked-map) (int amt) (int timeout-ms))))
-
-(defnk test-tuple
-  [values
-   :stream Utils/DEFAULT_STREAM_ID
-   :component "component"
-   :fields nil]
-  (Testing/testTuple
-    values
-    (doto (MkTupleParam. )
-      (.setStream stream)
-      (.setComponent component)
-      (.setFieldsList fields))))
-
-(defmacro with-timeout
-  [millis unit & body]
-  `(let [f# (future ~@body)]
-     (try
-       (.get f# ~millis ~unit)
-       (finally (future-cancel f#)))))

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/storm-clojure/src/clj/org/apache/storm/util.clj
----------------------------------------------------------------------
diff --git a/storm-clojure/src/clj/org/apache/storm/util.clj b/storm-clojure/src/clj/org/apache/storm/util.clj
new file mode 100644
index 0000000..9ad1f10
--- /dev/null
+++ b/storm-clojure/src/clj/org/apache/storm/util.clj
@@ -0,0 +1,134 @@
+;; 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.
+
+(ns org.apache.storm.util
+  (:import [java.util Map List HashMap])
+  (:import [org.apache.storm.generated ErrorInfo])
+  (:import [org.apache.storm.utils Utils])
+  (:import [java.util List Set])
+  (:use [clojure walk])
+  (:use [org.apache.storm log]))
+
+;; name-with-attributes by Konrad Hinsen:
+(defn name-with-attributes
+  "To be used in macro definitions.
+  Handles optional docstrings and attribute maps for a name to be defined
+  in a list of macro arguments. If the first macro argument is a string,
+  it is added as a docstring to name and removed from the macro argument
+  list. If afterwards the first macro argument is a map, its entries are
+  added to the name's metadata map and the map is removed from the
+  macro argument list. The return value is a vector containing the name
+  with its extended metadata map and the list of unprocessed macro
+  arguments."
+  [name macro-args]
+  (let [[docstring macro-args] (if (string? (first macro-args))
+                                 [(first macro-args) (next macro-args)]
+                                 [nil macro-args])
+        [attr macro-args] (if (map? (first macro-args))
+                            [(first macro-args) (next macro-args)]
+                            [{} macro-args])
+        attr (if docstring
+               (assoc attr :doc docstring)
+               attr)
+        attr (if (meta name)
+               (conj (meta name) attr)
+               attr)]
+    [(with-meta name attr) macro-args]))
+
+(defmacro defnk
+  "Define a function accepting keyword arguments. Symbols up to the first
+  keyword in the parameter list are taken as positional arguments.  Then
+  an alternating sequence of keywords and defaults values is expected. The
+  values of the keyword arguments are available in the function body by
+  virtue of the symbol corresponding to the keyword (cf. :keys destructuring).
+  defnk accepts an optional docstring as well as an optional metadata map."
+  [fn-name & fn-tail]
+  (let [[fn-name [args & body]] (name-with-attributes fn-name fn-tail)
+        [pos kw-vals] (split-with symbol? args)
+        syms (map #(-> % name symbol) (take-nth 2 kw-vals))
+        values (take-nth 2 (rest kw-vals))
+        sym-vals (apply hash-map (interleave syms values))
+        de-map {:keys (vec syms) :or sym-vals}]
+    `(defn ~fn-name
+       [~@pos & options#]
+       (let [~de-map (apply hash-map options#)]
+         ~@body))))
+
+(defmacro thrown-cause?
+  [klass & body]
+  `(try
+     ~@body
+     false
+     (catch Throwable t#
+       (let [tc# (Utils/exceptionCauseIsInstanceOf ~klass t#)]
+         (if (not tc#) (log-error t# "Exception did not match " ~klass))
+         tc#))))
+
+(defn clojurify-structure
+  [s]
+  (if s
+    (prewalk (fn [x]
+             (cond (instance? Map x) (into {} x)
+                   (instance? List x) (vec x)
+                   (instance? Set x) (into #{} x)
+                   ;; (Boolean. false) does not evaluate to false in an if.
+                   ;; This fixes that.
+                   (instance? Boolean x) (boolean x)
+                   true x))
+           s)))
+; move this func form convert.clj due to cyclic load dependency
+(defn clojurify-error [^ErrorInfo error]
+  (if error
+    {
+      :error (.get_error error)
+      :time-secs (.get_error_time_secs error)
+      :host (.get_host error)
+      :port (.get_port error)
+      }
+    ))
+
+;TODO: We're keeping this function around until all the code using it is properly tranlated to java
+;TODO: by properly having the for loop IN THE JAVA FUNCTION that originally used this function.
+(defn map-val
+  [afn amap]
+  (into {}
+        (for [[k v] amap]
+          [k (afn v)])))
+
+;TODO: We're keeping this function around until all the code using it is properly tranlated to java
+;TODO: by properly having the for loop IN THE JAVA FUNCTION that originally used this function.
+(defn filter-key
+  [afn amap]
+  (into {} (filter (fn [[k v]] (afn k)) amap)))
+
+;TODO: Once all the other clojure functions (100+ locations) are translated to java, this function becomes moot.
+(def not-nil? (complement nil?))
+
+(defmacro dofor [& body]
+  `(doall (for ~@body)))
+
+(defmacro -<>
+  ([x] x)
+  ([x form] (if (seq? form)
+              (with-meta
+                (let [[begin [_ & end]] (split-with #(not= % '<>) form)]
+                  (concat begin [x] end))
+                (meta form))
+              (list form x)))
+  ([x form & more] `(-<> (-<> ~x ~form) ~@more)))
+
+(defn hashmap-to-persistent [^HashMap m]
+  (zipmap (.keySet m) (.values m)))

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/storm-core/pom.xml
----------------------------------------------------------------------
diff --git a/storm-core/pom.xml b/storm-core/pom.xml
index b5f7086..0014aa5 100644
--- a/storm-core/pom.xml
+++ b/storm-core/pom.xml
@@ -377,7 +377,7 @@
     </dependencies>
     <build>
         <sourceDirectory>src/jvm</sourceDirectory>
-         <testSourceDirectory>test/jvm</testSourceDirectory>
+        <testSourceDirectory>test/jvm</testSourceDirectory>
         <resources>
             <resource>
                 <directory>../</directory>

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/storm-core/src/jvm/org/apache/storm/command/Activate.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/command/Activate.java b/storm-core/src/jvm/org/apache/storm/command/Activate.java
index 6a64bf6..45d4a99 100644
--- a/storm-core/src/jvm/org/apache/storm/command/Activate.java
+++ b/storm-core/src/jvm/org/apache/storm/command/Activate.java
@@ -31,7 +31,7 @@ public class Activate {
 
         NimbusClient.withConfiguredClient(new NimbusClient.WithNimbus() {
           @Override
-          public void run(Nimbus.Client nimbus) throws Exception {
+          public void run(Nimbus.Iface nimbus) throws Exception {
             nimbus.activate(name);
             LOG.info("Activated topology: {}", name);
           }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/storm-core/src/jvm/org/apache/storm/command/Deactivate.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/command/Deactivate.java b/storm-core/src/jvm/org/apache/storm/command/Deactivate.java
index 6b9dd11..4eafb83 100644
--- a/storm-core/src/jvm/org/apache/storm/command/Deactivate.java
+++ b/storm-core/src/jvm/org/apache/storm/command/Deactivate.java
@@ -31,7 +31,7 @@ public class Deactivate {
 
         NimbusClient.withConfiguredClient(new NimbusClient.WithNimbus() {
           @Override
-          public void run(Nimbus.Client nimbus) throws Exception {
+          public void run(Nimbus.Iface nimbus) throws Exception {
             nimbus.deactivate(name);
             LOG.info("Deactivated topology: {}", name);
           }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/storm-core/src/jvm/org/apache/storm/command/GetErrors.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/command/GetErrors.java b/storm-core/src/jvm/org/apache/storm/command/GetErrors.java
index ee15446..05c0ee4 100644
--- a/storm-core/src/jvm/org/apache/storm/command/GetErrors.java
+++ b/storm-core/src/jvm/org/apache/storm/command/GetErrors.java
@@ -40,7 +40,7 @@ public class GetErrors {
 
         NimbusClient.withConfiguredClient(new NimbusClient.WithNimbus() {
             @Override
-            public void run(Nimbus.Client client) throws Exception {
+            public void run(Nimbus.Iface client) throws Exception {
                 GetInfoOptions opts = new GetInfoOptions();
                 opts.set_num_err_choice(NumErrorsChoice.ONE);
                 String topologyId = Utils.getTopologyId(name, client);


[04/10] storm git commit: STORM-2447: add in storm local to avoid having server on worker classpath

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/trident/WordCountTridentRedis.java
----------------------------------------------------------------------
diff --git a/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/trident/WordCountTridentRedis.java b/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/trident/WordCountTridentRedis.java
index f5278fd..c162ca6 100644
--- a/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/trident/WordCountTridentRedis.java
+++ b/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/trident/WordCountTridentRedis.java
@@ -18,22 +18,20 @@
 package org.apache.storm.redis.trident;
 
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.generated.StormTopology;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Values;
+import org.apache.storm.redis.common.config.JedisPoolConfig;
 import org.apache.storm.redis.common.mapper.RedisLookupMapper;
 import org.apache.storm.redis.common.mapper.RedisStoreMapper;
 import org.apache.storm.redis.trident.state.RedisState;
 import org.apache.storm.redis.trident.state.RedisStateQuerier;
 import org.apache.storm.redis.trident.state.RedisStateUpdater;
-import org.apache.storm.redis.common.config.JedisPoolConfig;
 import org.apache.storm.trident.Stream;
 import org.apache.storm.trident.TridentState;
 import org.apache.storm.trident.TridentTopology;
 import org.apache.storm.trident.testing.FixedBatchSpout;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Values;
 
 public class WordCountTridentRedis {
     public static StormTopology buildTopology(String redisHost, Integer redisPort){
@@ -71,28 +69,17 @@ public class WordCountTridentRedis {
     }
 
     public static void main(String[] args) throws Exception {
-        if (args.length != 3) {
-            System.out.println("Usage: WordCountTrident 0(storm-local)|1(storm-cluster) redis-host redis-port");
+        if (args.length != 2) {
+            System.out.println("Usage: WordCountTrident redis-host redis-port");
             System.exit(1);
         }
 
-        Integer flag = Integer.valueOf(args[0]);
-        String redisHost = args[1];
-        Integer redisPort = Integer.valueOf(args[2]);
+        String redisHost = args[0];
+        Integer redisPort = Integer.valueOf(args[1]);
 
         Config conf = new Config();
         conf.setMaxSpoutPending(5);
-        if (flag == 0) {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology("test_wordCounter_for_redis", conf, buildTopology(redisHost, redisPort));) {
-                Thread.sleep(60 * 1000);
-            }
-            System.exit(0);
-        } else if(flag == 1) {
-            conf.setNumWorkers(3);
-            StormSubmitter.submitTopology("test_wordCounter_for_redis", conf, buildTopology(redisHost, redisPort));
-        } else {
-            System.out.println("Usage: WordCountTrident 0(storm-local)|1(storm-cluster) redis-host redis-port");
-        }
+        conf.setNumWorkers(3);
+        StormSubmitter.submitTopology("test_wordCounter_for_redis", conf, buildTopology(redisHost, redisPort));
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/trident/WordCountTridentRedisCluster.java
----------------------------------------------------------------------
diff --git a/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/trident/WordCountTridentRedisCluster.java b/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/trident/WordCountTridentRedisCluster.java
index b6e067d..687ac54 100644
--- a/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/trident/WordCountTridentRedisCluster.java
+++ b/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/trident/WordCountTridentRedisCluster.java
@@ -17,27 +17,25 @@
  */
 package org.apache.storm.redis.trident;
 
+import java.net.InetSocketAddress;
+import java.util.HashSet;
+import java.util.Set;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.generated.StormTopology;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Values;
+import org.apache.storm.redis.common.config.JedisClusterConfig;
 import org.apache.storm.redis.common.mapper.RedisLookupMapper;
 import org.apache.storm.redis.common.mapper.RedisStoreMapper;
 import org.apache.storm.redis.trident.state.RedisClusterState;
 import org.apache.storm.redis.trident.state.RedisClusterStateQuerier;
 import org.apache.storm.redis.trident.state.RedisClusterStateUpdater;
-import org.apache.storm.redis.common.config.JedisClusterConfig;
 import org.apache.storm.trident.Stream;
 import org.apache.storm.trident.TridentState;
 import org.apache.storm.trident.TridentTopology;
 import org.apache.storm.trident.testing.FixedBatchSpout;
-
-import java.net.InetSocketAddress;
-import java.util.HashSet;
-import java.util.Set;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Values;
 
 public class WordCountTridentRedisCluster {
     public static StormTopology buildTopology(String redisHostPort){
@@ -79,28 +77,17 @@ public class WordCountTridentRedisCluster {
     }
 
     public static void main(String[] args) throws Exception {
-        if (args.length != 2) {
-            System.out.println("Usage: WordCountTrident 0(storm-local)|1(storm-cluster) 127.0.0.1:6379,127.0.0.1:6380");
+        if (args.length != 1) {
+            System.out.println("Usage: WordCountTrident 127.0.0.1:6379,127.0.0.1:6380");
             System.exit(1);
         }
 
-        Integer flag = Integer.valueOf(args[0]);
-        String redisHostPort = args[1];
+        String redisHostPort = args[0];
 
         Config conf = new Config();
         conf.setMaxSpoutPending(5);
-        if (flag == 0) {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology("test_wordCounter_for_redis", conf, buildTopology(redisHostPort));) {
-                Thread.sleep(60 * 1000);
-            }
-            System.exit(0);
-        } else if(flag == 1) {
-            conf.setNumWorkers(3);
-            StormSubmitter.submitTopology("test_wordCounter_for_redis", conf, buildTopology(redisHostPort));
-        } else {
-            System.out.println("Usage: WordCountTrident 0(storm-local)|1(storm-cluster) redis-host redis-port");
-        }
+        conf.setNumWorkers(3);
+        StormSubmitter.submitTopology("test_wordCounter_for_redis", conf, buildTopology(redisHostPort));
     }
 
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/trident/WordCountTridentRedisClusterMap.java
----------------------------------------------------------------------
diff --git a/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/trident/WordCountTridentRedisClusterMap.java b/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/trident/WordCountTridentRedisClusterMap.java
index 0a025bd..8cf0f3c 100644
--- a/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/trident/WordCountTridentRedisClusterMap.java
+++ b/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/trident/WordCountTridentRedisClusterMap.java
@@ -17,17 +17,16 @@
  */
 package org.apache.storm.redis.trident;
 
+import java.net.InetSocketAddress;
+import java.util.HashSet;
+import java.util.Set;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.generated.StormTopology;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Values;
+import org.apache.storm.redis.common.config.JedisClusterConfig;
 import org.apache.storm.redis.common.mapper.RedisDataTypeDescription;
-import org.apache.storm.redis.common.mapper.TupleMapper;
 import org.apache.storm.redis.trident.state.RedisClusterMapState;
-import org.apache.storm.redis.common.config.JedisClusterConfig;
 import org.apache.storm.trident.Stream;
 import org.apache.storm.trident.TridentState;
 import org.apache.storm.trident.TridentTopology;
@@ -35,10 +34,8 @@ import org.apache.storm.trident.operation.builtin.MapGet;
 import org.apache.storm.trident.operation.builtin.Sum;
 import org.apache.storm.trident.state.StateFactory;
 import org.apache.storm.trident.testing.FixedBatchSpout;
-
-import java.net.InetSocketAddress;
-import java.util.HashSet;
-import java.util.Set;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Values;
 
 public class WordCountTridentRedisClusterMap {
     public static StormTopology buildTopology(String redisHostPort){
@@ -74,28 +71,17 @@ public class WordCountTridentRedisClusterMap {
     }
 
     public static void main(String[] args) throws Exception {
-        if (args.length != 2) {
-            System.out.println("Usage: WordCountTrident 0(storm-local)|1(storm-cluster) 127.0.0.1:6379,127.0.0.1:6380");
+        if (args.length != 1) {
+            System.out.println("Usage: WordCountTrident 127.0.0.1:6379,127.0.0.1:6380");
             System.exit(1);
         }
 
-        Integer flag = Integer.valueOf(args[0]);
-        String redisHostPort = args[1];
+        String redisHostPort = args[0];
 
         Config conf = new Config();
         conf.setMaxSpoutPending(5);
-        if (flag == 0) {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology("test_wordCounter_for_redis", conf, buildTopology(redisHostPort));) {
-                Thread.sleep(60 * 1000);
-            }
-            System.exit(0);
-        } else if(flag == 1) {
-            conf.setNumWorkers(3);
-            StormSubmitter.submitTopology("test_wordCounter_for_redis", conf, buildTopology(redisHostPort));
-        } else {
-            System.out.println("Usage: WordCountTrident 0(storm-local)|1(storm-cluster) redis-host redis-port");
-        }
+        conf.setNumWorkers(3);
+        StormSubmitter.submitTopology("test_wordCounter_for_redis", conf, buildTopology(redisHostPort));
     }
 
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/trident/WordCountTridentRedisMap.java
----------------------------------------------------------------------
diff --git a/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/trident/WordCountTridentRedisMap.java b/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/trident/WordCountTridentRedisMap.java
index b0cddcd..dac0a4d 100644
--- a/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/trident/WordCountTridentRedisMap.java
+++ b/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/trident/WordCountTridentRedisMap.java
@@ -18,15 +18,11 @@
 package org.apache.storm.redis.trident;
 
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.generated.StormTopology;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Values;
+import org.apache.storm.redis.common.config.JedisPoolConfig;
 import org.apache.storm.redis.common.mapper.RedisDataTypeDescription;
 import org.apache.storm.redis.trident.state.RedisMapState;
-import org.apache.storm.redis.common.config.JedisPoolConfig;
 import org.apache.storm.trident.Stream;
 import org.apache.storm.trident.TridentState;
 import org.apache.storm.trident.TridentTopology;
@@ -34,6 +30,8 @@ import org.apache.storm.trident.operation.builtin.MapGet;
 import org.apache.storm.trident.operation.builtin.Sum;
 import org.apache.storm.trident.state.StateFactory;
 import org.apache.storm.trident.testing.FixedBatchSpout;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Values;
 
 public class WordCountTridentRedisMap {
     public static StormTopology buildTopology(String redisHost, Integer redisPort){
@@ -66,29 +64,18 @@ public class WordCountTridentRedisMap {
     }
 
     public static void main(String[] args) throws Exception {
-        if (args.length != 3) {
-            System.out.println("Usage: WordCountTrident 0(storm-local)|1(storm-cluster) redis-host redis-port");
+        if (args.length != 2) {
+            System.out.println("Usage: WordCountTrident redis-host redis-port");
             System.exit(1);
         }
 
-        Integer flag = Integer.valueOf(args[0]);
-        String redisHost = args[1];
-        Integer redisPort = Integer.valueOf(args[2]);
+        String redisHost = args[0];
+        Integer redisPort = Integer.valueOf(args[1]);
 
         Config conf = new Config();
         conf.setMaxSpoutPending(5);
-        if (flag == 0) {
-            try (LocalCluster cluster = new LocalCluster();
-                LocalTopology topo = cluster.submitTopology("test_wordCounter_for_redis", conf, buildTopology(redisHost, redisPort));) {
-                Thread.sleep(60 * 1000);
-            }
-            System.exit(0);
-        } else if(flag == 1) {
-            conf.setNumWorkers(3);
-            StormSubmitter.submitTopology("test_wordCounter_for_redis", conf, buildTopology(redisHost, redisPort));
-        } else {
-            System.out.println("Usage: WordCountTrident 0(storm-local)|1(storm-cluster) redis-host redis-port");
-        }
+        conf.setNumWorkers(3);
+        StormSubmitter.submitTopology("test_wordCounter_for_redis", conf, buildTopology(redisHost, redisPort));
     }
 
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-solr-examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-solr-examples/pom.xml b/examples/storm-solr-examples/pom.xml
index d6d86fc..4ec049b 100644
--- a/examples/storm-solr-examples/pom.xml
+++ b/examples/storm-solr-examples/pom.xml
@@ -30,7 +30,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-server</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-solr-examples/src/main/java/org/apache/storm/solr/topology/SolrTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-solr-examples/src/main/java/org/apache/storm/solr/topology/SolrTopology.java b/examples/storm-solr-examples/src/main/java/org/apache/storm/solr/topology/SolrTopology.java
index 3b5b1b6..6e483d7 100644
--- a/examples/storm-solr-examples/src/main/java/org/apache/storm/solr/topology/SolrTopology.java
+++ b/examples/storm-solr-examples/src/main/java/org/apache/storm/solr/topology/SolrTopology.java
@@ -18,18 +18,16 @@
 
 package org.apache.storm.solr.topology;
 
+import java.io.IOException;
+
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.generated.StormTopology;
 import org.apache.storm.solr.config.SolrCommitStrategy;
 import org.apache.storm.solr.config.SolrConfig;
 
-import java.io.IOException;
-
 public abstract class SolrTopology {
     protected static String COLLECTION = "gettingstarted";
 
@@ -37,11 +35,11 @@ public abstract class SolrTopology {
         final StormTopology topology = getTopology();
         final Config config = getConfig();
 
-        if (args.length == 0) {
-            submitTopologyLocalCluster(topology, config);
-        } else {
-            submitTopologyRemoteCluster(args[0], topology, config);
+        String topoName = "test";
+        if (args.length > 0) {
+            topoName = args[0];
         }
+        submitTopologyRemoteCluster(topoName, topology, config);
     }
 
     protected abstract StormTopology getTopology() throws IOException;
@@ -50,15 +48,6 @@ public abstract class SolrTopology {
         StormSubmitter.submitTopology(arg, config, topology);
     }
 
-    protected void submitTopologyLocalCluster(StormTopology topology, Config config) throws Exception {
-        try (LocalCluster cluster = new LocalCluster();
-             LocalTopology topo = cluster.submitTopology("test", config, topology);) {
-            Thread.sleep(10000);
-            System.out.println("Killing topology per client's request");
-        }
-        System.exit(0);
-    }
-
     protected Config getConfig() {
         Config config = new Config();
         config.setDebug(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml
index 7d102e7..b66e4fe 100644
--- a/examples/storm-starter/pom.xml
+++ b/examples/storm-starter/pom.xml
@@ -71,32 +71,27 @@
         <groupId>org.apache.storm</groupId>
         <artifactId>storm-clojure</artifactId>
         <version>${project.version}</version>
-        <!--
-          Use "provided" scope to keep storm out of the jar-with-dependencies
-          For IntelliJ dev, intellij will load properly.
-        -->
-        <scope>${provided.scope}</scope>
     </dependency>
-      <!--
-      normally including 'storm-server' is OK for LocalCluster,
-      but 'storm-starter' also uses clojure implementation
-      so 'storm-starter' needs to include 'storm-core'
-      -->
+    <dependency>
+        <groupId>org.apache.storm</groupId>
+        <artifactId>storm-clojure-test</artifactId>
+        <version>${project.version}</version>
+        <scope>test</scope>
+    </dependency>
     <dependency>
       <groupId>org.apache.storm</groupId>
-      <artifactId>storm-core</artifactId>
+      <artifactId>storm-client</artifactId>
       <version>${project.version}</version>
-      <!--
-        Use "provided" scope to keep storm out of the jar-with-dependencies
-        For IntelliJ dev, intellij will load properly.
-      -->
+        <!--
+          Use "provided" scope to keep storm out of the jar-with-dependencies
+          For IntelliJ dev, intellij will load properly.
+        -->
         <scope>${provided.scope}</scope>
     </dependency>
       <dependency>
           <groupId>org.apache.storm</groupId>
           <artifactId>storm-client-misc</artifactId>
           <version>${project.version}</version>
-          <scope>${provided.scope}</scope>
       </dependency>
       <dependency>
           <groupId>org.apache.storm</groupId>

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/clj/org/apache/storm/starter/clj/exclamation.clj
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/clj/org/apache/storm/starter/clj/exclamation.clj b/examples/storm-starter/src/clj/org/apache/storm/starter/clj/exclamation.clj
index 6bcd35c..415f43f 100644
--- a/examples/storm-starter/src/clj/org/apache/storm/starter/clj/exclamation.clj
+++ b/examples/storm-starter/src/clj/org/apache/storm/starter/clj/exclamation.clj
@@ -14,7 +14,7 @@
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
 (ns org.apache.storm.starter.clj.exclamation
-  (:import [org.apache.storm StormSubmitter LocalCluster]
+  (:import [org.apache.storm StormSubmitter]
     [org.apache.storm.utils Utils]
            [org.apache.storm.testing TestWordSpout])
   (:use [org.apache.storm clojure config])
@@ -31,11 +31,6 @@
    {"exclaim1" (bolt-spec {"word" :shuffle} exclamation-bolt :p 3)
     "exclaim2" (bolt-spec {"exclaim1" :shuffle} exclamation-bolt :p 2)}))
 
-(defn run-local! []
-      (with-open [cluster (LocalCluster.)
-              topo (.submitTopology cluster "exclamation" {TOPOLOGY-DEBUG true} (mk-topology))]
-                 (Utils/sleep 10000)))
-
 (defn submit-topology! [name]
   (StormSubmitter/submitTopologyWithProgressBar
    name
@@ -45,6 +40,6 @@
 
 (defn -main
   ([]
-   (run-local!))
+   (submit-topology! "test"))
   ([name]
    (submit-topology! name)))

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/clj/org/apache/storm/starter/clj/rolling_top_words.clj
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/clj/org/apache/storm/starter/clj/rolling_top_words.clj b/examples/storm-starter/src/clj/org/apache/storm/starter/clj/rolling_top_words.clj
index d5eada5..9a9677c 100644
--- a/examples/storm-starter/src/clj/org/apache/storm/starter/clj/rolling_top_words.clj
+++ b/examples/storm-starter/src/clj/org/apache/storm/starter/clj/rolling_top_words.clj
@@ -17,7 +17,7 @@
   (:require [org.apache.storm [clojure :refer :all] [config :refer :all]]
             [org.apache.storm.starter.clj.bolts :refer
              [rolling-count-bolt intermediate-rankings-bolt total-rankings-bolt]])
-  (:import [org.apache.storm StormSubmitter LocalCluster]
+  (:import [org.apache.storm StormSubmitter]
     [org.apache.storm.utils Utils]
     [org.apache.storm.testing TestWordSpout])
   (:gen-class))
@@ -38,11 +38,6 @@
       total-ranker-id (bolt-spec {ranker-id :global}
                                  (total-rankings-bolt 5 2))})))
 
-(defn run-local! []
-      (with-open [cluster (LocalCluster.)
-              topo (.submitTopology cluster "slidingWindowCounts" {TOPOLOGY-DEBUG true} (mk-topology))]
-                 (Utils/sleep 60000)))
-
 (defn submit-topology! [name]
   (StormSubmitter/submitTopology
    name
@@ -52,6 +47,6 @@
 
 (defn -main
   ([]
-   (run-local!))
+   (submit-topology! "test"))
   ([name]
    (submit-topology! name)))

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/clj/org/apache/storm/starter/clj/word_count.clj
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/clj/org/apache/storm/starter/clj/word_count.clj b/examples/storm-starter/src/clj/org/apache/storm/starter/clj/word_count.clj
index cefa1e5..e3a52f5 100644
--- a/examples/storm-starter/src/clj/org/apache/storm/starter/clj/word_count.clj
+++ b/examples/storm-starter/src/clj/org/apache/storm/starter/clj/word_count.clj
@@ -14,7 +14,7 @@
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
 (ns org.apache.storm.starter.clj.word-count
-  (:import [org.apache.storm StormSubmitter LocalCluster]
+  (:import [org.apache.storm StormSubmitter]
            [org.apache.storm.utils Utils])
   (:use [org.apache.storm clojure config])
   (:gen-class))
@@ -74,11 +74,6 @@
                    word-count
                    :p 6)}))
 
-(defn run-local! []
-  (with-open [cluster (LocalCluster.)
-              topo (.submitTopology cluster "word-count" {TOPOLOGY-DEBUG true} (mk-topology))]
-    (Thread/sleep 10000)))
-
 (defn submit-topology! [name]
   (StormSubmitter/submitTopology
    name
@@ -88,7 +83,7 @@
 
 (defn -main
   ([]
-   (run-local!))
+   (submit-topology! "test"))
   ([name]
    (submit-topology! name)))
 

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/BasicDRPCTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/BasicDRPCTopology.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/BasicDRPCTopology.java
index 0f5f115..28d01f3 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/BasicDRPCTopology.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/BasicDRPCTopology.java
@@ -18,8 +18,6 @@
 package org.apache.storm.starter;
 
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalDRPC;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.drpc.LinearDRPCTopologyBuilder;
 import org.apache.storm.topology.BasicOutputCollector;
@@ -28,6 +26,7 @@ import org.apache.storm.topology.base.BaseBasicBolt;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Tuple;
 import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.DRPCClient;
 
 /**
  * This topology is a basic example of doing distributed RPC on top of Storm. It implements a function that appends a
@@ -36,42 +35,40 @@ import org.apache.storm.tuple.Values;
  * @see <a href="http://storm.apache.org/documentation/Distributed-RPC.html">Distributed RPC</a>
  */
 public class BasicDRPCTopology {
-  public static class ExclaimBolt extends BaseBasicBolt {
-    @Override
-    public void execute(Tuple tuple, BasicOutputCollector collector) {
-      String input = tuple.getString(1);
-      collector.emit(new Values(tuple.getValue(0), input + "!"));
-    }
+    public static class ExclaimBolt extends BaseBasicBolt {
+        @Override
+        public void execute(Tuple tuple, BasicOutputCollector collector) {
+            String input = tuple.getString(1);
+            collector.emit(new Values(tuple.getValue(0), input + "!"));
+        }
 
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer declarer) {
-      declarer.declare(new Fields("id", "result"));
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("id", "result"));
+        }
     }
 
-  }
-
-  public static void main(String[] args) throws Exception {
-    LinearDRPCTopologyBuilder builder = new LinearDRPCTopologyBuilder("exclamation");
-    builder.addBolt(new ExclaimBolt(), 3);
+    public static void main(String[] args) throws Exception {
+        LinearDRPCTopologyBuilder builder = new LinearDRPCTopologyBuilder("exclamation");
+        builder.addBolt(new ExclaimBolt(), 3);
 
-    Config conf = new Config();
+        Config conf = new Config();
+        String topoName = "DRPCExample";
 
-    if (args == null || args.length == 0) {
-      try (LocalDRPC drpc = new LocalDRPC();
-           LocalCluster cluster = new LocalCluster()) {
-
-        cluster.submitTopology("drpc-demo", conf, builder.createLocalTopology(drpc));
-
-        for (String word : new String[]{ "hello", "goodbye" }) {
-          System.out.println("Result for \"" + word + "\": " + drpc.execute("exclamation", word));
+        if (args != null && args.length > 0) {
+            topoName = args[0]; 
         }
 
-        Thread.sleep(10000);
-      }
-    }
-    else {
-      conf.setNumWorkers(3);
-      StormSubmitter.submitTopologyWithProgressBar(args[0], conf, builder.createRemoteTopology());
+        conf.setNumWorkers(3);
+        StormSubmitter.submitTopologyWithProgressBar(args[0], conf, builder.createRemoteTopology());
+        
+        if (args.length > 1) {
+            try (DRPCClient drpc = DRPCClient.getConfiguredClient(conf)) {
+                for (int i = 1; i < args.length; i++) {
+                    String word = args[i];
+                    System.out.println("Result for \"" + word + "\": " + drpc.execute("exclamation", word));
+                }
+            }
+        }
     }
-  }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/ExclamationTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/ExclamationTopology.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/ExclamationTopology.java
index 0b04709..ed93686 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/ExclamationTopology.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/ExclamationTopology.java
@@ -71,11 +71,7 @@ public class ExclamationTopology extends ConfigurableTopology {
 
     String topologyName = "test";
 
-    if (isLocal) {
-      ttl = 10;
-    } else {
-      conf.setNumWorkers(3);
-    }
+    conf.setNumWorkers(3);
 
     if (args != null && args.length > 0) {
       topologyName = args[0];

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/FastWordCountTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/FastWordCountTopology.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/FastWordCountTopology.java
index 5acf908..1c48480 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/FastWordCountTopology.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/FastWordCountTopology.java
@@ -117,7 +117,7 @@ public class FastWordCountTopology {
     }
   }
 
-  public static void printMetrics(Nimbus.Client client, String name) throws Exception {
+  public static void printMetrics(Nimbus.Iface client, String name) throws Exception {
     ClusterSummary summary = client.getClusterInfo();
     String id = null;
     for (TopologySummary ts: summary.get_topologies()) {
@@ -157,7 +157,7 @@ public class FastWordCountTopology {
     System.out.println("uptime: "+uptime+" acked: "+acked+" avgLatency: "+avgLatency+" acked/sec: "+(((double)acked)/uptime+" failed: "+failed));
   } 
 
-  public static void kill(Nimbus.Client client, String name) throws Exception {
+  public static void kill(Nimbus.Iface client, String name) throws Exception {
     KillOptions opts = new KillOptions();
     opts.set_wait_secs(0);
     client.killTopologyWithOpts(name, opts);
@@ -185,7 +185,7 @@ public class FastWordCountTopology {
 
     Map clusterConf = Utils.readStormConfig();
     clusterConf.putAll(Utils.readCommandLineOpts());
-    Nimbus.Client client = NimbusClient.getConfiguredClient(clusterConf).getClient();
+    Nimbus.Iface client = NimbusClient.getConfiguredClient(clusterConf).getClient();
 
     //Sleep for 5 mins
     for (int i = 0; i < 10; i++) {

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/InOrderDeliveryTest.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/InOrderDeliveryTest.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/InOrderDeliveryTest.java
index 78907ea..63be726 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/InOrderDeliveryTest.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/InOrderDeliveryTest.java
@@ -95,7 +95,7 @@ public class InOrderDeliveryTest {
     }
   }
 
-  public static void printMetrics(Nimbus.Client client, String name) throws Exception {
+  public static void printMetrics(Nimbus.Iface client, String name) throws Exception {
     ClusterSummary summary = client.getClusterInfo();
     String id = null;
     for (TopologySummary ts: summary.get_topologies()) {
@@ -135,7 +135,7 @@ public class InOrderDeliveryTest {
     System.out.println("uptime: "+uptime+" acked: "+acked+" avgLatency: "+avgLatency+" acked/sec: "+(((double)acked)/uptime+" failed: "+failed));
   } 
 
-  public static void kill(Nimbus.Client client, String name) throws Exception {
+  public static void kill(Nimbus.Iface client, String name) throws Exception {
     KillOptions opts = new KillOptions();
     opts.set_wait_secs(0);
     client.killTopologyWithOpts(name, opts);
@@ -161,7 +161,7 @@ public class InOrderDeliveryTest {
 
     Map clusterConf = Utils.readStormConfig();
     clusterConf.putAll(Utils.readCommandLineOpts());
-    Nimbus.Client client = NimbusClient.getConfiguredClient(clusterConf).getClient();
+    Nimbus.Iface client = NimbusClient.getConfiguredClient(clusterConf).getClient();
 
     //Sleep for 50 mins
     for (int i = 0; i < 50; i++) {

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/JoinBoltExample.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/JoinBoltExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/JoinBoltExample.java
index c23fad5..b71b64a 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/JoinBoltExample.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/JoinBoltExample.java
@@ -17,7 +17,10 @@
  */
 package org.apache.storm.starter;
 
+import java.util.concurrent.TimeUnit;
+
 import org.apache.storm.Config;
+import org.apache.storm.StormSubmitter;
 import org.apache.storm.bolt.JoinBolt;
 import org.apache.storm.starter.bolt.PrinterBolt;
 import org.apache.storm.testing.FeederSpout;
@@ -25,14 +28,14 @@ import org.apache.storm.topology.TopologyBuilder;
 import org.apache.storm.topology.base.BaseWindowedBolt;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Values;
-import org.apache.storm.utils.Utils;
-import org.apache.storm.LocalCluster;
-
-import java.util.concurrent.TimeUnit;
+import org.apache.storm.utils.NimbusClient;
 
 public class JoinBoltExample {
     public static void main(String[] args) throws Exception {
-
+        if (!NimbusClient.isLocalOverride()) {
+            throw new IllegalStateException("This example only works in local mode.  " 
+                    + "Run with storm local not storm jar");
+        }
         FeederSpout genderSpout = new FeederSpout(new Fields("id", "gender"));
         FeederSpout ageSpout = new FeederSpout(new Fields("id", "age"));
 
@@ -53,16 +56,11 @@ public class JoinBoltExample {
         builder.setBolt("printer", new PrinterBolt() ).shuffleGrouping("joiner");
 
         Config conf = new Config();
-
-        LocalCluster cluster = new LocalCluster();
-        cluster.submitTopology("join-example", conf, builder.createTopology());
+        StormSubmitter.submitTopologyWithProgressBar("join-example", conf, builder.createTopology());
 
         generateGenderData(genderSpout);
 
         generateAgeData(ageSpout);
-
-        Utils.sleep(30000);
-        cluster.shutdown();
     }
 
     private static void generateAgeData(FeederSpout ageSpout) {

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/ManualDRPC.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/ManualDRPC.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/ManualDRPC.java
index 5375ea6..110d0be 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/ManualDRPC.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/ManualDRPC.java
@@ -18,9 +18,7 @@
 package org.apache.storm.starter;
 
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
-import org.apache.storm.LocalDRPC;
+import org.apache.storm.StormSubmitter;
 import org.apache.storm.drpc.DRPCSpout;
 import org.apache.storm.drpc.ReturnResults;
 import org.apache.storm.topology.BasicOutputCollector;
@@ -30,6 +28,7 @@ import org.apache.storm.topology.base.BaseBasicBolt;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Tuple;
 import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.DRPCClient;
 
 public class ManualDRPC {
     public static class ExclamationBolt extends BaseBasicBolt {
@@ -50,19 +49,17 @@ public class ManualDRPC {
 
     public static void main(String[] args) throws Exception {
         TopologyBuilder builder = new TopologyBuilder();
-        try(LocalDRPC drpc = new LocalDRPC();
-            LocalCluster cluster = new LocalCluster();) {
 
-            DRPCSpout spout = new DRPCSpout("exclamation", drpc);
-            builder.setSpout("drpc", spout);
-            builder.setBolt("exclaim", new ExclamationBolt(), 3).shuffleGrouping("drpc");
-            builder.setBolt("return", new ReturnResults(), 3).shuffleGrouping("exclaim");
+        DRPCSpout spout = new DRPCSpout("exclamation");
+        builder.setSpout("drpc", spout);
+        builder.setBolt("exclaim", new ExclamationBolt(), 3).shuffleGrouping("drpc");
+        builder.setBolt("return", new ReturnResults(), 3).shuffleGrouping("exclaim");
 
-            Config conf = new Config();
-            try (LocalTopology topo = cluster.submitTopology("exclaim", conf, builder.createTopology())) {
-                System.out.println(drpc.execute("exclamation", "aaa"));
-                System.out.println(drpc.execute("exclamation", "bbb"));
-            }
+        Config conf = new Config();
+        StormSubmitter.submitTopology("exclaim", conf, builder.createTopology());
+        try (DRPCClient drpc = DRPCClient.getConfiguredClient(conf)) {
+            System.out.println(drpc.execute("exclamation", "aaa"));
+            System.out.println(drpc.execute("exclamation", "bbb"));
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/MultipleLoggerTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/MultipleLoggerTopology.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/MultipleLoggerTopology.java
index aca5c7b..26c4ee8 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/MultipleLoggerTopology.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/MultipleLoggerTopology.java
@@ -17,9 +17,9 @@
  */
 package org.apache.storm.starter;
 
+import java.util.Map;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.task.OutputCollector;
 import org.apache.storm.task.TopologyContext;
@@ -30,13 +30,9 @@ import org.apache.storm.topology.base.BaseRichBolt;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Tuple;
 import org.apache.storm.tuple.Values;
-import org.apache.storm.utils.Utils;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.Map;
-
 /**
  * This is a basic example of a Storm topology.
  */
@@ -90,15 +86,11 @@ public class MultipleLoggerTopology {
 
     Config conf = new Config();
     conf.setDebug(true);
-
+    String topoName = MultipleLoggerTopology.class.getName();
     if (args != null && args.length > 0) {
-      conf.setNumWorkers(2);
-      StormSubmitter.submitTopologyWithProgressBar(args[0], conf, builder.createTopology());
-    } else {
-      try (LocalCluster cluster = new LocalCluster();
-           LocalTopology topo = cluster.submitTopology("test", conf, builder.createTopology());) {
-          Utils.sleep(10000);
-      }
+      topoName = args[0];
     }
+    conf.setNumWorkers(2);
+    StormSubmitter.submitTopologyWithProgressBar(topoName, conf, builder.createTopology());
   }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/ReachTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/ReachTopology.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/ReachTopology.java
index 6b4aaa6..e90fcb0 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/ReachTopology.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/ReachTopology.java
@@ -17,9 +17,14 @@
  */
 package org.apache.storm.starter;
 
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalDRPC;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.coordination.BatchOutputCollector;
 import org.apache.storm.drpc.LinearDRPCTopologyBuilder;
@@ -31,8 +36,7 @@ import org.apache.storm.topology.base.BaseBatchBolt;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Tuple;
 import org.apache.storm.tuple.Values;
-
-import java.util.*;
+import org.apache.storm.utils.DRPCClient;
 
 /**
  * This is a good example of doing complex Distributed RPC on top of Storm. This program creates a topology that can
@@ -51,144 +55,139 @@ import java.util.*;
  * @see <a href="http://storm.apache.org/documentation/Distributed-RPC.html">Distributed RPC</a>
  */
 public class ReachTopology {
-  public static Map<String, List<String>> TWEETERS_DB = new HashMap<String, List<String>>() {{
-    put("foo.com/blog/1", Arrays.asList("sally", "bob", "tim", "george", "nathan"));
-    put("engineering.twitter.com/blog/5", Arrays.asList("adam", "david", "sally", "nathan"));
-    put("tech.backtype.com/blog/123", Arrays.asList("tim", "mike", "john"));
-  }};
-
-  public static Map<String, List<String>> FOLLOWERS_DB = new HashMap<String, List<String>>() {{
-    put("sally", Arrays.asList("bob", "tim", "alice", "adam", "jim", "chris", "jai"));
-    put("bob", Arrays.asList("sally", "nathan", "jim", "mary", "david", "vivian"));
-    put("tim", Arrays.asList("alex"));
-    put("nathan", Arrays.asList("sally", "bob", "adam", "harry", "chris", "vivian", "emily", "jordan"));
-    put("adam", Arrays.asList("david", "carissa"));
-    put("mike", Arrays.asList("john", "bob"));
-    put("john", Arrays.asList("alice", "nathan", "jim", "mike", "bob"));
-  }};
-
-  public static class GetTweeters extends BaseBasicBolt {
-    @Override
-    public void execute(Tuple tuple, BasicOutputCollector collector) {
-      Object id = tuple.getValue(0);
-      String url = tuple.getString(1);
-      List<String> tweeters = TWEETERS_DB.get(url);
-      if (tweeters != null) {
-        for (String tweeter : tweeters) {
-          collector.emit(new Values(id, tweeter));
+    public static Map<String, List<String>> TWEETERS_DB = new HashMap<String, List<String>>() {{
+        put("foo.com/blog/1", Arrays.asList("sally", "bob", "tim", "george", "nathan"));
+        put("engineering.twitter.com/blog/5", Arrays.asList("adam", "david", "sally", "nathan"));
+        put("tech.backtype.com/blog/123", Arrays.asList("tim", "mike", "john"));
+    }};
+
+    public static Map<String, List<String>> FOLLOWERS_DB = new HashMap<String, List<String>>() {{
+        put("sally", Arrays.asList("bob", "tim", "alice", "adam", "jim", "chris", "jai"));
+        put("bob", Arrays.asList("sally", "nathan", "jim", "mary", "david", "vivian"));
+        put("tim", Arrays.asList("alex"));
+        put("nathan", Arrays.asList("sally", "bob", "adam", "harry", "chris", "vivian", "emily", "jordan"));
+        put("adam", Arrays.asList("david", "carissa"));
+        put("mike", Arrays.asList("john", "bob"));
+        put("john", Arrays.asList("alice", "nathan", "jim", "mike", "bob"));
+    }};
+
+    public static class GetTweeters extends BaseBasicBolt {
+        @Override
+        public void execute(Tuple tuple, BasicOutputCollector collector) {
+            Object id = tuple.getValue(0);
+            String url = tuple.getString(1);
+            List<String> tweeters = TWEETERS_DB.get(url);
+            if (tweeters != null) {
+                for (String tweeter : tweeters) {
+                    collector.emit(new Values(id, tweeter));
+                }
+            }
         }
-      }
-    }
 
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer declarer) {
-      declarer.declare(new Fields("id", "tweeter"));
-    }
-  }
-
-  public static class GetFollowers extends BaseBasicBolt {
-    @Override
-    public void execute(Tuple tuple, BasicOutputCollector collector) {
-      Object id = tuple.getValue(0);
-      String tweeter = tuple.getString(1);
-      List<String> followers = FOLLOWERS_DB.get(tweeter);
-      if (followers != null) {
-        for (String follower : followers) {
-          collector.emit(new Values(id, follower));
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("id", "tweeter"));
         }
-      }
     }
 
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer declarer) {
-      declarer.declare(new Fields("id", "follower"));
-    }
-  }
-
-  public static class PartialUniquer extends BaseBatchBolt {
-    BatchOutputCollector _collector;
-    Object _id;
-    Set<String> _followers = new HashSet<String>();
+    public static class GetFollowers extends BaseBasicBolt {
+        @Override
+        public void execute(Tuple tuple, BasicOutputCollector collector) {
+            Object id = tuple.getValue(0);
+            String tweeter = tuple.getString(1);
+            List<String> followers = FOLLOWERS_DB.get(tweeter);
+            if (followers != null) {
+                for (String follower : followers) {
+                    collector.emit(new Values(id, follower));
+                }
+            }
+        }
 
-    @Override
-    public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, Object id) {
-      _collector = collector;
-      _id = id;
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("id", "follower"));
+        }
     }
 
-    @Override
-    public void execute(Tuple tuple) {
-      _followers.add(tuple.getString(1));
-    }
+    public static class PartialUniquer extends BaseBatchBolt {
+        BatchOutputCollector _collector;
+        Object _id;
+        Set<String> _followers = new HashSet<String>();
 
-    @Override
-    public void finishBatch() {
-      _collector.emit(new Values(_id, _followers.size()));
-    }
+        @Override
+        public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, Object id) {
+            _collector = collector;
+            _id = id;
+        }
 
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer declarer) {
-      declarer.declare(new Fields("id", "partial-count"));
-    }
-  }
+        @Override
+        public void execute(Tuple tuple) {
+            _followers.add(tuple.getString(1));
+        }
 
-  public static class CountAggregator extends BaseBatchBolt {
-    BatchOutputCollector _collector;
-    Object _id;
-    int _count = 0;
+        @Override
+        public void finishBatch() {
+            _collector.emit(new Values(_id, _followers.size()));
+        }
 
-    @Override
-    public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, Object id) {
-      _collector = collector;
-      _id = id;
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("id", "partial-count"));
+        }
     }
 
-    @Override
-    public void execute(Tuple tuple) {
-      _count += tuple.getInteger(1);
-    }
+    public static class CountAggregator extends BaseBatchBolt {
+        BatchOutputCollector _collector;
+        Object _id;
+        int _count = 0;
 
-    @Override
-    public void finishBatch() {
-      _collector.emit(new Values(_id, _count));
-    }
+        @Override
+        public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, Object id) {
+            _collector = collector;
+            _id = id;
+        }
 
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer declarer) {
-      declarer.declare(new Fields("id", "reach"));
-    }
-  }
+        @Override
+        public void execute(Tuple tuple) {
+            _count += tuple.getInteger(1);
+        }
 
-  public static LinearDRPCTopologyBuilder construct() {
-    LinearDRPCTopologyBuilder builder = new LinearDRPCTopologyBuilder("reach");
-    builder.addBolt(new GetTweeters(), 4);
-    builder.addBolt(new GetFollowers(), 12).shuffleGrouping();
-    builder.addBolt(new PartialUniquer(), 6).fieldsGrouping(new Fields("id", "follower"));
-    builder.addBolt(new CountAggregator(), 3).fieldsGrouping(new Fields("id"));
-    return builder;
-  }
+        @Override
+        public void finishBatch() {
+            _collector.emit(new Values(_id, _count));
+        }
 
-  public static void main(String[] args) throws Exception {
-    LinearDRPCTopologyBuilder builder = construct();
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("id", "reach"));
+        }
+    }
 
+    public static LinearDRPCTopologyBuilder construct() {
+        LinearDRPCTopologyBuilder builder = new LinearDRPCTopologyBuilder("reach");
+        builder.addBolt(new GetTweeters(), 4);
+        builder.addBolt(new GetFollowers(), 12).shuffleGrouping();
+        builder.addBolt(new PartialUniquer(), 6).fieldsGrouping(new Fields("id", "follower"));
+        builder.addBolt(new CountAggregator(), 3).fieldsGrouping(new Fields("id"));
+        return builder;
+    }
 
-    Config conf = new Config();
+    public static void main(String[] args) throws Exception {
+        LinearDRPCTopologyBuilder builder = construct();
 
-    if (args == null || args.length == 0) {
-      conf.setMaxTaskParallelism(3);
-      try (LocalDRPC drpc = new LocalDRPC();
-          LocalCluster cluster = new LocalCluster();) {
-        cluster.submitTopology("reach-drpc", conf, builder.createLocalTopology(drpc));
+        Config conf = new Config();
+        conf.setNumWorkers(6);
+        String topoName = "reach-drpc";
+        if (args.length > 0) {
+            topoName = args[0];
+        }
+        StormSubmitter.submitTopologyWithProgressBar(topoName, conf, builder.createRemoteTopology());
 
-        String[] urlsToTry = new String[]{ "foo.com/blog/1", "engineering.twitter.com/blog/5", "notaurl.com" };
-        for (String url : urlsToTry) {
-          System.out.println("Reach of " + url + ": " + drpc.execute("reach", url));
+        try (DRPCClient drpc = DRPCClient.getConfiguredClient(conf)) {
+            String[] urlsToTry = new String[]{ "foo.com/blog/1", "engineering.twitter.com/blog/5", "notaurl.com" };
+            for (String url : urlsToTry) {
+                System.out.println("Reach of " + url + ": " + drpc.execute("reach", url));
+            }
         }
-      }
-    }
-    else {
-      conf.setNumWorkers(6);
-      StormSubmitter.submitTopologyWithProgressBar(args[0], conf, builder.createRemoteTopology());
     }
-  }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/ResourceAwareExampleTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/ResourceAwareExampleTopology.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/ResourceAwareExampleTopology.java
index 788485d..57fcd5d 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/ResourceAwareExampleTopology.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/ResourceAwareExampleTopology.java
@@ -17,9 +17,9 @@
  */
 package org.apache.storm.starter;
 
+import java.util.Map;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.task.OutputCollector;
 import org.apache.storm.task.TopologyContext;
@@ -32,9 +32,6 @@ import org.apache.storm.topology.base.BaseRichBolt;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Tuple;
 import org.apache.storm.tuple.Values;
-import org.apache.storm.utils.Utils;
-
-import java.util.Map;
 
 public class ResourceAwareExampleTopology {
   public static class ExclamationBolt extends BaseRichBolt {
@@ -90,17 +87,12 @@ public class ResourceAwareExampleTopology {
     // Set strategy to schedule topology. If not specified, default to org.apache.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy
     conf.setTopologyStrategy(org.apache.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy.class);
 
+    String topoName = "test";
     if (args != null && args.length > 0) {
-      conf.setNumWorkers(3);
-
-      StormSubmitter.submitTopologyWithProgressBar(args[0], conf, builder.createTopology());
+        topoName = args[0];
     }
-    else {
+    conf.setNumWorkers(3);
 
-      try (LocalCluster cluster = new LocalCluster();
-           LocalTopology topo = cluster.submitTopology("test", conf, builder.createTopology());) {
-        Utils.sleep(10000);
-      }
-    }
+    StormSubmitter.submitTopologyWithProgressBar(topoName, conf, builder.createTopology());
   }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/RollingTopWords.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/RollingTopWords.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/RollingTopWords.java
index 7ebbaf8..78b2baf 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/RollingTopWords.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/RollingTopWords.java
@@ -57,15 +57,6 @@ public class RollingTopWords extends ConfigurableTopology {
    *
    * ```
    *
-   * # Runs in local mode (LocalCluster), with topology name "slidingWindowCounts"
-   * $ storm jar storm-starter-jar-with-dependencies.jar org.apache.storm.starter.RollingTopWords -local
-   * 
-   * # Runs in local mode (LocalCluster), with topology name "foobar"
-   * $ storm jar storm-starter-jar-with-dependencies.jar org.apache.storm.starter.RollingTopWords foobar -local
-   * 
-   * # Runs in local mode (LocalCluster) for 30 seconds, with topology name "foobar" 
-   * $ storm jar storm-starter-jar-with-dependencies.jar org.apache.storm.starter.RollingTopWords foobar -local -ttl 30
-   *
    * # Runs in remote/cluster mode, with topology name "production-topology"
    * $ storm jar storm-starter-jar-with-dependencies.jar org.apache.storm.starter.RollingTopWords production-topology ```
    *

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/SingleJoinExample.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/SingleJoinExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/SingleJoinExample.java
index 4eb9ed9..2e1bb94 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/SingleJoinExample.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/SingleJoinExample.java
@@ -18,52 +18,50 @@
 package org.apache.storm.starter;
 
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
+import org.apache.storm.StormSubmitter;
+import org.apache.storm.starter.bolt.SingleJoinBolt;
 import org.apache.storm.testing.FeederSpout;
 import org.apache.storm.topology.TopologyBuilder;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Values;
-import org.apache.storm.utils.Utils;
-import org.apache.storm.starter.bolt.SingleJoinBolt;
+import org.apache.storm.utils.NimbusClient;
 
 /** Example of using a simple custom join bolt
  *  NOTE: Prefer to use the built-in JoinBolt wherever applicable
  */
 
 public class SingleJoinExample {
-  public static void main(String[] args) throws Exception {
-    FeederSpout genderSpout = new FeederSpout(new Fields("id", "gender"));
-    FeederSpout ageSpout = new FeederSpout(new Fields("id", "age"));
+    public static void main(String[] args) throws Exception {
+        if (!NimbusClient.isLocalOverride()) {
+            throw new IllegalStateException("This example only works in local mode.  "
+                    + "Run with storm local not storm jar");
+        }
+        FeederSpout genderSpout = new FeederSpout(new Fields("id", "gender"));
+        FeederSpout ageSpout = new FeederSpout(new Fields("id", "age"));
 
-    TopologyBuilder builder = new TopologyBuilder();
-    builder.setSpout("gender", genderSpout);
-    builder.setSpout("age", ageSpout);
-    builder.setBolt("join", new SingleJoinBolt(new Fields("gender", "age"))).fieldsGrouping("gender", new Fields("id"))
+        TopologyBuilder builder = new TopologyBuilder();
+        builder.setSpout("gender", genderSpout);
+        builder.setSpout("age", ageSpout);
+        builder.setBolt("join", new SingleJoinBolt(new Fields("gender", "age"))).fieldsGrouping("gender", new Fields("id"))
         .fieldsGrouping("age", new Fields("id"));
 
-    Config conf = new Config();
-    conf.setDebug(true);
-
-    try (LocalCluster cluster = new LocalCluster();
-         LocalTopology topo = cluster.submitTopology("join-example", conf, builder.createTopology());) {
+        Config conf = new Config();
+        conf.setDebug(true);
+        StormSubmitter.submitTopology("join-example", conf, builder.createTopology());
 
-      for (int i = 0; i < 10; i++) {
-        String gender;
-        if (i % 2 == 0) {
-          gender = "male";
+        for (int i = 0; i < 10; i++) {
+            String gender;
+            if (i % 2 == 0) {
+                gender = "male";
+            }
+            else {
+                gender = "female";
+            }
+            genderSpout.feed(new Values(i, gender));
         }
-        else {
-          gender = "female";
-        }
-        genderSpout.feed(new Values(i, gender));
-      }
-
-      for (int i = 9; i >= 0; i--) {
-        ageSpout.feed(new Values(i, i + 20));
-      }
 
-      Utils.sleep(2000);
+        for (int i = 9; i >= 0; i--) {
+            ageSpout.feed(new Values(i, i + 20));
+        }
     }
-  }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/SkewedRollingTopWords.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/SkewedRollingTopWords.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/SkewedRollingTopWords.java
index 83ad4fc..4e18217 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/SkewedRollingTopWords.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/SkewedRollingTopWords.java
@@ -60,15 +60,6 @@ public class SkewedRollingTopWords extends ConfigurableTopology {
    *
    * ```
    *
-   * # Runs in local mode (LocalCluster), with topology name "slidingWindowCounts"
-   * $ storm jar storm-starter-jar-with-dependencies.jar org.apache.storm.starter.SkewedRollingTopWords -local
-   *
-   * # Runs in local mode (LocalCluster), with topology name "foobar"
-   * $ storm jar storm-starter-jar-with-dependencies.jar org.apache.storm.starter.SkewedRollingTopWords foobar -local
-   * 
-   * # Runs in local mode (LocalCluster) for 30 seconds, with topology name "foobar" 
-   * $ storm jar storm-starter-jar-with-dependencies.jar org.apache.storm.starter.SkewedRollingTopWords foobar -local -ttl 30
-   *
    * # Runs in remote/cluster mode, with topology name "production-topology"
    * $ storm jar storm-starter-jar-with-dependencies.jar org.apache.storm.starter.SkewedRollingTopWords production-topology ```
    *

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/SlidingTupleTsTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/SlidingTupleTsTopology.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/SlidingTupleTsTopology.java
index 3993b95..6204f8c 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/SlidingTupleTsTopology.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/SlidingTupleTsTopology.java
@@ -17,20 +17,16 @@
  */
 package org.apache.storm.starter;
 
+import java.util.concurrent.TimeUnit;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
-import org.apache.storm.topology.TopologyBuilder;
-import org.apache.storm.topology.base.BaseWindowedBolt;
-import org.apache.storm.utils.Utils;
 import org.apache.storm.starter.bolt.PrinterBolt;
 import org.apache.storm.starter.bolt.SlidingWindowSumBolt;
 import org.apache.storm.starter.spout.RandomIntegerSpout;
-
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.storm.topology.base.BaseWindowedBolt.Duration;
+import org.apache.storm.topology.TopologyBuilder;
+import org.apache.storm.topology.base.BaseWindowedBolt;
+import org.apache.storm.topology.base.BaseWindowedBolt.Duration;
 
 /**
  * Windowing based on tuple timestamp (e.g. the time when tuple is generated
@@ -48,15 +44,13 @@ public class SlidingTupleTsTopology {
         builder.setBolt("printer", new PrinterBolt(), 1).shuffleGrouping("slidingsum");
         Config conf = new Config();
         conf.setDebug(true);
-
+        String topoName = "test";
+        
         if (args != null && args.length > 0) {
-            conf.setNumWorkers(1);
-            StormSubmitter.submitTopologyWithProgressBar(args[0], conf, builder.createTopology());
-        } else {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology("test", conf, builder.createTopology());) {
-                Utils.sleep(40000);
-            }
+            topoName = args[0];
         }
+        
+        conf.setNumWorkers(1);
+        StormSubmitter.submitTopologyWithProgressBar(topoName, conf, builder.createTopology());
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/SlidingWindowTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/SlidingWindowTopology.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/SlidingWindowTopology.java
index 3c68f16..f00dac9 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/SlidingWindowTopology.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/SlidingWindowTopology.java
@@ -17,30 +17,26 @@
  */
 package org.apache.storm.starter;
 
+import java.util.List;
+import java.util.Map;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
+import org.apache.storm.starter.bolt.PrinterBolt;
+import org.apache.storm.starter.bolt.SlidingWindowSumBolt;
+import org.apache.storm.starter.spout.RandomIntegerSpout;
 import org.apache.storm.task.OutputCollector;
 import org.apache.storm.task.TopologyContext;
 import org.apache.storm.topology.OutputFieldsDeclarer;
 import org.apache.storm.topology.TopologyBuilder;
 import org.apache.storm.topology.base.BaseWindowedBolt;
+import org.apache.storm.topology.base.BaseWindowedBolt.Count;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Tuple;
 import org.apache.storm.tuple.Values;
-import org.apache.storm.utils.Utils;
 import org.apache.storm.windowing.TupleWindow;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.storm.starter.bolt.PrinterBolt;
-import org.apache.storm.starter.bolt.SlidingWindowSumBolt;
-import org.apache.storm.starter.spout.RandomIntegerSpout;
-
-import java.util.List;
-import java.util.Map;
-
-import static org.apache.storm.topology.base.BaseWindowedBolt.Count;
 
 /**
  * A sample topology that demonstrates the usage of {@link org.apache.storm.topology.IWindowedBolt}
@@ -95,14 +91,11 @@ public class SlidingWindowTopology {
         builder.setBolt("printer", new PrinterBolt(), 1).shuffleGrouping("tumblingavg");
         Config conf = new Config();
         conf.setDebug(true);
+        String topoName = "test";
         if (args != null && args.length > 0) {
-            conf.setNumWorkers(1);
-            StormSubmitter.submitTopologyWithProgressBar(args[0], conf, builder.createTopology());
-        } else {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology("test", conf, builder.createTopology());) {
-                Utils.sleep(40000);
-            }
+            topoName = args[0];
         }
+        conf.setNumWorkers(1);
+        StormSubmitter.submitTopologyWithProgressBar(topoName, conf, builder.createTopology());
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/StatefulTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/StatefulTopology.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/StatefulTopology.java
index cd8fa2c..36495fd 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/StatefulTopology.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/StatefulTopology.java
@@ -17,9 +17,9 @@
  */
 package org.apache.storm.starter;
 
+import java.util.Map;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.starter.spout.RandomIntegerSpout;
 import org.apache.storm.state.KeyValueState;
@@ -33,12 +33,9 @@ import org.apache.storm.topology.base.BaseStatefulBolt;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Tuple;
 import org.apache.storm.tuple.Values;
-import org.apache.storm.utils.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.Map;
-
 /**
  * An example topology that demonstrates the use of {@link org.apache.storm.topology.IStatefulBolt}
  * to manage state. To run the example,
@@ -129,15 +126,11 @@ public class StatefulTopology {
         builder.setBolt("total", new StatefulSumBolt("total"), 1).shuffleGrouping("printer");
         Config conf = new Config();
         conf.setDebug(false);
-
+        String topoName = "test";
         if (args != null && args.length > 0) {
-            conf.setNumWorkers(1);
-            StormSubmitter.submitTopologyWithProgressBar(args[0], conf, builder.createTopology());
-        } else {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topology = cluster.submitTopology("test", conf, builder.createTopology());) {
-                Utils.sleep(40000);
-            }
+            topoName = args[0];
         }
+        conf.setNumWorkers(1);
+        StormSubmitter.submitTopologyWithProgressBar(topoName, conf, builder.createTopology());
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/StatefulWindowingTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/StatefulWindowingTopology.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/StatefulWindowingTopology.java
index 0e9b6ea..8cce057 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/StatefulWindowingTopology.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/StatefulWindowingTopology.java
@@ -17,9 +17,9 @@
  */
 package org.apache.storm.starter;
 
+import java.util.Map;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.starter.bolt.PrinterBolt;
 import org.apache.storm.starter.spout.RandomIntegerSpout;
@@ -30,18 +30,14 @@ import org.apache.storm.task.TopologyContext;
 import org.apache.storm.topology.OutputFieldsDeclarer;
 import org.apache.storm.topology.TopologyBuilder;
 import org.apache.storm.topology.base.BaseStatefulWindowedBolt;
+import org.apache.storm.topology.base.BaseWindowedBolt.Count;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Tuple;
 import org.apache.storm.tuple.Values;
-import org.apache.storm.utils.Utils;
 import org.apache.storm.windowing.TupleWindow;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.Map;
-
-import static org.apache.storm.topology.base.BaseWindowedBolt.Count;
-
 /**
  * A simple example that demonstrates the usage of {@link org.apache.storm.topology.IStatefulWindowedBolt} to
  * save the state of the windowing operation to avoid re-computation in case of failures.
@@ -97,15 +93,13 @@ public class StatefulWindowingTopology {
         Config conf = new Config();
         conf.setDebug(false);
         //conf.put(Config.TOPOLOGY_STATE_PROVIDER, "org.apache.storm.redis.state.RedisKeyValueStateProvider");
+        
+        String topoName = "test";
         if (args != null && args.length > 0) {
-            conf.setNumWorkers(1);
-            StormSubmitter.submitTopologyWithProgressBar(args[0], conf, builder.createTopology());
-        } else {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology("test", conf, builder.createTopology());) {
-                Utils.sleep(40000);
-            }
+            topoName = args[0];
         }
+        conf.setNumWorkers(1);
+        StormSubmitter.submitTopologyWithProgressBar(topoName, conf, builder.createTopology());
     }
 
 }


[10/10] storm git commit: STORM-2447: CHANGELOG

Posted by ka...@apache.org.
STORM-2447: CHANGELOG


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

Branch: refs/heads/master
Commit: 78b074caac2410389d41db9d51fd441efc563c7b
Parents: a011de4
Author: Jungtaek Lim <ka...@gmail.com>
Authored: Wed Apr 12 17:28:52 2017 +0900
Committer: Jungtaek Lim <ka...@gmail.com>
Committed: Wed Apr 12 17:28:52 2017 +0900

----------------------------------------------------------------------
 CHANGELOG.md | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/78b074ca/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 1090a36..17c2df0 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
 \ufeff## 2.0.0
+ * STORM-2447: add in storm local to avoid having server on worker classpath
  * STORM-2458: Kafka Spout should allow hooking up custom logic when retries have exhanusted
  * STORM-2461: modify storm-hdfs's READEME.md and update storm-hdfs.md
  * STORM-2454: the default returned value of this method which named "isConsumerAutoCommitMode" in KafkaSpoutConfig.java should be false


[03/10] storm git commit: STORM-2447: add in storm local to avoid having server on worker classpath

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/ThroughputVsLatency.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/ThroughputVsLatency.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/ThroughputVsLatency.java
index d8137b0..f541cb9 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/ThroughputVsLatency.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/ThroughputVsLatency.java
@@ -17,13 +17,28 @@
  */
 package org.apache.storm.starter;
 
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.HdrHistogram.Histogram;
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
 import org.apache.storm.StormSubmitter;
-import org.apache.storm.misc.metric.HttpForwardingMetricsServer;
-import org.apache.storm.metric.api.IMetricsConsumer.TaskInfo;
+import org.apache.storm.generated.ClusterSummary;
+import org.apache.storm.generated.ExecutorSummary;
+import org.apache.storm.generated.KillOptions;
+import org.apache.storm.generated.Nimbus;
+import org.apache.storm.generated.SpoutStats;
+import org.apache.storm.generated.TopologyInfo;
+import org.apache.storm.generated.TopologySummary;
 import org.apache.storm.metric.api.IMetricsConsumer.DataPoint;
-import org.apache.storm.generated.*;
+import org.apache.storm.metric.api.IMetricsConsumer.TaskInfo;
+import org.apache.storm.metrics.hdrhistogram.HistogramMetric;
+import org.apache.storm.misc.metric.HttpForwardingMetricsServer;
 import org.apache.storm.spout.SpoutOutputCollector;
 import org.apache.storm.task.TopologyContext;
 import org.apache.storm.topology.BasicOutputCollector;
@@ -34,395 +49,328 @@ import org.apache.storm.topology.base.BaseRichSpout;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Tuple;
 import org.apache.storm.tuple.Values;
-import org.apache.storm.utils.Utils;
 import org.apache.storm.utils.NimbusClient;
 
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Random;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.storm.metrics.hdrhistogram.HistogramMetric;
-import org.HdrHistogram.Histogram;
-
 /**
  * WordCount but the spout goes at a predefined rate and we collect
  * proper latency statistics.
  */
 public class ThroughputVsLatency {
-  private static class SentWithTime {
-    public final String sentence;
-    public final long time;
+    private static class SentWithTime {
+        public final String sentence;
+        public final long time;
 
-    SentWithTime(String sentence, long time) {
-        this.sentence = sentence;
-        this.time = time;
-    }
-  }
-
-  public static class C {
-    LocalCluster _local = null;
-    Nimbus.Client _client = null;
-
-    public C(Map conf) throws Exception {
-      Map clusterConf = Utils.readStormConfig();
-      if (conf != null) {
-        clusterConf.putAll(conf);
-      }
-      Boolean isLocal = (Boolean)clusterConf.get("run.local");
-      if (isLocal != null && isLocal) {
-        _local = new LocalCluster();
-      } else {
-        _client = NimbusClient.getConfiguredClient(clusterConf).getClient();
-      }
+        SentWithTime(String sentence, long time) {
+            this.sentence = sentence;
+            this.time = time;
+        }
     }
 
-    public ClusterSummary getClusterInfo() throws Exception {
-      if (_local != null) {
-        return _local.getClusterInfo();
-      } else {
-        return _client.getClusterInfo();
-      }
-    }
+    public static class FastRandomSentenceSpout extends BaseRichSpout {
+        static final String[] SENTENCES = new String[]{ "the cow jumped over the moon", "an apple a day keeps the doctor away",
+                "four score and seven years ago", "snow white and the seven dwarfs", "i am at two with nature" };
+
+        SpoutOutputCollector _collector;
+        long _periodNano;
+        long _emitAmount;
+        Random _rand;
+        long _nextEmitTime;
+        long _emitsLeft;
+        HistogramMetric _histo;
+
+        public FastRandomSentenceSpout(long ratePerSecond) {
+            if (ratePerSecond > 0) {
+                _periodNano = Math.max(1, 1000000000/ratePerSecond);
+                _emitAmount = Math.max(1, (long)((ratePerSecond / 1000000000.0) * _periodNano));
+            } else {
+                _periodNano = Long.MAX_VALUE - 1;
+                _emitAmount = 1;
+            }
+        }
 
-    public TopologyInfo getTopologyInfo(String id) throws Exception {
-      if (_local != null) {
-        return _local.getTopologyInfo(id);
-      } else {
-        return _client.getTopologyInfo(id);
-      }
-    }
+        @Override
+        public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
+            _collector = collector;
+            _rand = ThreadLocalRandom.current();
+            _nextEmitTime = System.nanoTime();
+            _emitsLeft = _emitAmount;
+            _histo = new HistogramMetric(3600000000000L, 3);
+            context.registerMetric("comp-lat-histo", _histo, 10); //Update every 10 seconds, so we are not too far behind
+        }
 
-    public void killTopologyWithOpts(String name, KillOptions opts) throws Exception {
-      if (_local != null) {
-        _local.killTopologyWithOpts(name, opts);
-      } else {
-        _client.killTopologyWithOpts(name, opts);
-      }
-    }
+        @Override
+        public void nextTuple() {
+            if (_emitsLeft <= 0 && _nextEmitTime <= System.nanoTime()) {
+                _emitsLeft = _emitAmount;
+                _nextEmitTime = _nextEmitTime + _periodNano;
+            }
 
-    public void submitTopology(String name, Map stormConf, StormTopology topology) throws Exception {
-      if (_local != null) {
-        _local.submitTopology(name, stormConf, topology);
-      } else {
-        StormSubmitter.submitTopology(name, stormConf, topology);
-      }
-    }
+            if (_emitsLeft > 0) {
+                String sentence = SENTENCES[_rand.nextInt(SENTENCES.length)];
+                _collector.emit(new Values(sentence), new SentWithTime(sentence, _nextEmitTime - _periodNano));
+                _emitsLeft--;
+            }
+        }
 
-    public boolean isLocal() {
-      return _local != null;
-    }
-  }
-
-  public static class FastRandomSentenceSpout extends BaseRichSpout {
-    static final String[] SENTENCES = new String[]{ "the cow jumped over the moon", "an apple a day keeps the doctor away",
-          "four score and seven years ago", "snow white and the seven dwarfs", "i am at two with nature" };
-
-    SpoutOutputCollector _collector;
-    long _periodNano;
-    long _emitAmount;
-    Random _rand;
-    long _nextEmitTime;
-    long _emitsLeft;
-    HistogramMetric _histo;
-
-    public FastRandomSentenceSpout(long ratePerSecond) {
-        if (ratePerSecond > 0) {
-            _periodNano = Math.max(1, 1000000000/ratePerSecond);
-            _emitAmount = Math.max(1, (long)((ratePerSecond / 1000000000.0) * _periodNano));
-        } else {
-            _periodNano = Long.MAX_VALUE - 1;
-            _emitAmount = 1;
+        @Override
+        public void ack(Object id) {
+            long end = System.nanoTime();
+            SentWithTime st = (SentWithTime)id;
+            _histo.recordValue(end-st.time);
         }
-    }
 
-    @Override
-    public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
-      _collector = collector;
-      _rand = ThreadLocalRandom.current();
-      _nextEmitTime = System.nanoTime();
-      _emitsLeft = _emitAmount;
-      _histo = new HistogramMetric(3600000000000L, 3);
-      context.registerMetric("comp-lat-histo", _histo, 10); //Update every 10 seconds, so we are not too far behind
-    }
+        @Override
+        public void fail(Object id) {
+            SentWithTime st = (SentWithTime)id;
+            _collector.emit(new Values(st.sentence), id);
+        }
 
-    @Override
-    public void nextTuple() {
-      if (_emitsLeft <= 0 && _nextEmitTime <= System.nanoTime()) {
-          _emitsLeft = _emitAmount;
-          _nextEmitTime = _nextEmitTime + _periodNano;
-      }
-
-      if (_emitsLeft > 0) {
-          String sentence = SENTENCES[_rand.nextInt(SENTENCES.length)];
-          _collector.emit(new Values(sentence), new SentWithTime(sentence, _nextEmitTime - _periodNano));
-          _emitsLeft--;
-      }
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("sentence"));
+        }
     }
 
-    @Override
-    public void ack(Object id) {
-      long end = System.nanoTime();
-      SentWithTime st = (SentWithTime)id;
-      _histo.recordValue(end-st.time);
-    }
+    public static class SplitSentence extends BaseBasicBolt {
+        @Override
+        public void execute(Tuple tuple, BasicOutputCollector collector) {
+            String sentence = tuple.getString(0);
+            for (String word: sentence.split("\\s+")) {
+                collector.emit(new Values(word, 1));
+            }
+        }
 
-    @Override
-    public void fail(Object id) {
-      SentWithTime st = (SentWithTime)id;
-      _collector.emit(new Values(st.sentence), id);
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("word", "count"));
+        }
     }
 
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer declarer) {
-      declarer.declare(new Fields("sentence"));
-    }
-  }
-
-  public static class SplitSentence extends BaseBasicBolt {
-    @Override
-    public void execute(Tuple tuple, BasicOutputCollector collector) {
-      String sentence = tuple.getString(0);
-      for (String word: sentence.split("\\s+")) {
-          collector.emit(new Values(word, 1));
-      }
-    }
+    public static class WordCount extends BaseBasicBolt {
+        Map<String, Integer> counts = new HashMap<String, Integer>();
 
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer declarer) {
-      declarer.declare(new Fields("word", "count"));
-    }
-  }
-
-  public static class WordCount extends BaseBasicBolt {
-    Map<String, Integer> counts = new HashMap<String, Integer>();
-
-    @Override
-    public void execute(Tuple tuple, BasicOutputCollector collector) {
-      String word = tuple.getString(0);
-      Integer count = counts.get(word);
-      if (count == null)
-        count = 0;
-      count++;
-      counts.put(word, count);
-      collector.emit(new Values(word, count));
-    }
+        @Override
+        public void execute(Tuple tuple, BasicOutputCollector collector) {
+            String word = tuple.getString(0);
+            Integer count = counts.get(word);
+            if (count == null)
+                count = 0;
+            count++;
+            counts.put(word, count);
+            collector.emit(new Values(word, count));
+        }
 
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer declarer) {
-      declarer.declare(new Fields("word", "count"));
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("word", "count"));
+        }
     }
-  }
 
-  private static class MemMeasure {
-    private long _mem = 0;
-    private long _time = 0;
+    private static class MemMeasure {
+        private long _mem = 0;
+        private long _time = 0;
 
-    public synchronized void update(long mem) {
-        _mem = mem;
-        _time = System.currentTimeMillis();
-    }
+        public synchronized void update(long mem) {
+            _mem = mem;
+            _time = System.currentTimeMillis();
+        }
 
-    public synchronized long get() {
-        return isExpired() ? 0l : _mem;
-    }
+        public synchronized long get() {
+            return isExpired() ? 0l : _mem;
+        }
 
-    public synchronized boolean isExpired() {
-        return (System.currentTimeMillis() - _time) >= 20000;
-    }
-  }
-
-  private static final Histogram _histo = new Histogram(3600000000000L, 3);
-  private static final AtomicLong _systemCPU = new AtomicLong(0);
-  private static final AtomicLong _userCPU = new AtomicLong(0);
-  private static final AtomicLong _gcCount = new AtomicLong(0);
-  private static final AtomicLong _gcMs = new AtomicLong(0);
-  private static final ConcurrentHashMap<String, MemMeasure> _memoryBytes = new ConcurrentHashMap<String, MemMeasure>();
-
-  private static long readMemory() {
-    long total = 0;
-    for (MemMeasure mem: _memoryBytes.values()) {
-      total += mem.get();
-    }
-    return total;
-  }
-
-  private static long _prev_acked = 0;
-  private static long _prev_uptime = 0;
-
-  public static void printMetrics(C client, String name) throws Exception {
-    ClusterSummary summary = client.getClusterInfo();
-    String id = null;
-    for (TopologySummary ts: summary.get_topologies()) {
-      if (name.equals(ts.get_name())) {
-        id = ts.get_id();
-      }
+        public synchronized boolean isExpired() {
+            return (System.currentTimeMillis() - _time) >= 20000;
+        }
     }
-    if (id == null) {
-      throw new Exception("Could not find a topology named "+name);
+
+    private static final Histogram _histo = new Histogram(3600000000000L, 3);
+    private static final AtomicLong _systemCPU = new AtomicLong(0);
+    private static final AtomicLong _userCPU = new AtomicLong(0);
+    private static final AtomicLong _gcCount = new AtomicLong(0);
+    private static final AtomicLong _gcMs = new AtomicLong(0);
+    private static final ConcurrentHashMap<String, MemMeasure> _memoryBytes = new ConcurrentHashMap<String, MemMeasure>();
+
+    private static long readMemory() {
+        long total = 0;
+        for (MemMeasure mem: _memoryBytes.values()) {
+            total += mem.get();
+        }
+        return total;
     }
-    TopologyInfo info = client.getTopologyInfo(id);
-    int uptime = info.get_uptime_secs();
-    long acked = 0;
-    long failed = 0;
-    for (ExecutorSummary exec: info.get_executors()) {
-      if ("spout".equals(exec.get_component_id()) && exec.get_stats() != null && exec.get_stats().get_specific() != null) {
-        SpoutStats stats = exec.get_stats().get_specific().get_spout();
-        Map<String, Long> failedMap = stats.get_failed().get(":all-time");
-        Map<String, Long> ackedMap = stats.get_acked().get(":all-time");
-        if (ackedMap != null) {
-          for (String key: ackedMap.keySet()) {
-            if (failedMap != null) {
-              Long tmp = failedMap.get(key);
-              if (tmp != null) {
-                  failed += tmp;
-              }
+
+    private static long _prev_acked = 0;
+    private static long _prev_uptime = 0;
+
+    public static void printMetrics(Nimbus.Iface client, String name) throws Exception {
+        ClusterSummary summary = client.getClusterInfo();
+        String id = null;
+        for (TopologySummary ts: summary.get_topologies()) {
+            if (name.equals(ts.get_name())) {
+                id = ts.get_id();
             }
-            long ackVal = ackedMap.get(key);
-            acked += ackVal;
-          }
         }
-      }
-    }
-    long ackedThisTime = acked - _prev_acked;
-    long thisTime = uptime - _prev_uptime;
-    long nnpct, nnnpct, min, max;
-    double mean, stddev;
-    synchronized(_histo) {
-      nnpct = _histo.getValueAtPercentile(99.0);
-      nnnpct = _histo.getValueAtPercentile(99.9);
-      min = _histo.getMinValue();
-      max = _histo.getMaxValue();
-      mean = _histo.getMean();
-      stddev = _histo.getStdDeviation();
-      _histo.reset();
-    }
-    long user = _userCPU.getAndSet(0);
-    long sys = _systemCPU.getAndSet(0);
-    long gc = _gcMs.getAndSet(0);
-    double memMB = readMemory() / (1024.0 * 1024.0);
-    System.out.printf("uptime: %,4d acked: %,9d acked/sec: %,10.2f failed: %,8d " +
-                      "99%%: %,15d 99.9%%: %,15d min: %,15d max: %,15d mean: %,15.2f " +
-                      "stddev: %,15.2f user: %,10d sys: %,10d gc: %,10d mem: %,10.2f\n",
-                       uptime, ackedThisTime, (((double)ackedThisTime)/thisTime), failed, nnpct, nnnpct,
-                       min, max, mean, stddev, user, sys, gc, memMB);
-    _prev_uptime = uptime;
-    _prev_acked = acked;
-  }
-
-  public static void kill(C client, String name) throws Exception {
-    KillOptions opts = new KillOptions();
-    opts.set_wait_secs(0);
-    client.killTopologyWithOpts(name, opts);
-  }
-
-  public static void main(String[] args) throws Exception {
-    long ratePerSecond = 500;
-    if (args != null && args.length > 0) {
-        ratePerSecond = Long.valueOf(args[0]);
+        if (id == null) {
+            throw new Exception("Could not find a topology named "+name);
+        }
+        TopologyInfo info = client.getTopologyInfo(id);
+        int uptime = info.get_uptime_secs();
+        long acked = 0;
+        long failed = 0;
+        for (ExecutorSummary exec: info.get_executors()) {
+            if ("spout".equals(exec.get_component_id()) && exec.get_stats() != null && exec.get_stats().get_specific() != null) {
+                SpoutStats stats = exec.get_stats().get_specific().get_spout();
+                Map<String, Long> failedMap = stats.get_failed().get(":all-time");
+                Map<String, Long> ackedMap = stats.get_acked().get(":all-time");
+                if (ackedMap != null) {
+                    for (String key: ackedMap.keySet()) {
+                        if (failedMap != null) {
+                            Long tmp = failedMap.get(key);
+                            if (tmp != null) {
+                                failed += tmp;
+                            }
+                        }
+                        long ackVal = ackedMap.get(key);
+                        acked += ackVal;
+                    }
+                }
+            }
+        }
+        long ackedThisTime = acked - _prev_acked;
+        long thisTime = uptime - _prev_uptime;
+        long nnpct, nnnpct, min, max;
+        double mean, stddev;
+        synchronized(_histo) {
+            nnpct = _histo.getValueAtPercentile(99.0);
+            nnnpct = _histo.getValueAtPercentile(99.9);
+            min = _histo.getMinValue();
+            max = _histo.getMaxValue();
+            mean = _histo.getMean();
+            stddev = _histo.getStdDeviation();
+            _histo.reset();
+        }
+        long user = _userCPU.getAndSet(0);
+        long sys = _systemCPU.getAndSet(0);
+        long gc = _gcMs.getAndSet(0);
+        double memMB = readMemory() / (1024.0 * 1024.0);
+        System.out.printf("uptime: %,4d acked: %,9d acked/sec: %,10.2f failed: %,8d " +
+                "99%%: %,15d 99.9%%: %,15d min: %,15d max: %,15d mean: %,15.2f " +
+                "stddev: %,15.2f user: %,10d sys: %,10d gc: %,10d mem: %,10.2f\n",
+                uptime, ackedThisTime, (((double)ackedThisTime)/thisTime), failed, nnpct, nnnpct,
+                min, max, mean, stddev, user, sys, gc, memMB);
+        _prev_uptime = uptime;
+        _prev_acked = acked;
     }
 
-    int parallelism = 4;
-    if (args != null && args.length > 1) {
-        parallelism = Integer.valueOf(args[1]);
+    public static void kill(Nimbus.Iface client, String name) throws Exception {
+        KillOptions opts = new KillOptions();
+        opts.set_wait_secs(0);
+        client.killTopologyWithOpts(name, opts);
     }
 
-    int numMins = 5;
-    if (args != null && args.length > 2) {
-        numMins = Integer.valueOf(args[2]);
-    }
+    public static void main(String[] args) throws Exception {
+        long ratePerSecond = 500;
+        if (args != null && args.length > 0) {
+            ratePerSecond = Long.valueOf(args[0]);
+        }
 
-    String name = "wc-test";
-    if (args != null && args.length > 3) {
-        name = args[3];
-    }
+        int parallelism = 4;
+        if (args != null && args.length > 1) {
+            parallelism = Integer.valueOf(args[1]);
+        }
 
-    Config conf = new Config();
-    HttpForwardingMetricsServer metricServer = new HttpForwardingMetricsServer(conf) {
-        @Override
-        public void handle(TaskInfo taskInfo, Collection<DataPoint> dataPoints) {
-            String worker = taskInfo.srcWorkerHost + ":" + taskInfo.srcWorkerPort;
-            for (DataPoint dp: dataPoints) {
-                if ("comp-lat-histo".equals(dp.name) && dp.value instanceof Histogram) {
-                    synchronized(_histo) {
-                        _histo.add((Histogram)dp.value);
+        int numMins = 5;
+        if (args != null && args.length > 2) {
+            numMins = Integer.valueOf(args[2]);
+        }
+
+        String name = "wc-test";
+        if (args != null && args.length > 3) {
+            name = args[3];
+        }
+
+        Config conf = new Config();
+        HttpForwardingMetricsServer metricServer = new HttpForwardingMetricsServer(conf) {
+            @Override
+            public void handle(TaskInfo taskInfo, Collection<DataPoint> dataPoints) {
+                String worker = taskInfo.srcWorkerHost + ":" + taskInfo.srcWorkerPort;
+                for (DataPoint dp: dataPoints) {
+                    if ("comp-lat-histo".equals(dp.name) && dp.value instanceof Histogram) {
+                        synchronized(_histo) {
+                            _histo.add((Histogram)dp.value);
+                        }
+                    } else if ("CPU".equals(dp.name) && dp.value instanceof Map) {
+                        Map<Object, Object> m = (Map<Object, Object>)dp.value;
+                        Object sys = m.get("sys-ms");
+                        if (sys instanceof Number) {
+                            _systemCPU.getAndAdd(((Number)sys).longValue());
+                        }
+                        Object user = m.get("user-ms");
+                        if (user instanceof Number) {
+                            _userCPU.getAndAdd(((Number)user).longValue());
+                        }
+                    } else if (dp.name.startsWith("GC/") && dp.value instanceof Map) {
+                        Map<Object, Object> m = (Map<Object, Object>)dp.value;
+                        Object count = m.get("count");
+                        if (count instanceof Number) {
+                            _gcCount.getAndAdd(((Number)count).longValue());
+                        }
+                        Object time = m.get("timeMs");
+                        if (time instanceof Number) {
+                            _gcMs.getAndAdd(((Number)time).longValue());
+                        }
+                    } else if (dp.name.startsWith("memory/") && dp.value instanceof Map) {
+                        Map<Object, Object> m = (Map<Object, Object>)dp.value;
+                        Object val = m.get("usedBytes");
+                        if (val instanceof Number) {
+                            MemMeasure mm = _memoryBytes.get(worker);
+                            if (mm == null) {
+                                mm = new MemMeasure();
+                                MemMeasure tmp = _memoryBytes.putIfAbsent(worker, mm);
+                                mm = tmp == null ? mm : tmp; 
+                            }
+                            mm.update(((Number)val).longValue());
+                        }
                     }
-                } else if ("CPU".equals(dp.name) && dp.value instanceof Map) {
-                   Map<Object, Object> m = (Map<Object, Object>)dp.value;
-                   Object sys = m.get("sys-ms");
-                   if (sys instanceof Number) {
-                       _systemCPU.getAndAdd(((Number)sys).longValue());
-                   }
-                   Object user = m.get("user-ms");
-                   if (user instanceof Number) {
-                       _userCPU.getAndAdd(((Number)user).longValue());
-                   }
-                } else if (dp.name.startsWith("GC/") && dp.value instanceof Map) {
-                   Map<Object, Object> m = (Map<Object, Object>)dp.value;
-                   Object count = m.get("count");
-                   if (count instanceof Number) {
-                       _gcCount.getAndAdd(((Number)count).longValue());
-                   }
-                   Object time = m.get("timeMs");
-                   if (time instanceof Number) {
-                       _gcMs.getAndAdd(((Number)time).longValue());
-                   }
-                } else if (dp.name.startsWith("memory/") && dp.value instanceof Map) {
-                   Map<Object, Object> m = (Map<Object, Object>)dp.value;
-                   Object val = m.get("usedBytes");
-                   if (val instanceof Number) {
-                       MemMeasure mm = _memoryBytes.get(worker);
-                       if (mm == null) {
-                         mm = new MemMeasure();
-                         MemMeasure tmp = _memoryBytes.putIfAbsent(worker, mm);
-                         mm = tmp == null ? mm : tmp; 
-                       }
-                       mm.update(((Number)val).longValue());
-                   }
                 }
             }
+        };
+
+        metricServer.serve();
+        String url = metricServer.getUrl();
+
+        NimbusClient client = NimbusClient.getConfiguredClient(conf);
+        conf.setNumWorkers(parallelism);
+        conf.registerMetricsConsumer(org.apache.storm.metric.LoggingMetricsConsumer.class);
+        conf.registerMetricsConsumer(org.apache.storm.misc.metric.HttpForwardingMetricsConsumer.class, url, 1);
+        Map<String, String> workerMetrics = new HashMap<String, String>();
+        if (!NimbusClient.isLocalOverride()) {
+            //sigar uses JNI and does not work in local mode
+            workerMetrics.put("CPU", "org.apache.storm.metrics.sigar.CPUMetric");
         }
-    };
-
-    metricServer.serve();
-    String url = metricServer.getUrl();
-
-    C cluster = new C(conf);
-    conf.setNumWorkers(parallelism);
-    conf.registerMetricsConsumer(org.apache.storm.metric.LoggingMetricsConsumer.class);
-    conf.registerMetricsConsumer(org.apache.storm.misc.metric.HttpForwardingMetricsConsumer.class, url, 1);
-    Map<String, String> workerMetrics = new HashMap<String, String>();
-    if (!cluster.isLocal()) {
-      //sigar uses JNI and does not work in local mode
-      workerMetrics.put("CPU", "org.apache.storm.metrics.sigar.CPUMetric");
-    }
-    conf.put(Config.TOPOLOGY_WORKER_METRICS, workerMetrics);
-    conf.put(Config.TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS, 10);
-    conf.put(Config.TOPOLOGY_WORKER_GC_CHILDOPTS,
-      "-XX:+UseConcMarkSweepGC -XX:+UseParNewGC -XX:+UseConcMarkSweepGC -XX:NewSize=128m -XX:CMSInitiatingOccupancyFraction=70 -XX:-CMSConcurrentMTEnabled");
-    conf.put(Config.TOPOLOGY_WORKER_CHILDOPTS, "-Xmx2g");
+        conf.put(Config.TOPOLOGY_WORKER_METRICS, workerMetrics);
+        conf.put(Config.TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS, 10);
+        conf.put(Config.TOPOLOGY_WORKER_GC_CHILDOPTS,
+                "-XX:+UseConcMarkSweepGC -XX:+UseParNewGC -XX:+UseConcMarkSweepGC -XX:NewSize=128m -XX:CMSInitiatingOccupancyFraction=70 -XX:-CMSConcurrentMTEnabled");
+        conf.put(Config.TOPOLOGY_WORKER_CHILDOPTS, "-Xmx2g");
 
-    TopologyBuilder builder = new TopologyBuilder();
+        TopologyBuilder builder = new TopologyBuilder();
 
-    int numEach = 4 * parallelism;
-    builder.setSpout("spout", new FastRandomSentenceSpout(ratePerSecond/numEach), numEach);
+        int numEach = 4 * parallelism;
+        builder.setSpout("spout", new FastRandomSentenceSpout(ratePerSecond/numEach), numEach);
 
-    builder.setBolt("split", new SplitSentence(), numEach).shuffleGrouping("spout");
-    builder.setBolt("count", new WordCount(), numEach).fieldsGrouping("split", new Fields("word"));
+        builder.setBolt("split", new SplitSentence(), numEach).shuffleGrouping("spout");
+        builder.setBolt("count", new WordCount(), numEach).fieldsGrouping("split", new Fields("word"));
 
-    try {
-        cluster.submitTopology(name, conf, builder.createTopology());
+        try {
+            StormSubmitter.submitTopology(name, conf, builder.createTopology());
 
-        for (int i = 0; i < numMins * 2; i++) {
-            Thread.sleep(30 * 1000);
-            printMetrics(cluster, name);
+            for (int i = 0; i < numMins * 2; i++) {
+                Thread.sleep(30 * 1000);
+                printMetrics(client.getClient(), name);
+            }
+        } finally {
+            kill(client.getClient(), name);
         }
-    } finally {
-        kill(cluster, name);
     }
-    System.exit(0);
-  }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/TransactionalGlobalCount.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/TransactionalGlobalCount.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/TransactionalGlobalCount.java
index 5a310f6..c1e9555 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/TransactionalGlobalCount.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/TransactionalGlobalCount.java
@@ -17,9 +17,14 @@
  */
 package org.apache.storm.starter;
 
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
+import org.apache.storm.StormSubmitter;
 import org.apache.storm.coordination.BatchOutputCollector;
 import org.apache.storm.task.TopologyContext;
 import org.apache.storm.testing.MemoryTransactionalSpout;
@@ -32,12 +37,7 @@ import org.apache.storm.transactional.TransactionalTopologyBuilder;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Tuple;
 import org.apache.storm.tuple.Values;
-
-import java.math.BigInteger;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
+import org.apache.storm.utils.NimbusClient;
 
 /**
  * This is a basic example of a transactional topology. It keeps a count of the number of tuples seen so far in a
@@ -46,128 +46,129 @@ import java.util.Map;
  * @see <a href="http://storm.apache.org/documentation/Transactional-topologies.html">Transactional topologies</a>
  */
 public class TransactionalGlobalCount {
-  public static final int PARTITION_TAKE_PER_BATCH = 3;
-  public static final Map<Integer, List<List<Object>>> DATA = new HashMap<Integer, List<List<Object>>>() {{
-    put(0, new ArrayList<List<Object>>() {{
-      add(new Values("cat"));
-      add(new Values("dog"));
-      add(new Values("chicken"));
-      add(new Values("cat"));
-      add(new Values("dog"));
-      add(new Values("apple"));
-    }});
-    put(1, new ArrayList<List<Object>>() {{
-      add(new Values("cat"));
-      add(new Values("dog"));
-      add(new Values("apple"));
-      add(new Values("banana"));
-    }});
-    put(2, new ArrayList<List<Object>>() {{
-      add(new Values("cat"));
-      add(new Values("cat"));
-      add(new Values("cat"));
-      add(new Values("cat"));
-      add(new Values("cat"));
-      add(new Values("dog"));
-      add(new Values("dog"));
-      add(new Values("dog"));
-      add(new Values("dog"));
-    }});
-  }};
-
-  public static class Value {
-    int count = 0;
-    BigInteger txid;
-  }
-
-  public static Map<String, Value> DATABASE = new HashMap<String, Value>();
-  public static final String GLOBAL_COUNT_KEY = "GLOBAL-COUNT";
-
-  public static class BatchCount extends BaseBatchBolt {
-    Object _id;
-    BatchOutputCollector _collector;
-
-    int _count = 0;
-
-    @Override
-    public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, Object id) {
-      _collector = collector;
-      _id = id;
+    public static final int PARTITION_TAKE_PER_BATCH = 3;
+    public static final Map<Integer, List<List<Object>>> DATA = new HashMap<Integer, List<List<Object>>>() {{
+        put(0, new ArrayList<List<Object>>() {{
+            add(new Values("cat"));
+            add(new Values("dog"));
+            add(new Values("chicken"));
+            add(new Values("cat"));
+            add(new Values("dog"));
+            add(new Values("apple"));
+        }});
+        put(1, new ArrayList<List<Object>>() {{
+            add(new Values("cat"));
+            add(new Values("dog"));
+            add(new Values("apple"));
+            add(new Values("banana"));
+        }});
+        put(2, new ArrayList<List<Object>>() {{
+            add(new Values("cat"));
+            add(new Values("cat"));
+            add(new Values("cat"));
+            add(new Values("cat"));
+            add(new Values("cat"));
+            add(new Values("dog"));
+            add(new Values("dog"));
+            add(new Values("dog"));
+            add(new Values("dog"));
+        }});
+    }};
+
+    public static class Value {
+        int count = 0;
+        BigInteger txid;
     }
 
-    @Override
-    public void execute(Tuple tuple) {
-      _count++;
-    }
+    public static Map<String, Value> DATABASE = new HashMap<String, Value>();
+    public static final String GLOBAL_COUNT_KEY = "GLOBAL-COUNT";
 
-    @Override
-    public void finishBatch() {
-      _collector.emit(new Values(_id, _count));
-    }
+    public static class BatchCount extends BaseBatchBolt {
+        Object _id;
+        BatchOutputCollector _collector;
 
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer declarer) {
-      declarer.declare(new Fields("id", "count"));
-    }
-  }
+        int _count = 0;
 
-  public static class UpdateGlobalCount extends BaseTransactionalBolt implements ICommitter {
-    TransactionAttempt _attempt;
-    BatchOutputCollector _collector;
+        @Override
+        public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, Object id) {
+            _collector = collector;
+            _id = id;
+        }
 
-    int _sum = 0;
+        @Override
+        public void execute(Tuple tuple) {
+            _count++;
+        }
 
-    @Override
-    public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, TransactionAttempt attempt) {
-      _collector = collector;
-      _attempt = attempt;
-    }
+        @Override
+        public void finishBatch() {
+            _collector.emit(new Values(_id, _count));
+        }
 
-    @Override
-    public void execute(Tuple tuple) {
-      _sum += tuple.getInteger(1);
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("id", "count"));
+        }
     }
 
-    @Override
-    public void finishBatch() {
-      Value val = DATABASE.get(GLOBAL_COUNT_KEY);
-      Value newval;
-      if (val == null || !val.txid.equals(_attempt.getTransactionId())) {
-        newval = new Value();
-        newval.txid = _attempt.getTransactionId();
-        if (val == null) {
-          newval.count = _sum;
+    public static class UpdateGlobalCount extends BaseTransactionalBolt implements ICommitter {
+        TransactionAttempt _attempt;
+        BatchOutputCollector _collector;
+
+        int _sum = 0;
+
+        @Override
+        public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, TransactionAttempt attempt) {
+            _collector = collector;
+            _attempt = attempt;
         }
-        else {
-          newval.count = _sum + val.count;
+
+        @Override
+        public void execute(Tuple tuple) {
+            _sum += tuple.getInteger(1);
         }
-        DATABASE.put(GLOBAL_COUNT_KEY, newval);
-      }
-      else {
-        newval = val;
-      }
-      _collector.emit(new Values(_attempt, newval.count));
-    }
 
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer declarer) {
-      declarer.declare(new Fields("id", "sum"));
+        @Override
+        public void finishBatch() {
+            Value val = DATABASE.get(GLOBAL_COUNT_KEY);
+            Value newval;
+            if (val == null || !val.txid.equals(_attempt.getTransactionId())) {
+                newval = new Value();
+                newval.txid = _attempt.getTransactionId();
+                if (val == null) {
+                    newval.count = _sum;
+                }
+                else {
+                    newval.count = _sum + val.count;
+                }
+                DATABASE.put(GLOBAL_COUNT_KEY, newval);
+            }
+            else {
+                newval = val;
+            }
+            _collector.emit(new Values(_attempt, newval.count));
+        }
+
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("id", "sum"));
+        }
     }
-  }
-
-  public static void main(String[] args) throws Exception {
-    MemoryTransactionalSpout spout = new MemoryTransactionalSpout(DATA, new Fields("word"), PARTITION_TAKE_PER_BATCH);
-    TransactionalTopologyBuilder builder = new TransactionalTopologyBuilder("global-count", "spout", spout, 3);
-    builder.setBolt("partial-count", new BatchCount(), 5).noneGrouping("spout");
-    builder.setBolt("sum", new UpdateGlobalCount()).globalGrouping("partial-count");
-
-    Config config = new Config();
-    config.setDebug(true);
-    config.setMaxSpoutPending(3);
- 
-    try (LocalCluster cluster = new LocalCluster();
-         LocalTopology topo = cluster.submitTopology("global-count-topology", config, builder.buildTopology());) {
-        Thread.sleep(3000);
+
+    public static void main(String[] args) throws Exception {
+        if (!NimbusClient.isLocalOverride()) {
+            throw new IllegalStateException("This example only works in local mode.  "
+                    + "Run with storm local not storm jar");
+        }
+        MemoryTransactionalSpout spout = new MemoryTransactionalSpout(DATA, new Fields("word"), PARTITION_TAKE_PER_BATCH);
+        TransactionalTopologyBuilder builder = new TransactionalTopologyBuilder("global-count", "spout", spout, 3);
+        builder.setBolt("partial-count", new BatchCount(), 5).noneGrouping("spout");
+        builder.setBolt("sum", new UpdateGlobalCount()).globalGrouping("partial-count");
+
+        Config config = new Config();
+        config.setDebug(true);
+        config.setMaxSpoutPending(3);
+
+        StormSubmitter.submitTopology("global-count-topology", config, builder.buildTopology());
     }
-  }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/TransactionalWords.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/TransactionalWords.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/TransactionalWords.java
index 4965565..e922bab 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/TransactionalWords.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/TransactionalWords.java
@@ -17,9 +17,14 @@
  */
 package org.apache.storm.starter;
 
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
+import org.apache.storm.StormSubmitter;
 import org.apache.storm.coordination.BatchOutputCollector;
 import org.apache.storm.task.TopologyContext;
 import org.apache.storm.testing.MemoryTransactionalSpout;
@@ -33,12 +38,7 @@ import org.apache.storm.transactional.TransactionalTopologyBuilder;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Tuple;
 import org.apache.storm.tuple.Values;
-
-import java.math.BigInteger;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
+import org.apache.storm.utils.NimbusClient;
 
 /**
  * This class defines a more involved transactional topology then TransactionalGlobalCount. This topology processes a
@@ -51,193 +51,194 @@ import java.util.Map;
  * between buckets as their counts accumulate.
  */
 public class TransactionalWords {
-  public static class CountValue {
-    Integer prev_count = null;
-    int count = 0;
-    BigInteger txid = null;
-  }
-
-  public static class BucketValue {
-    int count = 0;
-    BigInteger txid;
-  }
-
-  public static final int BUCKET_SIZE = 10;
-
-  public static Map<String, CountValue> COUNT_DATABASE = new HashMap<String, CountValue>();
-  public static Map<Integer, BucketValue> BUCKET_DATABASE = new HashMap<Integer, BucketValue>();
-
-
-  public static final int PARTITION_TAKE_PER_BATCH = 3;
-
-  public static final Map<Integer, List<List<Object>>> DATA = new HashMap<Integer, List<List<Object>>>() {{
-    put(0, new ArrayList<List<Object>>() {{
-      add(new Values("cat"));
-      add(new Values("dog"));
-      add(new Values("chicken"));
-      add(new Values("cat"));
-      add(new Values("dog"));
-      add(new Values("apple"));
-    }});
-    put(1, new ArrayList<List<Object>>() {{
-      add(new Values("cat"));
-      add(new Values("dog"));
-      add(new Values("apple"));
-      add(new Values("banana"));
-    }});
-    put(2, new ArrayList<List<Object>>() {{
-      add(new Values("cat"));
-      add(new Values("cat"));
-      add(new Values("cat"));
-      add(new Values("cat"));
-      add(new Values("cat"));
-      add(new Values("dog"));
-      add(new Values("dog"));
-      add(new Values("dog"));
-      add(new Values("dog"));
-    }});
-  }};
-
-  public static class KeyedCountUpdater extends BaseTransactionalBolt implements ICommitter {
-    Map<String, Integer> _counts = new HashMap<String, Integer>();
-    BatchOutputCollector _collector;
-    TransactionAttempt _id;
-
-    int _count = 0;
-
-    @Override
-    public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, TransactionAttempt id) {
-      _collector = collector;
-      _id = id;
+    public static class CountValue {
+        Integer prev_count = null;
+        int count = 0;
+        BigInteger txid = null;
     }
 
-    @Override
-    public void execute(Tuple tuple) {
-      String key = tuple.getString(1);
-      Integer curr = _counts.get(key);
-      if (curr == null)
-        curr = 0;
-      _counts.put(key, curr + 1);
+    public static class BucketValue {
+        int count = 0;
+        BigInteger txid;
     }
 
-    @Override
-    public void finishBatch() {
-      for (String key : _counts.keySet()) {
-        CountValue val = COUNT_DATABASE.get(key);
-        CountValue newVal;
-        if (val == null || !val.txid.equals(_id)) {
-          newVal = new CountValue();
-          newVal.txid = _id.getTransactionId();
-          if (val != null) {
-            newVal.prev_count = val.count;
-            newVal.count = val.count;
-          }
-          newVal.count = newVal.count + _counts.get(key);
-          COUNT_DATABASE.put(key, newVal);
+    public static final int BUCKET_SIZE = 10;
+
+    public static Map<String, CountValue> COUNT_DATABASE = new HashMap<String, CountValue>();
+    public static Map<Integer, BucketValue> BUCKET_DATABASE = new HashMap<Integer, BucketValue>();
+
+
+    public static final int PARTITION_TAKE_PER_BATCH = 3;
+
+    public static final Map<Integer, List<List<Object>>> DATA = new HashMap<Integer, List<List<Object>>>() {{
+        put(0, new ArrayList<List<Object>>() {{
+            add(new Values("cat"));
+            add(new Values("dog"));
+            add(new Values("chicken"));
+            add(new Values("cat"));
+            add(new Values("dog"));
+            add(new Values("apple"));
+        }});
+        put(1, new ArrayList<List<Object>>() {{
+            add(new Values("cat"));
+            add(new Values("dog"));
+            add(new Values("apple"));
+            add(new Values("banana"));
+        }});
+        put(2, new ArrayList<List<Object>>() {{
+            add(new Values("cat"));
+            add(new Values("cat"));
+            add(new Values("cat"));
+            add(new Values("cat"));
+            add(new Values("cat"));
+            add(new Values("dog"));
+            add(new Values("dog"));
+            add(new Values("dog"));
+            add(new Values("dog"));
+        }});
+    }};
+
+    public static class KeyedCountUpdater extends BaseTransactionalBolt implements ICommitter {
+        Map<String, Integer> _counts = new HashMap<String, Integer>();
+        BatchOutputCollector _collector;
+        TransactionAttempt _id;
+
+        int _count = 0;
+
+        @Override
+        public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, TransactionAttempt id) {
+            _collector = collector;
+            _id = id;
         }
-        else {
-          newVal = val;
+
+        @Override
+        public void execute(Tuple tuple) {
+            String key = tuple.getString(1);
+            Integer curr = _counts.get(key);
+            if (curr == null)
+                curr = 0;
+            _counts.put(key, curr + 1);
         }
-        _collector.emit(new Values(_id, key, newVal.count, newVal.prev_count));
-      }
-    }
 
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer declarer) {
-      declarer.declare(new Fields("id", "key", "count", "prev-count"));
-    }
-  }
-
-  public static class Bucketize extends BaseBasicBolt {
-    @Override
-    public void execute(Tuple tuple, BasicOutputCollector collector) {
-      TransactionAttempt attempt = (TransactionAttempt) tuple.getValue(0);
-      int curr = tuple.getInteger(2);
-      Integer prev = tuple.getInteger(3);
-
-      int currBucket = curr / BUCKET_SIZE;
-      Integer prevBucket = null;
-      if (prev != null) {
-        prevBucket = prev / BUCKET_SIZE;
-      }
-
-      if (prevBucket == null) {
-        collector.emit(new Values(attempt, currBucket, 1));
-      }
-      else if (currBucket != prevBucket) {
-        collector.emit(new Values(attempt, currBucket, 1));
-        collector.emit(new Values(attempt, prevBucket, -1));
-      }
+        @Override
+        public void finishBatch() {
+            for (String key : _counts.keySet()) {
+                CountValue val = COUNT_DATABASE.get(key);
+                CountValue newVal;
+                if (val == null || !val.txid.equals(_id)) {
+                    newVal = new CountValue();
+                    newVal.txid = _id.getTransactionId();
+                    if (val != null) {
+                        newVal.prev_count = val.count;
+                        newVal.count = val.count;
+                    }
+                    newVal.count = newVal.count + _counts.get(key);
+                    COUNT_DATABASE.put(key, newVal);
+                }
+                else {
+                    newVal = val;
+                }
+                _collector.emit(new Values(_id, key, newVal.count, newVal.prev_count));
+            }
+        }
+
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("id", "key", "count", "prev-count"));
+        }
     }
 
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer declarer) {
-      declarer.declare(new Fields("attempt", "bucket", "delta"));
+    public static class Bucketize extends BaseBasicBolt {
+        @Override
+        public void execute(Tuple tuple, BasicOutputCollector collector) {
+            TransactionAttempt attempt = (TransactionAttempt) tuple.getValue(0);
+            int curr = tuple.getInteger(2);
+            Integer prev = tuple.getInteger(3);
+
+            int currBucket = curr / BUCKET_SIZE;
+            Integer prevBucket = null;
+            if (prev != null) {
+                prevBucket = prev / BUCKET_SIZE;
+            }
+
+            if (prevBucket == null) {
+                collector.emit(new Values(attempt, currBucket, 1));
+            }
+            else if (currBucket != prevBucket) {
+                collector.emit(new Values(attempt, currBucket, 1));
+                collector.emit(new Values(attempt, prevBucket, -1));
+            }
+        }
+
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("attempt", "bucket", "delta"));
+        }
     }
-  }
 
-  public static class BucketCountUpdater extends BaseTransactionalBolt {
-    Map<Integer, Integer> _accum = new HashMap<Integer, Integer>();
-    BatchOutputCollector _collector;
-    TransactionAttempt _attempt;
+    public static class BucketCountUpdater extends BaseTransactionalBolt {
+        Map<Integer, Integer> _accum = new HashMap<Integer, Integer>();
+        BatchOutputCollector _collector;
+        TransactionAttempt _attempt;
 
-    int _count = 0;
+        int _count = 0;
 
-    @Override
-    public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, TransactionAttempt attempt) {
-      _collector = collector;
-      _attempt = attempt;
-    }
+        @Override
+        public void prepare(Map conf, TopologyContext context, BatchOutputCollector collector, TransactionAttempt attempt) {
+            _collector = collector;
+            _attempt = attempt;
+        }
 
-    @Override
-    public void execute(Tuple tuple) {
-      Integer bucket = tuple.getInteger(1);
-      Integer delta = tuple.getInteger(2);
-      Integer curr = _accum.get(bucket);
-      if (curr == null)
-        curr = 0;
-      _accum.put(bucket, curr + delta);
-    }
+        @Override
+        public void execute(Tuple tuple) {
+            Integer bucket = tuple.getInteger(1);
+            Integer delta = tuple.getInteger(2);
+            Integer curr = _accum.get(bucket);
+            if (curr == null)
+                curr = 0;
+            _accum.put(bucket, curr + delta);
+        }
 
-    @Override
-    public void finishBatch() {
-      for (Integer bucket : _accum.keySet()) {
-        BucketValue currVal = BUCKET_DATABASE.get(bucket);
-        BucketValue newVal;
-        if (currVal == null || !currVal.txid.equals(_attempt.getTransactionId())) {
-          newVal = new BucketValue();
-          newVal.txid = _attempt.getTransactionId();
-          newVal.count = _accum.get(bucket);
-          if (currVal != null)
-            newVal.count += currVal.count;
-          BUCKET_DATABASE.put(bucket, newVal);
+        @Override
+        public void finishBatch() {
+            for (Integer bucket : _accum.keySet()) {
+                BucketValue currVal = BUCKET_DATABASE.get(bucket);
+                BucketValue newVal;
+                if (currVal == null || !currVal.txid.equals(_attempt.getTransactionId())) {
+                    newVal = new BucketValue();
+                    newVal.txid = _attempt.getTransactionId();
+                    newVal.count = _accum.get(bucket);
+                    if (currVal != null)
+                        newVal.count += currVal.count;
+                    BUCKET_DATABASE.put(bucket, newVal);
+                }
+                else {
+                    newVal = currVal;
+                }
+                _collector.emit(new Values(_attempt, bucket, newVal.count));
+            }
         }
-        else {
-          newVal = currVal;
+
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("id", "bucket", "count"));
         }
-        _collector.emit(new Values(_attempt, bucket, newVal.count));
-      }
     }
 
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer declarer) {
-      declarer.declare(new Fields("id", "bucket", "count"));
-    }
-  }
-
-  public static void main(String[] args) throws Exception {
-    MemoryTransactionalSpout spout = new MemoryTransactionalSpout(DATA, new Fields("word"), PARTITION_TAKE_PER_BATCH);
-    TransactionalTopologyBuilder builder = new TransactionalTopologyBuilder("top-n-words", "spout", spout, 2);
-    builder.setBolt("count", new KeyedCountUpdater(), 5).fieldsGrouping("spout", new Fields("word"));
-    builder.setBolt("bucketize", new Bucketize()).noneGrouping("count");
-    builder.setBolt("buckets", new BucketCountUpdater(), 5).fieldsGrouping("bucketize", new Fields("bucket"));
-    Config config = new Config();
-    config.setDebug(true);
-    config.setMaxSpoutPending(3);
- 
-    try (LocalCluster cluster = new LocalCluster();
-         LocalTopology topo = cluster.submitTopology("top-n-topology", config, builder.buildTopology());) {
-      Thread.sleep(3000);
+    public static void main(String[] args) throws Exception {
+        if (!NimbusClient.isLocalOverride()) {
+            throw new IllegalStateException("This example only works in local mode.  "
+                    + "Run with storm local not storm jar");
+        }
+        MemoryTransactionalSpout spout = new MemoryTransactionalSpout(DATA, new Fields("word"), PARTITION_TAKE_PER_BATCH);
+        TransactionalTopologyBuilder builder = new TransactionalTopologyBuilder("top-n-words", "spout", spout, 2);
+        builder.setBolt("count", new KeyedCountUpdater(), 5).fieldsGrouping("spout", new Fields("word"));
+        builder.setBolt("bucketize", new Bucketize()).noneGrouping("count");
+        builder.setBolt("buckets", new BucketCountUpdater(), 5).fieldsGrouping("bucketize", new Fields("bucket"));
+        Config config = new Config();
+        config.setDebug(true);
+        config.setMaxSpoutPending(3);
+
+        StormSubmitter.submitTopology("top-n-topology", config, builder.buildTopology());
     }
-  }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/WordCountTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/WordCountTopology.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/WordCountTopology.java
index 576fcd9..59e3e4b 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/WordCountTopology.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/WordCountTopology.java
@@ -78,7 +78,7 @@ public class WordCountTopology extends ConfigurableTopology {
     ConfigurableTopology.start(new WordCountTopology(), args);
   }
 
-  protected int run(String[] args) {
+  protected int run(String[] args) throws Exception {
 
     TopologyBuilder builder = new TopologyBuilder();
 
@@ -91,17 +91,11 @@ public class WordCountTopology extends ConfigurableTopology {
 
     String topologyName = "word-count";
 
-    if (isLocal) {
-      conf.setMaxTaskParallelism(3);
-      ttl = 10;
-    } else {
-      conf.setNumWorkers(3);
-    }
+    conf.setNumWorkers(3);
 
     if (args != null && args.length > 0) {
       topologyName = args[0];
     }
-
     return submit(topologyName, conf, builder);
   }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/WordCountTopologyNode.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/WordCountTopologyNode.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/WordCountTopologyNode.java
index 45014fc..39f5932 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/WordCountTopologyNode.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/WordCountTopologyNode.java
@@ -17,41 +17,43 @@
  */
 package org.apache.storm.starter;
 
+import java.util.HashMap;
+import java.util.Map;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.spout.ShellSpout;
 import org.apache.storm.task.ShellBolt;
-import org.apache.storm.topology.*;
+import org.apache.storm.topology.BasicOutputCollector;
+import org.apache.storm.topology.IRichBolt;
+import org.apache.storm.topology.IRichSpout;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.TopologyBuilder;
 import org.apache.storm.topology.base.BaseBasicBolt;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Tuple;
 import org.apache.storm.tuple.Values;
 
-import java.util.HashMap;
-import java.util.Map;
-
 /**
  * This topology demonstrates Storm's stream groupings and multilang capabilities.
  */
 public class WordCountTopologyNode {
-  public static class SplitSentence extends ShellBolt implements IRichBolt {
+    public static class SplitSentence extends ShellBolt implements IRichBolt {
 
-    public SplitSentence() {
-      super("node", "splitsentence.js");
-    }
+        public SplitSentence() {
+            super("node", "splitsentence.js");
+        }
 
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer declarer) {
-      declarer.declare(new Fields("word"));
-    }
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("word"));
+        }
 
-    @Override
-    public Map<String, Object> getComponentConfiguration() {
-      return null;
+        @Override
+        public Map<String, Object> getComponentConfiguration() {
+            return null;
+        }
     }
-  }
 
     public static class RandomSentence extends ShellSpout implements IRichSpout {
 
@@ -70,51 +72,42 @@ public class WordCountTopologyNode {
         }
     }
 
-  public static class WordCount extends BaseBasicBolt {
-    Map<String, Integer> counts = new HashMap<String, Integer>();
-
-    @Override
-    public void execute(Tuple tuple, BasicOutputCollector collector) {
-      String word = tuple.getString(0);
-      Integer count = counts.get(word);
-      if (count == null)
-        count = 0;
-      count++;
-      counts.put(word, count);
-      collector.emit(new Values(word, count));
-    }
-
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer declarer) {
-      declarer.declare(new Fields("word", "count"));
-    }
-  }
-
-  public static void main(String[] args) throws Exception {
+    public static class WordCount extends BaseBasicBolt {
+        Map<String, Integer> counts = new HashMap<String, Integer>();
 
-    TopologyBuilder builder = new TopologyBuilder();
-
-    builder.setSpout("spout", new RandomSentence(), 5);
+        @Override
+        public void execute(Tuple tuple, BasicOutputCollector collector) {
+            String word = tuple.getString(0);
+            Integer count = counts.get(word);
+            if (count == null)
+                count = 0;
+            count++;
+            counts.put(word, count);
+            collector.emit(new Values(word, count));
+        }
 
-    builder.setBolt("split", new SplitSentence(), 8).shuffleGrouping("spout");
-    builder.setBolt("count", new WordCount(), 12).fieldsGrouping("split", new Fields("word"));
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("word", "count"));
+        }
+    }
 
-    Config conf = new Config();
-    conf.setDebug(true);
+    public static void main(String[] args) throws Exception {
 
+        TopologyBuilder builder = new TopologyBuilder();
 
-    if (args != null && args.length > 0) {
-      conf.setNumWorkers(3);
+        builder.setSpout("spout", new RandomSentence(), 5);
 
-      StormSubmitter.submitTopologyWithProgressBar(args[0], conf, builder.createTopology());
-    }
-    else {
-      conf.setMaxTaskParallelism(3);
+        builder.setBolt("split", new SplitSentence(), 8).shuffleGrouping("spout");
+        builder.setBolt("count", new WordCount(), 12).fieldsGrouping("split", new Fields("word"));
 
-      try (LocalCluster cluster = new LocalCluster();
-           LocalTopology topo = cluster.submitTopology("word-count", conf, builder.createTopology());) {
-          Thread.sleep(10000);
-      }
+        Config conf = new Config();
+        conf.setDebug(true);
+        String topoName = "word-count";
+        if (args != null && args.length > 0) {
+            topoName = args[0];
+        }
+        conf.setNumWorkers(3);
+        StormSubmitter.submitTopologyWithProgressBar(topoName, conf, builder.createTopology());
     }
-  }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java
index cbc5d45..dae774b 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java
@@ -18,7 +18,6 @@
 package org.apache.storm.starter.streams;
 
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.starter.spout.RandomIntegerSpout;
 import org.apache.storm.streams.Pair;
@@ -27,7 +26,6 @@ import org.apache.storm.streams.operations.CombinerAggregator;
 import org.apache.storm.streams.operations.mappers.ValueMapper;
 import org.apache.storm.streams.windowing.TumblingWindows;
 import org.apache.storm.topology.base.BaseWindowedBolt;
-import org.apache.storm.utils.Utils;
 
 /**
  * An example that illustrates the global aggregate
@@ -48,15 +46,12 @@ public class AggregateExample {
                 .print();
 
         Config config = new Config();
+        String topoName = "AGG_EXAMPLE";
         if (args.length > 0) {
-            config.setNumWorkers(1);
-            StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build());
-        } else {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalCluster.LocalTopology topo = cluster.submitTopology("test", config, builder.build())) {
-                Utils.sleep(60_000);
-            }
+            topoName = args[0];
         }
+        config.setNumWorkers(1);
+        StormSubmitter.submitTopologyWithProgressBar(topoName, config, builder.build());
     }
 
     private static class Avg implements CombinerAggregator<Integer, Pair<Integer, Integer>, Double> {

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/BranchExample.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/BranchExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/BranchExample.java
index 027b432..f8d865f 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/BranchExample.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/BranchExample.java
@@ -18,14 +18,12 @@
 package org.apache.storm.starter.streams;
 
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.starter.spout.RandomIntegerSpout;
 import org.apache.storm.streams.Stream;
 import org.apache.storm.streams.StreamBuilder;
 import org.apache.storm.streams.operations.Predicate;
 import org.apache.storm.streams.operations.mappers.ValueMapper;
-import org.apache.storm.utils.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -57,15 +55,13 @@ public class BranchExample {
         evenAndOdd[1].forEach(x -> LOG.info("ODD > " + x));
 
         Config config = new Config();
+        String topoName = "branchExample";
         if (args.length > 0) {
-            config.setNumWorkers(1);
-            StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build());
-        } else {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalCluster.LocalTopology topo = cluster.submitTopology("test", config, builder.build())) {
-                Utils.sleep(60_000);
-            }
+            topoName = args[0];
         }
+         
+        config.setNumWorkers(1);
+        StormSubmitter.submitTopologyWithProgressBar(topoName, config, builder.build());
     }
 
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/GroupByKeyAndWindowExample.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/GroupByKeyAndWindowExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/GroupByKeyAndWindowExample.java
index dd7e97f..d989433 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/GroupByKeyAndWindowExample.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/GroupByKeyAndWindowExample.java
@@ -17,8 +17,11 @@
  */
 package org.apache.storm.starter.streams;
 
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.spout.SpoutOutputCollector;
 import org.apache.storm.streams.PairStream;
@@ -35,10 +38,6 @@ import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Values;
 import org.apache.storm.utils.Utils;
 
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-
 /**
  * An example that shows the usage of {@link PairStream#groupByKeyAndWindow(Window)}
  * and {@link PairStream#reduceByKeyAndWindow(Reducer, Window)}
@@ -72,15 +71,12 @@ public class GroupByKeyAndWindowExample {
                 .print();
 
         Config config = new Config();
+        String topoName = GroupByKeyAndWindowExample.class.getName();
         if (args.length > 0) {
-            config.setNumWorkers(1);
-            StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build());
-        } else {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalCluster.LocalTopology topo = cluster.submitTopology("test", config, builder.build())) {
-                Utils.sleep(60_000);
-            }
+            topoName = args[0];
         }
+        config.setNumWorkers(1);
+        StormSubmitter.submitTopologyWithProgressBar(topoName, config, builder.build());
     }
 
     private static class StockQuotes extends BaseRichSpout {

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/JoinExample.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/JoinExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/JoinExample.java
index 4aa6253..ae8be3d 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/JoinExample.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/JoinExample.java
@@ -17,8 +17,9 @@
  */
 package org.apache.storm.starter.streams;
 
+import java.util.Map;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.spout.SpoutOutputCollector;
 import org.apache.storm.streams.PairStream;
@@ -29,14 +30,11 @@ import org.apache.storm.streams.windowing.TumblingWindows;
 import org.apache.storm.task.TopologyContext;
 import org.apache.storm.topology.OutputFieldsDeclarer;
 import org.apache.storm.topology.base.BaseRichSpout;
+import org.apache.storm.topology.base.BaseWindowedBolt.Duration;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Values;
 import org.apache.storm.utils.Utils;
 
-import java.util.Map;
-
-import static org.apache.storm.topology.base.BaseWindowedBolt.Duration;
-
 /**
  * An example that demonstrates the usage of {@link PairStream#join(PairStream)} to join
  * multiple streams.
@@ -67,16 +65,12 @@ public class JoinExample {
                 .print();
 
         Config config = new Config();
+        String topoName = JoinExample.class.getName();
         if (args.length > 0) {
-            config.setNumWorkers(1);
-            StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build());
-        } else {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalCluster.LocalTopology topo = cluster.submitTopology("test", config, builder.build())) {
-                Utils.sleep(60_000);
-            }
+            topoName = args[0];
         }
-
+        config.setNumWorkers(1);
+        StormSubmitter.submitTopologyWithProgressBar(topoName, config, builder.build());
     }
 
     private static class NumberSpout extends BaseRichSpout {

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StateQueryExample.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StateQueryExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StateQueryExample.java
index ab6cac3..d7eb16b 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StateQueryExample.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StateQueryExample.java
@@ -17,8 +17,9 @@
  */
 package org.apache.storm.starter.streams;
 
+import java.util.Map;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.spout.SpoutOutputCollector;
 import org.apache.storm.streams.Pair;
@@ -34,8 +35,6 @@ import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Values;
 import org.apache.storm.utils.Utils;
 
-import java.util.Map;
-
 /**
  * An example that uses {@link Stream#stateQuery(StreamState)} to query the state
  * <p>
@@ -79,16 +78,12 @@ public class StateQueryExample {
         Config config = new Config();
         // use redis based state store for persistence
         config.put(Config.TOPOLOGY_STATE_PROVIDER, "org.apache.storm.redis.state.RedisKeyValueStateProvider");
-
+        String topoName = "test";
         if (args.length > 0) {
-            config.setNumWorkers(1);
-            StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build());
-        } else {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalCluster.LocalTopology topo = cluster.submitTopology("test", config, builder.build())) {
-                Utils.sleep(60_000);
-            }
+            topoName = args[0];
         }
+        config.setNumWorkers(1);
+        StormSubmitter.submitTopologyWithProgressBar(topoName, config, builder.build());
     }
 
     private static class QuerySpout extends BaseRichSpout {

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StatefulWordCount.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StatefulWordCount.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StatefulWordCount.java
index ddd318a..aba19f3 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StatefulWordCount.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StatefulWordCount.java
@@ -18,7 +18,6 @@
 package org.apache.storm.starter.streams;
 
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.streams.Pair;
 import org.apache.storm.streams.PairStream;
@@ -28,7 +27,6 @@ import org.apache.storm.streams.operations.mappers.ValueMapper;
 import org.apache.storm.streams.windowing.TumblingWindows;
 import org.apache.storm.testing.TestWordSpout;
 import org.apache.storm.topology.base.BaseWindowedBolt;
-import org.apache.storm.utils.Utils;
 
 /**
  * A stateful word count that uses {@link PairStream#updateStateByKey(StateUpdater)} to
@@ -74,15 +72,11 @@ public class StatefulWordCount {
         Config config = new Config();
         // use redis based state store for persistence
         config.put(Config.TOPOLOGY_STATE_PROVIDER, "org.apache.storm.redis.state.RedisKeyValueStateProvider");
-
+        String topoName = "test";
         if (args.length > 0) {
-            config.setNumWorkers(1);
-            StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build());
-        } else {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalCluster.LocalTopology topo = cluster.submitTopology("test", config, builder.build())) {
-                Utils.sleep(60_000);
-            }
+            topoName = args[0];
         }
+        config.setNumWorkers(1);
+        StormSubmitter.submitTopologyWithProgressBar(topoName, config, builder.build());
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/TypedTupleExample.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/TypedTupleExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/TypedTupleExample.java
index 11e89bf..a74151b 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/TypedTupleExample.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/TypedTupleExample.java
@@ -18,7 +18,6 @@
 package org.apache.storm.starter.streams;
 
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.starter.spout.RandomIntegerSpout;
 import org.apache.storm.streams.Pair;
@@ -28,15 +27,12 @@ import org.apache.storm.streams.StreamBuilder;
 import org.apache.storm.streams.operations.mappers.TupleValueMappers;
 import org.apache.storm.streams.tuple.Tuple3;
 import org.apache.storm.streams.windowing.TumblingWindows;
-import org.apache.storm.utils.Utils;
-
-import static org.apache.storm.topology.base.BaseWindowedBolt.Count;
+import org.apache.storm.topology.base.BaseWindowedBolt.Count;
 
 /**
  * An example that illustrates the usage of typed tuples (TupleN<..>) and {@link TupleValueMappers}.
  */
 public class TypedTupleExample {
-    @SuppressWarnings("unchecked")
     public static void main(String[] args) throws Exception {
         StreamBuilder builder = new StreamBuilder();
         /**
@@ -49,15 +45,12 @@ public class TypedTupleExample {
 
         pairs.window(TumblingWindows.of(Count.of(10))).groupByKey().print();
 
-        Config config = new Config();
+        String topoName = "test";
         if (args.length > 0) {
-            config.setNumWorkers(1);
-            StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build());
-        } else {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalCluster.LocalTopology topo = cluster.submitTopology("test", config, builder.build())) {
-                Utils.sleep(60_000);
-            }
+            topoName = args[0];
         }
+        Config config = new Config();
+        config.setNumWorkers(1);
+        StormSubmitter.submitTopologyWithProgressBar(topoName, config, builder.build());
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WindowedWordCount.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WindowedWordCount.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WindowedWordCount.java
index 0f30b7c..50c8aad 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WindowedWordCount.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WindowedWordCount.java
@@ -17,19 +17,16 @@
  */
 package org.apache.storm.starter.streams;
 
+import java.util.Arrays;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.starter.spout.RandomSentenceSpout;
 import org.apache.storm.streams.Pair;
 import org.apache.storm.streams.StreamBuilder;
 import org.apache.storm.streams.operations.mappers.ValueMapper;
 import org.apache.storm.streams.windowing.TumblingWindows;
-import org.apache.storm.utils.Utils;
-
-import java.util.Arrays;
-
-import static org.apache.storm.topology.base.BaseWindowedBolt.Duration;
+import org.apache.storm.topology.base.BaseWindowedBolt.Duration;
 
 /**
  * A windowed word count example
@@ -66,14 +63,11 @@ public class WindowedWordCount {
                 .print();
 
         Config config = new Config();
+        String topoName = "test";
         if (args.length > 0) {
-            config.setNumWorkers(1);
-            StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build());
-        } else {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalCluster.LocalTopology topo = cluster.submitTopology("test", config, builder.build())) {
-                Utils.sleep(60_000);
-            }
+            topoName = args[0];
         }
+        config.setNumWorkers(1);
+        StormSubmitter.submitTopologyWithProgressBar(topoName, config, builder.build());
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WordCountToBolt.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WordCountToBolt.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WordCountToBolt.java
index 1c0aae1..360f0ad 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WordCountToBolt.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WordCountToBolt.java
@@ -18,7 +18,6 @@
 package org.apache.storm.starter.streams;
 
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.redis.bolt.RedisStoreBolt;
 import org.apache.storm.redis.common.config.JedisPoolConfig;
@@ -30,7 +29,6 @@ import org.apache.storm.streams.operations.mappers.ValueMapper;
 import org.apache.storm.testing.TestWordSpout;
 import org.apache.storm.topology.IRichBolt;
 import org.apache.storm.tuple.ITuple;
-import org.apache.storm.utils.Utils;
 
 /**
  * An example that computes word counts and finally emits the results to an
@@ -67,15 +65,12 @@ public class WordCountToBolt {
                 .to(redisStoreBolt);
 
         Config config = new Config();
+        String topoName = "test";
         if (args.length > 0) {
-            config.setNumWorkers(1);
-            StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build());
-        } else {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalCluster.LocalTopology topo = cluster.submitTopology("test", config, builder.build())) {
-                Utils.sleep(60_000);
-            }
+            topoName = args[0];
         }
+        config.setNumWorkers(1);
+        StormSubmitter.submitTopologyWithProgressBar(topoName, config, builder.build());
     }
 
     // Maps a storm tuple to redis key and value

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentHBaseWindowingStoreTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentHBaseWindowingStoreTopology.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentHBaseWindowingStoreTopology.java
index 70a23b8..fa9274d 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentHBaseWindowingStoreTopology.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentHBaseWindowingStoreTopology.java
@@ -18,9 +18,9 @@
  */
 package org.apache.storm.starter.trident;
 
+import java.util.HashMap;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.generated.StormTopology;
 import org.apache.storm.hbase.trident.windowing.HBaseWindowsStoreFactory;
@@ -35,12 +35,9 @@ import org.apache.storm.trident.windowing.WindowsStoreFactory;
 import org.apache.storm.trident.windowing.config.TumblingCountWindow;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Values;
-import org.apache.storm.utils.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.HashMap;
-
 /**
  * Sample application of trident windowing which uses {@link HBaseWindowsStoreFactory}'s store for storing tuples in window.
  *
@@ -76,17 +73,12 @@ public class TridentHBaseWindowingStoreTopology {
 
         // window-state table should already be created with cf:tuples column
         HBaseWindowsStoreFactory windowStoreFactory = new HBaseWindowsStoreFactory(new HashMap<String, Object>(), "window-state", "cf".getBytes("UTF-8"), "tuples".getBytes("UTF-8"));
-
-        if (args.length == 0) {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology("wordCounterWithWindowing", conf, buildTopology(windowStoreFactory));) {
-                Utils.sleep(120 * 1000);
-            }
-            System.exit(0);
-        } else {
-            conf.setNumWorkers(3);
-            StormSubmitter.submitTopologyWithProgressBar(args[0], conf, buildTopology(windowStoreFactory));
+        String topoName = "wordCounterWithWindowing";
+        if (args.length > 0) {
+            topoName = args[0];
         }
+        conf.setNumWorkers(3);
+        StormSubmitter.submitTopologyWithProgressBar(topoName, conf, buildTopology(windowStoreFactory));
     }
 
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentMapExample.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentMapExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentMapExample.java
index 5ddace8..c5f73ff 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentMapExample.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentMapExample.java
@@ -17,10 +17,10 @@
  */
 package org.apache.storm.starter.trident;
 
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
-import org.apache.storm.LocalDRPC;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.generated.StormTopology;
 import org.apache.storm.trident.TridentState;
@@ -39,9 +39,7 @@ import org.apache.storm.trident.testing.MemoryMapState;
 import org.apache.storm.trident.tuple.TridentTuple;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Values;
-
-import java.util.ArrayList;
-import java.util.List;
+import org.apache.storm.utils.DRPCClient;
 
 /**
  * A simple example that demonstrates the usage of {@link org.apache.storm.trident.Stream#map(MapFunction)} and
@@ -74,7 +72,7 @@ public class TridentMapExample {
         }
     };
 
-    public static StormTopology buildTopology(LocalDRPC drpc) {
+    public static StormTopology buildTopology() {
         FixedBatchSpout spout = new FixedBatchSpout(
                 new Fields("word"), 3, new Values("the cow jumped over the moon"),
                 new Values("the man went to the store and bought some candy"), new Values("four score and seven years ago"),
@@ -96,7 +94,7 @@ public class TridentMapExample {
                 .persistentAggregate(new MemoryMapState.Factory(), new Count(), new Fields("count"))
                 .parallelismHint(16);
 
-        topology.newDRPCStream("words", drpc)
+        topology.newDRPCStream("words")
                 .flatMap(split, new Fields("word"))
                 .groupBy(new Fields("word"))
                 .stateQuery(wordCounts, new Fields("word"), new MapGet(), new Fields("count"))
@@ -108,18 +106,17 @@ public class TridentMapExample {
     public static void main(String[] args) throws Exception {
         Config conf = new Config();
         conf.setMaxSpoutPending(20);
-        if (args.length == 0) {
-            try (LocalDRPC drpc = new LocalDRPC();
-                 LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology("wordCounter", conf, buildTopology(drpc));) {
-                for (int i = 0; i < 100; i++) {
-                    System.out.println("DRPC RESULT: " + drpc.execute("words", "CAT THE DOG JUMPED"));
-                    Thread.sleep(1000);
-                }
+        String topoName = "wordCounter";
+        if (args.length > 0) {
+            topoName = args[0];
+        }
+        conf.setNumWorkers(3);
+        StormSubmitter.submitTopologyWithProgressBar(topoName, conf, buildTopology());
+        try (DRPCClient drpc = DRPCClient.getConfiguredClient(conf)) {
+            for (int i = 0; i < 10; i++) {
+                System.out.println("DRPC RESULT: " + drpc.execute("words", "CAT THE DOG JUMPED"));
+                Thread.sleep(1000);
             }
-        } else {
-            conf.setNumWorkers(3);
-            StormSubmitter.submitTopologyWithProgressBar(args[0], conf, buildTopology(null));
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentMinMaxOfDevicesTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentMinMaxOfDevicesTopology.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentMinMaxOfDevicesTopology.java
index e5a775b..1cc33a9 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentMinMaxOfDevicesTopology.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/trident/TridentMinMaxOfDevicesTopology.java
@@ -17,9 +17,12 @@
  */
 package org.apache.storm.starter.trident;
 
+import java.io.Serializable;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.generated.StormTopology;
 import org.apache.storm.starter.spout.RandomNumberGeneratorSpout;
@@ -30,12 +33,6 @@ import org.apache.storm.trident.testing.FixedBatchSpout;
 import org.apache.storm.trident.tuple.TridentTuple;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Values;
-import org.apache.storm.utils.Utils;
-
-import java.io.Serializable;
-import java.util.Comparator;
-import java.util.List;
-import java.util.concurrent.ThreadLocalRandom;
 
 /**
  * This class demonstrates different usages of
@@ -113,16 +110,8 @@ public class TridentMinMaxOfDevicesTopology {
         StormTopology topology = buildDevicesTopology();
         Config conf = new Config();
         conf.setMaxSpoutPending(20);
-        if (args.length == 0) {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology("devices-topology", conf, topology);) {
-                Utils.sleep(60 * 1000);
-            }
-            System.exit(0);
-        } else {
-            conf.setNumWorkers(3);
-            StormSubmitter.submitTopologyWithProgressBar("devices-topology", conf, topology);
-        }
+        conf.setNumWorkers(3);
+        StormSubmitter.submitTopologyWithProgressBar("devices-topology", conf, topology);
     }
 
     static class SpeedComparator implements Comparator<TridentTuple>, Serializable {


[06/10] storm git commit: STORM-2447: add in storm local to avoid having server on worker classpath

Posted by ka...@apache.org.
STORM-2447: add in storm local to avoid having server on worker classpath


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

Branch: refs/heads/master
Commit: b254ede46a25466749cd48ebd4bcb56dd791ec4a
Parents: 4eb6507
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Thu Apr 6 13:58:41 2017 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Thu Apr 6 13:58:41 2017 -0500

----------------------------------------------------------------------
 bin/storm.py                                    | 143 ++++-
 docs/Clojure-DSL.md                             |   2 +-
 docs/Distributed-RPC.md                         |  27 +-
 docs/Local-mode.md                              |  65 +-
 docs/Tutorial.md                                |  31 +-
 docs/flux.md                                    |  33 +-
 docs/storm-hbase.md                             |  16 +-
 examples/storm-elasticsearch-examples/pom.xml   |   2 +-
 .../elasticsearch/bolt/EsIndexTopology.java     |  26 +-
 .../trident/TridentEsTopology.java              |  23 +-
 examples/storm-hbase-examples/pom.xml           |   2 +-
 .../storm/hbase/topology/LookupWordCount.java   |  27 +-
 .../hbase/topology/PersistentWordCount.java     |  33 +-
 .../storm/hbase/trident/WordCountTrident.java   |  27 +-
 examples/storm-hdfs-examples/pom.xml            |   2 +-
 .../storm/hdfs/bolt/HdfsFileTopology.java       |  52 +-
 .../storm/hdfs/bolt/SequenceFileTopology.java   |  51 +-
 .../storm/hdfs/spout/HdfsSpoutTopology.java     |   6 +-
 .../storm/hdfs/trident/TridentFileTopology.java |  37 +-
 .../hdfs/trident/TridentSequenceTopology.java   |  38 +-
 examples/storm-hive-examples/pom.xml            |   2 +-
 .../storm/hive/bolt/BucketTestHiveTopology.java |  44 +-
 .../apache/storm/hive/bolt/HiveTopology.java    |  30 +-
 .../hive/bolt/HiveTopologyPartitioned.java      |  29 +-
 .../storm/hive/trident/TridentHiveTopology.java |  70 +-
 examples/storm-jdbc-examples/pom.xml            |   2 +-
 .../jdbc/topology/AbstractUserTopology.java     |  31 +-
 examples/storm-jms-examples/pom.xml             |  37 +-
 .../storm/jms/example/ExampleJmsTopology.java   |  23 +-
 examples/storm-kafka-client-examples/pom.xml    |   4 +-
 .../TridentKafkaClientWordCountNamedTopics.java |  70 +-
 examples/storm-kafka-examples/pom.xml           |   2 +-
 .../storm/kafka/trident/DrpcResultsPrinter.java |  18 +-
 .../storm/kafka/trident/LocalSubmitter.java     | 106 ---
 .../trident/TridentKafkaConsumerTopology.java   |  17 +-
 .../kafka/trident/TridentKafkaTopology.java     |  13 +-
 .../kafka/trident/TridentKafkaWordCount.java    |  48 +-
 examples/storm-mongodb-examples/pom.xml         |   2 +-
 .../storm/mongodb/topology/InsertWordCount.java |  25 +-
 .../storm/mongodb/topology/LookupWordCount.java |  18 +-
 .../storm/mongodb/topology/UpdateWordCount.java |  24 +-
 .../storm/mongodb/trident/WordCountTrident.java |  20 +-
 .../mongodb/trident/WordCountTridentMap.java    |  22 +-
 examples/storm-mqtt-examples/pom.xml            |   2 +-
 examples/storm-opentsdb-examples/pom.xml        |   2 +-
 .../opentsdb/SampleOpenTsdbBoltTopology.java    |  27 +-
 .../opentsdb/SampleOpenTsdbTridentTopology.java |  26 +-
 examples/storm-perf/pom.xml                     |   2 +-
 .../perf/ConstSpoutIdBoltNullBoltTopo.java      |  37 +-
 .../storm/perf/ConstSpoutNullBoltTopo.java      |  42 +-
 .../apache/storm/perf/ConstSpoutOnlyTopo.java   |  32 +-
 .../storm/perf/FileReadWordCountTopo.java       |  42 +-
 .../storm/perf/StrGenSpoutHdfsBoltTopo.java     |  45 +-
 .../storm/perf/utils/BasicMetricsCollector.java |  49 +-
 .../org/apache/storm/perf/utils/Helper.java     | 134 ++--
 .../apache/storm/perf/utils/MetricsSample.java  |  15 +-
 examples/storm-pmml-examples/pom.xml            |   4 +-
 .../storm/pmml/JpmmlRunnerTestTopology.java     |  44 +-
 examples/storm-redis-examples/pom.xml           |   2 +-
 .../storm/redis/topology/LookupWordCount.java   |  35 +-
 .../redis/topology/PersistentWordCount.java     |  32 +-
 .../redis/topology/WhitelistWordCount.java      |  25 +-
 .../redis/trident/WordCountTridentRedis.java    |  31 +-
 .../trident/WordCountTridentRedisCluster.java   |  37 +-
 .../WordCountTridentRedisClusterMap.java        |  38 +-
 .../redis/trident/WordCountTridentRedisMap.java |  31 +-
 examples/storm-solr-examples/pom.xml            |   2 +-
 .../storm/solr/topology/SolrTopology.java       |  23 +-
 examples/storm-starter/pom.xml                  |  27 +-
 .../apache/storm/starter/clj/exclamation.clj    |   9 +-
 .../storm/starter/clj/rolling_top_words.clj     |   9 +-
 .../org/apache/storm/starter/clj/word_count.clj |   9 +-
 .../apache/storm/starter/BasicDRPCTopology.java |  61 +-
 .../storm/starter/ExclamationTopology.java      |   6 +-
 .../storm/starter/FastWordCountTopology.java    |   6 +-
 .../storm/starter/InOrderDeliveryTest.java      |   6 +-
 .../apache/storm/starter/JoinBoltExample.java   |  20 +-
 .../org/apache/storm/starter/ManualDRPC.java    |  25 +-
 .../storm/starter/MultipleLoggerTopology.java   |  20 +-
 .../org/apache/storm/starter/ReachTopology.java | 239 ++++---
 .../starter/ResourceAwareExampleTopology.java   |  20 +-
 .../apache/storm/starter/RollingTopWords.java   |   9 -
 .../apache/storm/starter/SingleJoinExample.java |  60 +-
 .../storm/starter/SkewedRollingTopWords.java    |   9 -
 .../storm/starter/SlidingTupleTsTopology.java   |  28 +-
 .../storm/starter/SlidingWindowTopology.java    |  29 +-
 .../apache/storm/starter/StatefulTopology.java  |  19 +-
 .../starter/StatefulWindowingTopology.java      |  22 +-
 .../storm/starter/ThroughputVsLatency.java      | 638 +++++++++----------
 .../storm/starter/TransactionalGlobalCount.java | 235 +++----
 .../storm/starter/TransactionalWords.java       | 355 ++++++-----
 .../apache/storm/starter/WordCountTopology.java |  10 +-
 .../storm/starter/WordCountTopologyNode.java    | 107 ++--
 .../storm/starter/streams/AggregateExample.java |  13 +-
 .../storm/starter/streams/BranchExample.java    |  14 +-
 .../streams/GroupByKeyAndWindowExample.java     |  20 +-
 .../storm/starter/streams/JoinExample.java      |  20 +-
 .../starter/streams/StateQueryExample.java      |  17 +-
 .../starter/streams/StatefulWordCount.java      |  14 +-
 .../starter/streams/TypedTupleExample.java      |  19 +-
 .../starter/streams/WindowedWordCount.java      |  20 +-
 .../storm/starter/streams/WordCountToBolt.java  |  13 +-
 .../TridentHBaseWindowingStoreTopology.java     |  22 +-
 .../starter/trident/TridentMapExample.java      |  35 +-
 .../trident/TridentMinMaxOfDevicesTopology.java |  25 +-
 .../TridentMinMaxOfVehiclesTopology.java        |  25 +-
 .../storm/starter/trident/TridentReach.java     | 190 +++---
 .../TridentWindowingInmemoryStoreTopology.java  |  36 +-
 .../storm/starter/trident/TridentWordCount.java |  82 ++-
 external/storm-eventhubs/pom.xml                |   7 -
 .../storm/eventhubs/samples/EventCount.java     |  39 +-
 .../main/java/org/apache/storm/flux/Flux.java   |  72 +--
 flux/pom.xml                                    |   9 +-
 pom.xml                                         |   3 +-
 .../jvm/org/apache/storm/drpc/DRPCSpout.java    |  10 +-
 .../storm/security/auth/ThriftClient.java       |   6 +-
 .../security/auth/ThriftConnectionType.java     |  29 +-
 .../storm/topology/ConfigurableTopology.java    | 151 +++++
 .../jvm/org/apache/storm/utils/DRPCClient.java  |  78 ++-
 .../org/apache/storm/utils/NimbusClient.java    |  45 +-
 .../src/jvm/org/apache/storm/utils/Utils.java   |   2 +-
 storm-clojure-test/pom.xml                      |  66 ++
 .../src/clj/org/apache/storm/testing.clj        | 270 ++++++++
 storm-clojure/pom.xml                           |  15 +-
 .../src/clj/org/apache/storm/config.clj         |  28 +
 storm-clojure/src/clj/org/apache/storm/log.clj  |  34 +
 .../src/clj/org/apache/storm/testing.clj        | 270 --------
 storm-clojure/src/clj/org/apache/storm/util.clj | 134 ++++
 storm-core/pom.xml                              |   2 +-
 .../jvm/org/apache/storm/command/Activate.java  |   2 +-
 .../org/apache/storm/command/Deactivate.java    |   2 +-
 .../jvm/org/apache/storm/command/GetErrors.java |   2 +-
 .../org/apache/storm/command/KillTopology.java  |   2 +-
 .../apache/storm/command/ListTopologies.java    |   2 +-
 .../jvm/org/apache/storm/command/Monitor.java   |   2 +-
 .../jvm/org/apache/storm/command/Rebalance.java |   2 +-
 .../org/apache/storm/command/SetLogLevel.java   |   2 +-
 .../src/jvm/org/apache/storm/utils/Monitor.java |   6 +-
 .../java/org/apache/storm/LocalCluster.java     | 296 ++++++++-
 .../storm/topology/ConfigurableTopology.java    | 184 ------
 140 files changed, 3089 insertions(+), 3285 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/bin/storm.py
----------------------------------------------------------------------
diff --git a/bin/storm.py b/bin/storm.py
index 41cd8f4..257dcdd 100755
--- a/bin/storm.py
+++ b/bin/storm.py
@@ -128,9 +128,12 @@ def get_jars_full(adir):
             ret.append(os.path.join(adir, f))
     return ret
 
-def get_classpath(extrajars, daemon=True):
+def get_classpath(extrajars, daemon=True, client=False):
     ret = get_jars_full(STORM_DIR)
-    ret.extend(get_jars_full(STORM_LIB_DIR))
+    if client:
+        ret.extend(get_jars_full(STORM_WORKER_LIB_DIR))
+    else :
+        ret.extend(get_jars_full(STORM_LIB_DIR))
     ret.extend(get_jars_full(os.path.join(STORM_DIR, "extlib")))
     if daemon:
         ret.extend(get_jars_full(os.path.join(STORM_DIR, "extlib-daemon")))
@@ -235,7 +238,7 @@ def parse_args(string):
     args = [re.compile(r"'((?:[^'\\]|\\.)*)'").sub('\\1', x) for x in args]
     return [re.compile(r'\\(.)').sub('\\1', x) for x in args]
 
-def exec_storm_class(klass, jvmtype="-server", jvmopts=[], extrajars=[], args=[], fork=False, daemon=True, daemonName=""):
+def exec_storm_class(klass, jvmtype="-server", jvmopts=[], extrajars=[], args=[], fork=False, daemon=True, client=False, daemonName=""):
     global CONFFILE
     storm_log_dir = confvalue("storm.log.dir",[CLUSTER_CONF_DIR])
     if(storm_log_dir == None or storm_log_dir == "null"):
@@ -248,7 +251,7 @@ def exec_storm_class(klass, jvmtype="-server", jvmopts=[], extrajars=[], args=[]
         "-Dstorm.log.dir=" + storm_log_dir,
         "-Djava.library.path=" + confvalue("java.library.path", extrajars, daemon),
         "-Dstorm.conf.file=" + CONFFILE,
-        "-cp", get_classpath(extrajars, daemon),
+        "-cp", get_classpath(extrajars, daemon, client=client),
     ] + jvmopts + [klass] + list(args)
     print("Running: " + " ".join(all_args))
     sys.stdout.flush()
@@ -266,30 +269,7 @@ def exec_storm_class(klass, jvmtype="-server", jvmopts=[], extrajars=[], args=[]
         os.execvp(JAVA_CMD, all_args)
     return exit_code
 
-def jar(jarfile, klass, *args):
-    """Syntax: [storm jar topology-jar-path class ...]
-
-    Runs the main method of class with the specified arguments.
-    The storm jars and configs in ~/.storm are put on the classpath.
-    The process is configured so that StormSubmitter
-    (http://storm.apache.org/releases/current/javadocs/org/apache/storm/StormSubmitter.html)
-    will upload the jar at topology-jar-path when the topology is submitted.
-
-    When you want to ship other jars which is not included to application jar, you can pass them to --jars option with comma-separated string.
-    For example, --jars "your-local-jar.jar,your-local-jar2.jar" will load your-local-jar.jar and your-local-jar2.jar.
-    And when you want to ship maven artifacts and its transitive dependencies, you can pass them to --artifacts with comma-separated string.
-    You can also exclude some dependencies like what you're doing in maven pom.
-    Please add exclusion artifacts with '^' separated string after the artifact.
-    For example, --artifacts "redis.clients:jedis:2.9.0,org.apache.kafka:kafka_2.10:0.8.2.2^org.slf4j:slf4j-log4j12" will load jedis and kafka artifact and all of transitive dependencies but exclude slf4j-log4j12 from kafka.
-
-    When you need to pull the artifacts from other than Maven Central, you can pass remote repositories to --artifactRepositories option with comma-separated string.
-    Repository format is "<name>^<url>". '^' is taken as separator because URL allows various characters.
-    For example, --artifactRepositories "jboss-repository^http://repository.jboss.com/maven2,HDPRepo^http://repo.hortonworks.com/content/groups/public/" will add JBoss and HDP repositories for dependency resolver.
-
-    Complete example of options is here: `./bin/storm jar example/storm-starter/storm-starter-topologies-*.jar org.apache.storm.starter.RollingTopWords blobstore-remote2 remote --jars "./external/storm-redis/storm-redis-1.1.0.jar,./external/storm-kafka/storm-kafka-1.1.0.jar" --artifacts "redis.clients:jedis:2.9.0,org.apache.kafka:kafka_2.10:0.8.2.2^org.slf4j:slf4j-log4j12" --artifactRepositories "jboss-repository^http://repository.jboss.com/maven2,HDPRepo^http://repo.hortonworks.com/content/groups/public/"`
-
-    When you pass jars and/or artifacts options, StormSubmitter will upload them when the topology is submitted, and they will be included to classpath of both the process which runs the class, and also workers for that topology.
-    """
+def run_client_jar(jarfile, klass, args, daemon=False, client=True, extrajvmopts=[]):
     global DEP_JARS_OPTS, DEP_ARTIFACTS_OPTS, DEP_ARTIFACTS_REPOSITORIES_OPTS
 
     local_jars = DEP_JARS_OPTS
@@ -307,9 +287,10 @@ def jar(jarfile, klass, *args):
                 jvmtype="-client",
                 extrajars=extra_jars,
                 args=args,
-                daemon=False,
+                daemon=daemon,
+                client=client,
                 fork=True,
-                jvmopts=JAR_JVM_OPTS + ["-Dstorm.jar=" + tmpjar] +
+                jvmopts=JAR_JVM_OPTS + extrajvmopts + ["-Dstorm.jar=" + tmpjar] +
                         ["-Dstorm.dependency.jars=" + ",".join(local_jars)] +
                         ["-Dstorm.dependency.artifacts=" + json.dumps(artifact_to_file_jars)])
         os.remove(tmpjar)
@@ -324,10 +305,63 @@ def jar(jarfile, klass, *args):
             extrajars=extra_jars,
             args=args,
             daemon=False,
-            jvmopts=JAR_JVM_OPTS + ["-Dstorm.jar=" + jarfile] +
+            jvmopts=JAR_JVM_OPTS + extrajvmopts + ["-Dstorm.jar=" + jarfile] +
                     ["-Dstorm.dependency.jars=" + ",".join(local_jars)] +
                     ["-Dstorm.dependency.artifacts=" + json.dumps(artifact_to_file_jars)])
 
+def local(jarfile, klass, *args):
+    """Syntax: [storm local topology-jar-path class ...]
+
+    Runs the main method of class with the specified arguments but pointing to a local cluster
+    The storm jars and configs in ~/.storm are put on the classpath.
+    The process is configured so that StormSubmitter
+    (http://storm.apache.org/releases/current/javadocs/org/apache/storm/StormSubmitter.html)
+    and others will interact with a local cluster instead of the one configured by default.
+
+    Most options should work just like with the storm jar command.
+
+    local also adds in the option --local-ttl which sets the number of seconds the
+    local cluster will run for before it shuts down.
+
+    --java-debug lets you turn on java debugging and set the parameters passed to -agentlib:jdwp on the JDK
+    --java-debug transport=dt_socket,address=localhost:8000
+    will open up a debugging server on port 8000.
+    """
+    [ttl, debug_args, args] = parse_local_opts(args)
+    extrajvmopts = ["-Dstorm.local.sleeptime=" + ttl]
+    if debug_args != None:
+        extrajvmopts = extrajvmopts + ["-agentlib:jdwp=" + debug_args]
+    run_client_jar(jarfile, "org.apache.storm.LocalCluster", [klass] + list(args), client=False, daemon=False, extrajvmopts=extrajvmopts)
+
+def jar(jarfile, klass, *args):
+    """Syntax: [storm jar topology-jar-path class ...]
+
+    Runs the main method of class with the specified arguments.
+    The storm worker dependencies and configs in ~/.storm are put on the classpath.
+    The process is configured so that StormSubmitter
+    (http://storm.apache.org/releases/current/javadocs/org/apache/storm/StormSubmitter.html)
+    will upload the jar at topology-jar-path when the topology is submitted.
+
+    When you want to ship other jars which is not included to application jar, you can pass them to --jars option with comma-separated string.
+    For example, --jars "your-local-jar.jar,your-local-jar2.jar" will load your-local-jar.jar and your-local-jar2.jar.
+    And when you want to ship maven artifacts and its transitive dependencies, you can pass them to --artifacts with comma-separated string.
+    You can also exclude some dependencies like what you're doing in maven pom.
+    Please add exclusion artifacts with '^' separated string after the artifact.
+    For example, --artifacts "redis.clients:jedis:2.9.0,org.apache.kafka:kafka_2.10:0.8.2.2^org.slf4j:slf4j-log4j12" will load jedis and kafka artifact and all of transitive dependencies but exclude slf4j-log4j12 from kafka.
+
+    When you need to pull the artifacts from other than Maven Central, you can pass remote repositories to --artifactRepositories option with comma-separated string.
+    Repository format is "<name>^<url>". '^' is taken as separator because URL allows various characters.
+    For example, --artifactRepositories "jboss-repository^http://repository.jboss.com/maven2,HDPRepo^http://repo.hortonworks.com/content/groups/public/" will add JBoss and HDP repositories for dependency resolver.
+
+    Complete example of options is here: `./bin/storm jar example/storm-starter/storm-starter-topologies-*.jar org.apache.storm.starter.RollingTopWords blobstore-remote2 remote --jars "./external/storm-redis/storm-redis-1.1.0.jar,./external/storm-kafka/storm-kafka-1.1.0.jar" --artifacts "redis.clients:jedis:2.9.0,org.apache.kafka:kafka_2.10:0.8.2.2^org.slf4j:slf4j-log4j12" --artifactRepositories "jboss-repository^http://repository.jboss.com/maven2,HDPRepo^http://repo.hortonworks.com/content/groups/public/"`
+
+    When you pass jars and/or artifacts options, StormSubmitter will upload them when the topology is submitted, and they will be included to classpath of both the process which runs the class, and also workers for that topology.
+
+    If for some reason you need to have the full storm classpath, not just the one for the worker you may include the command line option `--storm-server-classpath`.  Please be careful because this will add things to the classpath that will not be on the worker classpath and could result in the worker not running.
+    """
+    [server_class_path, args] = parse_jar_opts(args) 
+    run_client_jar(jarfile, klass, list(args), client=not server_class_path, daemon=False)
+
 def sql(sql_file, topology_name):
     """Syntax: [storm sql sql-file topology-name], or [storm sql sql-file --explain] when activating explain mode
 
@@ -802,7 +836,14 @@ def print_classpath():
 
     Prints the classpath used by the storm client when running commands.
     """
-    print(get_classpath([]))
+    print(get_classpath([], client=True))
+
+def print_server_classpath():
+    """Syntax: [storm server_classpath]
+
+    Prints the classpath used by the storm servers when running commands.
+    """
+    print(get_classpath([], daemon=True))
 
 def monitor(*args):
     """Syntax: [storm monitor topology-name [-i interval-secs] [-m component-id] [-s stream-id] [-w [emitted | transferred]]]
@@ -845,9 +886,9 @@ def unknown_command(*args):
     print_usage()
     sys.exit(254)
 
-COMMANDS = {"jar": jar, "kill": kill, "shell": shell, "nimbus": nimbus, "ui": ui, "logviewer": logviewer,
+COMMANDS = {"local": local, "jar": jar, "kill": kill, "shell": shell, "nimbus": nimbus, "ui": ui, "logviewer": logviewer,
             "drpc": drpc, "supervisor": supervisor, "localconfvalue": print_localconfvalue,
-            "remoteconfvalue": print_remoteconfvalue, "repl": repl, "classpath": print_classpath,
+            "remoteconfvalue": print_remoteconfvalue, "repl": repl, "classpath": print_classpath, "server_classpath": print_server_classpath,
             "activate": activate, "deactivate": deactivate, "rebalance": rebalance, "help": print_usage,
             "list": listtopos, "dev-zookeeper": dev_zookeeper, "version": version, "monitor": monitor,
             "upload-credentials": upload_credentials, "pacemaker": pacemaker, "heartbeats": heartbeats, "blobstore": blobstore,
@@ -860,6 +901,40 @@ def parse_config(config_list):
         for config in config_list:
             CONFIG_OPTS.append(config)
 
+def parse_local_opts(args):
+    curr = list(args[:])
+    curr.reverse()
+    ttl = "20"
+    debug_args = None
+    args_list = []
+
+    while len(curr) > 0:
+        token = curr.pop()
+        if token == "--local-ttl":
+            ttl = curr.pop()
+        elif token == "--java-debug":
+            debug_args = curr.pop()
+        else:
+            args_list.append(token)
+
+    return ttl, debug_args, args_list
+
+
+def parse_jar_opts(args):
+    curr = list(args[:])
+    curr.reverse()
+    server_class_path = False
+    args_list = []
+
+    while len(curr) > 0:
+        token = curr.pop()
+        if token == "--storm-server-classpath":
+            server_class_path = True
+        else:
+            args_list.append(token)
+
+    return server_class_path, args_list
+
 def parse_config_opts(args):
     curr = args[:]
     curr.reverse()

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/docs/Clojure-DSL.md
----------------------------------------------------------------------
diff --git a/docs/Clojure-DSL.md b/docs/Clojure-DSL.md
index 1aa3393..56bb54f 100644
--- a/docs/Clojure-DSL.md
+++ b/docs/Clojure-DSL.md
@@ -252,7 +252,7 @@ The following example illustrates how to use this spout in a `spout-spec`:
 
 ### Running topologies in local mode or on a cluster
 
-That's all there is to the Clojure DSL. To submit topologies in remote mode or local mode, just use the `StormSubmitter` or `LocalCluster` classes just like you would from Java.
+That's all there is to the Clojure DSL. To submit topologies in remote mode or local mode, just use the `StormSubmitter` class just like you would from Java.
 
 To create topology configs, it's easiest to use the `org.apache.storm.config` namespace which defines constants for all of the possible configs. The constants are the same as the static constants in the `Config` class, except with dashes instead of underscores. For example, here's a topology config that sets the number of workers to 15 and configures the topology in debug mode:
 

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/docs/Distributed-RPC.md
----------------------------------------------------------------------
diff --git a/docs/Distributed-RPC.md b/docs/Distributed-RPC.md
index b20419a..3547934 100644
--- a/docs/Distributed-RPC.md
+++ b/docs/Distributed-RPC.md
@@ -16,6 +16,13 @@ DRPCClient client = new DRPCClient("drpc-host", 3772);
 String result = client.execute("reach", "http://twitter.com");
 ```
 
+or if you just want to use a preconfigured client you can call.  The exact host will be selected randomly from the configured set of hosts
+
+```java
+DRPCClient client = DRPCClient.getConfiguredClient(conf);
+String result = client.execute("reach", "http://twitter.com");
+```
+
 The distributed RPC workflow looks like this:
 
 ![Tasks in a topology](images/drpc-workflow.png)
@@ -57,23 +64,9 @@ In this example, `ExclaimBolt` simply appends a "!" to the second field of the t
 
 ### Local mode DRPC
 
-DRPC can be run in local mode. Here's how to run the above example in local mode:
-
-```java
-LocalDRPC drpc = new LocalDRPC();
-LocalCluster cluster = new LocalCluster();
-
-cluster.submitTopology("drpc-demo", conf, builder.createLocalTopology(drpc));
-
-System.out.println("Results for 'hello':" + drpc.execute("exclamation", "hello"));
-
-cluster.shutdown();
-drpc.shutdown();
-```
-
-First you create a `LocalDRPC` object. This object simulates a DRPC server in process, just like how `LocalCluster` simulates a Storm cluster in process. Then you create the `LocalCluster` to run the topology in local mode. `LinearDRPCTopologyBuilder` has separate methods for creating local topologies and remote topologies. In local mode the `LocalDRPC` object does not bind to any ports so the topology needs to know about the object to communicate with it. This is why `createLocalTopology` takes in the `LocalDRPC` object as input.
-
-After launching the topology, you can do DRPC invocations using the `execute` method on `LocalDRPC`.
+In the past to use DRPC in local mode it took creating a special LocalDRPC instance.  This can still be used when writing tests for your code, but in the current version of storm when you run in local mode a LocalDRPC
+instance is also created, and any DRPCClient created will link to it instead of the outside world.  This means that any interaction you want to test needs to be a part of the script that launches the topology, just like
+with LocalDRPC.
 
 ### Remote mode DRPC
 

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/docs/Local-mode.md
----------------------------------------------------------------------
diff --git a/docs/Local-mode.md b/docs/Local-mode.md
index e3d9666..a9e3a28 100644
--- a/docs/Local-mode.md
+++ b/docs/Local-mode.md
@@ -3,27 +3,82 @@ title: Local Mode
 layout: documentation
 documentation: true
 ---
-Local mode simulates a Storm cluster in process and is useful for developing and testing topologies. Running topologies in local mode is similar to running topologies [on a cluster](Running-topologies-on-a-production-cluster.html). 
+Local mode simulates a Storm cluster in process and is useful for developing and testing topologies. Running topologies in local mode is similar to running topologies [on a cluster](Running-topologies-on-a-production-cluster.html).
+
+To run a topology in local mode you have two options.  The most common option is to run your topology with `storm local` instead of `storm jar`
+
+This will bring up a local simulated cluster and force all interactions with nimbus to go through the simulated cluster instead of going to a separate process.
+
+If you want to do some automated testing but without actually launching a storm cluster you can use the same classes internally that `storm local` does.
+
+To do this you first need to pull in the dependencies needed to access these classes.  For the java API you should depend on `storm-server` as a `test` dependency.
 
 To create an in-process cluster, simply use the `LocalCluster` class. For example:
 
 ```java
 import org.apache.storm.LocalCluster;
 
-LocalCluster cluster = new LocalCluster();
+...
+
+try (LocalCluster cluster = new LocalCluster()) {
+    //Interact with the cluster...
+}
 ```
 
 You can then submit topologies using the `submitTopology` method on the `LocalCluster` object. Just like the corresponding method on [StormSubmitter](javadocs/org/apache/storm/StormSubmitter.html), `submitTopology` takes a name, a topology configuration, and the topology object. You can then kill a topology using the `killTopology` method which takes the topology name as an argument.
 
-To shutdown a local cluster, simple call:
+The `LocalCluster` is an `AutoCloseable` and will shut down when close is called. 
+
+many of the Nimbus APIs are also available through the LocalCluster.
+
+### DRPC
+
+DRPC can be run in local mode as well. Here's how to run the above example in local mode:
 
 ```java
-cluster.shutdown();
+try (LocalDRPC drpc = new LocalDRPC();
+     LocalCluster cluster = new LocalCluster();
+     LocalTopology topo = cluster.submitTopology("drpc-demo", conf, builder.createLocalTopology(drpc))) {
+
+    System.out.println("Results for 'hello':" + drpc.execute("exclamation", "hello"));
+}
 ```
 
+First you create a `LocalDRPC` object. This object simulates a DRPC server in process, just like how `LocalCluster` simulates a Storm cluster in process. Then you create the `LocalCluster` to run the topology in local mode. `LinearDRPCTopologyBuilder` has separate methods for creating local topologies and remote topologies. In local mode the `LocalDRPC` object does not bind to any ports so the topology needs to know about the object to communicate with it. This is why `createLocalTopology` takes in the `LocalDRPC` object as input.
+
+After launching the topology, you can do DRPC invocations using the `execute` method on `LocalDRPC`.
+
+Because all of the objects used are instances of AutoCloseable when the try blocks scope ends the topology is killed, the cluster is shut down and the drpc server also shuts down.
+
+### Clojure API
+
+Storm also offers a clojure API for testing.
+
+[This blog post](http://www.pixelmachine.org/2011/12/21/Testing-Storm-Topologies-Part-2.html) talk about this, but is a little out of date.  To get this functionality you need to include the `storm-clojure-test` dependency.  This will pull in a lot of storm itself that should not be packaged with your topology, sp please make sure it is a test dependency only,.
+
+### Debugging your topology with an IDE
+
+One of the great use cases for local mode is to be able to walk through the code execution of your bolts and spouts using an IDE.  You can do this on the command line by adding the `--java-debug` option followed by the paramer you would pass to jdwp. This makes it simple to launch the local cluster with `-agentlib:jdwp=` turned on.
+
+When running from within an IDE itself you can modify your code run run withing a call to `LocalCluster.withLocalModeOverride`
+
+```java
+public static void main(final String args[]) {
+    LocalCluster.withLocalModeOverride(() -> originalMain(args), 10);
+}
+```
+
+Or you could also modify the IDE to run "org.apache.storm.LocalCluster" instead of your main class when launching, and pass in the name of the class as an argument to it.  This will also trigger local mode, and is what `storm local` does behind the scenes. 
+
 ### Common configurations for local mode
 
 You can see a full list of configurations [here](javadocs/org/apache/storm/Config.html).
 
 1. **Config.TOPOLOGY_MAX_TASK_PARALLELISM**: This config puts a ceiling on the number of threads spawned for a single component. Oftentimes production topologies have a lot of parallelism (hundreds of threads) which places unreasonable load when trying to test the topology in local mode. This config lets you easy control that parallelism.
-2. **Config.TOPOLOGY_DEBUG**: When this is set to true, Storm will log a message every time a tuple is emitted from any spout or bolt. This is extremely useful for debugging.
+2. **Config.TOPOLOGY_DEBUG**: When this is set to true, Storm will log a message every time a tuple is emitted from any spout or bolt. This is extremely useful for debugging.A
+
+These, like all other configs, can be set on the command line when launching your toplogy with the `-c` flag.  The flag is of the form `-c <conf_name>=<JSON_VALUE>`  so to enable debugging when launching your topology in local mode you could run
+
+```
+storm local topology.jar <MY_MAIN_CLASS> -c topology.debug=true
+``` 

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/docs/Tutorial.md
----------------------------------------------------------------------
diff --git a/docs/Tutorial.md b/docs/Tutorial.md
index 5dad834..f71c209 100644
--- a/docs/Tutorial.md
+++ b/docs/Tutorial.md
@@ -206,36 +206,9 @@ public static class ExclamationBolt extends BaseRichBolt {
 
 Let's see how to run the `ExclamationTopology` in local mode and see that it's working.
 
-Storm has two modes of operation: local mode and distributed mode. In local mode, Storm executes completely in process by simulating worker nodes with threads. Local mode is useful for testing and development of topologies. When you run the topologies in storm-starter, they'll run in local mode and you'll be able to see what messages each component is emitting. You can read more about running topologies in local mode on [Local mode](Local-mode.html).
+Storm has two modes of operation: local mode and distributed mode. In local mode, Storm executes completely in process by simulating worker nodes with threads. Local mode is useful for testing and development of topologies. You can read more about running topologies in local mode on [Local mode](Local-mode.html).
 
-In distributed mode, Storm operates as a cluster of machines. When you submit a topology to the master, you also submit all the code necessary to run the topology. The master will take care of distributing your code and allocating workers to run your topology. If workers go down, the master will reassign them somewhere else. You can read more about running topologies on a cluster on [Running topologies on a production cluster](Running-topologies-on-a-production-cluster.html)]. 
-
-Here's the code that runs `ExclamationTopology` in local mode:
-
-```java
-Config conf = new Config();
-conf.setDebug(true);
-conf.setNumWorkers(2);
-
-LocalCluster cluster = new LocalCluster();
-cluster.submitTopology("test", conf, builder.createTopology());
-Utils.sleep(10000);
-cluster.killTopology("test");
-cluster.shutdown();
-```
-
-First, the code defines an in-process cluster by creating a `LocalCluster` object. Submitting topologies to this virtual cluster is identical to submitting topologies to distributed clusters. It submits a topology to the `LocalCluster` by calling `submitTopology`, which takes as arguments a name for the running topology, a configuration for the topology, and then the topology itself.
-
-The name is used to identify the topology so that you can kill it later on. A topology will run indefinitely until you kill it.
-
-The configuration is used to tune various aspects of the running topology. The two configurations specified here are very common:
-
-1. **TOPOLOGY_WORKERS** (set with `setNumWorkers`) specifies how many _processes_ you want allocated around the cluster to execute the topology. Each component in the topology will execute as many _threads_. The number of threads allocated to a given component is configured through the `setBolt` and `setSpout` methods. Those _threads_ exist within worker _processes_. Each worker _process_ contains within it some number of _threads_ for some number of components. For instance, you may have 300 threads specified across all your components and 50 worker processes specified in your config. Each worker process will execute 6 threads, each of which of could belong to a different component. You tune the performance of Storm topologies by tweaking the parallelism for each component and the number of worker processes those threads should run within.
-2. **TOPOLOGY_DEBUG** (set with `setDebug`), when set to true, tells Storm to log every message every emitted by a component. This is useful in local mode when testing topologies, but you probably want to keep this turned off when running topologies on the cluster.
-
-There's many other configurations you can set for the topology. The various configurations are detailed on [the Javadoc for Config](javadocs/org/apache/storm/Config.html).
-
-To learn about how to set up your development environment so that you can run topologies in local mode (such as in Eclipse), see [Creating a new Storm project](Creating-a-new-Storm-project.html).
+To run a topology in local mode run the command `storm local` instead of `storm jar`.
 
 ## Stream groupings
 

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/docs/flux.md
----------------------------------------------------------------------
diff --git a/docs/flux.md b/docs/flux.md
index 7ebc590..886f55d 100644
--- a/docs/flux.md
+++ b/docs/flux.md
@@ -20,38 +20,7 @@ order to change configuration.
 
 ## About
 Flux is a framework and set of utilities that make defining and deploying Apache Storm topologies less painful and
-deveoper-intensive.
-
-Have you ever found yourself repeating this pattern?:
-
-```java
-
-public static void main(String[] args) throws Exception {
-    // logic to determine if we're running locally or not...
-    // create necessary config options...
-    boolean runLocal = shouldRunLocal();
-    if(runLocal){
-        LocalCluster cluster = new LocalCluster();
-        cluster.submitTopology(name, conf, topology);
-    } else {
-        StormSubmitter.submitTopology(name, conf, topology);
-    }
-}
-```
-
-Wouldn't something like this be easier:
-
-```bash
-storm jar mytopology.jar org.apache.storm.flux.Flux --local config.yaml
-```
-
-or:
-
-```bash
-storm jar mytopology.jar org.apache.storm.flux.Flux --remote config.yaml
-```
-
-Another pain point often mentioned is the fact that the wiring for a Topology graph is often tied up in Java code,
+deveoper-intensive. One of the pain points often mentioned is the fact that the wiring for a Topology graph is often tied up in Java code,
 and that any changes require recompilation and repackaging of the topology jar file. Flux aims to alleviate that
 pain by allowing you to package all your Storm components in a single jar, and use an external text file to define
 the layout and configuration of your topologies.

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/docs/storm-hbase.md
----------------------------------------------------------------------
diff --git a/docs/storm-hbase.md b/docs/storm-hbase.md
index 7f4fb62..7f71346 100644
--- a/docs/storm-hbase.md
+++ b/docs/storm-hbase.md
@@ -223,18 +223,12 @@ public class PersistentWordCount {
         builder.setBolt(COUNT_BOLT, bolt, 1).shuffleGrouping(WORD_SPOUT);
         builder.setBolt(HBASE_BOLT, hbase, 1).fieldsGrouping(COUNT_BOLT, new Fields("word"));
 
-
-        if (args.length == 0) {
-            LocalCluster cluster = new LocalCluster();
-            cluster.submitTopology("test", config, builder.createTopology());
-            Thread.sleep(10000);
-            cluster.killTopology("test");
-            cluster.shutdown();
-            System.exit(0);
-        } else {
-            config.setNumWorkers(3);
-            StormSubmitter.submitTopology(args[0], config, builder.createTopology());
+        String topoName = "test";
+        if (args.length > 0) {
+            topoName = args[0];
         }
+        config.setNumWorkers(3);
+        StormSubmitter.submitTopology(topoName, config, builder.createTopology());
     }
 }
 ```

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-elasticsearch-examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-elasticsearch-examples/pom.xml b/examples/storm-elasticsearch-examples/pom.xml
index b351a2b..64c8229 100644
--- a/examples/storm-elasticsearch-examples/pom.xml
+++ b/examples/storm-elasticsearch-examples/pom.xml
@@ -29,7 +29,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-server</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-elasticsearch-examples/src/main/java/org/apache/storm/elasticsearch/bolt/EsIndexTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-elasticsearch-examples/src/main/java/org/apache/storm/elasticsearch/bolt/EsIndexTopology.java b/examples/storm-elasticsearch-examples/src/main/java/org/apache/storm/elasticsearch/bolt/EsIndexTopology.java
index 0a518cd..3cd2bc8 100644
--- a/examples/storm-elasticsearch-examples/src/main/java/org/apache/storm/elasticsearch/bolt/EsIndexTopology.java
+++ b/examples/storm-elasticsearch-examples/src/main/java/org/apache/storm/elasticsearch/bolt/EsIndexTopology.java
@@ -17,9 +17,16 @@
  */
 package org.apache.storm.elasticsearch.bolt;
 
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
+import org.apache.storm.StormSubmitter;
+import org.apache.storm.elasticsearch.common.EsConfig;
+import org.apache.storm.elasticsearch.common.EsConstants;
+import org.apache.storm.elasticsearch.common.EsTestUtil;
+import org.apache.storm.elasticsearch.common.EsTupleMapper;
 import org.apache.storm.spout.SpoutOutputCollector;
 import org.apache.storm.task.TopologyContext;
 import org.apache.storm.topology.OutputFieldsDeclarer;
@@ -27,14 +34,6 @@ import org.apache.storm.topology.TopologyBuilder;
 import org.apache.storm.topology.base.BaseRichSpout;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Values;
-import org.apache.storm.elasticsearch.common.EsConfig;
-import org.apache.storm.elasticsearch.common.EsConstants;
-import org.apache.storm.elasticsearch.common.EsTestUtil;
-import org.apache.storm.elasticsearch.common.EsTupleMapper;
-
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
 
 public class EsIndexTopology {
 
@@ -54,12 +53,7 @@ public class EsIndexTopology {
 
         EsTestUtil.startEsNode();
         EsTestUtil.waitForSeconds(5);
-
-        try (LocalCluster cluster = new LocalCluster();
-            LocalTopology topo = cluster.submitTopology(TOPOLOGY_NAME, config, builder.createTopology());) {
-            EsTestUtil.waitForSeconds(20);
-        }
-        System.exit(0);
+        StormSubmitter.submitTopology(TOPOLOGY_NAME, config, builder.createTopology());
     }
 
     public static class UserDataSpout extends BaseRichSpout {

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-elasticsearch-examples/src/main/java/org/apache/storm/elasticsearch/trident/TridentEsTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-elasticsearch-examples/src/main/java/org/apache/storm/elasticsearch/trident/TridentEsTopology.java b/examples/storm-elasticsearch-examples/src/main/java/org/apache/storm/elasticsearch/trident/TridentEsTopology.java
index 4293c8c..307a991 100644
--- a/examples/storm-elasticsearch-examples/src/main/java/org/apache/storm/elasticsearch/trident/TridentEsTopology.java
+++ b/examples/storm-elasticsearch-examples/src/main/java/org/apache/storm/elasticsearch/trident/TridentEsTopology.java
@@ -17,24 +17,27 @@
  */
 package org.apache.storm.elasticsearch.trident;
 
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Values;
+import org.apache.storm.StormSubmitter;
 import org.apache.storm.elasticsearch.common.EsConfig;
 import org.apache.storm.elasticsearch.common.EsConstants;
 import org.apache.storm.elasticsearch.common.EsTestUtil;
 import org.apache.storm.elasticsearch.common.EsTupleMapper;
+import org.apache.storm.task.TopologyContext;
 import org.apache.storm.trident.Stream;
 import org.apache.storm.trident.TridentState;
 import org.apache.storm.trident.TridentTopology;
 import org.apache.storm.trident.operation.TridentCollector;
 import org.apache.storm.trident.spout.IBatchSpout;
 import org.apache.storm.trident.state.StateFactory;
-
-import java.util.*;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Values;
 
 public class TridentEsTopology {
 
@@ -56,11 +59,7 @@ public class TridentEsTopology {
         EsTestUtil.startEsNode();
         EsTestUtil.waitForSeconds(5);
 
-        try (LocalCluster cluster = new LocalCluster();
-             LocalTopology topo = cluster.submitTopology(TOPOLOGY_NAME, null, topology.build());) {
-            EsTestUtil.waitForSeconds(20);
-        }
-        System.exit(0);
+        StormSubmitter.submitTopology(TOPOLOGY_NAME, new Config(), topology.build());
     }
 
     public static class FixedBatchSpout implements IBatchSpout {

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-hbase-examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-hbase-examples/pom.xml b/examples/storm-hbase-examples/pom.xml
index 3014486..e9d3282 100644
--- a/examples/storm-hbase-examples/pom.xml
+++ b/examples/storm-hbase-examples/pom.xml
@@ -30,7 +30,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-server</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-hbase-examples/src/main/java/org/apache/storm/hbase/topology/LookupWordCount.java
----------------------------------------------------------------------
diff --git a/examples/storm-hbase-examples/src/main/java/org/apache/storm/hbase/topology/LookupWordCount.java b/examples/storm-hbase-examples/src/main/java/org/apache/storm/hbase/topology/LookupWordCount.java
index afbfafd..a10f34f 100644
--- a/examples/storm-hbase-examples/src/main/java/org/apache/storm/hbase/topology/LookupWordCount.java
+++ b/examples/storm-hbase-examples/src/main/java/org/apache/storm/hbase/topology/LookupWordCount.java
@@ -17,18 +17,16 @@
  */
 package org.apache.storm.hbase.topology;
 
+import java.util.HashMap;
+import java.util.Map;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
-import org.apache.storm.topology.TopologyBuilder;
-import org.apache.storm.tuple.Fields;
 import org.apache.storm.hbase.bolt.HBaseLookupBolt;
 import org.apache.storm.hbase.bolt.mapper.HBaseProjectionCriteria;
 import org.apache.storm.hbase.bolt.mapper.SimpleHBaseMapper;
-
-import java.util.HashMap;
-import java.util.Map;
+import org.apache.storm.topology.TopologyBuilder;
+import org.apache.storm.tuple.Fields;
 
 
 public class LookupWordCount {
@@ -63,17 +61,14 @@ public class LookupWordCount {
         builder.setSpout(WORD_SPOUT, spout, 1);
         builder.setBolt(LOOKUP_BOLT, hBaseLookupBolt, 1).shuffleGrouping(WORD_SPOUT);
         builder.setBolt(TOTAL_COUNT_BOLT, totalBolt, 1).fieldsGrouping(LOOKUP_BOLT, new Fields("columnName"));
-
+        String topoName = "test";
         if (args.length == 1) {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology("test", config, builder.createTopology());) {
-                Thread.sleep(30000);
-            }
-            System.exit(0);
-        } else if (args.length == 2) {
-            StormSubmitter.submitTopology(args[1], config, builder.createTopology());
-        } else{
+            topoName = args[1];
+        } else if (args.length > 1) {
             System.out.println("Usage: LookupWordCount <hbase.rootdir>");
+            return;
         }
+            
+        StormSubmitter.submitTopology(topoName, config, builder.createTopology());
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-hbase-examples/src/main/java/org/apache/storm/hbase/topology/PersistentWordCount.java
----------------------------------------------------------------------
diff --git a/examples/storm-hbase-examples/src/main/java/org/apache/storm/hbase/topology/PersistentWordCount.java b/examples/storm-hbase-examples/src/main/java/org/apache/storm/hbase/topology/PersistentWordCount.java
index ed866e9..700a7cc 100644
--- a/examples/storm-hbase-examples/src/main/java/org/apache/storm/hbase/topology/PersistentWordCount.java
+++ b/examples/storm-hbase-examples/src/main/java/org/apache/storm/hbase/topology/PersistentWordCount.java
@@ -18,19 +18,16 @@
 package org.apache.storm.hbase.topology;
 
 
+import java.util.HashMap;
+import java.util.Map;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
-import org.apache.storm.topology.TopologyBuilder;
-import org.apache.storm.tuple.Fields;
 import org.apache.storm.hbase.bolt.HBaseBolt;
 import org.apache.storm.hbase.bolt.mapper.SimpleHBaseMapper;
 import org.apache.storm.hbase.security.HBaseSecurityUtil;
-
-
-import java.util.HashMap;
-import java.util.Map;
+import org.apache.storm.topology.TopologyBuilder;
+import org.apache.storm.tuple.Fields;
 
 
 public class PersistentWordCount {
@@ -68,24 +65,20 @@ public class PersistentWordCount {
         builder.setBolt(COUNT_BOLT, bolt, 1).shuffleGrouping(WORD_SPOUT);
         builder.setBolt(HBASE_BOLT, hbase, 1).fieldsGrouping(COUNT_BOLT, new Fields("word"));
 
-
-        if (args.length == 1) {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology("test", config, builder.createTopology());) {
-                Thread.sleep(30000);
-            }
-            System.exit(0);
-        } else if (args.length == 2) {
-            StormSubmitter.submitTopology(args[1], config, builder.createTopology());
+        String topoName = "test";
+        if (args.length == 2) {
+            topoName = args[0];
         } else if (args.length == 4) {
             System.out.println("hdfs url: " + args[0] + ", keytab file: " + args[2] + 
-                ", principal name: " + args[3] + ", toplogy name: " + args[1]);
+                    ", principal name: " + args[3] + ", toplogy name: " + args[1]);
             hbConf.put(HBaseSecurityUtil.STORM_KEYTAB_FILE_KEY, args[2]);
             hbConf.put(HBaseSecurityUtil.STORM_USER_NAME_KEY, args[3]);
             config.setNumWorkers(3);
-            StormSubmitter.submitTopology(args[1], config, builder.createTopology());
-        } else {
+            topoName = args[1];
+        } else if (args.length != 1) {
             System.out.println("Usage: PersistentWordCount <hbase.rootdir> [topology name] [keytab file] [principal name]");
+            return;
         }
+        StormSubmitter.submitTopology(topoName, config, builder.createTopology());
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-hbase-examples/src/main/java/org/apache/storm/hbase/trident/WordCountTrident.java
----------------------------------------------------------------------
diff --git a/examples/storm-hbase-examples/src/main/java/org/apache/storm/hbase/trident/WordCountTrident.java b/examples/storm-hbase-examples/src/main/java/org/apache/storm/hbase/trident/WordCountTrident.java
index 62a2005..c81512e 100644
--- a/examples/storm-hbase-examples/src/main/java/org/apache/storm/hbase/trident/WordCountTrident.java
+++ b/examples/storm-hbase-examples/src/main/java/org/apache/storm/hbase/trident/WordCountTrident.java
@@ -17,14 +17,10 @@
  */
 package org.apache.storm.hbase.trident;
 
+import org.apache.hadoop.hbase.client.Durability;
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.generated.StormTopology;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Values;
-import org.apache.hadoop.hbase.client.Durability;
 import org.apache.storm.hbase.bolt.mapper.HBaseProjectionCriteria;
 import org.apache.storm.hbase.bolt.mapper.HBaseValueMapper;
 import org.apache.storm.hbase.topology.WordCountValueMapper;
@@ -39,6 +35,8 @@ import org.apache.storm.trident.TridentState;
 import org.apache.storm.trident.TridentTopology;
 import org.apache.storm.trident.state.StateFactory;
 import org.apache.storm.trident.testing.FixedBatchSpout;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Values;
 
 public class WordCountTrident {
     public static StormTopology buildTopology(String hbaseRoot){
@@ -86,19 +84,16 @@ public class WordCountTrident {
     public static void main(String[] args) throws Exception {
         Config conf = new Config();
         conf.setMaxSpoutPending(5);
-        if (args.length == 1) {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology("wordCounter", conf, buildTopology(args[0]));) {
-                Thread.sleep(60 * 1000);
-            }
-            System.exit(0);
-        }
-        else if(args.length == 2) {
-            conf.setNumWorkers(3);
-            StormSubmitter.submitTopology(args[1], conf, buildTopology(args[0]));
-        } else{
+        String topoName = "wordCounter";
+        
+        if (args.length == 2) {
+            topoName = args[1];
+        } else if (args.length > 2) {
             System.out.println("Usage: TridentFileTopology <hdfs url> [topology name]");
+            return;
         }
+        conf.setNumWorkers(3);
+        StormSubmitter.submitTopology(topoName, conf, buildTopology(args[0]));
     }
 
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-hdfs-examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-hdfs-examples/pom.xml b/examples/storm-hdfs-examples/pom.xml
index 3d63529..34b4ef5 100644
--- a/examples/storm-hdfs-examples/pom.xml
+++ b/examples/storm-hdfs-examples/pom.xml
@@ -30,7 +30,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-server</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/bolt/HdfsFileTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/bolt/HdfsFileTopology.java b/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/bolt/HdfsFileTopology.java
index a5c2fe3..ea3ab94 100644
--- a/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/bolt/HdfsFileTopology.java
+++ b/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/bolt/HdfsFileTopology.java
@@ -17,10 +17,24 @@
  */
 package org.apache.storm.hdfs.bolt;
 
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
+import org.apache.storm.hdfs.bolt.format.DefaultFileNameFormat;
+import org.apache.storm.hdfs.bolt.format.DelimitedRecordFormat;
+import org.apache.storm.hdfs.bolt.format.FileNameFormat;
+import org.apache.storm.hdfs.bolt.format.RecordFormat;
+import org.apache.storm.hdfs.bolt.rotation.FileRotationPolicy;
+import org.apache.storm.hdfs.bolt.rotation.TimedRotationPolicy;
+import org.apache.storm.hdfs.bolt.sync.CountSyncPolicy;
+import org.apache.storm.hdfs.bolt.sync.SyncPolicy;
+import org.apache.storm.hdfs.common.rotation.MoveFileAction;
 import org.apache.storm.spout.SpoutOutputCollector;
 import org.apache.storm.task.OutputCollector;
 import org.apache.storm.task.TopologyContext;
@@ -31,26 +45,8 @@ import org.apache.storm.topology.base.BaseRichSpout;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Tuple;
 import org.apache.storm.tuple.Values;
-import org.apache.storm.hdfs.bolt.format.DefaultFileNameFormat;
-import org.apache.storm.hdfs.bolt.format.DelimitedRecordFormat;
-import org.apache.storm.hdfs.bolt.format.FileNameFormat;
-import org.apache.storm.hdfs.bolt.format.RecordFormat;
-import org.apache.storm.hdfs.bolt.rotation.FileRotationPolicy;
-import org.apache.storm.hdfs.bolt.rotation.FileSizeRotationPolicy;
-import org.apache.storm.hdfs.bolt.rotation.FileSizeRotationPolicy.Units;
-import org.apache.storm.hdfs.bolt.rotation.TimedRotationPolicy;
-import org.apache.storm.hdfs.bolt.sync.CountSyncPolicy;
-import org.apache.storm.hdfs.bolt.sync.SyncPolicy;
-import org.apache.storm.hdfs.common.rotation.MoveFileAction;
 import org.yaml.snakeyaml.Yaml;
 
-import java.io.FileInputStream;
-import java.io.InputStream;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-
 public class HdfsFileTopology {
     static final String SENTENCE_SPOUT_ID = "sentence-spout";
     static final String BOLT_ID = "my-bolt";
@@ -97,18 +93,14 @@ public class HdfsFileTopology {
         // SentenceSpout --> MyBolt
         builder.setBolt(BOLT_ID, bolt, 4)
                 .shuffleGrouping(SENTENCE_SPOUT_ID);
-
-        if (args.length == 2) {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology(TOPOLOGY_NAME, config, builder.createTopology());) {
-                waitForSeconds(120);
-            }
-            System.exit(0);
-        } else if (args.length == 3) {
-            StormSubmitter.submitTopology(args[2], config, builder.createTopology());
-        } else{
+        String topoName = TOPOLOGY_NAME;
+        if (args.length == 3) {
+            topoName = args[2];
+        } else if (args.length > 3) {
             System.out.println("Usage: HdfsFileTopology [hdfs url] [hdfs yaml config file] <topology name>");
+            return;
         }
+        StormSubmitter.submitTopology(topoName, config, builder.createTopology());
     }
 
     public static void waitForSeconds(int seconds) {

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/bolt/SequenceFileTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/bolt/SequenceFileTopology.java b/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/bolt/SequenceFileTopology.java
index abe1ebd..0ef3868 100644
--- a/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/bolt/SequenceFileTopology.java
+++ b/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/bolt/SequenceFileTopology.java
@@ -17,10 +17,25 @@
  */
 package org.apache.storm.hdfs.bolt;
 
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.hadoop.io.SequenceFile;
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
+import org.apache.storm.hdfs.bolt.format.DefaultFileNameFormat;
+import org.apache.storm.hdfs.bolt.format.DefaultSequenceFormat;
+import org.apache.storm.hdfs.bolt.format.FileNameFormat;
+import org.apache.storm.hdfs.bolt.rotation.FileRotationPolicy;
+import org.apache.storm.hdfs.bolt.rotation.FileSizeRotationPolicy;
+import org.apache.storm.hdfs.bolt.rotation.FileSizeRotationPolicy.Units;
+import org.apache.storm.hdfs.bolt.sync.CountSyncPolicy;
+import org.apache.storm.hdfs.bolt.sync.SyncPolicy;
+import org.apache.storm.hdfs.common.rotation.MoveFileAction;
 import org.apache.storm.spout.SpoutOutputCollector;
 import org.apache.storm.task.OutputCollector;
 import org.apache.storm.task.TopologyContext;
@@ -31,24 +46,8 @@ import org.apache.storm.topology.base.BaseRichSpout;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Tuple;
 import org.apache.storm.tuple.Values;
-import org.apache.storm.hdfs.bolt.format.*;
-import org.apache.storm.hdfs.bolt.rotation.FileRotationPolicy;
-import org.apache.storm.hdfs.bolt.rotation.FileSizeRotationPolicy;
-import org.apache.storm.hdfs.bolt.rotation.FileSizeRotationPolicy.Units;
-import org.apache.storm.hdfs.bolt.sync.CountSyncPolicy;
-import org.apache.storm.hdfs.bolt.sync.SyncPolicy;
-import org.apache.storm.hdfs.common.rotation.MoveFileAction;
-
-import org.apache.hadoop.io.SequenceFile;
 import org.yaml.snakeyaml.Yaml;
 
-import java.io.FileInputStream;
-import java.io.InputStream;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-
 public class SequenceFileTopology {
     static final String SENTENCE_SPOUT_ID = "sentence-spout";
     static final String BOLT_ID = "my-bolt";
@@ -97,18 +96,14 @@ public class SequenceFileTopology {
         builder.setBolt(BOLT_ID, bolt, 4)
                 .shuffleGrouping(SENTENCE_SPOUT_ID);
 
-
-        if (args.length == 2) {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology(TOPOLOGY_NAME, config, builder.createTopology());) {
-                waitForSeconds(120);
-            }
-            System.exit(0);
-        } else if(args.length == 3) {
-            StormSubmitter.submitTopology(args[2], config, builder.createTopology());
-        } else{
+        String topoName = TOPOLOGY_NAME;
+        if (args.length == 3) {
+            topoName = args[2];
+        } else if (args.length > 3) {
             System.out.println("Usage: SequenceFileTopology [hdfs url] [hdfs yaml config file] <topology name>");
+            return;
         }
+        StormSubmitter.submitTopology(topoName, config, builder.createTopology());
     }
 
     public static void waitForSeconds(int seconds) {

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/spout/HdfsSpoutTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/spout/HdfsSpoutTopology.java b/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/spout/HdfsSpoutTopology.java
index 4eab557..580cb91 100644
--- a/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/spout/HdfsSpoutTopology.java
+++ b/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/spout/HdfsSpoutTopology.java
@@ -134,7 +134,7 @@ public class HdfsSpoutTopology {
     // 4 - submit topology, wait for a few min and terminate it
     Map clusterConf = Utils.readStormConfig();
     StormSubmitter.submitTopologyWithProgressBar(topologyName, conf, builder.createTopology());
-    Nimbus.Client client = NimbusClient.getConfiguredClient(clusterConf).getClient();
+    Nimbus.Iface client = NimbusClient.getConfiguredClient(clusterConf).getClient();
 
     // 5 - Print metrics every 30 sec, kill topology after 20 min
     for (int i = 0; i < 40; i++) {
@@ -144,13 +144,13 @@ public class HdfsSpoutTopology {
     kill(client, topologyName);
   } // main
 
-  private static void kill(Nimbus.Client client, String topologyName) throws Exception {
+  private static void kill(Nimbus.Iface client, String topologyName) throws Exception {
     KillOptions opts = new KillOptions();
     opts.set_wait_secs(0);
     client.killTopologyWithOpts(topologyName, opts);
   }
 
-  static void printMetrics(Nimbus.Client client, String name) throws Exception {
+  static void printMetrics(Nimbus.Iface client, String name) throws Exception {
     ClusterSummary summary = client.getClusterInfo();
     String id = null;
     for (TopologySummary ts: summary.get_topologies()) {

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/trident/TridentFileTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/trident/TridentFileTopology.java b/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/trident/TridentFileTopology.java
index 1e830a1..af76c00 100644
--- a/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/trident/TridentFileTopology.java
+++ b/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/trident/TridentFileTopology.java
@@ -17,27 +17,27 @@
  */
 package org.apache.storm.hdfs.trident;
 
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.util.Map;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.generated.StormTopology;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Values;
-import org.apache.storm.hdfs.common.rotation.MoveFileAction;
-import org.apache.storm.hdfs.trident.format.*;
+import org.apache.storm.hdfs.trident.format.DefaultFileNameFormat;
+import org.apache.storm.hdfs.trident.format.DelimitedRecordFormat;
+import org.apache.storm.hdfs.trident.format.FileNameFormat;
+import org.apache.storm.hdfs.trident.format.RecordFormat;
 import org.apache.storm.hdfs.trident.rotation.FileRotationPolicy;
 import org.apache.storm.hdfs.trident.rotation.FileSizeRotationPolicy;
 import org.apache.storm.trident.Stream;
 import org.apache.storm.trident.TridentState;
 import org.apache.storm.trident.TridentTopology;
 import org.apache.storm.trident.state.StateFactory;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Values;
 import org.yaml.snakeyaml.Yaml;
 
-import java.io.FileInputStream;
-import java.io.InputStream;
-import java.util.Map;
-
 public class TridentFileTopology {
 
     public static StormTopology buildTopology(String hdfsUrl){
@@ -85,17 +85,14 @@ public class TridentFileTopology {
         Map<String, Object> yamlConf = (Map<String, Object>) yaml.load(in);
         in.close();
         conf.put("hdfs.config", yamlConf);
-
-        if (args.length == 2) {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology("wordCounter", conf, buildTopology(args[0]));) {
-                Thread.sleep(120 * 1000);
-            }
-        } else if(args.length == 3) {
-            conf.setNumWorkers(3);
-            StormSubmitter.submitTopology(args[2], conf, buildTopology(args[0]));
-        } else{
+        String topoName = "wordCounter";
+        if (args.length == 3) {
+            topoName = args[2];
+        } else if (args.length > 3) {
             System.out.println("Usage: TridentFileTopology [hdfs url] [hdfs yaml config file] <topology name>");
+            return;
         }
+        conf.setNumWorkers(3);
+        StormSubmitter.submitTopology(topoName, conf, buildTopology(args[0]));
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/trident/TridentSequenceTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/trident/TridentSequenceTopology.java b/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/trident/TridentSequenceTopology.java
index 3248363..525770b 100644
--- a/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/trident/TridentSequenceTopology.java
+++ b/examples/storm-hdfs-examples/src/main/java/org/apache/storm/hdfs/trident/TridentSequenceTopology.java
@@ -17,27 +17,27 @@
  */
 package org.apache.storm.hdfs.trident;
 
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.util.Map;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.generated.StormTopology;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Values;
 import org.apache.storm.hdfs.common.rotation.MoveFileAction;
-import org.apache.storm.hdfs.trident.format.*;
+import org.apache.storm.hdfs.trident.format.DefaultFileNameFormat;
+import org.apache.storm.hdfs.trident.format.DefaultSequenceFormat;
+import org.apache.storm.hdfs.trident.format.FileNameFormat;
 import org.apache.storm.hdfs.trident.rotation.FileRotationPolicy;
 import org.apache.storm.hdfs.trident.rotation.FileSizeRotationPolicy;
 import org.apache.storm.trident.Stream;
 import org.apache.storm.trident.TridentState;
 import org.apache.storm.trident.TridentTopology;
 import org.apache.storm.trident.state.StateFactory;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Values;
 import org.yaml.snakeyaml.Yaml;
 
-import java.io.FileInputStream;
-import java.io.InputStream;
-import java.util.Map;
-
 public class TridentSequenceTopology {
 
     public static StormTopology buildTopology(String hdfsUrl){
@@ -82,17 +82,15 @@ public class TridentSequenceTopology {
         Map<String, Object> yamlConf = (Map<String, Object>) yaml.load(in);
         in.close();
         conf.put("hdfs.config", yamlConf);
-
-        if (args.length == 2) {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology("wordCounter", conf, buildTopology(args[0]));) {
-                Thread.sleep(120 * 1000);
-            }
-        } else if(args.length == 3) {
-            conf.setNumWorkers(3);
-            StormSubmitter.submitTopology(args[2], conf, buildTopology(args[0]));
-        } else{
-            System.out.println("Usage: TridentSequenceTopology [hdfs url] [hdfs yaml config file] <topology name>");
+        String topoName = "wordCounter";
+        if (args.length == 3) {
+            topoName = args[2];
+        } else if (args.length > 3) {
+            System.out.println("Usage: TridentSequenceTopology <hdfs_config_yaml> [<topology name>]");
+            return;
         }
+
+        conf.setNumWorkers(3);
+        StormSubmitter.submitTopology(topoName, conf, buildTopology(args[0]));
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-hive-examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-hive-examples/pom.xml b/examples/storm-hive-examples/pom.xml
index 0c074e8..ec7a4c1 100644
--- a/examples/storm-hive-examples/pom.xml
+++ b/examples/storm-hive-examples/pom.xml
@@ -30,7 +30,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-server</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-hive-examples/src/main/java/org/apache/storm/hive/bolt/BucketTestHiveTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-hive-examples/src/main/java/org/apache/storm/hive/bolt/BucketTestHiveTopology.java b/examples/storm-hive-examples/src/main/java/org/apache/storm/hive/bolt/BucketTestHiveTopology.java
index e80a118..50ab532 100644
--- a/examples/storm-hive-examples/src/main/java/org/apache/storm/hive/bolt/BucketTestHiveTopology.java
+++ b/examples/storm-hive-examples/src/main/java/org/apache/storm/hive/bolt/BucketTestHiveTopology.java
@@ -18,10 +18,19 @@
 
 package org.apache.storm.hive.bolt;
 
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
+import org.apache.storm.hive.bolt.mapper.DelimitedRecordHiveMapper;
+import org.apache.storm.hive.common.HiveOptions;
 import org.apache.storm.spout.SpoutOutputCollector;
 import org.apache.storm.task.TopologyContext;
 import org.apache.storm.topology.OutputFieldsDeclarer;
@@ -30,18 +39,6 @@ import org.apache.storm.topology.base.BaseRichSpout;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Values;
 
-import org.apache.storm.hive.bolt.mapper.DelimitedRecordHiveMapper;
-import org.apache.storm.hive.common.HiveOptions;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-
 
 public class BucketTestHiveTopology {
     static final String USER_SPOUT_ID = "user-spout";
@@ -94,22 +91,11 @@ public class BucketTestHiveTopology {
         // SentenceSpout --> MyBolt
         builder.setBolt(BOLT_ID, hiveBolt, 14)
                 .shuffleGrouping(USER_SPOUT_ID);
-        if (args.length == 6) {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology(TOPOLOGY_NAME, config, builder.createTopology());) {
-                waitForSeconds(20);
-            }
-            System.exit(0);
-        } else {
-            StormSubmitter.submitTopology(args[7], config, builder.createTopology());
-        }
-    }
-
-    public static void waitForSeconds(int seconds) {
-        try {
-            Thread.sleep(seconds * 1000);
-        } catch (InterruptedException e) {
+        String topoName = TOPOLOGY_NAME;
+        if (args.length > 6) {
+            topoName = args[7];
         }
+        StormSubmitter.submitTopology(args[7], config, builder.createTopology());
     }
 
     public static class UserDataSpout extends BaseRichSpout {

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-hive-examples/src/main/java/org/apache/storm/hive/bolt/HiveTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-hive-examples/src/main/java/org/apache/storm/hive/bolt/HiveTopology.java b/examples/storm-hive-examples/src/main/java/org/apache/storm/hive/bolt/HiveTopology.java
index e69c68f..39c9d5f 100644
--- a/examples/storm-hive-examples/src/main/java/org/apache/storm/hive/bolt/HiveTopology.java
+++ b/examples/storm-hive-examples/src/main/java/org/apache/storm/hive/bolt/HiveTopology.java
@@ -18,10 +18,14 @@
 
 package org.apache.storm.hive.bolt;
 
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
+import org.apache.storm.hive.bolt.mapper.DelimitedRecordHiveMapper;
+import org.apache.storm.hive.common.HiveOptions;
 import org.apache.storm.spout.SpoutOutputCollector;
 import org.apache.storm.task.TopologyContext;
 import org.apache.storm.topology.OutputFieldsDeclarer;
@@ -30,13 +34,6 @@ import org.apache.storm.topology.base.BaseRichSpout;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Values;
 
-import org.apache.storm.hive.bolt.mapper.DelimitedRecordHiveMapper;
-import org.apache.storm.hive.common.HiveOptions;
-
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-
 
 public class HiveTopology {
     static final String USER_SPOUT_ID = "user-spout";
@@ -75,17 +72,12 @@ public class HiveTopology {
         // SentenceSpout --> MyBolt
         builder.setBolt(BOLT_ID, hiveBolt, 1)
                 .shuffleGrouping(USER_SPOUT_ID);
-        if (args.length == 3) {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology(TOPOLOGY_NAME, config, builder.createTopology());) {
-                waitForSeconds(20);
-            }
-            System.exit(0);
-        } else if(args.length >= 4) {
-            StormSubmitter.submitTopology(args[3], config, builder.createTopology());
-        } else {
-            System.out.println("Usage: HiveTopology metastoreURI dbName tableName [topologyNamey] [keytab file] [principal name]");
+        
+        String topoName = TOPOLOGY_NAME;
+        if(args.length >= 4) {
+            topoName = args[3];
         }
+        StormSubmitter.submitTopology(topoName, config, builder.createTopology());
     }
 
     public static void waitForSeconds(int seconds) {

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-hive-examples/src/main/java/org/apache/storm/hive/bolt/HiveTopologyPartitioned.java
----------------------------------------------------------------------
diff --git a/examples/storm-hive-examples/src/main/java/org/apache/storm/hive/bolt/HiveTopologyPartitioned.java b/examples/storm-hive-examples/src/main/java/org/apache/storm/hive/bolt/HiveTopologyPartitioned.java
index f64e626..dc51708 100644
--- a/examples/storm-hive-examples/src/main/java/org/apache/storm/hive/bolt/HiveTopologyPartitioned.java
+++ b/examples/storm-hive-examples/src/main/java/org/apache/storm/hive/bolt/HiveTopologyPartitioned.java
@@ -18,10 +18,14 @@
 
 package org.apache.storm.hive.bolt;
 
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
+import org.apache.storm.hive.bolt.mapper.DelimitedRecordHiveMapper;
+import org.apache.storm.hive.common.HiveOptions;
 import org.apache.storm.spout.SpoutOutputCollector;
 import org.apache.storm.task.TopologyContext;
 import org.apache.storm.topology.OutputFieldsDeclarer;
@@ -31,13 +35,6 @@ import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Values;
 import org.apache.storm.utils.Utils;
 
-import org.apache.storm.hive.bolt.mapper.DelimitedRecordHiveMapper;
-import org.apache.storm.hive.common.HiveOptions;
-
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-
 
 public class HiveTopologyPartitioned {
     static final String USER_SPOUT_ID = "hive-user-spout-partitioned";
@@ -77,17 +74,11 @@ public class HiveTopologyPartitioned {
         // SentenceSpout --> MyBolt
         builder.setBolt(BOLT_ID, hiveBolt, 1)
                 .shuffleGrouping(USER_SPOUT_ID);
-        if (args.length == 3) {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology(TOPOLOGY_NAME, config, builder.createTopology());) {
-                waitForSeconds(20);
-            }
-            System.exit(0);
-        } else if(args.length >= 4) {
-            StormSubmitter.submitTopology(args[3], config, builder.createTopology());
-        } else {
-            System.out.println("Usage: HiveTopologyPartitioned metastoreURI dbName tableName [topologyNamey] [keytab file] [principal name]");
+        String topoName = TOPOLOGY_NAME;
+        if (args.length > 3) {
+            topoName = args[3];
         }
+        StormSubmitter.submitTopology(topoName, config, builder.createTopology());
     }
 
     public static void waitForSeconds(int seconds) {

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-hive-examples/src/main/java/org/apache/storm/hive/trident/TridentHiveTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-hive-examples/src/main/java/org/apache/storm/hive/trident/TridentHiveTopology.java b/examples/storm-hive-examples/src/main/java/org/apache/storm/hive/trident/TridentHiveTopology.java
index 4505561..5204f5b 100644
--- a/examples/storm-hive-examples/src/main/java/org/apache/storm/hive/trident/TridentHiveTopology.java
+++ b/examples/storm-hive-examples/src/main/java/org/apache/storm/hive/trident/TridentHiveTopology.java
@@ -19,30 +19,26 @@
 package org.apache.storm.hive.trident;
 
 
-import org.apache.storm.hive.bolt.mapper.DelimitedRecordHiveMapper;
-import org.apache.storm.hive.common.HiveOptions;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.generated.StormTopology;
+import org.apache.storm.hive.bolt.mapper.DelimitedRecordHiveMapper;
+import org.apache.storm.hive.common.HiveOptions;
 import org.apache.storm.hooks.SubmitterHookException;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Values;
 import org.apache.storm.task.TopologyContext;
-import org.apache.storm.trident.operation.TridentCollector;
-import org.apache.storm.trident.spout.IBatchSpout;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
 import org.apache.storm.trident.Stream;
 import org.apache.storm.trident.TridentState;
 import org.apache.storm.trident.TridentTopology;
+import org.apache.storm.trident.operation.TridentCollector;
+import org.apache.storm.trident.spout.IBatchSpout;
 import org.apache.storm.trident.state.StateFactory;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Values;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -96,31 +92,27 @@ public class TridentHiveTopology {
         String tblName = args[2];
         Config conf = new Config();
         conf.setMaxSpoutPending(5);
-        if(args.length == 3) {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology("tridentHiveTopology", conf, buildTopology(metaStoreURI, dbName, tblName,null,null));) {
-                LOG.info("waiting for 60 seconds");
-                waitForSeconds(60);
-            }
-            System.exit(0);
-        } else if(args.length == 4) {
-            try {
-                StormSubmitter.submitTopology(args[3], conf, buildTopology(metaStoreURI, dbName, tblName,null,null));
-            } catch(SubmitterHookException e) {
-                LOG.warn("Topology is submitted but invoking ISubmitterHook failed", e);
-            } catch (Exception e) {
-                LOG.warn("Failed to submit topology ", e);
-            }
-        } else if (args.length == 6) {
-            try {
-                StormSubmitter.submitTopology(args[3], conf, buildTopology(metaStoreURI, dbName, tblName,args[4],args[5]));
-            } catch(SubmitterHookException e) {
-                LOG.warn("Topology is submitted but invoking ISubmitterHook failed", e);
-            } catch (Exception e) {
-                LOG.warn("Failed to submit topology ", e);
-            }
-        } else {
-            LOG.info("Usage: TridentHiveTopology metastoreURI dbName tableName [topologyNamey]");
+        String topoName = "tridentHiveTopology";
+        String keytab = null;
+        String principal = null;
+        
+        if (args.length > 3) {
+            topoName = args[3];
+        }
+        if (args.length == 6) {
+            keytab = args[4];
+            principal = args[5];
+        } else if (args.length != 3 && args.length != 4) {
+            LOG.info("Usage: TridentHiveTopology metastoreURI dbName tableName [topologyName] [keytab principal]");
+            return;
+        }
+        
+        try {
+            StormSubmitter.submitTopology(args[3], conf, buildTopology(metaStoreURI, dbName, tblName,null,null));
+        } catch(SubmitterHookException e) {
+            LOG.warn("Topology is submitted but invoking ISubmitterHook failed", e);
+        } catch (Exception e) {
+            LOG.warn("Failed to submit topology ", e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-jdbc-examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-jdbc-examples/pom.xml b/examples/storm-jdbc-examples/pom.xml
index ab35c6e..eb3279c 100644
--- a/examples/storm-jdbc-examples/pom.xml
+++ b/examples/storm-jdbc-examples/pom.xml
@@ -30,7 +30,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-server</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-jdbc-examples/src/main/java/org/apache/storm/jdbc/topology/AbstractUserTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-jdbc-examples/src/main/java/org/apache/storm/jdbc/topology/AbstractUserTopology.java b/examples/storm-jdbc-examples/src/main/java/org/apache/storm/jdbc/topology/AbstractUserTopology.java
index 19cdf74..79ca987 100644
--- a/examples/storm-jdbc-examples/src/main/java/org/apache/storm/jdbc/topology/AbstractUserTopology.java
+++ b/examples/storm-jdbc-examples/src/main/java/org/apache/storm/jdbc/topology/AbstractUserTopology.java
@@ -17,27 +17,26 @@
  */
 package org.apache.storm.jdbc.topology;
 
+import java.sql.Types;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.storm.Config;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.generated.StormTopology;
-import org.apache.storm.tuple.Fields;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
 import org.apache.storm.jdbc.common.Column;
 import org.apache.storm.jdbc.common.ConnectionProvider;
 import org.apache.storm.jdbc.common.HikariCPConnectionProvider;
 import org.apache.storm.jdbc.common.JdbcClient;
-import org.apache.storm.jdbc.mapper.JdbcMapper;
 import org.apache.storm.jdbc.mapper.JdbcLookupMapper;
-import org.apache.storm.jdbc.mapper.SimpleJdbcMapper;
+import org.apache.storm.jdbc.mapper.JdbcMapper;
 import org.apache.storm.jdbc.mapper.SimpleJdbcLookupMapper;
+import org.apache.storm.jdbc.mapper.SimpleJdbcMapper;
 import org.apache.storm.jdbc.spout.UserSpout;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
+import org.apache.storm.tuple.Fields;
 
-import java.sql.Types;
-import java.util.List;
-import java.util.Map;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 
 public abstract class AbstractUserTopology {
     private static final List<String> setupSqls = Lists.newArrayList(
@@ -99,15 +98,11 @@ public abstract class AbstractUserTopology {
         List<Column> queryParamColumns = Lists.newArrayList(new Column("user_id", Types.INTEGER));
         this.jdbcLookupMapper = new SimpleJdbcLookupMapper(outputFields, queryParamColumns);
         this.connectionProvider = new HikariCPConnectionProvider(map);
-        if (args.length == 4) {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology("test", config, getTopology());) {
-                Thread.sleep(30000);
-            }
-            System.exit(0);
-        } else {
-            StormSubmitter.submitTopology(args[4], config, getTopology());
+        String topoName = "test";
+        if (args.length > 4) {
+            topoName = args[4];
         }
+        StormSubmitter.submitTopology(topoName, config, getTopology());
     }
 
     public abstract StormTopology getTopology();


[05/10] storm git commit: STORM-2447: add in storm local to avoid having server on worker classpath

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-jms-examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-jms-examples/pom.xml b/examples/storm-jms-examples/pom.xml
index 1e10fde..591cf7d 100644
--- a/examples/storm-jms-examples/pom.xml
+++ b/examples/storm-jms-examples/pom.xml
@@ -60,7 +60,7 @@
         </dependency>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-server</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <!-- keep storm out of the jar-with-dependencies -->
             <scope>${provided.scope}</scope>
@@ -112,39 +112,6 @@
                 </executions>
 
             </plugin>
-
-            <plugin>
-                <groupId>org.codehaus.mojo</groupId>
-                <artifactId>exec-maven-plugin</artifactId>
-                <version>1.2.1</version>
-                <executions>
-                    <execution>
-                        <goals>
-                            <goal>exec</goal>
-                        </goals>
-                    </execution>
-                </executions>
-                <configuration>
-                    <executable>java</executable>
-                    <includeProjectDependencies>true</includeProjectDependencies>
-                    <includePluginDependencies>true</includePluginDependencies>
-                    <mainClass>org.apache.storm.jms.example.ExampleJmsTopology</mainClass>
-                    <systemProperties>
-                        <systemProperty>
-                            <key>log4j.configuration</key>
-                            <value>file:./src/main/resources/log4j.properties</value>
-                        </systemProperty>
-                    </systemProperties>
-                </configuration>
-                <dependencies>
-                    <dependency>
-                        <groupId>org.apache.storm</groupId>
-                        <artifactId>storm-server</artifactId>
-                        <version>${project.version}</version>
-                        <type>jar</type>
-                    </dependency>
-                </dependencies>
-            </plugin>
         </plugins>
     </build>
-</project>
\ No newline at end of file
+</project>

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-jms-examples/src/main/java/org/apache/storm/jms/example/ExampleJmsTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-jms-examples/src/main/java/org/apache/storm/jms/example/ExampleJmsTopology.java b/examples/storm-jms-examples/src/main/java/org/apache/storm/jms/example/ExampleJmsTopology.java
index 82dbd5b..accb052 100644
--- a/examples/storm-jms-examples/src/main/java/org/apache/storm/jms/example/ExampleJmsTopology.java
+++ b/examples/storm-jms-examples/src/main/java/org/apache/storm/jms/example/ExampleJmsTopology.java
@@ -23,8 +23,6 @@ import javax.jms.Session;
 import javax.jms.TextMessage;
 
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.jms.JmsMessageProducer;
 import org.apache.storm.jms.JmsProvider;
@@ -34,7 +32,6 @@ import org.apache.storm.jms.spout.JmsSpout;
 import org.apache.storm.topology.TopologyBuilder;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.ITuple;
-import org.apache.storm.utils.Utils;
 
 public class ExampleJmsTopology {
     public static final String JMS_QUEUE_SPOUT = "JMS_QUEUE_SPOUT";
@@ -111,21 +108,13 @@ public class ExampleJmsTopology {
                 JMS_TOPIC_SPOUT);
 
         Config conf = new Config();
-
+        String topoName = "storm-jms-example";
         if (args.length > 0) {
-            conf.setNumWorkers(3);
-
-            StormSubmitter.submitTopology(args[0], conf,
-                    builder.createTopology());
-        } else {
-
-            conf.setDebug(true);
-
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology("storm-jms-example", conf, builder.createTopology());) {
-                Utils.sleep(60000);
-            }
+            topoName = args[0];
         }
-    }
+        conf.setNumWorkers(3);
 
+        StormSubmitter.submitTopology(topoName, conf,
+                builder.createTopology());
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-kafka-client-examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-kafka-client-examples/pom.xml b/examples/storm-kafka-client-examples/pom.xml
index e319346..d0877c8 100644
--- a/examples/storm-kafka-client-examples/pom.xml
+++ b/examples/storm-kafka-client-examples/pom.xml
@@ -34,7 +34,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-server</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>
@@ -134,4 +134,4 @@
         </plugins>
     </build>
 
-</project>
\ No newline at end of file
+</project>

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-kafka-client-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaClientWordCountNamedTopics.java
----------------------------------------------------------------------
diff --git a/examples/storm-kafka-client-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaClientWordCountNamedTopics.java b/examples/storm-kafka-client-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaClientWordCountNamedTopics.java
index edd1f09..9ab00f5 100644
--- a/examples/storm-kafka-client-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaClientWordCountNamedTopics.java
+++ b/examples/storm-kafka-client-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaClientWordCountNamedTopics.java
@@ -18,6 +18,13 @@
 
 package org.apache.storm.kafka.trident;
 
+import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST;
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.storm.Config;
 import org.apache.storm.StormSubmitter;
@@ -33,13 +40,6 @@ import org.apache.storm.kafka.spout.trident.KafkaTridentSpoutOpaque;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Values;
 
-import java.io.Serializable;
-import java.util.Arrays;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST;
-
 public class TridentKafkaClientWordCountNamedTopics {
     private static final String TOPIC_1 = "test-trident";
     private static final String TOPIC_2 = "test-trident-1";
@@ -93,49 +93,19 @@ public class TridentKafkaClientWordCountNamedTopics {
 
             System.out.printf("Running with broker_url: [%s], topics: [%s, %s]\n", brokerUrl, topic1, topic2);
 
-            Config tpConf = LocalSubmitter.defaultConfig(true);
-
-            if (args.length == 4) { //Submit Remote
-                // Producers
-                StormSubmitter.submitTopology(topic1 + "-producer", tpConf, KafkaProducerTopology.newTopology(brokerUrl, topic1));
-                StormSubmitter.submitTopology(topic2 + "-producer", tpConf, KafkaProducerTopology.newTopology(brokerUrl, topic2));
-                // Consumer
-                StormSubmitter.submitTopology("topics-consumer", tpConf, TridentKafkaConsumerTopology.newTopology(newKafkaTridentSpoutOpaque()));
-
-                // Print results to console, which also causes the print filter in the consumer topology to print the results in the worker log
-                Thread.sleep(2000);
-                DrpcResultsPrinter.remoteClient().printResults(60, 1, TimeUnit.SECONDS);
-
-            } else { //Submit Local
-
-                final LocalSubmitter localSubmitter = LocalSubmitter.newInstance();
-                final String topic1Tp = "topic1-producer";
-                final String topic2Tp = "topic2-producer";
-                final String consTpName = "topics-consumer";
-
-                try {
-                    // Producers
-                    localSubmitter.submit(topic1Tp, tpConf, KafkaProducerTopology.newTopology(brokerUrl, topic1));
-                    localSubmitter.submit(topic2Tp, tpConf, KafkaProducerTopology.newTopology(brokerUrl, topic2));
-                    // Consumer
-                    try {
-                        localSubmitter.submit(consTpName, tpConf, TridentKafkaConsumerTopology.newTopology(
-                                localSubmitter.getDrpc(), newKafkaTridentSpoutOpaque()));
-                        // print
-                        localSubmitter.printResults(15, 1, TimeUnit.SECONDS);
-                    } catch (Exception e) {
-                        e.printStackTrace();
-                    }
-
-                } finally {
-                    // kill
-                    localSubmitter.kill(topic1Tp);
-                    localSubmitter.kill(topic2Tp);
-                    localSubmitter.kill(consTpName);
-                    // shutdown
-                    localSubmitter.shutdown();
-                }
-            }
+            Config tpConf = new Config();
+            tpConf.setDebug(true);
+            tpConf.setMaxSpoutPending(5);
+
+            // Producers
+            StormSubmitter.submitTopology(topic1 + "-producer", tpConf, KafkaProducerTopology.newTopology(brokerUrl, topic1));
+            StormSubmitter.submitTopology(topic2 + "-producer", tpConf, KafkaProducerTopology.newTopology(brokerUrl, topic2));
+            // Consumer
+            StormSubmitter.submitTopology("topics-consumer", tpConf, TridentKafkaConsumerTopology.newTopology(newKafkaTridentSpoutOpaque()));
+
+            // Print results to console, which also causes the print filter in the consumer topology to print the results in the worker log
+            Thread.sleep(2000);
+            DrpcResultsPrinter.remoteClient().printResults(60, 1, TimeUnit.SECONDS);
         }
         System.exit(0);     // Kill all the non daemon threads
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-kafka-examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-kafka-examples/pom.xml b/examples/storm-kafka-examples/pom.xml
index 8e89aec..f2d927d 100644
--- a/examples/storm-kafka-examples/pom.xml
+++ b/examples/storm-kafka-examples/pom.xml
@@ -35,7 +35,7 @@
         </dependency>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-server</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/DrpcResultsPrinter.java
----------------------------------------------------------------------
diff --git a/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/DrpcResultsPrinter.java b/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/DrpcResultsPrinter.java
index b08fc96..3476d9f 100644
--- a/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/DrpcResultsPrinter.java
+++ b/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/DrpcResultsPrinter.java
@@ -18,17 +18,16 @@
 
 package org.apache.storm.kafka.trident;
 
-import org.apache.storm.LocalDRPC;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
 import org.apache.storm.generated.DistributedRPC;
-import org.apache.thrift.transport.TTransportException;
-import org.apache.storm.utils.Utils;
 import org.apache.storm.utils.DRPCClient;
+import org.apache.storm.utils.Utils;
+import org.apache.thrift.transport.TTransportException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
 public class DrpcResultsPrinter {
     private static final Logger LOG = LoggerFactory.getLogger(DrpcResultsPrinter.class);
 
@@ -39,13 +38,6 @@ public class DrpcResultsPrinter {
     }
 
     /**
-     * @return local DRPC client running on the same JVML
-     */
-    public static DrpcResultsPrinter localClient() {
-        return new DrpcResultsPrinter(new LocalDRPC());
-    }
-
-    /**
      * @return remote DRPC client running on local host, on port 3772, with defaults.yaml config
      */
     public static DrpcResultsPrinter remoteClient() {

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/LocalSubmitter.java
----------------------------------------------------------------------
diff --git a/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/LocalSubmitter.java b/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/LocalSubmitter.java
deleted file mode 100644
index 9666695..0000000
--- a/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/LocalSubmitter.java
+++ /dev/null
@@ -1,106 +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.storm.kafka.trident;
-
-import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalDRPC;
-import org.apache.storm.generated.StormTopology;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.TimeUnit;
-
-public class LocalSubmitter {
-    protected static final Logger LOG = LoggerFactory.getLogger(LocalSubmitter.class);
-
-    private LocalDRPC drpc;
-    private LocalCluster cluster;
-
-    public LocalSubmitter(LocalDRPC drpc, LocalCluster cluster) {
-        this.drpc = drpc;
-        this.cluster = cluster;
-    }
-
-    public static LocalSubmitter newInstance() {
-        try {
-            return new LocalSubmitter(new LocalDRPC(), new LocalCluster());
-        } catch (Exception e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    public static Config defaultConfig() {
-        return defaultConfig(false);
-    }
-
-    public static Config defaultConfig(boolean debug) {
-        final Config conf = new Config();
-        conf.setMaxSpoutPending(20);
-        conf.setDebug(debug);
-        return conf;
-    }
-
-    public LocalSubmitter(StormTopology topology, LocalDRPC drpc, LocalCluster cluster, String name) {
-        this(drpc, cluster);
-    }
-
-    public void submit(String name, Config config, StormTopology topology) {
-        try {
-            cluster.submitTopology(name, config, topology);
-        } catch (Exception e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    /**
-     * Prints the DRPC results for the amount of time specified
-     */
-    public void printResults(int num, int time, TimeUnit unit) {
-        for (int i = 0; i < num; i++) {
-            try {
-                LOG.info("--- DRPC RESULT: " + drpc.execute("words", "the and apple snow jumped"));
-                System.out.println();
-                Thread.sleep(unit.toMillis(time));
-            } catch (Exception e) {
-                throw new RuntimeException(e);
-            }
-        }
-    }
-
-    public void kill (String name) {
-        try {
-            cluster.killTopology(name);
-        } catch (Exception e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    public void shutdown() {
-        cluster.shutdown();
-    }
-
-    public LocalDRPC getDrpc() {
-        return drpc;
-    }
-
-    public LocalCluster getCluster() {
-        return cluster;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaConsumerTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaConsumerTopology.java b/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaConsumerTopology.java
index a39eba1..e571210 100644
--- a/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaConsumerTopology.java
+++ b/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaConsumerTopology.java
@@ -18,7 +18,6 @@
 
 package org.apache.storm.kafka.trident;
 
-import org.apache.storm.LocalDRPC;
 import org.apache.storm.generated.StormTopology;
 import org.apache.storm.starter.trident.DebugMemoryMapState;
 import org.apache.storm.trident.Stream;
@@ -30,7 +29,6 @@ import org.apache.storm.trident.operation.builtin.Debug;
 import org.apache.storm.trident.operation.builtin.FilterNull;
 import org.apache.storm.trident.operation.builtin.MapGet;
 import org.apache.storm.trident.spout.ITridentDataSource;
-import org.apache.storm.trident.testing.MemoryMapState;
 import org.apache.storm.trident.testing.Split;
 import org.apache.storm.trident.tuple.TridentTuple;
 import org.apache.storm.tuple.Fields;
@@ -44,22 +42,13 @@ public class TridentKafkaConsumerTopology {
      * See {@link TridentKafkaConsumerTopology#newTopology(LocalDRPC, ITridentDataSource)}
      */
     public static StormTopology newTopology(ITridentDataSource tridentSpout) {
-        return newTopology(null, tridentSpout);
-    }
-
-    /**
-     * @param drpc The DRPC stream to be used in querying the word counts. Can be null in distributed mode
-     * @return a trident topology that consumes sentences from the kafka topic specified using a
-     * {@link TransactionalTridentKafkaSpout} computes the word count and stores it in a {@link MemoryMapState}.
-     */
-    public static StormTopology newTopology(LocalDRPC drpc, ITridentDataSource tridentSpout) {
         final TridentTopology tridentTopology = new TridentTopology();
-        addDRPCStream(tridentTopology, addTridentState(tridentTopology, tridentSpout), drpc);
+        addDRPCStream(tridentTopology, addTridentState(tridentTopology, tridentSpout));
         return tridentTopology.build();
     }
 
-    private static Stream addDRPCStream(TridentTopology tridentTopology, final TridentState state, LocalDRPC drpc) {
-        return tridentTopology.newDRPCStream("words", drpc)
+    private static Stream addDRPCStream(TridentTopology tridentTopology, final TridentState state) {
+        return tridentTopology.newDRPCStream("words")
                 .each(new Fields("args"), new Split(), new Fields("word"))
                 .groupBy(new Fields("word"))
                 .stateQuery(state, new Fields("word"), new MapGet(), new Fields("count"))

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaTopology.java b/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaTopology.java
index 11ea899..ad785b8 100644
--- a/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaTopology.java
+++ b/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaTopology.java
@@ -18,9 +18,10 @@
 
 package org.apache.storm.kafka.trident;
 
+import java.util.Properties;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
+import org.apache.storm.StormSubmitter;
 import org.apache.storm.generated.StormTopology;
 import org.apache.storm.kafka.trident.mapper.FieldNameBasedTupleToKafkaMapper;
 import org.apache.storm.kafka.trident.selector.DefaultTopicSelector;
@@ -30,8 +31,6 @@ import org.apache.storm.trident.testing.FixedBatchSpout;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.Values;
 
-import java.util.Properties;
-
 public class TridentKafkaTopology {
 
     private static StormTopology buildTopology(String brokerConnectionString) {
@@ -79,10 +78,6 @@ public class TridentKafkaTopology {
         if(args.length < 1) {
             System.out.println("Please provide kafka broker url ,e.g. localhost:9092");
         }
-
-        try (LocalCluster cluster = new LocalCluster();
-             LocalTopology topo = cluster.submitTopology("wordCounter", new Config(), buildTopology(args[0]));) {
-            Thread.sleep(60 * 1000);
-        }
+        StormSubmitter.submitTopology("wordCounter", new Config(), buildTopology(args[0]));
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaWordCount.java
----------------------------------------------------------------------
diff --git a/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaWordCount.java b/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaWordCount.java
index 84dc380..6aa54ea 100644
--- a/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaWordCount.java
+++ b/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaWordCount.java
@@ -77,40 +77,24 @@ public class TridentKafkaWordCount implements Serializable {
     public static void main(String[] args) throws Exception {
         final String[] zkBrokerUrl = parseUrl(args);
         final String topicName = "test";
-        Config tpConf = LocalSubmitter.defaultConfig();
+        Config tpConf = new Config();
+        tpConf.setMaxSpoutPending(20);
+        String prodTpName = "kafkaBolt";
+        String consTpName = "wordCounter";
 
-        if (args.length == 3)  { //Submit Remote
-            // Producer
-            StormSubmitter.submitTopology(args[2] + "-producer", tpConf, KafkaProducerTopology.newTopology(zkBrokerUrl[1], topicName));
-            // Consumer
-            StormSubmitter.submitTopology(args[2] + "-consumer", tpConf, TridentKafkaConsumerTopology.newTopology(
-                    new TransactionalTridentKafkaSpout(newTridentKafkaConfig(zkBrokerUrl[0]))));
-
-            // Print results to console, which also causes the print filter in the consumer topology to print the results in the worker log
-            Thread.sleep(2000);
-            DrpcResultsPrinter.remoteClient().printResults(60, 1, TimeUnit.SECONDS);
-        } else { //Submit Local
-            final LocalSubmitter localSubmitter = LocalSubmitter.newInstance();
-            final String prodTpName = "kafkaBolt";
-            final String consTpName = "wordCounter";
-
-            try {
-                // Producer
-                localSubmitter.submit(prodTpName, tpConf, KafkaProducerTopology.newTopology(zkBrokerUrl[1], topicName));
-                // Consumer
-                localSubmitter.submit(consTpName, tpConf, TridentKafkaConsumerTopology.newTopology(localSubmitter.getDrpc(),
-                        new TransactionalTridentKafkaSpout(newTridentKafkaConfig(zkBrokerUrl[0]))));
-
-                // print
-                new DrpcResultsPrinter(localSubmitter.getDrpc()).printResults(60, 1, TimeUnit.SECONDS);
-            } finally {
-                // kill
-                localSubmitter.kill(prodTpName);
-                localSubmitter.kill(consTpName);
-                // shutdown
-                localSubmitter.shutdown();
-            }
+        if (args.length == 3)  {
+            prodTpName = args[2] + "-producer";
+            consTpName = args[2] + "-consumer";
         }
+        // Producer
+        StormSubmitter.submitTopology(prodTpName, tpConf, KafkaProducerTopology.newTopology(zkBrokerUrl[1], topicName));
+        // Consumer
+        StormSubmitter.submitTopology(consTpName, tpConf, TridentKafkaConsumerTopology.newTopology(
+                new TransactionalTridentKafkaSpout(newTridentKafkaConfig(zkBrokerUrl[0]))));
+
+        // Print results to console, which also causes the print filter in the consumer topology to print the results in the worker log
+        Thread.sleep(2000);
+        DrpcResultsPrinter.remoteClient().printResults(60, 1, TimeUnit.SECONDS);
     }
 
     private static String[] parseUrl(String[] args) {

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-mongodb-examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-mongodb-examples/pom.xml b/examples/storm-mongodb-examples/pom.xml
index e4a2a92..16200fc 100644
--- a/examples/storm-mongodb-examples/pom.xml
+++ b/examples/storm-mongodb-examples/pom.xml
@@ -30,7 +30,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-server</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/InsertWordCount.java
----------------------------------------------------------------------
diff --git a/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/InsertWordCount.java b/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/InsertWordCount.java
index 6f71b1c..3b27fd1 100644
--- a/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/InsertWordCount.java
+++ b/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/InsertWordCount.java
@@ -18,17 +18,12 @@
 package org.apache.storm.mongodb.topology;
 
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
-import org.apache.storm.topology.TopologyBuilder;
-import org.apache.storm.tuple.Fields;
 import org.apache.storm.mongodb.bolt.MongoInsertBolt;
 import org.apache.storm.mongodb.common.mapper.MongoMapper;
 import org.apache.storm.mongodb.common.mapper.SimpleMongoMapper;
-
-import java.util.HashMap;
-import java.util.Map;
+import org.apache.storm.topology.TopologyBuilder;
+import org.apache.storm.tuple.Fields;
 
 public class InsertWordCount {
     private static final String WORD_SPOUT = "WORD_SPOUT";
@@ -65,17 +60,13 @@ public class InsertWordCount {
         builder.setBolt(COUNT_BOLT, bolt, 1).shuffleGrouping(WORD_SPOUT);
         builder.setBolt(INSERT_BOLT, insertBolt, 1).fieldsGrouping(COUNT_BOLT, new Fields("word"));
 
-
-        if (args.length == 2) {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology("test", config, builder.createTopology());) {
-                Thread.sleep(30000);
-            }
-            System.exit(0);
-        } else if (args.length == 3) {
-            StormSubmitter.submitTopology(args[2], config, builder.createTopology());
-        } else{
+        String topoName = "test";
+        if (args.length == 3) {
+            topoName = args[2];
+        } else if (args.length > 3) {
             System.out.println("Usage: InsertWordCount <mongodb url> <mongodb collection> [topology name]");
+            return;
         }
+        StormSubmitter.submitTopology(topoName, config, builder.createTopology());
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/LookupWordCount.java
----------------------------------------------------------------------
diff --git a/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/LookupWordCount.java b/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/LookupWordCount.java
index 5140685..c9d43bd 100644
--- a/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/LookupWordCount.java
+++ b/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/LookupWordCount.java
@@ -18,7 +18,6 @@
 package org.apache.storm.mongodb.topology;
 
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.mongodb.bolt.MongoLookupBolt;
 import org.apache.storm.mongodb.common.QueryFilterCreator;
@@ -64,17 +63,14 @@ public class LookupWordCount {
         builder.setBolt(LOOKUP_BOLT, lookupBolt, 1).shuffleGrouping(WORD_SPOUT);
         builder.setBolt(TOTAL_COUNT_BOLT, totalBolt, 1).fieldsGrouping(LOOKUP_BOLT, new Fields("word"));
 
-        if (args.length == 2) {
-            LocalCluster cluster = new LocalCluster();
-            cluster.submitTopology("test", config, builder.createTopology());
-            Thread.sleep(30000);
-            cluster.killTopology("test");
-            cluster.shutdown();
-            System.exit(0);
-        } else if (args.length == 3) {
-            StormSubmitter.submitTopology(args[2], config, builder.createTopology());
-        } else{
+        String topoName = "test";
+        if (args.length == 3) {
+            topoName = args[2];
+        } else if (args.length > 3) {
             System.out.println("Usage: LookupWordCount <mongodb url> <mongodb collection> [topology name]");
+            return;
         }
+        
+        StormSubmitter.submitTopology(topoName, config, builder.createTopology());
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/UpdateWordCount.java
----------------------------------------------------------------------
diff --git a/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/UpdateWordCount.java b/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/UpdateWordCount.java
index b4af4ca..8e4f757 100644
--- a/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/UpdateWordCount.java
+++ b/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/topology/UpdateWordCount.java
@@ -18,16 +18,14 @@
 package org.apache.storm.mongodb.topology;
 
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
-import org.apache.storm.mongodb.common.mapper.MongoUpdateMapper;
-import org.apache.storm.topology.TopologyBuilder;
-import org.apache.storm.tuple.Fields;
 import org.apache.storm.mongodb.bolt.MongoUpdateBolt;
 import org.apache.storm.mongodb.common.QueryFilterCreator;
 import org.apache.storm.mongodb.common.SimpleQueryFilterCreator;
+import org.apache.storm.mongodb.common.mapper.MongoUpdateMapper;
 import org.apache.storm.mongodb.common.mapper.SimpleMongoUpdateMapper;
+import org.apache.storm.topology.TopologyBuilder;
+import org.apache.storm.tuple.Fields;
 
 public class UpdateWordCount {
     private static final String WORD_SPOUT = "WORD_SPOUT";
@@ -73,17 +71,13 @@ public class UpdateWordCount {
         builder.setBolt(COUNT_BOLT, bolt, 1).shuffleGrouping(WORD_SPOUT);
         builder.setBolt(UPDATE_BOLT, updateBolt, 1).fieldsGrouping(COUNT_BOLT, new Fields("word"));
 
-
-        if (args.length == 2) {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology("test", config, builder.createTopology());) {
-                Thread.sleep(30000);
-            }
-            System.exit(0);
-        } else if (args.length == 3) {
-            StormSubmitter.submitTopology(args[2], config, builder.createTopology());
-        } else{
+        String topoName = "test";
+        if (args.length == 3) {
+            topoName = args[2];
+        } else if (args.length > 3) {
             System.out.println("Usage: UpdateWordCount <mongodb url> <mongodb collection> [topology name]");
+            return;
         }
+        StormSubmitter.submitTopology(topoName, config, builder.createTopology());
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/trident/WordCountTrident.java
----------------------------------------------------------------------
diff --git a/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/trident/WordCountTrident.java b/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/trident/WordCountTrident.java
index 14dccbd..ef5a39a 100644
--- a/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/trident/WordCountTrident.java
+++ b/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/trident/WordCountTrident.java
@@ -18,8 +18,6 @@
 package org.apache.storm.mongodb.trident;
 
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.generated.StormTopology;
 import org.apache.storm.mongodb.common.mapper.MongoMapper;
@@ -74,19 +72,15 @@ public class WordCountTrident {
     public static void main(String[] args) throws Exception {
         Config conf = new Config();
         conf.setMaxSpoutPending(5);
-        if (args.length == 2) {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology("wordCounter", conf, buildTopology(args[0], args[1]));) {
-                Thread.sleep(60 * 1000);
-            }
-            System.exit(0);
-        }
-        else if(args.length == 3) {
-            conf.setNumWorkers(3);
-            StormSubmitter.submitTopology(args[2], conf, buildTopology(args[0], args[1]));
-        } else{
+        String topoName = "wordCounter";
+        if (args.length == 3) {
+            topoName = args[2];
+        } else if (args.length > 3 || args.length < 2) {
             System.out.println("Usage: WordCountTrident <mongodb url> <mongodb collection> [topology name]");
+            return;
         }
+        conf.setNumWorkers(3);
+        StormSubmitter.submitTopology(topoName, conf, buildTopology(args[0], args[1]));
     }
 
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/trident/WordCountTridentMap.java
----------------------------------------------------------------------
diff --git a/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/trident/WordCountTridentMap.java b/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/trident/WordCountTridentMap.java
index 83c0caf..0ff2c51 100644
--- a/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/trident/WordCountTridentMap.java
+++ b/examples/storm-mongodb-examples/src/main/java/org/apache/storm/mongodb/trident/WordCountTridentMap.java
@@ -18,14 +18,13 @@
 package org.apache.storm.mongodb.trident;
 
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.generated.StormTopology;
 import org.apache.storm.mongodb.common.QueryFilterCreator;
 import org.apache.storm.mongodb.common.SimpleQueryFilterCreator;
 import org.apache.storm.mongodb.common.mapper.MongoMapper;
 import org.apache.storm.mongodb.common.mapper.SimpleMongoMapper;
-import org.apache.storm.mongodb.trident.state.*;
+import org.apache.storm.mongodb.trident.state.MongoMapState;
 import org.apache.storm.trident.Stream;
 import org.apache.storm.trident.TridentState;
 import org.apache.storm.trident.TridentTopology;
@@ -76,20 +75,15 @@ public class WordCountTridentMap {
     public static void main(String[] args) throws Exception {
         Config conf = new Config();
         conf.setMaxSpoutPending(5);
-        if (args.length == 2) {
-            LocalCluster cluster = new LocalCluster();
-            cluster.submitTopology("wordCounter", conf, buildTopology(args[0], args[1]));
-            Thread.sleep(60 * 1000);
-            cluster.killTopology("wordCounter");
-            cluster.shutdown();
-            System.exit(0);
-        }
-        else if(args.length == 3) {
-            conf.setNumWorkers(3);
-            StormSubmitter.submitTopology(args[2], conf, buildTopology(args[0], args[1]));
-        } else{
+        String topoName = "wordCounter";
+        if (args.length == 3) {
+            topoName = args[2];
+        } else if (args.length > 3 || args.length < 2) {
             System.out.println("Usage: WordCountTrident <mongodb url> <mongodb collection> [topology name]");
+            return;
         }
+        conf.setNumWorkers(3);
+        StormSubmitter.submitTopology(topoName, conf, buildTopology(args[0], args[1]));
     }
 
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-mqtt-examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-mqtt-examples/pom.xml b/examples/storm-mqtt-examples/pom.xml
index 41ff820..a178c5b 100644
--- a/examples/storm-mqtt-examples/pom.xml
+++ b/examples/storm-mqtt-examples/pom.xml
@@ -34,7 +34,7 @@
   <dependencies>
    <dependency>
       <groupId>org.apache.storm</groupId>
-      <artifactId>storm-server</artifactId>
+      <artifactId>storm-client</artifactId>
       <version>${project.version}</version>
       <scope>${provided.scope}</scope>
     </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-opentsdb-examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-opentsdb-examples/pom.xml b/examples/storm-opentsdb-examples/pom.xml
index 9f550e6..4ec799b 100644
--- a/examples/storm-opentsdb-examples/pom.xml
+++ b/examples/storm-opentsdb-examples/pom.xml
@@ -30,7 +30,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-server</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-opentsdb-examples/src/main/java/org/apache/storm/opentsdb/SampleOpenTsdbBoltTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-opentsdb-examples/src/main/java/org/apache/storm/opentsdb/SampleOpenTsdbBoltTopology.java b/examples/storm-opentsdb-examples/src/main/java/org/apache/storm/opentsdb/SampleOpenTsdbBoltTopology.java
index 7dc65fd..009366f 100644
--- a/examples/storm-opentsdb-examples/src/main/java/org/apache/storm/opentsdb/SampleOpenTsdbBoltTopology.java
+++ b/examples/storm-opentsdb-examples/src/main/java/org/apache/storm/opentsdb/SampleOpenTsdbBoltTopology.java
@@ -18,17 +18,15 @@
  */
 package org.apache.storm.opentsdb;
 
+import java.util.Collections;
+
+import org.apache.storm.Config;
+import org.apache.storm.StormSubmitter;
 import org.apache.storm.opentsdb.bolt.OpenTsdbBolt;
 import org.apache.storm.opentsdb.bolt.TupleOpenTsdbDatapointMapper;
 import org.apache.storm.opentsdb.client.OpenTsdbClient;
-import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
-import org.apache.storm.StormSubmitter;
 import org.apache.storm.topology.TopologyBuilder;
 
-import java.util.Collections;
-
 /**
  * Sample application to use OpenTSDB bolt.
  */
@@ -51,19 +49,12 @@ public class SampleOpenTsdbBoltTopology {
 
         Config conf = new Config();
         conf.setDebug(true);
-
+        String topoName = "word-count";
         if (args.length > 1) {
-            conf.setNumWorkers(3);
-
-            StormSubmitter.submitTopologyWithProgressBar(args[1], conf, topologyBuilder.createTopology());
-        } else {
-            conf.setMaxTaskParallelism(3);
-
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology("word-count", conf, topologyBuilder.createTopology());) {
-                Thread.sleep(30000);
-            }
-            System.exit(0);
+            topoName = args[1];
         }
+        conf.setNumWorkers(3);
+
+        StormSubmitter.submitTopologyWithProgressBar(topoName, conf, topologyBuilder.createTopology());
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-opentsdb-examples/src/main/java/org/apache/storm/opentsdb/SampleOpenTsdbTridentTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-opentsdb-examples/src/main/java/org/apache/storm/opentsdb/SampleOpenTsdbTridentTopology.java b/examples/storm-opentsdb-examples/src/main/java/org/apache/storm/opentsdb/SampleOpenTsdbTridentTopology.java
index 3220068..8ac950b 100644
--- a/examples/storm-opentsdb-examples/src/main/java/org/apache/storm/opentsdb/SampleOpenTsdbTridentTopology.java
+++ b/examples/storm-opentsdb-examples/src/main/java/org/apache/storm/opentsdb/SampleOpenTsdbTridentTopology.java
@@ -18,14 +18,14 @@
  */
 package org.apache.storm.opentsdb;
 
+import java.util.Collections;
+
+import org.apache.storm.Config;
+import org.apache.storm.StormSubmitter;
 import org.apache.storm.opentsdb.bolt.TupleOpenTsdbDatapointMapper;
 import org.apache.storm.opentsdb.client.OpenTsdbClient;
 import org.apache.storm.opentsdb.trident.OpenTsdbStateFactory;
 import org.apache.storm.opentsdb.trident.OpenTsdbStateUpdater;
-import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
-import org.apache.storm.StormSubmitter;
 import org.apache.storm.trident.Stream;
 import org.apache.storm.trident.TridentTopology;
 import org.apache.storm.trident.operation.Consumer;
@@ -33,8 +33,6 @@ import org.apache.storm.trident.tuple.TridentTuple;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.Collections;
-
 /**
  * Sample trident topology to store time series metrics in to OpenTsdb.
  */
@@ -67,20 +65,12 @@ public class SampleOpenTsdbTridentTopology {
 
         Config conf = new Config();
         conf.setDebug(true);
-
+        String topoName = "word-count";
         if (args.length > 1) {
-            conf.setNumWorkers(3);
-
-            StormSubmitter.submitTopologyWithProgressBar(args[1], conf, tridentTopology.build());
-        } else {
-            conf.setMaxTaskParallelism(3);
-
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology("word-count", conf, tridentTopology.build())) {
-                Thread.sleep(30000);
-            }
-            System.exit(0);
+            topoName = args[1];
         }
+        conf.setNumWorkers(3);
 
+        StormSubmitter.submitTopologyWithProgressBar(topoName, conf, tridentTopology.build());
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-perf/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-perf/pom.xml b/examples/storm-perf/pom.xml
index 260eb06..1625090 100644
--- a/examples/storm-perf/pom.xml
+++ b/examples/storm-perf/pom.xml
@@ -83,7 +83,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-server</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <!--
               Use "provided" scope to keep storm out of the jar-with-dependencies

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-perf/src/main/java/org/apache/storm/perf/ConstSpoutIdBoltNullBoltTopo.java
----------------------------------------------------------------------
diff --git a/examples/storm-perf/src/main/java/org/apache/storm/perf/ConstSpoutIdBoltNullBoltTopo.java b/examples/storm-perf/src/main/java/org/apache/storm/perf/ConstSpoutIdBoltNullBoltTopo.java
index 11c63d3..69df3fb 100644
--- a/examples/storm-perf/src/main/java/org/apache/storm/perf/ConstSpoutIdBoltNullBoltTopo.java
+++ b/examples/storm-perf/src/main/java/org/apache/storm/perf/ConstSpoutIdBoltNullBoltTopo.java
@@ -19,7 +19,6 @@
 package org.apache.storm.perf;
 
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
 import org.apache.storm.generated.StormTopology;
 import org.apache.storm.perf.bolt.DevNullBolt;
 import org.apache.storm.perf.bolt.IdBolt;
@@ -49,7 +48,7 @@ public class ConstSpoutIdBoltNullBoltTopo {
     public static final String BOLT2_COUNT = "bolt2.count";
     public static final String SPOUT_COUNT = "spout.count";
 
-    public static StormTopology getTopology(Map conf) {
+    public static StormTopology getTopology(Map<String, Object> conf) {
 
         // 1 -  Setup Spout   --------
         ConstSpout spout = new ConstSpout("some data").withOutputFields("str");
@@ -75,27 +74,19 @@ public class ConstSpoutIdBoltNullBoltTopo {
 
 
     public static void main(String[] args) throws Exception {
-
-        if (args.length <= 0) {
-            // submit to local cluster
-            Config conf = new Config();
-            LocalCluster cluster = Helper.runOnLocalCluster(TOPOLOGY_NAME, getTopology(conf));
-
-            Helper.setupShutdownHook(cluster, TOPOLOGY_NAME);
-            while (true) {//  run indefinitely till Ctrl-C
-                Thread.sleep(20_000_000);
-            }
-        } else {
-            // submit to real cluster
-            if (args.length >2) {
-                System.err.println("args: runDurationSec  [optionalConfFile]");
-                return;
-            }
-            Integer durationSec = Integer.parseInt(args[0]);
-            Map topoConf =  (args.length==2) ? Utils.findAndReadConfigFile(args[1])  : new Config();
-
-            //  Submit topology to storm cluster
-            Helper.runOnClusterAndPrintMetrics(durationSec, TOPOLOGY_NAME, topoConf, getTopology(topoConf));
+        int runTime = -1;
+        Config topoConf = new Config();
+        if (args.length > 0) {
+            runTime = Integer.parseInt(args[0]);
+        }
+        if (args.length > 1) {
+            topoConf.putAll(Utils.findAndReadConfigFile(args[1]));
+        }
+        if (args.length > 2) {
+            System.err.println("args: [runDurationSec]  [optionalConfFile]");
+            return;
         }
+        //  Submit topology to storm cluster
+        Helper.runOnClusterAndPrintMetrics(runTime, TOPOLOGY_NAME, topoConf, getTopology(topoConf));
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-perf/src/main/java/org/apache/storm/perf/ConstSpoutNullBoltTopo.java
----------------------------------------------------------------------
diff --git a/examples/storm-perf/src/main/java/org/apache/storm/perf/ConstSpoutNullBoltTopo.java b/examples/storm-perf/src/main/java/org/apache/storm/perf/ConstSpoutNullBoltTopo.java
index 92c2787..298c73e 100755
--- a/examples/storm-perf/src/main/java/org/apache/storm/perf/ConstSpoutNullBoltTopo.java
+++ b/examples/storm-perf/src/main/java/org/apache/storm/perf/ConstSpoutNullBoltTopo.java
@@ -18,8 +18,9 @@
 
 package org.apache.storm.perf;
 
+import java.util.Map;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
 import org.apache.storm.generated.StormTopology;
 import org.apache.storm.perf.bolt.DevNullBolt;
 import org.apache.storm.perf.spout.ConstSpout;
@@ -28,8 +29,6 @@ import org.apache.storm.topology.BoltDeclarer;
 import org.apache.storm.topology.TopologyBuilder;
 import org.apache.storm.utils.Utils;
 
-import java.util.Map;
-
 /***
  * This topo helps measure the messaging speed between a spout and a bolt.
  *  Spout generates a stream of a fixed string.
@@ -51,7 +50,7 @@ public class ConstSpoutNullBoltTopo {
     public static final String SHUFFLE_GROUPING = "shuffle";
     public static final String DEFAULT_GROUPING = LOCAL_GROPING;
 
-    public static StormTopology getTopology(Map conf) {
+    public static StormTopology getTopology(Map<String, Object> conf) {
 
         // 1 -  Setup Spout   --------
         ConstSpout spout = new ConstSpout("some data").withOutputFields("str");
@@ -78,29 +77,20 @@ public class ConstSpoutNullBoltTopo {
      * ConstSpout -> DevNullBolt with configurable grouping (default localOrShuffle)
      */
     public static void main(String[] args) throws Exception {
-
-        if(args.length <= 0) {
-            // For IDE based profiling ... submit topology to local cluster
-            Config conf = new Config();
-            final LocalCluster cluster = Helper.runOnLocalCluster(TOPOLOGY_NAME, getTopology(conf));
-
-            Helper.setupShutdownHook(cluster, TOPOLOGY_NAME);
-            while (true) {//  run indefinitely till Ctrl-C
-                Thread.sleep(20_000_000);
-            }
-
-        } else {
-            // For measuring perf against a Storm cluster
-            if (args.length > 2) {
-                System.err.println("args: runDurationSec  [optionalConfFile]");
-                return;
-            }
-            Integer durationSec = Integer.parseInt(args[0]);
-            Map topoConf =  (args.length==2) ? Utils.findAndReadConfigFile(args[1])  : new Config();
-
-            //  Submit topology to storm cluster
-            Helper.runOnClusterAndPrintMetrics(durationSec, TOPOLOGY_NAME, topoConf, getTopology(topoConf));
+        int runTime = -1;
+        Config topoConf = new Config();
+        if (args.length > 0) {
+            runTime = Integer.parseInt(args[0]);
+        }
+        if (args.length > 1) {
+            topoConf.putAll(Utils.findAndReadConfigFile(args[1]));
+        }
+        if (args.length > 2) {
+            System.err.println("args: [runDurationSec]  [optionalConfFile]");
+            return;
         }
+        //  Submit topology to storm cluster
+        Helper.runOnClusterAndPrintMetrics(runTime, TOPOLOGY_NAME, topoConf, getTopology(topoConf));
     }
 
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-perf/src/main/java/org/apache/storm/perf/ConstSpoutOnlyTopo.java
----------------------------------------------------------------------
diff --git a/examples/storm-perf/src/main/java/org/apache/storm/perf/ConstSpoutOnlyTopo.java b/examples/storm-perf/src/main/java/org/apache/storm/perf/ConstSpoutOnlyTopo.java
index 721ae3d..94bd17f 100755
--- a/examples/storm-perf/src/main/java/org/apache/storm/perf/ConstSpoutOnlyTopo.java
+++ b/examples/storm-perf/src/main/java/org/apache/storm/perf/ConstSpoutOnlyTopo.java
@@ -19,11 +19,11 @@
 package org.apache.storm.perf;
 
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
 import org.apache.storm.generated.StormTopology;
 import org.apache.storm.perf.spout.ConstSpout;
 import org.apache.storm.perf.utils.Helper;
 import org.apache.storm.topology.TopologyBuilder;
+import org.apache.storm.utils.Utils;
 
 
 /***
@@ -52,23 +52,19 @@ public class ConstSpoutOnlyTopo {
      * ConstSpout only topology  (No bolts)
      */
     public static void main(String[] args) throws Exception {
-        if(args.length <= 0) {
-            // For IDE based profiling ... submit topology to local cluster
-            LocalCluster cluster = Helper.runOnLocalCluster(TOPOLOGY_NAME, getTopology());
-
-            Helper.setupShutdownHook(cluster, TOPOLOGY_NAME);
-            while (true) {//  run indefinitely till Ctrl-C
-                Thread.sleep(20_000_000);
-            }
-        } else {
-            //  Submit topology to storm cluster
-            if (args.length != 1) {
-                System.err.println("args: runDurationSec");
-                return;
-            }
-            Integer durationSec = Integer.parseInt(args[0]);
-
-            Helper.runOnClusterAndPrintMetrics(durationSec, TOPOLOGY_NAME, new Config(), getTopology());
+        int runTime = -1;
+        Config topoConf = new Config();
+        if (args.length > 0) {
+            runTime = Integer.parseInt(args[0]);
+        }
+        if (args.length > 1) {
+            topoConf.putAll(Utils.findAndReadConfigFile(args[1]));
+        }
+        if (args.length > 2) {
+            System.err.println("args: [runDurationSec]  [optionalConfFile]");
+            return;
         }
+        //  Submit topology to storm cluster
+        Helper.runOnClusterAndPrintMetrics(runTime, TOPOLOGY_NAME, topoConf, getTopology());
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-perf/src/main/java/org/apache/storm/perf/FileReadWordCountTopo.java
----------------------------------------------------------------------
diff --git a/examples/storm-perf/src/main/java/org/apache/storm/perf/FileReadWordCountTopo.java b/examples/storm-perf/src/main/java/org/apache/storm/perf/FileReadWordCountTopo.java
index d518c86..e64dd36 100644
--- a/examples/storm-perf/src/main/java/org/apache/storm/perf/FileReadWordCountTopo.java
+++ b/examples/storm-perf/src/main/java/org/apache/storm/perf/FileReadWordCountTopo.java
@@ -18,8 +18,9 @@
 package org.apache.storm.perf;
 
 
+import java.util.Map;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
 import org.apache.storm.generated.StormTopology;
 import org.apache.storm.perf.bolt.CountBolt;
 import org.apache.storm.perf.bolt.SplitSentenceBolt;
@@ -29,9 +30,6 @@ import org.apache.storm.topology.TopologyBuilder;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.utils.Utils;
 
-
-import java.util.Map;
-
 /***
  * This topo helps measure speed of word count.
  *  Spout loads a file into memory on initialization, then emits the lines in an endless loop.
@@ -54,7 +52,7 @@ public class FileReadWordCountTopo {
     public static final int DEFAULT_COUNT_BOLT_NUM = 2;
 
 
-    public static StormTopology getTopology(Map config) {
+    public static StormTopology getTopology(Map<String, Object> config) {
 
         final int spoutNum = Helper.getInt(config, SPOUT_NUM, DEFAULT_SPOUT_NUM);
         final int spBoltNum = Helper.getInt(config, SPLIT_NUM, DEFAULT_SPLIT_BOLT_NUM);
@@ -70,27 +68,19 @@ public class FileReadWordCountTopo {
     }
 
     public static void main(String[] args) throws Exception {
-        if(args.length <= 0) {
-            // For IDE based profiling ... submit topology to local cluster
-            Config conf = new Config();
-            conf.put(INPUT_FILE, "resources/randomwords.txt");
-            LocalCluster cluster = Helper.runOnLocalCluster(TOPOLOGY_NAME, getTopology(conf));
-
-            Helper.setupShutdownHook(cluster, TOPOLOGY_NAME);
-            while (true) {//  run indefinitely till Ctrl-C
-                Thread.sleep(20_000_000);
-            }
-        } else {
-            //  Submit to Storm cluster
-            if (args.length !=2) {
-                System.err.println("args: runDurationSec  confFile");
-                return;
-            }
-            Integer durationSec = Integer.parseInt(args[0]);
-            Map topoConf = Utils.findAndReadConfigFile(args[1]);
-
-            Helper.runOnClusterAndPrintMetrics(durationSec, TOPOLOGY_NAME, topoConf, getTopology(topoConf));
-
+        int runTime = -1;
+        Config topoConf = new Config();
+        if (args.length > 0) {
+            runTime = Integer.parseInt(args[0]);
+        }
+        if (args.length > 1) {
+            topoConf.putAll(Utils.findAndReadConfigFile(args[1]));
+        }
+        if (args.length > 2) {
+            System.err.println("args: [runDurationSec]  [optionalConfFile]");
+            return;
         }
+        //  Submit topology to storm cluster
+        Helper.runOnClusterAndPrintMetrics(runTime, TOPOLOGY_NAME, topoConf, getTopology(topoConf));
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-perf/src/main/java/org/apache/storm/perf/StrGenSpoutHdfsBoltTopo.java
----------------------------------------------------------------------
diff --git a/examples/storm-perf/src/main/java/org/apache/storm/perf/StrGenSpoutHdfsBoltTopo.java b/examples/storm-perf/src/main/java/org/apache/storm/perf/StrGenSpoutHdfsBoltTopo.java
index 5b97540..ed81a4a 100755
--- a/examples/storm-perf/src/main/java/org/apache/storm/perf/StrGenSpoutHdfsBoltTopo.java
+++ b/examples/storm-perf/src/main/java/org/apache/storm/perf/StrGenSpoutHdfsBoltTopo.java
@@ -19,7 +19,8 @@
 
 package org.apache.storm.perf;
 
-import org.apache.storm.LocalCluster;
+import java.util.Map;
+
 import org.apache.storm.generated.StormTopology;
 import org.apache.storm.hdfs.bolt.HdfsBolt;
 import org.apache.storm.hdfs.bolt.format.DefaultFileNameFormat;
@@ -35,8 +36,6 @@ import org.apache.storm.topology.TopologyBuilder;
 import org.apache.storm.tuple.Tuple;
 import org.apache.storm.utils.Utils;
 
-import java.util.Map;
-
 /***
  * This topo helps measure speed of writing to Hdfs
  *  Spout generates fixed length random strings.
@@ -64,7 +63,7 @@ public class StrGenSpoutHdfsBoltTopo {
     public static final String BOLT_ID = "hdfsBolt";
 
 
-    public static StormTopology getTopology(Map topoConf) {
+    public static StormTopology getTopology(Map<String, Object> topoConf) {
         final int hdfsBatch = Helper.getInt(topoConf, HDFS_BATCH, DEFAULT_HDFS_BATCH);
 
         // 1 -  Setup StringGen Spout   --------
@@ -104,30 +103,30 @@ public class StrGenSpoutHdfsBoltTopo {
 
     /** Spout generates random strings and HDFS bolt writes them to a text file */
     public static void main(String[] args) throws Exception {
-        if(args.length <= 0) {
-            // submit to local cluster
-            Map topoConf = Utils.findAndReadConfigFile("conf/HdfsSpoutTopo.yaml");
-            LocalCluster cluster = Helper.runOnLocalCluster(TOPOLOGY_NAME, getTopology(topoConf));
-
-            Helper.setupShutdownHook(cluster, TOPOLOGY_NAME);
-            while (true) {//  run indefinitely till Ctrl-C
-                Thread.sleep(20_000_000);
-            }
-        } else {
-            //  Submit to Storm cluster
-            if (args.length !=2) {
-                System.err.println("args: runDurationSec confFile");
-                return;
-            }
-            Integer durationSec = Integer.parseInt(args[0]);
-            Map topoConf = Utils.findAndReadConfigFile(args[1]);
-
-            Helper.runOnClusterAndPrintMetrics(durationSec, TOPOLOGY_NAME, topoConf, getTopology(topoConf));
+        String confFile = "conf/HdfsSpoutTopo.yaml";
+        int runTime = -1; //Run until Ctrl-C
+        if (args.length > 0) {
+            runTime = Integer.parseInt(args[0]);
+        }
+
+        if (args.length > 1) {
+            confFile = args[1];
+        }
+
+        //  Submit to Storm cluster
+        if (args.length > 2) {
+            System.err.println("args: [runDurationSec] [confFile]");
+            return;
         }
+
+        Map<String, Object> topoConf = Utils.findAndReadConfigFile(confFile);
+
+        Helper.runOnClusterAndPrintMetrics(runTime, TOPOLOGY_NAME, topoConf, getTopology(topoConf));
     }
 
 
     public static class LineWriter implements RecordFormat {
+        private static final long serialVersionUID = 7524288317405514146L;
         private String lineDelimiter = System.lineSeparator();
         private String fieldName;
 

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-perf/src/main/java/org/apache/storm/perf/utils/BasicMetricsCollector.java
----------------------------------------------------------------------
diff --git a/examples/storm-perf/src/main/java/org/apache/storm/perf/utils/BasicMetricsCollector.java b/examples/storm-perf/src/main/java/org/apache/storm/perf/utils/BasicMetricsCollector.java
index e01de0e..a008888 100755
--- a/examples/storm-perf/src/main/java/org/apache/storm/perf/utils/BasicMetricsCollector.java
+++ b/examples/storm-perf/src/main/java/org/apache/storm/perf/utils/BasicMetricsCollector.java
@@ -18,7 +18,6 @@
 
 package org.apache.storm.perf.utils;
 
-import org.apache.storm.LocalCluster;
 import org.apache.storm.generated.Nimbus;
 import org.apache.storm.utils.Utils;
 import org.apache.log4j.Logger;
@@ -27,10 +26,7 @@ import java.io.PrintWriter;
 import java.util.*;
 
 
-public class BasicMetricsCollector  {
-
-    private LocalCluster localCluster = null;
-    private Nimbus.Client client = null;
+public class BasicMetricsCollector implements AutoCloseable {
     private PrintWriter dataWriter;
     private long startTime=0;
 
@@ -80,20 +76,8 @@ public class BasicMetricsCollector  {
     private double maxLatency = 0;
 
     boolean first = true;
-
-    public BasicMetricsCollector(Nimbus.Client client, String topoName, Map stormConfig) {
-        this(topoName, stormConfig);
-        this.client = client;
-        this.localCluster = null;
-    }
-
-    public BasicMetricsCollector(LocalCluster localCluster, String topoName, Map stormConfig) {
-        this(topoName, stormConfig);
-        this.client = null;
-        this.localCluster = localCluster;
-    }
-
-    private BasicMetricsCollector(String topoName, Map stormConfig) {
+    
+    public BasicMetricsCollector(String topoName, Map<String, Object> stormConfig) {
         Set<MetricsItem> items = getMetricsToCollect();
         this.config = new MetricsCollectorConfig(topoName, stormConfig);
         collectTopologyStats = collectTopologyStats(items);
@@ -104,14 +88,13 @@ public class BasicMetricsCollector  {
         dataWriter = new PrintWriter(System.err);
     }
 
-
     private Set<MetricsItem>  getMetricsToCollect() {
         Set<MetricsItem> result = new HashSet<>();
         result.add(MetricsItem.ALL);
         return result;
     }
 
-    public void collect(Nimbus.Client client) {
+    public void collect(Nimbus.Iface client) {
         try {
             if (!first) {
                 this.lastSample = this.curSample;
@@ -130,25 +113,7 @@ public class BasicMetricsCollector  {
         }
     }
 
-    public void collect(LocalCluster localCluster) {
-        try {
-            if (!first) {
-                this.lastSample = this.curSample;
-                this.curSample = MetricsSample.factory(localCluster, config.name);
-                updateStats(dataWriter);
-                writeLine(dataWriter);
-            } else {
-                LOG.info("Getting baseline metrics sample.");
-                writeHeader(dataWriter);
-                this.curSample = MetricsSample.factory(localCluster, config.name);
-                first = false;
-                startTime = System.currentTimeMillis();
-            }
-        } catch (Exception e) {
-            LOG.error("storm metrics failed! ", e);
-        }
-    }
-
+    @Override
     public void close() {
         dataWriter.close();
     }
@@ -287,13 +252,13 @@ public class BasicMetricsCollector  {
         private static final Logger LOG = Logger.getLogger(MetricsCollectorConfig.class);
 
         // storm configuration
-        public final Map stormConfig;
+        public final Map<String, Object> stormConfig;
         // storm topology name
         public final String name;
         // benchmark label
         public final String label;
 
-        public MetricsCollectorConfig(String topoName, Map stormConfig) {
+        public MetricsCollectorConfig(String topoName, Map<String, Object> stormConfig) {
             this.stormConfig = stormConfig;
             String labelStr = (String) stormConfig.get("benchmark.label");
             this.name = topoName;

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-perf/src/main/java/org/apache/storm/perf/utils/Helper.java
----------------------------------------------------------------------
diff --git a/examples/storm-perf/src/main/java/org/apache/storm/perf/utils/Helper.java b/examples/storm-perf/src/main/java/org/apache/storm/perf/utils/Helper.java
index 465ff33..d26078a 100755
--- a/examples/storm-perf/src/main/java/org/apache/storm/perf/utils/Helper.java
+++ b/examples/storm-perf/src/main/java/org/apache/storm/perf/utils/Helper.java
@@ -18,39 +18,23 @@
 
 package org.apache.storm.perf.utils;
 
-import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
+import java.util.Map;
+
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.generated.KillOptions;
 import org.apache.storm.generated.Nimbus;
 import org.apache.storm.generated.StormTopology;
-import org.apache.storm.utils.Utils;
-import org.apache.storm.utils.ObjectReader;
 import org.apache.storm.utils.NimbusClient;
-
-import java.util.Map;
+import org.apache.storm.utils.ObjectReader;
+import org.apache.storm.utils.Utils;
 
 
 public class Helper {
 
-  public static void kill(Nimbus.Client client, String topoName) throws Exception {
-    KillOptions opts = new KillOptions();
-    opts.set_wait_secs(0);
-    client.killTopologyWithOpts(topoName, opts);
-  }
-
-  public static void killAndShutdownCluster(LocalCluster cluster, String topoName) throws Exception {
-    KillOptions opts = new KillOptions();
-    opts.set_wait_secs(0);
-    cluster.killTopologyWithOpts(topoName, opts);
-    cluster.shutdown();
-  }
-
-
-    public static LocalCluster runOnLocalCluster(String topoName, StormTopology topology) throws Exception {
-        LocalCluster cluster = new LocalCluster();
-        cluster.submitTopology(topoName, new Config(), topology);
-        return cluster;
+    public static void kill(Nimbus.Iface client, String topoName) throws Exception {
+        KillOptions opts = new KillOptions();
+        opts.set_wait_secs(0);
+        client.killTopologyWithOpts(topoName, opts);
     }
 
     public static int getInt(Map map, Object key, int def) {
@@ -61,72 +45,52 @@ public class Helper {
         return (String) map.get(key);
     }
 
-    public static void collectMetricsAndKill(String topologyName, Integer pollInterval, Integer duration) throws Exception {
-        Map clusterConf = Utils.readStormConfig();
-        Nimbus.Client client = NimbusClient.getConfiguredClient(clusterConf).getClient();
-        BasicMetricsCollector metricsCollector = new BasicMetricsCollector(client, topologyName, clusterConf);
-
-        int times = duration / pollInterval;
-        metricsCollector.collect(client);
-        for (int i = 0; i < times; i++) {
-            Thread.sleep(pollInterval * 1000);
-            metricsCollector.collect(client);
+    public static void collectMetricsAndKill(String topologyName, Integer pollInterval, int duration) throws Exception {
+        Map<String, Object> clusterConf = Utils.readStormConfig();
+        Nimbus.Iface client = NimbusClient.getConfiguredClient(clusterConf).getClient();
+        try (BasicMetricsCollector metricsCollector = new BasicMetricsCollector(topologyName, clusterConf)) {
+
+            if (duration > 0) {
+                int times = duration / pollInterval;
+                metricsCollector.collect(client);
+                for (int i = 0; i < times; i++) {
+                    Thread.sleep(pollInterval * 1000);
+                    metricsCollector.collect(client);
+                }
+            } else {
+                while (true) { //until Ctrl-C
+                    metricsCollector.collect(client);
+                    Thread.sleep(pollInterval * 1000);
+                }
+            }
+        } finally {
+            kill(client, topologyName);
         }
-        metricsCollector.close();
-        kill(client, topologyName);
     }
 
-    public static void collectLocalMetricsAndKill(LocalCluster localCluster, String topologyName, Integer pollInterval, Integer duration, Map clusterConf) throws Exception {
-        BasicMetricsCollector metricsCollector = new BasicMetricsCollector(localCluster, topologyName, clusterConf);
-
-        int times = duration / pollInterval;
-        metricsCollector.collect(localCluster);
-        for (int i = 0; i < times; i++) {
-            Thread.sleep(pollInterval * 1000);
-            metricsCollector.collect(localCluster);
-        }
-        metricsCollector.close();
-        killAndShutdownCluster(localCluster, topologyName);
+    /** Kill topo on Ctrl-C */
+    public static void setupShutdownHook(final String topoName) {
+        Map clusterConf = Utils.readStormConfig();
+        final Nimbus.Iface client = NimbusClient.getConfiguredClient(clusterConf).getClient();
+        Runtime.getRuntime().addShutdownHook(new Thread() {
+            public void run() {
+                try {
+                    Helper.kill(client, topoName);
+                    System.out.println("Killed Topology");
+                } catch (Exception e) {
+                    e.printStackTrace();
+                }
+            }
+        });
     }
 
-    /** Kill topo and Shutdown local cluster on Ctrl-C */
-  public static void setupShutdownHook(final LocalCluster cluster, final String topoName) {
-    Runtime.getRuntime().addShutdownHook(new Thread() {
-      public void run() {
-          try {
-              cluster.killTopology(topoName);
-              System.out.println("Killed Topology");
-          } catch (Exception e) {
-              System.err.println("Encountered error in killing topology: " + e);
-          }
-        cluster.shutdown();
-      }
-    });
-  }
-
-  /** Kill topo on Ctrl-C */
-  public static void setupShutdownHook(final String topoName) {
-    Map clusterConf = Utils.readStormConfig();
-    final Nimbus.Client client = NimbusClient.getConfiguredClient(clusterConf).getClient();
-    Runtime.getRuntime().addShutdownHook(new Thread() {
-      public void run() {
-        try {
-          Helper.kill(client, topoName);
-          System.out.println("Killed Topology");
-        } catch (Exception e) {
-          e.printStackTrace();
-        }
-      }
-    });
-  }
-
-    public static void runOnClusterAndPrintMetrics(Integer durationSec, String topoName, Map topoConf, StormTopology topology) throws Exception {
-      // submit topology
-      StormSubmitter.submitTopologyWithProgressBar(topoName, topoConf, topology);
-      setupShutdownHook(topoName); // handle Ctrl-C
+    public static void runOnClusterAndPrintMetrics(int durationSec, String topoName, Map topoConf, StormTopology topology) throws Exception {
+        // submit topology
+        StormSubmitter.submitTopologyWithProgressBar(topoName, topoConf, topology);
+        setupShutdownHook(topoName); // handle Ctrl-C
 
-      // poll metrics every minute, then kill topology after specified duration
-      Integer pollIntervalSec = 60;
-      collectMetricsAndKill(topoName, pollIntervalSec, durationSec);
+        // poll metrics every minute, then kill topology after specified duration
+        Integer pollIntervalSec = 60;
+        collectMetricsAndKill(topoName, pollIntervalSec, durationSec);
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-perf/src/main/java/org/apache/storm/perf/utils/MetricsSample.java
----------------------------------------------------------------------
diff --git a/examples/storm-perf/src/main/java/org/apache/storm/perf/utils/MetricsSample.java b/examples/storm-perf/src/main/java/org/apache/storm/perf/utils/MetricsSample.java
index a934120..fbcea13 100755
--- a/examples/storm-perf/src/main/java/org/apache/storm/perf/utils/MetricsSample.java
+++ b/examples/storm-perf/src/main/java/org/apache/storm/perf/utils/MetricsSample.java
@@ -18,7 +18,9 @@
 
 package org.apache.storm.perf.utils;
 
-import org.apache.storm.LocalCluster;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.storm.generated.ClusterSummary;
 import org.apache.storm.generated.ExecutorSpecificStats;
 import org.apache.storm.generated.ExecutorStats;
@@ -29,9 +31,6 @@ import org.apache.storm.generated.TopologyInfo;
 import org.apache.storm.generated.TopologySummary;
 import org.apache.storm.utils.Utils;
 
-import java.util.List;
-import java.util.Map;
-
 public class MetricsSample {
 
     private long sampleTime = -1;
@@ -54,7 +53,7 @@ public class MetricsSample {
     private int totalSlots = 0;
     private int usedSlots = 0;
 
-    public static MetricsSample factory(Nimbus.Client client, String topologyName) throws Exception {
+    public static MetricsSample factory(Nimbus.Iface client, String topologyName) throws Exception {
         // "************ Sampling Metrics *****************
 
         ClusterSummary clusterSummary = client.getClusterInfo();
@@ -72,12 +71,6 @@ public class MetricsSample {
         return sample;
     }
 
-    public static MetricsSample factory(LocalCluster localCluster, String topologyName) throws Exception {
-        TopologyInfo topologyInfo = localCluster.getTopologyInfo(topologyName);;
-        return getMetricsSample(topologyInfo);
-    }
-
-
     private static MetricsSample getMetricsSample(TopologyInfo topInfo) {
         List<ExecutorSummary> executorSummaries = topInfo.get_executors();
 

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-pmml-examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-pmml-examples/pom.xml b/examples/storm-pmml-examples/pom.xml
index d202076..a1699d2 100644
--- a/examples/storm-pmml-examples/pom.xml
+++ b/examples/storm-pmml-examples/pom.xml
@@ -34,7 +34,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-server</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>
@@ -76,4 +76,4 @@
             </plugin>
         </plugins>
     </build>
-</project>
\ No newline at end of file
+</project>

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-pmml-examples/src/main/java/org/apache/storm/pmml/JpmmlRunnerTestTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-pmml-examples/src/main/java/org/apache/storm/pmml/JpmmlRunnerTestTopology.java b/examples/storm-pmml-examples/src/main/java/org/apache/storm/pmml/JpmmlRunnerTestTopology.java
index 1e5521e..c330f98 100644
--- a/examples/storm-pmml-examples/src/main/java/org/apache/storm/pmml/JpmmlRunnerTestTopology.java
+++ b/examples/storm-pmml-examples/src/main/java/org/apache/storm/pmml/JpmmlRunnerTestTopology.java
@@ -18,11 +18,15 @@
 
 package org.apache.storm.pmml;
 
-import com.google.common.collect.Lists;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.List;
 
 import org.apache.commons.compress.utils.IOUtils;
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.generated.StormTopology;
 import org.apache.storm.pmml.model.ModelOutputs;
@@ -36,14 +40,7 @@ import org.apache.storm.topology.base.BaseBasicBolt;
 import org.apache.storm.tuple.Tuple;
 import org.apache.storm.utils.Utils;
 
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.util.Arrays;
-import java.util.List;
+import com.google.common.collect.Lists;
 
 /**
  * Topology that loads a PMML Model and raw input data from a CSV file. The {@link RawInputFromCSVSpout}
@@ -66,8 +63,7 @@ public class JpmmlRunnerTestTopology {
     private File rawInputs;           // Raw input data to be scored (predicted)
     private File pmml;                // PMML Model read from file - null if using Blobstore
     private String blobKey;           // PMML Model downloaded from Blobstore - null if using File
-    private boolean isLocal;
-    private String tplgyName;
+    private String tplgyName = "test";
 
     public static void main(String[] args) throws Exception {
         try {
@@ -89,7 +85,6 @@ public class JpmmlRunnerTestTopology {
             printUsage();
         } else {
             try {
-                isLocal = true;
                 for (int i = 0; i < args.length; ) {
                     switch (args[i]) {
                         case "-f":
@@ -106,7 +101,6 @@ public class JpmmlRunnerTestTopology {
                             break;
                         default:
                             tplgyName = args[i];
-                            isLocal = false;
                             i++;
                             break;
                     }
@@ -155,11 +149,7 @@ public class JpmmlRunnerTestTopology {
     private void run() throws Exception {
         System.out.println(String.format("Running topology using PMML model loaded from [%s] and raw input data loaded from [%s]",
                 blobKey != null ? "Blobstore with blob key [" + blobKey + "]" : pmml.getAbsolutePath(), rawInputs.getAbsolutePath()));
-        if (isLocal) {
-            submitTopologyLocalCluster(newTopology(), newConfig());
-        } else {
-            submitTopologyRemoteCluster(newTopology(), newConfig());
-        }
+        submitTopologyRemoteCluster(newTopology(), newConfig());
     }
 
     private StormTopology newTopology() throws Exception {
@@ -171,26 +161,10 @@ public class JpmmlRunnerTestTopology {
         return builder.createTopology();
     }
 
-    private void submitTopologyLocalCluster(StormTopology topology, Config config) throws Exception {
-        LocalCluster cluster = new LocalCluster();
-        cluster.submitTopology(tplgyName, config, topology);
-        stopWaitingForInput();
-    }
-
     private void submitTopologyRemoteCluster(StormTopology topology, Config config) throws Exception {
         StormSubmitter.submitTopology(tplgyName, config, topology);
     }
 
-    private void stopWaitingForInput() {
-        try {
-            System.out.println("PRESS ENTER TO STOP");
-            new BufferedReader(new InputStreamReader(System.in)).readLine();
-            System.exit(0);
-        } catch (IOException e) {
-            e.printStackTrace();
-        }
-    }
-
     private Config newConfig() {
         Config config = new Config();
         config.setDebug(true);

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-redis-examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-redis-examples/pom.xml b/examples/storm-redis-examples/pom.xml
index 9317daf..d03d8b7 100644
--- a/examples/storm-redis-examples/pom.xml
+++ b/examples/storm-redis-examples/pom.xml
@@ -30,7 +30,7 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.storm</groupId>
-            <artifactId>storm-server</artifactId>
+            <artifactId>storm-client</artifactId>
             <version>${project.version}</version>
             <scope>${provided.scope}</scope>
         </dependency>

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/topology/LookupWordCount.java
----------------------------------------------------------------------
diff --git a/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/topology/LookupWordCount.java b/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/topology/LookupWordCount.java
index 17a088b..43798d0 100644
--- a/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/topology/LookupWordCount.java
+++ b/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/topology/LookupWordCount.java
@@ -17,10 +17,16 @@
  */
 package org.apache.storm.redis.topology;
 
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
+import org.apache.storm.redis.bolt.RedisLookupBolt;
+import org.apache.storm.redis.common.config.JedisPoolConfig;
+import org.apache.storm.redis.common.mapper.RedisDataTypeDescription;
+import org.apache.storm.redis.common.mapper.RedisLookupMapper;
 import org.apache.storm.task.OutputCollector;
 import org.apache.storm.task.TopologyContext;
 import org.apache.storm.topology.OutputFieldsDeclarer;
@@ -30,18 +36,10 @@ import org.apache.storm.tuple.Fields;
 import org.apache.storm.tuple.ITuple;
 import org.apache.storm.tuple.Tuple;
 import org.apache.storm.tuple.Values;
-import com.google.common.collect.Lists;
-import org.apache.storm.redis.bolt.RedisLookupBolt;
-import org.apache.storm.redis.common.config.JedisPoolConfig;
-import org.apache.storm.redis.common.mapper.RedisDataTypeDescription;
-import org.apache.storm.redis.common.mapper.RedisLookupMapper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
+import com.google.common.collect.Lists;
 
 public class LookupWordCount {
     private static final String WORD_SPOUT = "WORD_SPOUT";
@@ -109,17 +107,14 @@ public class LookupWordCount {
         builder.setBolt(LOOKUP_BOLT, lookupBolt, 1).shuffleGrouping(WORD_SPOUT);
         builder.setBolt(PRINT_BOLT, printBolt, 1).shuffleGrouping(LOOKUP_BOLT);
 
-        if (args.length == 2) {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology("test", config, builder.createTopology());) {
-                Thread.sleep(30000);
-            }
-            System.exit(0);
-        } else if (args.length == 3) {
-            StormSubmitter.submitTopology(args[2], config, builder.createTopology());
-        } else{
+        String topoName = "test";
+        if (args.length == 3) {
+            topoName = args[2];
+        } else if (args.length > 3) {
             System.out.println("Usage: LookupWordCount <redis host> <redis port> (topology name)");
+            return;
         }
+        StormSubmitter.submitTopology(topoName, config, builder.createTopology());
     }
 
     private static RedisLookupMapper setupLookupMapper() {

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/topology/PersistentWordCount.java
----------------------------------------------------------------------
diff --git a/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/topology/PersistentWordCount.java b/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/topology/PersistentWordCount.java
index 36cec89..4c7b0dc 100644
--- a/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/topology/PersistentWordCount.java
+++ b/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/topology/PersistentWordCount.java
@@ -18,25 +18,14 @@
 package org.apache.storm.redis.topology;
 
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.topology.TopologyBuilder;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.ITuple;
-import org.apache.storm.tuple.Tuple;
-import org.apache.storm.redis.bolt.AbstractRedisBolt;
 import org.apache.storm.redis.bolt.RedisStoreBolt;
-import org.apache.storm.redis.common.config.JedisClusterConfig;
 import org.apache.storm.redis.common.config.JedisPoolConfig;
 import org.apache.storm.redis.common.mapper.RedisDataTypeDescription;
 import org.apache.storm.redis.common.mapper.RedisStoreMapper;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import redis.clients.jedis.JedisCommands;
-import redis.clients.jedis.exceptions.JedisConnectionException;
-import redis.clients.jedis.exceptions.JedisException;
+import org.apache.storm.topology.TopologyBuilder;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.ITuple;
 
 public class PersistentWordCount {
     private static final String WORD_SPOUT = "WORD_SPOUT";
@@ -72,17 +61,14 @@ public class PersistentWordCount {
         builder.setBolt(COUNT_BOLT, bolt, 1).fieldsGrouping(WORD_SPOUT, new Fields("word"));
         builder.setBolt(STORE_BOLT, storeBolt, 1).shuffleGrouping(COUNT_BOLT);
 
-        if (args.length == 2) {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology("test", config, builder.createTopology());) {
-                Thread.sleep(30000);
-            }
-            System.exit(0);
-        } else if (args.length == 3) {
-            StormSubmitter.submitTopology(args[2], config, builder.createTopology());
-        } else {
+        String topoName = "test";
+        if (args.length == 3) {
+            topoName = args[2];
+        } else if (args.length > 3) {
             System.out.println("Usage: PersistentWordCount <redis host> <redis port> (topology name)");
+            return;
         }
+        StormSubmitter.submitTopology(topoName, config, builder.createTopology());
     }
 
     private static RedisStoreMapper setupStoreMapper() {

http://git-wip-us.apache.org/repos/asf/storm/blob/b254ede4/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/topology/WhitelistWordCount.java
----------------------------------------------------------------------
diff --git a/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/topology/WhitelistWordCount.java b/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/topology/WhitelistWordCount.java
index 1cccd9b..4ac410a 100644
--- a/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/topology/WhitelistWordCount.java
+++ b/examples/storm-redis-examples/src/main/java/org/apache/storm/redis/topology/WhitelistWordCount.java
@@ -17,9 +17,10 @@
  */
 package org.apache.storm.redis.topology;
 
+import java.util.Map;
+import java.util.Random;
+
 import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.StormSubmitter;
 import org.apache.storm.redis.bolt.RedisFilterBolt;
 import org.apache.storm.redis.common.config.JedisPoolConfig;
@@ -36,9 +37,6 @@ import org.apache.storm.tuple.Tuple;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.Map;
-import java.util.Random;
-
 public class WhitelistWordCount {
     private static final String WORD_SPOUT = "WORD_SPOUT";
     private static final String WHITELIST_BOLT = "WHITELIST_BOLT";
@@ -106,17 +104,14 @@ public class WhitelistWordCount {
         builder.setBolt(COUNT_BOLT, wordCounterBolt, 1).fieldsGrouping(WHITELIST_BOLT, new Fields("word"));
         builder.setBolt(PRINT_BOLT, printBolt, 1).shuffleGrouping(COUNT_BOLT);
 
-        if (args.length == 2) {
-            try (LocalCluster cluster = new LocalCluster();
-                 LocalTopology topo = cluster.submitTopology("test", config, builder.createTopology());) {
-                Thread.sleep(30000);
-            }
-            System.exit(0);
-        } else if (args.length == 3) {
-            StormSubmitter.submitTopology(args[2], config, builder.createTopology());
-        } else{
-            System.out.println("Usage: WhitelistWordCount <redis host> <redis port> (topology name)");
+        String topoName = "test";
+        if (args.length == 3) {
+            topoName = args[2];
+        } else if (args.length > 3) {
+            System.out.println("Usage: WhitelistWordCount <redis host> <redis port> [topology name]");
+            return;
         }
+        StormSubmitter.submitTopology(topoName, config, builder.createTopology());
     }
 
     private static RedisFilterMapper setupWhitelistMapper() {


[07/10] storm git commit: STORM-2447: Fixed compilation issue with int test

Posted by ka...@apache.org.
STORM-2447: Fixed compilation issue with int test


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

Branch: refs/heads/master
Commit: 9eced5310464f87ca52eb92b6224ef5abe531514
Parents: b254ede
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Thu Apr 6 14:50:14 2017 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Thu Apr 6 14:50:14 2017 -0500

----------------------------------------------------------------------
 .../java/org/apache/storm/ExclamationTopology.java  | 16 ++++++----------
 .../org/apache/storm/st/wrapper/StormCluster.java   |  4 ++--
 2 files changed, 8 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/9eced531/integration-test/src/main/java/org/apache/storm/ExclamationTopology.java
----------------------------------------------------------------------
diff --git a/integration-test/src/main/java/org/apache/storm/ExclamationTopology.java b/integration-test/src/main/java/org/apache/storm/ExclamationTopology.java
index 79f2ba3..372ee56 100644
--- a/integration-test/src/main/java/org/apache/storm/ExclamationTopology.java
+++ b/integration-test/src/main/java/org/apache/storm/ExclamationTopology.java
@@ -18,7 +18,6 @@
 package org.apache.storm;
 
 import org.apache.storm.generated.StormTopology;
-import org.apache.storm.LocalCluster.LocalTopology;
 import org.apache.storm.task.OutputCollector;
 import org.apache.storm.task.TopologyContext;
 import org.apache.storm.testing.TestWordSpout;
@@ -66,16 +65,13 @@ public class ExclamationTopology {
 
     Config conf = new Config();
     conf.setDebug(true);
-
-    if (args != null && args.length > 0) {
-      conf.setNumWorkers(3);
-      StormSubmitter.submitTopologyWithProgressBar(args[0], conf, topology);
-    } else {
-      try (LocalCluster cluster = new LocalCluster();
-           LocalTopology topo = cluster.submitTopology("test", conf, topology);) {
-        Utils.sleep(10000);
-      }
+    String topoName = "test";
+    if (args.length > 0) {
+        topoName = args[0];
     }
+
+    conf.setNumWorkers(3);
+    StormSubmitter.submitTopologyWithProgressBar(topoName, conf, topology);
   }
 
   public static StormTopology getStormTopology() {

http://git-wip-us.apache.org/repos/asf/storm/blob/9eced531/integration-test/src/test/java/org/apache/storm/st/wrapper/StormCluster.java
----------------------------------------------------------------------
diff --git a/integration-test/src/test/java/org/apache/storm/st/wrapper/StormCluster.java b/integration-test/src/test/java/org/apache/storm/st/wrapper/StormCluster.java
index f97a259..6d6f940 100644
--- a/integration-test/src/test/java/org/apache/storm/st/wrapper/StormCluster.java
+++ b/integration-test/src/test/java/org/apache/storm/st/wrapper/StormCluster.java
@@ -41,7 +41,7 @@ import java.util.Map;
 
 public class StormCluster {
     private static Logger log = LoggerFactory.getLogger(StormCluster.class);
-    private final Nimbus.Client client;
+    private final Nimbus.Iface client;
 
     public StormCluster() {
         Map conf = getConfig();
@@ -103,7 +103,7 @@ public class StormCluster {
         return client.getTopologyInfo(topologySummary.get_id());
     }
 
-    public Nimbus.Client getNimbusClient() {
+    public Nimbus.Iface getNimbusClient() {
         return client;
     }
 


[08/10] storm git commit: Merge branch 'apache-master' into STORM-2447

Posted by ka...@apache.org.
Merge branch 'apache-master' into STORM-2447


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

Branch: refs/heads/master
Commit: 71dc2e019d84580fbcdd5eabf267016393324872
Parents: 9eced53 1850dd5
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Fri Apr 7 08:31:56 2017 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Fri Apr 7 08:31:56 2017 -0500

----------------------------------------------------------------------
 CHANGELOG.md                                    |  2 +
 docs/storm-hbase.md                             | 46 +++++++++++++++++---
 external/storm-hbase/README.md                  | 38 +++++++++++-----
 .../storm/kafka/spout/KafkaSpoutConfig.java     |  2 +-
 4 files changed, 69 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/71dc2e01/docs/storm-hbase.md
----------------------------------------------------------------------
diff --cc docs/storm-hbase.md
index 7f71346,e873f31..1710e1b
--- a/docs/storm-hbase.md
+++ b/docs/storm-hbase.md
@@@ -223,12 -248,25 +248,21 @@@ public class PersistentWordCount 
          builder.setBolt(COUNT_BOLT, bolt, 1).shuffleGrouping(WORD_SPOUT);
          builder.setBolt(HBASE_BOLT, hbase, 1).fieldsGrouping(COUNT_BOLT, new Fields("word"));
  
 -
 -        if (args.length == 1) {
 -            try (LocalCluster cluster = new LocalCluster();
 -                 LocalTopology topo = cluster.submitTopology("test", config, builder.createTopology());) {
 -                Thread.sleep(30000);
 -            }
 -            System.exit(0);
 -        } else if (args.length == 2) {
 -            StormSubmitter.submitTopology(args[1], config, builder.createTopology());
 -        } else if (args.length == 4) {
 +        String topoName = "test";
-         if (args.length > 0) {
-             topoName = args[0];
++        if (args.length > 1) {
++            topoName = args[1];
++        }
++        if (args.length == 4) {
+             System.out.println("hdfs url: " + args[0] + ", keytab file: " + args[2] + 
 -                ", principal name: " + args[3] + ", toplogy name: " + args[1]);
++                ", principal name: " + args[3] + ", toplogy name: " + topoName);
+             hbConf.put(HBaseSecurityUtil.STORM_KEYTAB_FILE_KEY, args[2]);
+             hbConf.put(HBaseSecurityUtil.STORM_USER_NAME_KEY, args[3]);
 -            config.setNumWorkers(3);
 -            StormSubmitter.submitTopology(args[1], config, builder.createTopology());
 -        } else {
++        } else if (args.length == 3 || args.length > 4) {
+             System.out.println("Usage: PersistentWordCount <hbase.rootdir> [topology name] [keytab file] [principal name]");
++            return;
          }
 +        config.setNumWorkers(3);
 +        StormSubmitter.submitTopology(topoName, config, builder.createTopology());
      }
  }
  ```