A native heron api example
diff --git a/heron/examples/src/java/BUILD b/heron/examples/src/java/BUILD
index c74ab26..d96b80d 100644
--- a/heron/examples/src/java/BUILD
+++ b/heron/examples/src/java/BUILD
@@ -6,6 +6,7 @@
     deps = [
         "//heron/common/src/java:basics-java",
         "//heron/storm/src/java:storm-compatibility-java",
+        "//heron/api/src/java:api-java",
     ],
     create_executable = 0,
 )
diff --git a/heron/examples/src/java/com/twitter/heron/examples/ExclamationTopology.java b/heron/examples/src/java/com/twitter/heron/examples/ExclamationTopology.java
index 3a53362..71c185f3 100644
--- a/heron/examples/src/java/com/twitter/heron/examples/ExclamationTopology.java
+++ b/heron/examples/src/java/com/twitter/heron/examples/ExclamationTopology.java
@@ -15,21 +15,23 @@
 package com.twitter.heron.examples;
 
 import java.util.Map;
+import java.util.Random;
 
+import com.twitter.heron.api.Config;
+import com.twitter.heron.api.HeronSubmitter;
+import com.twitter.heron.api.bolt.BaseRichBolt;
+import com.twitter.heron.api.bolt.OutputCollector;
+import com.twitter.heron.api.metric.GlobalMetrics;
+import com.twitter.heron.api.spout.BaseRichSpout;
+import com.twitter.heron.api.spout.SpoutOutputCollector;
+import com.twitter.heron.api.topology.OutputFieldsDeclarer;
+import com.twitter.heron.api.topology.TopologyBuilder;
+import com.twitter.heron.api.topology.TopologyContext;
+import com.twitter.heron.api.tuple.Fields;
+import com.twitter.heron.api.tuple.Tuple;
+import com.twitter.heron.api.tuple.Values;
 import com.twitter.heron.common.basics.ByteAmount;
 
-import backtype.storm.Config;
-import backtype.storm.LocalCluster;
-import backtype.storm.StormSubmitter;
-import backtype.storm.metric.api.GlobalMetrics;
-import backtype.storm.task.OutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.topology.TopologyBuilder;
-import backtype.storm.topology.base.BaseRichBolt;
-import backtype.storm.tuple.Tuple;
-import backtype.storm.utils.Utils;
-
 /**
  * This is a basic example of a Storm topology.
  */
@@ -55,16 +57,43 @@
     conf.setContainerDiskRequested(ByteAmount.fromGigabytes(5));
     conf.setContainerCpuRequested(5);
 
-    if (args != null && args.length > 0) {
-      conf.setNumStmgrs(parallelism);
-      StormSubmitter.submitTopology(args[0], conf, builder.createTopology());
-    } else {
-      System.out.println("Topology name not provided as an argument, running in simulator mode.");
-      LocalCluster cluster = new LocalCluster();
-      cluster.submitTopology("test", conf, builder.createTopology());
-      Utils.sleep(10000);
-      cluster.killTopology("test");
-      cluster.shutdown();
+    conf.setNumStmgrs(parallelism);
+    HeronSubmitter.submitTopology(args[0], conf, builder.createTopology());
+  }
+
+  public static class TestWordSpout extends BaseRichSpout {
+
+    private static final long serialVersionUID = -3217886193225455451L;
+    private SpoutOutputCollector collector;
+    private String[] words;
+    private Random rand;
+
+    @SuppressWarnings("rawtypes")
+    public void open(
+        Map conf,
+        TopologyContext context,
+        SpoutOutputCollector acollector) {
+      collector = acollector;
+      words = new String[]{"nathan", "mike", "jackson", "golda", "bertels"};
+      rand = new Random();
+    }
+
+    public void close() {
+    }
+
+    public void nextTuple() {
+      final String word = words[rand.nextInt(words.length)];
+      collector.emit(new Values(word));
+    }
+
+    public void ack(Object msgId) {
+    }
+
+    public void fail(Object msgId) {
+    }
+
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+      declarer.declare(new Fields("word"));
     }
   }