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/05 23:19:04 UTC

[01/23] storm git commit: STORM-2453 Move non-connectors into the top directory

Repository: storm
Updated Branches:
  refs/heads/master e84b39d1e -> c652d3ffc


http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/SerdeUtils.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/SerdeUtils.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/SerdeUtils.java
new file mode 100644
index 0000000..6b3dfc9
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/SerdeUtils.java
@@ -0,0 +1,123 @@
+/*
+ * 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.sql.runtime.utils;
+
+import static org.apache.commons.lang.StringUtils.isNotEmpty;
+
+import com.google.common.base.Preconditions;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.util.Utf8;
+import org.apache.storm.spout.Scheme;
+import org.apache.storm.sql.runtime.IOutputSerializer;
+import org.apache.storm.sql.runtime.serde.avro.AvroScheme;
+import org.apache.storm.sql.runtime.serde.avro.AvroSerializer;
+import org.apache.storm.sql.runtime.serde.csv.CsvScheme;
+import org.apache.storm.sql.runtime.serde.csv.CsvSerializer;
+import org.apache.storm.sql.runtime.serde.json.JsonScheme;
+import org.apache.storm.sql.runtime.serde.json.JsonSerializer;
+import org.apache.storm.sql.runtime.serde.tsv.TsvScheme;
+import org.apache.storm.sql.runtime.serde.tsv.TsvSerializer;
+import org.apache.storm.utils.Utils;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+public final class SerdeUtils {
+    public static Scheme getScheme(String inputFormatClass, Properties properties, List<String> fieldNames) {
+        Scheme scheme;
+        if (isNotEmpty(inputFormatClass)) {
+            if (JsonScheme.class.getName().equals(inputFormatClass)) {
+                scheme = new JsonScheme(fieldNames);
+            } else if (TsvScheme.class.getName().equals(inputFormatClass)) {
+                String delimiter = properties.getProperty("input.tsv.delimiter", "\t");
+                scheme = new TsvScheme(fieldNames, delimiter.charAt(0));
+            } else if (CsvScheme.class.getName().equals(inputFormatClass)) {
+                scheme = new CsvScheme(fieldNames);
+            } else if (AvroScheme.class.getName().equals(inputFormatClass)) {
+                String schemaString = properties.getProperty("input.avro.schema");
+                Preconditions.checkArgument(isNotEmpty(schemaString), "input.avro.schema can not be empty");
+                scheme = new AvroScheme(schemaString, fieldNames);
+            } else {
+                scheme = Utils.newInstance(inputFormatClass);
+            }
+        } else {
+            //use JsonScheme as the default scheme
+            scheme = new JsonScheme(fieldNames);
+        }
+        return scheme;
+    }
+
+    public static IOutputSerializer getSerializer(String outputFormatClass, Properties properties, List<String> fieldNames) {
+        IOutputSerializer serializer;
+        if (isNotEmpty(outputFormatClass)) {
+            if (JsonSerializer.class.getName().equals(outputFormatClass)) {
+                serializer = new JsonSerializer(fieldNames);
+            } else if (TsvSerializer.class.getName().equals(outputFormatClass)) {
+                String delimiter = properties.getProperty("output.tsv.delimiter", "\t");
+                serializer = new TsvSerializer(fieldNames, delimiter.charAt(0));
+            } else if (CsvSerializer.class.getName().equals(outputFormatClass)) {
+                serializer = new CsvSerializer(fieldNames);
+            } else if (AvroSerializer.class.getName().equals(outputFormatClass)) {
+                String schemaString = properties.getProperty("output.avro.schema");
+                Preconditions.checkArgument(isNotEmpty(schemaString), "output.avro.schema can not be empty");
+                serializer = new AvroSerializer(schemaString, fieldNames);
+            } else {
+                serializer = Utils.newInstance(outputFormatClass);
+            }
+        } else {
+            //use JsonSerializer as the default serializer
+            serializer = new JsonSerializer(fieldNames);
+        }
+        return serializer;
+    }
+
+    public static Object convertAvroUtf8(Object value){
+        Object ret;
+        if (value instanceof Utf8) {
+            ret = value.toString();
+        } else if (value instanceof Map<?, ?>) {
+            ret = convertAvroUtf8Map((Map<Object,Object>)value);
+        } else if (value instanceof GenericData.Array) {
+            ret = convertAvroUtf8Array((GenericData.Array)value);
+        } else {
+            ret = value;
+        }
+        return ret;
+    }
+
+    public static Object convertAvroUtf8Map(Map<Object,Object> value) {
+        Map<Object, Object> map = new HashMap<>(value.size());
+        for (Map.Entry<Object, Object> entry : value.entrySet()) {
+            Object k = convertAvroUtf8(entry.getKey());
+            Object v = convertAvroUtf8(entry.getValue());
+            map.put(k, v);
+        }
+        return map;
+    }
+
+    public static Object convertAvroUtf8Array(GenericData.Array value){
+        List<Object> ls = new ArrayList<>(value.size());
+        for(Object o : value){
+            ls.add(convertAvroUtf8(o));
+        }
+        return ls;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/Utils.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/Utils.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/Utils.java
new file mode 100644
index 0000000..a0f3af3
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/Utils.java
@@ -0,0 +1,55 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.runtime.utils;
+
+import java.util.LinkedList;
+import java.util.List;
+
+public final class Utils {
+
+    /**
+     * This method for splitting string into parts by a delimiter
+     * It has higher performance than String.split(String regex)
+     *
+     * @param data
+     * @param delimiter
+     * @return
+     */
+    public static List<String> split(String data, char delimiter){
+        List<String> list = new LinkedList<>();
+        //do not use .toCharArray avoid system copy
+        StringBuilder sb = new StringBuilder(512);
+        int len = data.length();
+        for (int i=0; i < len; i++) {
+            char ch = data.charAt(i);
+            if(ch == delimiter){
+                list.add(sb.toString());
+                sb.setLength(0);
+                if(i == len - 1){
+                    list.add("");
+                }
+            }else{
+                sb.append(ch);
+            }
+        }
+        if (sb.length() > 0) {
+            list.add(sb.toString());
+        }
+        return list;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider b/sql/storm-sql-runtime/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
new file mode 100644
index 0000000..9a945f7
--- /dev/null
+++ b/sql/storm-sql-runtime/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
@@ -0,0 +1,16 @@
+# 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.
+
+org.apache.storm.sql.runtime.datasource.socket.SocketDataSourcesProvider
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestAvroSerializer.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestAvroSerializer.java b/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestAvroSerializer.java
new file mode 100644
index 0000000..174bfde
--- /dev/null
+++ b/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestAvroSerializer.java
@@ -0,0 +1,72 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.storm.sql.runtime.serde.avro.AvroScheme;
+import org.apache.storm.sql.runtime.serde.avro.AvroSerializer;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertArrayEquals;
+
+public class TestAvroSerializer {
+  private static final String schemaString = "{\"type\":\"record\"," +
+          "\"name\":\"avrotest\"," +
+          "\"fields\":[{\"name\":\"ID\",\"type\":\"int\"}," +
+          "{ \"name\":\"val\", \"type\":\"string\" }]}";
+
+  private static final String schemaString1 = "{\"type\":\"record\"," +
+          "\"name\":\"avrotest\"," +
+          "\"fields\":[{\"name\":\"ID\",\"type\":\"int\"}," +
+          "{ \"type\":{\"type\":\"map\",\"values\": \"long\"}, \"name\":\"val1\" }," +
+          "{ \"type\":{\"type\":\"array\",\"items\": \"string\"}, \"name\":\"val2\" }]}";
+
+  @Test
+  public void testAvroSchemeAndSerializer() {
+    List<String> fields = Lists.newArrayList("ID", "val");
+    List<Object> o = Lists.newArrayList(1, "2");
+
+    AvroSerializer serializer = new AvroSerializer(schemaString, fields);
+    ByteBuffer byteBuffer = serializer.write(o, null);
+
+    AvroScheme scheme = new AvroScheme(schemaString, fields);
+    assertArrayEquals(o.toArray(), scheme.deserialize(byteBuffer).toArray());
+  }
+
+  @Test
+  public void testAvroComplexSchemeAndSerializer() {
+    List<String> fields = Lists.newArrayList("ID", "val1", "val2");
+
+    Map<String,Long> mp = Maps.newHashMap();;
+    mp.put("l1",1234L);
+    mp.put("l2",56789L);
+    List<String> ls = Lists.newArrayList("s1", "s2");
+    List<Object> o = Lists.newArrayList(1, mp, ls);
+
+    AvroSerializer serializer = new AvroSerializer(schemaString1, fields);
+    ByteBuffer byteBuffer = serializer.write(o, null);
+
+    AvroScheme scheme = new AvroScheme(schemaString1, fields);
+    assertArrayEquals(o.toArray(), scheme.deserialize(byteBuffer).toArray());
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestCsvSerializer.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestCsvSerializer.java b/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestCsvSerializer.java
new file mode 100644
index 0000000..0108949
--- /dev/null
+++ b/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestCsvSerializer.java
@@ -0,0 +1,54 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql;
+
+import com.google.common.collect.Lists;
+import org.apache.storm.sql.runtime.serde.csv.CsvScheme;
+import org.apache.storm.sql.runtime.serde.csv.CsvSerializer;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import static org.junit.Assert.assertArrayEquals;
+
+public class TestCsvSerializer {
+
+  @Test
+  public void testCsvSchemeAndSerializer() {
+    List<String> fields = Lists.newArrayList("ID", "val");
+    List<Object> o = Lists.newArrayList("1", "2");
+
+    CsvSerializer serializer = new CsvSerializer(fields);
+    ByteBuffer byteBuffer = serializer.write(o, null);
+
+    CsvScheme scheme = new CsvScheme(fields);
+    assertArrayEquals(o.toArray(), scheme.deserialize(byteBuffer).toArray());
+
+    // Fields with embedded commas or double-quote characters
+    fields = Lists.newArrayList("ID", "val", "v");
+    o = Lists.newArrayList("1,9", "2,\"3\",5", "\"8\"");
+
+    serializer = new CsvSerializer(fields);
+    byteBuffer = serializer.write(o, null);
+
+    scheme = new CsvScheme(fields);
+    assertArrayEquals(o.toArray(), scheme.deserialize(byteBuffer).toArray());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestJsonRepresentation.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestJsonRepresentation.java b/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestJsonRepresentation.java
new file mode 100644
index 0000000..6ca1877
--- /dev/null
+++ b/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestJsonRepresentation.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql;
+
+import org.apache.storm.sql.runtime.serde.json.JsonScheme;
+import org.apache.storm.sql.runtime.serde.json.JsonSerializer;
+import org.apache.storm.utils.Utils;
+import com.google.common.collect.Lists;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+import java.util.List;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+public class TestJsonRepresentation {
+  @Test
+  public void testJsonScheme() {
+    final List<String> fields = Lists.newArrayList("ID", "val");
+    final String s = "{\"ID\": 1, \"val\": \"2\"}";
+    JsonScheme scheme = new JsonScheme(fields);
+    List<Object> o = scheme.deserialize(ByteBuffer.wrap(s.getBytes(Charset.defaultCharset())));
+    assertArrayEquals(new Object[] {1, "2"}, o.toArray());
+  }
+
+  @Test
+  public void testJsonSerializer() {
+    final List<String> fields = Lists.newArrayList("ID", "val");
+    List<Object> o = Lists.<Object> newArrayList(1, "2");
+    JsonSerializer s = new JsonSerializer(fields);
+    ByteBuffer buf = s.write(o, null);
+    byte[] b = Utils.toByteArray(buf);
+    assertEquals("{\"ID\":1,\"val\":\"2\"}", new String(b));
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestTsvSerializer.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestTsvSerializer.java b/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestTsvSerializer.java
new file mode 100644
index 0000000..1798828
--- /dev/null
+++ b/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestTsvSerializer.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql;
+
+import com.google.common.collect.Lists;
+import org.apache.storm.sql.runtime.serde.tsv.TsvScheme;
+import org.apache.storm.sql.runtime.serde.tsv.TsvSerializer;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import static org.junit.Assert.assertArrayEquals;
+
+public class TestTsvSerializer {
+
+  @Test
+  public void testTsvSchemeAndSerializer() {
+    final char delimiter = '\t';
+
+    List<String> fields = Lists.newArrayList("ID", "val");
+    List<Object> o = Lists.newArrayList("1", "2");
+
+    TsvSerializer serializer = new TsvSerializer(fields, delimiter);
+    ByteBuffer byteBuffer = serializer.write(o, null);
+
+    TsvScheme scheme = new TsvScheme(fields, delimiter);
+    assertArrayEquals(o.toArray(), scheme.deserialize(byteBuffer).toArray());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestUtils.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestUtils.java b/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestUtils.java
new file mode 100644
index 0000000..0a3bac6
--- /dev/null
+++ b/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestUtils.java
@@ -0,0 +1,584 @@
+/*
+ * *
+ *  * 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
+ *  * <p>
+ *  * http://www.apache.org/licenses/LICENSE-2.0
+ *  * <p>
+ *  * 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.sql;
+
+import org.apache.storm.sql.runtime.ChannelContext;
+import org.apache.storm.sql.runtime.ChannelHandler;
+import org.apache.storm.sql.runtime.DataSource;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.sql.runtime.SimpleSqlTridentConsumer;
+import org.apache.storm.task.IMetricsContext;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.trident.operation.TridentCollector;
+import org.apache.storm.trident.operation.TridentOperationContext;
+import org.apache.storm.trident.spout.IBatchSpout;
+import org.apache.storm.trident.state.State;
+import org.apache.storm.trident.state.StateFactory;
+import org.apache.storm.trident.state.StateUpdater;
+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.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+
+public class TestUtils {
+  public static class MyPlus {
+    public static Integer evaluate(Integer x, Integer y) {
+      return x + y;
+    }
+  }
+
+  public static class MyConcat {
+    public static String init() {
+      return "";
+    }
+    public static String add(String accumulator, String val) {
+      return accumulator + val;
+    }
+    public static String result(String accumulator) {
+      return accumulator;
+    }
+  }
+
+
+  public static class TopN {
+    public static PriorityQueue<Integer> init() {
+      return new PriorityQueue<>();
+    }
+    public static PriorityQueue<Integer> add(PriorityQueue<Integer> accumulator, Integer n, Integer val) {
+      if (n <= 0) {
+        return accumulator;
+      }
+      if (accumulator.size() >= n) {
+        if (val > accumulator.peek()) {
+          accumulator.remove();
+          accumulator.add(val);
+        }
+      } else {
+        accumulator.add(val);
+      }
+      return accumulator;
+    }
+    public static List<Integer> result(PriorityQueue<Integer> accumulator) {
+      List<Integer> res = new ArrayList<>(accumulator);
+      Collections.reverse(res);
+      return res;
+    }
+  }
+
+
+  public static class MockDataSource implements DataSource {
+    private final ArrayList<Values> RECORDS = new ArrayList<>();
+
+    public MockDataSource() {
+      for (int i = 0; i < 5; ++i) {
+        RECORDS.add(new Values(i, "x", null));
+      }
+    }
+
+    @Override
+    public void open(ChannelContext ctx) {
+      for (Values v : RECORDS) {
+        ctx.emit(v);
+      }
+      ctx.fireChannelInactive();
+    }
+  }
+
+  public static class MockGroupDataSource implements DataSource {
+    private final ArrayList<Values> RECORDS = new ArrayList<>();
+
+    public MockGroupDataSource() {
+      for (int i = 0; i < 10; ++i) {
+        RECORDS.add(new Values(i/3, i, (i+1)* 0.5, "x", i/2));
+      }
+    }
+
+    @Override
+    public void open(ChannelContext ctx) {
+      for (Values v : RECORDS) {
+        ctx.emit(v);
+      }
+      // force evaluation of the aggregate function on the last group
+      ctx.flush();
+      ctx.fireChannelInactive();
+    }
+  }
+
+  public static class MockEmpDataSource implements DataSource {
+    private final ArrayList<Values> RECORDS = new ArrayList<>();
+
+    public MockEmpDataSource() {
+      RECORDS.add(new Values(1, "emp1", 1));
+      RECORDS.add(new Values(2, "emp2", 1));
+      RECORDS.add(new Values(3, "emp3", 2));
+    }
+
+    @Override
+    public void open(ChannelContext ctx) {
+      for (Values v : RECORDS) {
+        ctx.emit(v);
+      }
+      ctx.flush();
+      ctx.fireChannelInactive();
+    }
+  }
+
+  public static class MockDeptDataSource implements DataSource {
+    private final ArrayList<Values> RECORDS = new ArrayList<>();
+
+    public MockDeptDataSource() {
+      RECORDS.add(new Values(1, "dept1"));
+      RECORDS.add(new Values(2, "dept2"));
+      RECORDS.add(new Values(3, "dept3"));
+    }
+
+    @Override
+    public void open(ChannelContext ctx) {
+      for (Values v : RECORDS) {
+        ctx.emit(v);
+      }
+      ctx.flush();
+      ctx.fireChannelInactive();
+    }
+  }
+
+  public static class MockNestedDataSource implements DataSource {
+    private final ArrayList<Values> RECORDS = new ArrayList<>();
+
+    public MockNestedDataSource() {
+      List<Integer> ints = Arrays.asList(100, 200, 300);
+      for (int i = 0; i < 5; ++i) {
+        Map<String, Integer> map = new HashMap<>();
+        map.put("b", i);
+        map.put("c", i*i);
+        Map<String, Map<String, Integer>> mm = new HashMap<>();
+        mm.put("a", map);
+        RECORDS.add(new Values(i, map, mm, ints));
+      }
+    }
+
+    @Override
+    public void open(ChannelContext ctx) {
+      for (Values v : RECORDS) {
+        ctx.emit(v);
+      }
+      ctx.fireChannelInactive();
+    }
+  }
+
+  public static class MockState implements State {
+    /**
+     * Collect all values in a static variable as the instance will go through serialization and deserialization.
+     * NOTE: This should be cleared before or after running each test.
+     */
+    private transient static final List<List<Object> > VALUES = new ArrayList<>();
+
+    public static List<List<Object>> getCollectedValues() {
+      return VALUES;
+    }
+
+    @Override
+    public void beginCommit(Long txid) {
+      // NOOP
+    }
+
+    @Override
+    public void commit(Long txid) {
+      // NOOP
+    }
+
+    public void updateState(List<TridentTuple> tuples, TridentCollector collector) {
+      for (TridentTuple tuple : tuples) {
+        VALUES.add(tuple.getValues());
+      }
+    }
+  }
+
+  public static class MockStateFactory implements StateFactory {
+
+    @Override
+    public State makeState(Map conf, IMetricsContext metrics, int partitionIndex, int numPartitions) {
+      return new MockState();
+    }
+  }
+
+  public static class MockStateUpdater implements StateUpdater<MockState> {
+
+    @Override
+    public void updateState(MockState state, List<TridentTuple> tuples, TridentCollector collector) {
+      state.updateState(tuples, collector);
+    }
+
+    @Override
+    public void prepare(Map conf, TridentOperationContext context) {
+      // NOOP
+    }
+
+    @Override
+    public void cleanup() {
+      // NOOP
+    }
+  }
+
+  public static class MockSqlTridentDataSource implements ISqlTridentDataSource {
+    @Override
+    public IBatchSpout getProducer() {
+      return new MockSpout();
+    }
+
+    @Override
+    public SqlTridentConsumer getConsumer() {
+      return new SimpleSqlTridentConsumer(new MockStateFactory(), new MockStateUpdater());
+    }
+
+    private static class MockSpout implements IBatchSpout {
+      private final ArrayList<Values> RECORDS = new ArrayList<>();
+      private final Fields OUTPUT_FIELDS = new Fields("ID", "NAME", "ADDR");
+
+      public MockSpout() {
+        RECORDS.add(new Values(0, "a", "y"));
+        RECORDS.add(new Values(1, "ab", "y"));
+        RECORDS.add(new Values(2, "abc", "y"));
+        RECORDS.add(new Values(3, "abcd", "y"));
+        RECORDS.add(new Values(4, "abcde", "y"));
+      }
+
+      private boolean emitted = false;
+
+      @Override
+      public void open(Map conf, TopologyContext context) {
+      }
+
+      @Override
+      public void emitBatch(long batchId, TridentCollector collector) {
+        if (emitted) {
+          return;
+        }
+
+        for (Values r : RECORDS) {
+          collector.emit(r);
+        }
+        emitted = true;
+      }
+
+      @Override
+      public void ack(long batchId) {
+      }
+
+      @Override
+      public void close() {
+      }
+
+      @Override
+      public Map<String, Object> getComponentConfiguration() {
+        return null;
+      }
+
+      @Override
+      public Fields getOutputFields() {
+        return OUTPUT_FIELDS;
+      }
+    }
+  }
+
+  public static class MockSqlTridentGroupedDataSource implements ISqlTridentDataSource {
+    @Override
+    public IBatchSpout getProducer() {
+      return new MockGroupedSpout();
+    }
+
+    @Override
+    public SqlTridentConsumer getConsumer() {
+      return new SimpleSqlTridentConsumer(new MockStateFactory(), new MockStateUpdater());
+    }
+
+    private static class MockGroupedSpout implements IBatchSpout {
+      private final ArrayList<Values> RECORDS = new ArrayList<>();
+      private final Fields OUTPUT_FIELDS = new Fields("ID", "GRPID", "NAME", "ADDR", "AGE", "SCORE");
+
+      public MockGroupedSpout() {
+        for (int i = 0; i < 5; ++i) {
+          RECORDS.add(new Values(i, 0, "x", "y", 5 - i, i * 10));
+        }
+      }
+
+      private boolean emitted = false;
+
+      @Override
+      public void open(Map conf, TopologyContext context) {
+      }
+
+      @Override
+      public void emitBatch(long batchId, TridentCollector collector) {
+        if (emitted) {
+          return;
+        }
+
+        for (Values r : RECORDS) {
+          collector.emit(r);
+        }
+        emitted = true;
+      }
+
+      @Override
+      public void ack(long batchId) {
+      }
+
+      @Override
+      public void close() {
+      }
+
+      @Override
+      public Map<String, Object> getComponentConfiguration() {
+        return null;
+      }
+
+      @Override
+      public Fields getOutputFields() {
+        return OUTPUT_FIELDS;
+      }
+    }
+  }
+
+  public static class MockSqlTridentJoinDataSourceEmp implements ISqlTridentDataSource {
+    @Override
+    public IBatchSpout getProducer() {
+      return new MockSpout();
+    }
+
+    @Override
+    public SqlTridentConsumer getConsumer() {
+      return new SimpleSqlTridentConsumer(new MockStateFactory(), new MockStateUpdater());
+    }
+
+    private static class MockSpout implements IBatchSpout {
+      private final ArrayList<Values> RECORDS = new ArrayList<>();
+      private final Fields OUTPUT_FIELDS = new Fields("EMPID", "EMPNAME", "DEPTID");
+
+      public MockSpout() {
+        for (int i = 0; i < 5; ++i) {
+          RECORDS.add(new Values(i, "emp-" + i, i % 2));
+        }
+        for (int i = 10; i < 15; ++i) {
+          RECORDS.add(new Values(i, "emp-" + i, i));
+        }
+      }
+
+      private boolean emitted = false;
+
+      @Override
+      public void open(Map conf, TopologyContext context) {
+      }
+
+      @Override
+      public void emitBatch(long batchId, TridentCollector collector) {
+        if (emitted) {
+          return;
+        }
+
+        for (Values r : RECORDS) {
+          collector.emit(r);
+        }
+        emitted = true;
+      }
+
+      @Override
+      public void ack(long batchId) {
+      }
+
+      @Override
+      public void close() {
+      }
+
+      @Override
+      public Map<String, Object> getComponentConfiguration() {
+        return null;
+      }
+
+      @Override
+      public Fields getOutputFields() {
+        return OUTPUT_FIELDS;
+      }
+    }
+  }
+
+  public static class MockSqlTridentJoinDataSourceDept implements ISqlTridentDataSource {
+    @Override
+    public IBatchSpout getProducer() {
+      return new MockSpout();
+    }
+
+    @Override
+    public SqlTridentConsumer getConsumer() {
+      return new SimpleSqlTridentConsumer(new MockStateFactory(), new MockStateUpdater());
+    }
+
+    private static class MockSpout implements IBatchSpout {
+      private final ArrayList<Values> RECORDS = new ArrayList<>();
+      private final Fields OUTPUT_FIELDS = new Fields("DEPTID", "DEPTNAME");
+
+      public MockSpout() {
+        for (int i = 0; i < 5; ++i) {
+          RECORDS.add(new Values(i, "dept-" + i));
+        }
+      }
+
+      private boolean emitted = false;
+
+      @Override
+      public void open(Map conf, TopologyContext context) {
+      }
+
+      @Override
+      public void emitBatch(long batchId, TridentCollector collector) {
+        if (emitted) {
+          return;
+        }
+
+        for (Values r : RECORDS) {
+          collector.emit(r);
+        }
+        emitted = true;
+      }
+
+      @Override
+      public void ack(long batchId) {
+      }
+
+      @Override
+      public void close() {
+      }
+
+      @Override
+      public Map<String, Object> getComponentConfiguration() {
+        return null;
+      }
+
+      @Override
+      public Fields getOutputFields() {
+        return OUTPUT_FIELDS;
+      }
+    }
+  }
+
+  public static class MockSqlTridentNestedDataSource implements ISqlTridentDataSource {
+    @Override
+    public IBatchSpout getProducer() {
+      return new MockSpout();
+    }
+
+    @Override
+    public SqlTridentConsumer getConsumer() {
+      return new SimpleSqlTridentConsumer(new MockStateFactory(), new MockStateUpdater());
+    }
+
+    private static class MockSpout implements IBatchSpout {
+      private final ArrayList<Values> RECORDS = new ArrayList<>();
+      private final Fields OUTPUT_FIELDS = new Fields("ID", "MAPFIELD", "NESTEDMAPFIELD", "ARRAYFIELD");
+
+      public MockSpout() {
+        List<Integer> ints = Arrays.asList(100, 200, 300);
+        for (int i = 0; i < 5; ++i) {
+          Map<String, Integer> map = new HashMap<>();
+          map.put("b", i);
+          map.put("c", i*i);
+          Map<String, Map<String, Integer>> mm = new HashMap<>();
+          mm.put("a", map);
+          RECORDS.add(new Values(i, map, mm, ints));
+        }
+      }
+
+      private boolean emitted = false;
+
+      @Override
+      public void open(Map conf, TopologyContext context) {
+      }
+
+      @Override
+      public void emitBatch(long batchId, TridentCollector collector) {
+        if (emitted) {
+          return;
+        }
+
+        for (Values r : RECORDS) {
+          collector.emit(r);
+        }
+        emitted = true;
+      }
+
+      @Override
+      public void ack(long batchId) {
+      }
+
+      @Override
+      public void close() {
+      }
+
+      @Override
+      public Map<String, Object> getComponentConfiguration() {
+        return null;
+      }
+
+      @Override
+      public Fields getOutputFields() {
+        return OUTPUT_FIELDS;
+      }
+    }
+  }
+
+  public static class CollectDataChannelHandler implements ChannelHandler {
+    private final List<Values> values;
+
+    public CollectDataChannelHandler(List<Values> values) {
+      this.values = values;
+    }
+
+    @Override
+    public void dataReceived(ChannelContext ctx, Values data) {
+      values.add(data);
+    }
+
+    @Override
+    public void channelInactive(ChannelContext ctx) {}
+
+    @Override
+    public void exceptionCaught(Throwable cause) {
+      throw new RuntimeException(cause);
+    }
+
+    @Override
+    public void flush(ChannelContext ctx) {}
+
+    @Override
+    public void setSource(ChannelContext ctx, Object source) {}
+  }
+
+  public static long monotonicNow() {
+    final long NANOSECONDS_PER_MILLISECOND = 1000000;
+    return System.nanoTime() / NANOSECONDS_PER_MILLISECOND;
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/test/org/apache/storm/sql/runtime/datasource/socket/TestSocketDataSourceProvider.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/test/org/apache/storm/sql/runtime/datasource/socket/TestSocketDataSourceProvider.java b/sql/storm-sql-runtime/src/test/org/apache/storm/sql/runtime/datasource/socket/TestSocketDataSourceProvider.java
new file mode 100644
index 0000000..1e8edee
--- /dev/null
+++ b/sql/storm-sql-runtime/src/test/org/apache/storm/sql/runtime/datasource/socket/TestSocketDataSourceProvider.java
@@ -0,0 +1,94 @@
+/**
+ * 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.sql.runtime.datasource.socket;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import org.apache.storm.sql.runtime.DataSourcesRegistry;
+import org.apache.storm.sql.runtime.FieldInfo;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.sql.runtime.serde.json.JsonSerializer;
+import org.apache.storm.sql.runtime.datasource.socket.trident.SocketState;
+import org.apache.storm.sql.runtime.datasource.socket.trident.SocketStateUpdater;
+import org.apache.storm.trident.state.StateUpdater;
+import org.apache.storm.trident.tuple.TridentTuple;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class TestSocketDataSourceProvider {
+    private static final List<FieldInfo> FIELDS = ImmutableList.of(
+            new FieldInfo("ID", int.class, true),
+            new FieldInfo("val", String.class, false));
+    private static final List<String> FIELD_NAMES = ImmutableList.of("ID", "val");
+    private static final JsonSerializer SERIALIZER = new JsonSerializer(FIELD_NAMES);
+
+    @Test
+    public void testSocketSink() throws IOException {
+        ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(
+                URI.create("socket://localhost:8888"), null, null, new Properties(), FIELDS);
+        Assert.assertNotNull(ds);
+
+        ISqlTridentDataSource.SqlTridentConsumer consumer = ds.getConsumer();
+
+        Assert.assertEquals(SocketState.Factory.class, consumer.getStateFactory().getClass());
+        Assert.assertEquals(SocketStateUpdater.class, consumer.getStateUpdater().getClass());
+
+        // makeState() fails on creating State so we just mock SocketState anyway
+        SocketState mockState = mock(SocketState.class);
+        StateUpdater stateUpdater = consumer.getStateUpdater();
+
+        List<TridentTuple> tupleList = mockTupleList();
+
+        stateUpdater.updateState(mockState, tupleList, null);
+        for (TridentTuple t : tupleList) {
+            String serializedValue = new String(SERIALIZER.write(t.getValues(), null).array());
+            verify(mockState).write(serializedValue + "\n");
+        }
+    }
+
+    private static List<TridentTuple> mockTupleList() {
+        List<TridentTuple> tupleList = new ArrayList<>();
+        TridentTuple t0 = mock(TridentTuple.class);
+        TridentTuple t1 = mock(TridentTuple.class);
+        when(t0.getValueByField("ID")).thenReturn(1);
+        when(t0.getValueByField("val")).thenReturn("2");
+        doReturn(Lists.<Object>newArrayList(1, "2")).when(t0).getValues();
+        when(t0.size()).thenReturn(2);
+
+        when(t1.getValueByField("ID")).thenReturn(2);
+        when(t1.getValueByField("val")).thenReturn("3");
+        doReturn(Lists.<Object>newArrayList(2, "3")).when(t1).getValues();
+        when(t1.size()).thenReturn(2);
+
+        tupleList.add(t0);
+        tupleList.add(t1);
+        return tupleList;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/storm-dist/binary/final-package/src/main/assembly/binary.xml
----------------------------------------------------------------------
diff --git a/storm-dist/binary/final-package/src/main/assembly/binary.xml b/storm-dist/binary/final-package/src/main/assembly/binary.xml
index a6aab07..7ee1445 100644
--- a/storm-dist/binary/final-package/src/main/assembly/binary.xml
+++ b/storm-dist/binary/final-package/src/main/assembly/binary.xml
@@ -298,57 +298,6 @@
             </includes>
         </fileSet>
         <fileSet>
-            <directory>${project.basedir}/../../../external/sql/storm-sql-core/target/app-assembler/repo</directory>
-            <outputDirectory>external/sql/storm-sql-core</outputDirectory>
-            <includes>
-                <include>*jar</include>
-            </includes>
-        </fileSet>
-        <fileSet>
-            <directory>${project.basedir}/../../../external/sql/storm-sql-external/storm-sql-kafka/target</directory>
-            <outputDirectory>external/sql/storm-sql-external/storm-sql-kafka</outputDirectory>
-            <includes>
-                <include>storm*jar</include>
-            </includes>
-        </fileSet>
-        <fileSet>
-            <directory>${project.basedir}/../../../external/sql/storm-sql-external/storm-sql-redis/target</directory>
-            <outputDirectory>external/sql/storm-sql-external/storm-sql-redis</outputDirectory>
-            <includes>
-                <include>storm*jar</include>
-            </includes>
-        </fileSet>
-        <fileSet>
-            <directory>${project.basedir}/../../../external/sql/storm-sql-external/storm-sql-mongodb/target</directory>
-            <outputDirectory>external/sql/storm-sql-external/storm-sql-mongodb</outputDirectory>
-            <includes>
-                <include>storm*jar</include>
-            </includes>
-        </fileSet>
-        <fileSet>
-            <directory>${project.basedir}/../../../external/sql/storm-sql-external/storm-sql-hdfs/target</directory>
-            <outputDirectory>external/sql/storm-sql-external/storm-sql-hdfs</outputDirectory>
-            <includes>
-                <include>storm*jar</include>
-            </includes>
-        </fileSet>
-        <fileSet>
-            <directory>${project.basedir}/../../../external/sql/storm-sql-runtime/target/app-assembler/repo</directory>
-            <outputDirectory>external/sql/storm-sql-runtime</outputDirectory>
-            <includes>
-                <include>*jar</include>
-            </includes>
-        </fileSet>
-
-        <fileSet>
-            <directory>${project.basedir}/../../../external/sql</directory>
-            <outputDirectory>external/sql</outputDirectory>
-            <includes>
-                <include>README.*</include>
-            </includes>
-        </fileSet>
-
-        <fileSet>
             <directory>${project.basedir}/../../../external/storm-mqtt</directory>
             <outputDirectory>external/storm-mqtt</outputDirectory>
             <includes>
@@ -426,13 +375,6 @@
             </includes>
         </fileSet>
         <fileSet>
-            <directory>${project.basedir}/../../../external/storm-submit-tools/target</directory>
-            <outputDirectory>external/storm-submit-tools</outputDirectory>
-            <includes>
-                <include>storm*jar</include>
-            </includes>
-        </fileSet>
-        <fileSet>
             <directory>${project.basedir}/../../../external/storm-jms</directory>
             <outputDirectory>external/storm-jms</outputDirectory>
             <includes>
@@ -504,7 +446,7 @@
         <!-- $STORM_HOME/toollib -->
         <fileSet>
             <directory>${project.basedir}/../../../external/storm-kafka-monitor/target</directory>
-            <outputDirectory>toollib</outputDirectory>
+            <outputDirectory>lib-tools/storm-kafka-monitor</outputDirectory>
             <includes>
                 <include>storm*jar</include>
             </includes>
@@ -539,6 +481,32 @@
             </includes>
         </fileSet>
 
+        <!-- storm-sql -->
+        <fileSet>
+            <directory>${project.basedir}/../../../sql/storm-sql-core/target/app-assembler/repo</directory>
+            <outputDirectory>lib-tools/sql/core</outputDirectory>
+            <includes>
+                <include>*jar</include>
+            </includes>
+        </fileSet>
+        <fileSet>
+            <directory>${project.basedir}/../../../sql/storm-sql-runtime/target/app-assembler/repo</directory>
+            <outputDirectory>lib-tools/sql/runtime</outputDirectory>
+            <includes>
+                <include>*jar</include>
+            </includes>
+        </fileSet>
+
+        <!-- storm-submit-tools -->
+        <fileSet>
+            <directory>${project.basedir}/../../../storm-submit-tools/target</directory>
+            <outputDirectory>lib-tools/submit-tools</outputDirectory>
+            <includes>
+                <include>storm*jar</include>
+            </includes>
+        </fileSet>
+
+
     </fileSets>
 
     <files>

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/storm-submit-tools/pom.xml
----------------------------------------------------------------------
diff --git a/storm-submit-tools/pom.xml b/storm-submit-tools/pom.xml
new file mode 100644
index 0000000..c58930b
--- /dev/null
+++ b/storm-submit-tools/pom.xml
@@ -0,0 +1,216 @@
+<?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">
+    <parent>
+        <artifactId>storm</artifactId>
+        <groupId>org.apache.storm</groupId>
+        <version>2.0.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>storm-submit-tools</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>commons-lang</groupId>
+            <artifactId>commons-lang</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>com.googlecode.json-simple</groupId>
+            <artifactId>json-simple</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+        </dependency>
+
+        <!-- Aether :: maven dependency resolution -->
+        <dependency>
+            <groupId>org.apache.maven</groupId>
+            <artifactId>maven-plugin-api</artifactId>
+            <version>3.0</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.codehaus.plexus</groupId>
+                    <artifactId>plexus-utils</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.sonatype.sisu</groupId>
+                    <artifactId>sisu-inject-plexus</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.maven</groupId>
+                    <artifactId>maven-model</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.sonatype.aether</groupId>
+            <artifactId>aether-api</artifactId>
+            <version>1.12</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.sonatype.aether</groupId>
+            <artifactId>aether-util</artifactId>
+            <version>1.12</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.sonatype.aether</groupId>
+            <artifactId>aether-impl</artifactId>
+            <version>1.12</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.maven</groupId>
+            <artifactId>maven-aether-provider</artifactId>
+            <version>3.0.3</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.sonatype.aether</groupId>
+                    <artifactId>aether-api</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.sonatype.aether</groupId>
+                    <artifactId>aether-spi</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.sonatype.aether</groupId>
+                    <artifactId>aether-util</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.sonatype.aether</groupId>
+                    <artifactId>aether-impl</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.codehaus.plexus</groupId>
+                    <artifactId>plexus-utils</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.sonatype.aether</groupId>
+            <artifactId>aether-connector-file</artifactId>
+            <version>1.12</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.sonatype.aether</groupId>
+            <artifactId>aether-connector-wagon</artifactId>
+            <version>1.12</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.maven.wagon</groupId>
+                    <artifactId>wagon-provider-api</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.maven.wagon</groupId>
+            <artifactId>wagon-provider-api</artifactId>
+            <version>1.0</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.codehaus.plexus</groupId>
+                    <artifactId>plexus-utils</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.maven.wagon</groupId>
+            <artifactId>wagon-http-lightweight</artifactId>
+            <version>1.0</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.maven.wagon</groupId>
+                    <artifactId>wagon-http-shared</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.maven.wagon</groupId>
+            <artifactId>wagon-http</artifactId>
+            <version>1.0</version>
+            <exclusions>
+            </exclusions>
+        </dependency>
+
+        <!-- storm-core is needed only for test (surefire) -->
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-core</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-shade-plugin</artifactId>
+                <configuration>
+                    <keepDependenciesWithProvidedScope>false</keepDependenciesWithProvidedScope>
+                    <createDependencyReducedPom>true</createDependencyReducedPom>
+                    <filters>
+                        <filter>
+                            <artifact>*:*</artifact>
+                            <excludes>
+                                <exclude>META-INF/*.SF</exclude>
+                                <exclude>META-INF/*.sf</exclude>
+                                <exclude>META-INF/*.DSA</exclude>
+                                <exclude>META-INF/*.dsa</exclude>
+                                <exclude>META-INF/*.RSA</exclude>
+                                <exclude>META-INF/*.rsa</exclude>
+                                <exclude>META-INF/*.EC</exclude>
+                                <exclude>META-INF/*.ec</exclude>
+                                <exclude>META-INF/MSFTSIG.SF</exclude>
+                                <exclude>META-INF/MSFTSIG.RSA</exclude>
+                            </excludes>
+                        </filter>
+                    </filters>
+                </configuration>
+                <executions>
+                    <execution>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>shade</goal>
+                        </goals>
+                        <configuration>
+                            <transformers>
+                                <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
+                                <transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
+                                </transformer>
+                            </transformers>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/storm-submit-tools/src/main/java/org/apache/storm/submit/command/DependencyResolverMain.java
----------------------------------------------------------------------
diff --git a/storm-submit-tools/src/main/java/org/apache/storm/submit/command/DependencyResolverMain.java b/storm-submit-tools/src/main/java/org/apache/storm/submit/command/DependencyResolverMain.java
new file mode 100644
index 0000000..ac50110
--- /dev/null
+++ b/storm-submit-tools/src/main/java/org/apache/storm/submit/command/DependencyResolverMain.java
@@ -0,0 +1,158 @@
+/**
+ * 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.submit.command;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+import org.apache.commons.lang.StringUtils;
+import org.apache.storm.submit.dependency.AetherUtils;
+import org.apache.storm.submit.dependency.DependencyResolver;
+import org.json.simple.JSONValue;
+import org.sonatype.aether.artifact.Artifact;
+import org.sonatype.aether.graph.Dependency;
+import org.sonatype.aether.repository.RemoteRepository;
+import org.sonatype.aether.resolution.ArtifactResult;
+
+import java.io.File;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+public class DependencyResolverMain {
+
+    public static void main(String[] args) {
+        if (args.length < 1) {
+            throw new IllegalArgumentException("artifacts must be presented.");
+        }
+
+        String artifactsArg = args[0];
+
+        // DO NOT CHANGE THIS TO SYSOUT
+        System.err.println("DependencyResolver input - artifacts: " + artifactsArg);
+        List<Dependency> dependencies = parseArtifactArgs(artifactsArg);
+
+        List<RemoteRepository> repositories;
+        if (args.length > 1) {
+            String remoteRepositoryArg = args[1];
+
+            // DO NOT CHANGE THIS TO SYSOUT
+            System.err.println("DependencyResolver input - repositories: " + remoteRepositoryArg);
+
+            repositories = parseRemoteRepositoryArgs(remoteRepositoryArg);
+        } else {
+            repositories = Collections.emptyList();
+        }
+
+        try {
+            String localMavenRepoPath = getOrDefaultLocalMavenRepositoryPath("local-repo");
+            DependencyResolver resolver = new DependencyResolver(localMavenRepoPath, repositories);
+
+            List<ArtifactResult> artifactResults = resolver.resolve(dependencies);
+
+            Iterable<ArtifactResult> missingArtifacts = filterMissingArtifacts(artifactResults);
+            if (missingArtifacts.iterator().hasNext()) {
+                printMissingArtifactsToSysErr(missingArtifacts);
+                throw new RuntimeException("Some artifacts are not resolved");
+            }
+
+            System.out.println(JSONValue.toJSONString(transformArtifactResultToArtifactToPaths(artifactResults)));
+            System.out.flush();
+        } catch (Throwable e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private static Iterable<ArtifactResult> filterMissingArtifacts(List<ArtifactResult> artifactResults) {
+        return Iterables.filter(artifactResults, new Predicate<ArtifactResult>() {
+            @Override
+            public boolean apply(ArtifactResult artifactResult) {
+                return artifactResult.isMissing();
+            }
+        });
+    }
+
+    private static void printMissingArtifactsToSysErr(Iterable<ArtifactResult> missingArtifacts) {
+        for (ArtifactResult artifactResult : missingArtifacts) {
+            System.err.println("ArtifactResult : " + artifactResult + " / Errors : " + artifactResult.getExceptions());
+        }
+    }
+
+    private static List<Dependency> parseArtifactArgs(String artifactArgs) {
+        List<String> artifacts = Arrays.asList(artifactArgs.split(","));
+        List<Dependency> dependencies = new ArrayList<>(artifacts.size());
+        for (String artifactOpt : artifacts) {
+            if (artifactOpt.trim().isEmpty()) {
+                continue;
+            }
+
+            dependencies.add(AetherUtils.parseDependency(artifactOpt));
+        }
+
+        return dependencies;
+    }
+
+    private static List<RemoteRepository> parseRemoteRepositoryArgs(String remoteRepositoryArg) {
+        List<String> repositories = Arrays.asList(remoteRepositoryArg.split(","));
+        List<RemoteRepository> remoteRepositories = new ArrayList<>(repositories.size());
+        for (String repositoryOpt : repositories) {
+            if (repositoryOpt.trim().isEmpty()) {
+                continue;
+            }
+
+            remoteRepositories.add(AetherUtils.parseRemoteRepository(repositoryOpt));
+        }
+
+        return remoteRepositories;
+    }
+
+    private static Map<String, String> transformArtifactResultToArtifactToPaths(List<ArtifactResult> artifactResults) {
+        Map<String, String> artifactToPath = new LinkedHashMap<>();
+        for (ArtifactResult artifactResult : artifactResults) {
+            Artifact artifact = artifactResult.getArtifact();
+            artifactToPath.put(AetherUtils.artifactToString(artifact), artifact.getFile().getAbsolutePath());
+        }
+        return artifactToPath;
+    }
+
+    private static String getOrDefaultLocalMavenRepositoryPath(String defaultPath) {
+        String localMavenRepoPathStr = getLocalMavenRepositoryPath();
+        if (StringUtils.isNotEmpty(localMavenRepoPathStr)) {
+            Path localMavenRepoPath = new File(localMavenRepoPathStr).toPath();
+            if (Files.exists(localMavenRepoPath) && Files.isDirectory(localMavenRepoPath)) {
+                return localMavenRepoPathStr;
+            }
+        }
+
+        return defaultPath;
+    }
+
+    private static String getLocalMavenRepositoryPath() {
+        String userHome = System.getProperty("user.home");
+        if (StringUtils.isNotEmpty(userHome)) {
+            return userHome + File.separator + ".m2" + File.separator + "repository";
+        }
+
+        return null;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/AetherUtils.java
----------------------------------------------------------------------
diff --git a/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/AetherUtils.java b/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/AetherUtils.java
new file mode 100644
index 0000000..086be2b
--- /dev/null
+++ b/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/AetherUtils.java
@@ -0,0 +1,91 @@
+/**
+ * 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.submit.dependency;
+
+import org.sonatype.aether.artifact.Artifact;
+import org.sonatype.aether.graph.Dependency;
+import org.sonatype.aether.graph.Exclusion;
+import org.sonatype.aether.repository.RemoteRepository;
+import org.sonatype.aether.util.artifact.DefaultArtifact;
+import org.sonatype.aether.util.artifact.JavaScopes;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+public class AetherUtils {
+    private AetherUtils() {
+    }
+
+    public static Dependency parseDependency(String dependency) {
+        List<String> dependencyAndExclusions = Arrays.asList(dependency.split("\\^"));
+        Collection<Exclusion> exclusions = new ArrayList<>();
+        for (int idx = 1 ; idx < dependencyAndExclusions.size() ; idx++) {
+            exclusions.add(AetherUtils.createExclusion(dependencyAndExclusions.get(idx)));
+        }
+
+        Artifact artifact = new DefaultArtifact(dependencyAndExclusions.get(0));
+        return new Dependency(artifact, JavaScopes.COMPILE, false, exclusions);
+    }
+
+    public static Exclusion createExclusion(String exclusionString) {
+        String[] parts = exclusionString.split(":");
+
+        // length of parts should be greater than 0
+        String groupId = parts[0];
+
+        String artifactId = "*";
+        String classifier = "*";
+        String extension = "*";
+
+        int len = parts.length;
+        if (len > 1) {
+            artifactId = parts[1];
+        }
+        if (len > 2) {
+            classifier = parts[2];
+        }
+        if (len > 3) {
+            extension = parts[3];
+        }
+
+        return new Exclusion(groupId, artifactId, classifier, extension);
+    }
+
+    public static String artifactToString(Artifact artifact) {
+        StringBuilder buffer = new StringBuilder(128);
+        buffer.append(artifact.getGroupId());
+        buffer.append(':').append(artifact.getArtifactId());
+        buffer.append(':').append(artifact.getExtension());
+        if (artifact.getClassifier().length() > 0) {
+            buffer.append(':').append(artifact.getClassifier());
+        }
+        buffer.append(':').append(artifact.getVersion());
+        return buffer.toString();
+    }
+
+    public static RemoteRepository parseRemoteRepository(String repository) {
+        String[] parts = repository.split("\\^");
+        if (parts.length < 2) {
+            throw new IllegalArgumentException("Bad remote repository form: " + repository);
+        }
+
+        return new RemoteRepository(parts[0], "default", parts[1]);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/Booter.java
----------------------------------------------------------------------
diff --git a/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/Booter.java b/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/Booter.java
new file mode 100644
index 0000000..816e3cc
--- /dev/null
+++ b/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/Booter.java
@@ -0,0 +1,51 @@
+/**
+ * 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.submit.dependency;
+
+import org.apache.maven.repository.internal.MavenRepositorySystemSession;
+import org.sonatype.aether.RepositorySystem;
+import org.sonatype.aether.RepositorySystemSession;
+import org.sonatype.aether.repository.LocalRepository;
+import org.sonatype.aether.repository.RemoteRepository;
+
+import java.io.File;
+
+/**
+ * Manage mvn repository.
+ */
+public class Booter {
+    public static RepositorySystem newRepositorySystem() {
+        return RepositorySystemFactory.newRepositorySystem();
+    }
+
+    public static RepositorySystemSession newRepositorySystemSession(
+            RepositorySystem system, String localRepoPath) {
+        MavenRepositorySystemSession session = new MavenRepositorySystemSession();
+
+        LocalRepository localRepo =
+                new LocalRepository(new File(localRepoPath).getAbsolutePath());
+        session.setLocalRepositoryManager(system.newLocalRepositoryManager(localRepo));
+
+        return session;
+    }
+
+    public static RemoteRepository newCentralRepository() {
+        return new RemoteRepository("central", "default", "http://repo1.maven.org/maven2/");
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/DependencyResolver.java
----------------------------------------------------------------------
diff --git a/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/DependencyResolver.java b/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/DependencyResolver.java
new file mode 100644
index 0000000..4534344
--- /dev/null
+++ b/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/DependencyResolver.java
@@ -0,0 +1,98 @@
+/**
+ * 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.submit.dependency;
+
+import org.sonatype.aether.RepositorySystem;
+import org.sonatype.aether.RepositorySystemSession;
+import org.sonatype.aether.collection.CollectRequest;
+import org.sonatype.aether.graph.Dependency;
+import org.sonatype.aether.graph.DependencyFilter;
+import org.sonatype.aether.repository.RemoteRepository;
+import org.sonatype.aether.resolution.ArtifactResolutionException;
+import org.sonatype.aether.resolution.ArtifactResult;
+import org.sonatype.aether.resolution.DependencyRequest;
+import org.sonatype.aether.resolution.DependencyResolutionException;
+import org.sonatype.aether.util.artifact.JavaScopes;
+import org.sonatype.aether.util.filter.DependencyFilterUtils;
+
+import java.io.File;
+import java.net.MalformedURLException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class DependencyResolver {
+    private final RepositorySystem system = Booter.newRepositorySystem();
+    private final RepositorySystemSession session;
+
+    private final List<RemoteRepository> remoteRepositories;
+
+    public DependencyResolver(String localRepoPath) {
+        this(localRepoPath, Collections.emptyList());
+    }
+
+    public DependencyResolver(String localRepoPath, List<RemoteRepository> repositories) {
+        localRepoPath = handleRelativePath(localRepoPath);
+
+        session = Booter.newRepositorySystemSession(system, localRepoPath);
+
+        remoteRepositories = new ArrayList<>();
+        remoteRepositories.add(Booter.newCentralRepository());
+        remoteRepositories.addAll(repositories);
+    }
+
+    private String handleRelativePath(String localRepoPath) {
+        File repoDir = new File(localRepoPath);
+        if (!repoDir.isAbsolute()) {
+            // find homedir
+            String home = System.getProperty("storm.home");
+            if (home == null) {
+                home = ".";
+            }
+
+            localRepoPath = home + "/" + localRepoPath;
+        }
+        return localRepoPath;
+    }
+
+    public List<ArtifactResult> resolve(List<Dependency> dependencies) throws MalformedURLException,
+            DependencyResolutionException, ArtifactResolutionException {
+
+        DependencyFilter classpathFilter = DependencyFilterUtils
+                .classpathFilter(JavaScopes.COMPILE, JavaScopes.RUNTIME);
+
+        if (dependencies.size() == 0) {
+            return Collections.EMPTY_LIST;
+        }
+
+        CollectRequest collectRequest = new CollectRequest();
+        collectRequest.setRoot(dependencies.get(0));
+        for (int idx = 1; idx < dependencies.size(); idx++) {
+            collectRequest.addDependency(dependencies.get(idx));
+        }
+
+        for (RemoteRepository repository : remoteRepositories) {
+            collectRequest.addRepository(repository);
+        }
+
+        DependencyRequest dependencyRequest = new DependencyRequest(collectRequest, classpathFilter);
+        return system.resolveDependencies(session, dependencyRequest).getArtifactResults();
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/RepositorySystemFactory.java
----------------------------------------------------------------------
diff --git a/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/RepositorySystemFactory.java b/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/RepositorySystemFactory.java
new file mode 100644
index 0000000..f1fa2db
--- /dev/null
+++ b/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/RepositorySystemFactory.java
@@ -0,0 +1,67 @@
+/**
+ * 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.submit.dependency;
+
+import org.apache.maven.repository.internal.DefaultServiceLocator;
+import org.apache.maven.wagon.Wagon;
+import org.apache.maven.wagon.providers.http.HttpWagon;
+import org.apache.maven.wagon.providers.http.LightweightHttpWagon;
+import org.sonatype.aether.RepositorySystem;
+import org.sonatype.aether.connector.file.FileRepositoryConnectorFactory;
+import org.sonatype.aether.connector.wagon.WagonProvider;
+import org.sonatype.aether.connector.wagon.WagonRepositoryConnectorFactory;
+import org.sonatype.aether.spi.connector.RepositoryConnectorFactory;
+
+/**
+ * Get maven repository instance.
+ */
+public class RepositorySystemFactory {
+    public static RepositorySystem newRepositorySystem() {
+        DefaultServiceLocator locator = new DefaultServiceLocator();
+        locator.addService(RepositoryConnectorFactory.class, FileRepositoryConnectorFactory.class);
+        locator.addService(RepositoryConnectorFactory.class, WagonRepositoryConnectorFactory.class);
+        locator.setServices(WagonProvider.class, new ManualWagonProvider());
+
+        return locator.getService(RepositorySystem.class);
+    }
+
+    /**
+     * ManualWagonProvider
+     */
+    public static class ManualWagonProvider implements WagonProvider {
+
+        @Override
+        public Wagon lookup(String roleHint) throws Exception {
+            if ("http".equals(roleHint)) {
+                return new LightweightHttpWagon();
+            }
+
+            if ("https".equals(roleHint)) {
+                return new HttpWagon();
+            }
+
+            return null;
+        }
+
+        @Override
+        public void release(Wagon arg0) {
+
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/AetherUtilsTest.java
----------------------------------------------------------------------
diff --git a/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/AetherUtilsTest.java b/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/AetherUtilsTest.java
new file mode 100644
index 0000000..9951621
--- /dev/null
+++ b/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/AetherUtilsTest.java
@@ -0,0 +1,102 @@
+/**
+ * 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.submit.dependency;
+
+import com.google.common.collect.Lists;
+import org.junit.Test;
+import org.sonatype.aether.artifact.Artifact;
+import org.sonatype.aether.graph.Dependency;
+import org.sonatype.aether.graph.Exclusion;
+import org.sonatype.aether.util.artifact.DefaultArtifact;
+import org.sonatype.aether.util.artifact.JavaScopes;
+
+import java.util.List;
+
+import static org.junit.Assert.*;
+
+public class AetherUtilsTest {
+    @Test
+    public void parseDependency() throws Exception {
+        String testDependency = "testgroup:testartifact:1.0.0^testgroup:testexcartifact^testgroup:*";
+
+        Dependency dependency = AetherUtils.parseDependency(testDependency);
+
+        assertEquals("testgroup", dependency.getArtifact().getGroupId());
+        assertEquals("testartifact", dependency.getArtifact().getArtifactId());
+        assertEquals("1.0.0", dependency.getArtifact().getVersion());
+        assertEquals(JavaScopes.COMPILE, dependency.getScope());
+
+        assertEquals(2, dependency.getExclusions().size());
+
+        List<Exclusion> exclusions = Lists.newArrayList(dependency.getExclusions());
+
+        Exclusion exclusion = exclusions.get(0);
+        assertEquals("testgroup", exclusion.getGroupId());
+        assertEquals("testexcartifact", exclusion.getArtifactId());
+        assertEquals(JavaScopes.COMPILE, dependency.getScope());
+
+        exclusion = exclusions.get(1);
+        assertEquals("testgroup", exclusion.getGroupId());
+        assertEquals("*", exclusion.getArtifactId());
+        assertEquals(JavaScopes.COMPILE, dependency.getScope());
+    }
+
+    @Test
+    public void createExclusion() throws Exception {
+        String testExclusion = "group";
+        Exclusion exclusion = AetherUtils.createExclusion(testExclusion);
+
+        assertEquals("group", exclusion.getGroupId());
+        assertEquals("*", exclusion.getArtifactId());
+        assertEquals("*", exclusion.getClassifier());
+        assertEquals("*", exclusion.getExtension());
+
+        testExclusion = "group:artifact";
+        exclusion = AetherUtils.createExclusion(testExclusion);
+
+        assertEquals("group", exclusion.getGroupId());
+        assertEquals("artifact", exclusion.getArtifactId());
+        assertEquals("*", exclusion.getClassifier());
+        assertEquals("*", exclusion.getExtension());
+
+        testExclusion = "group:artifact:site";
+        exclusion = AetherUtils.createExclusion(testExclusion);
+
+        assertEquals("group", exclusion.getGroupId());
+        assertEquals("artifact", exclusion.getArtifactId());
+        assertEquals("site", exclusion.getClassifier());
+        assertEquals("*", exclusion.getExtension());
+
+        testExclusion = "group:artifact:site:jar";
+        exclusion = AetherUtils.createExclusion(testExclusion);
+
+        assertEquals("group", exclusion.getGroupId());
+        assertEquals("artifact", exclusion.getArtifactId());
+        assertEquals("site", exclusion.getClassifier());
+        assertEquals("jar", exclusion.getExtension());
+    }
+
+    @Test
+    public void artifactToString() throws Exception {
+        Artifact testArtifact = new DefaultArtifact("org.apache.storm:storm-core:1.0.0");
+
+        String ret = AetherUtils.artifactToString(testArtifact);
+        assertEquals("org.apache.storm:storm-core:jar:1.0.0", ret);
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/DependencyResolverTest.java
----------------------------------------------------------------------
diff --git a/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/DependencyResolverTest.java b/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/DependencyResolverTest.java
new file mode 100644
index 0000000..086abad
--- /dev/null
+++ b/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/DependencyResolverTest.java
@@ -0,0 +1,83 @@
+/**
+ * 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.submit.dependency;
+
+import com.google.common.collect.Lists;
+import org.apache.commons.io.FileUtils;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.sonatype.aether.graph.Dependency;
+import org.sonatype.aether.resolution.ArtifactResult;
+import org.sonatype.aether.util.artifact.DefaultArtifact;
+import org.sonatype.aether.util.artifact.JavaScopes;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.List;
+
+import static org.junit.Assert.assertTrue;
+
+public class DependencyResolverTest {
+    private static Path tempDirForTest;
+
+    private DependencyResolver sut;
+
+    @BeforeClass
+    public static void setUpBeforeClass() throws Exception {
+        tempDirForTest = Files.createTempDirectory("dr-test");
+    }
+
+    @AfterClass
+    public static void tearDownAfterClass() throws Exception {
+        FileUtils.deleteQuietly(tempDirForTest.toFile());
+    }
+
+    @Before
+    public void setUp() {
+        sut = new DependencyResolver(tempDirForTest.toAbsolutePath().toString());
+    }
+
+    @Test
+    public void resolveValid() throws Exception {
+        // please pick small artifact which has small transitive dependency
+        // and let's mark as Ignore if we want to run test even without internet or maven central is often not stable
+        Dependency dependency = new Dependency(new DefaultArtifact("org.apache.storm:flux-core:1.0.0"), JavaScopes.COMPILE);
+        List<ArtifactResult> results = sut.resolve(Lists.newArrayList(dependency));
+
+        assertTrue(results.size() > 0);
+        // it should be org.apache.storm:flux-core:jar:1.0.0 and commons-cli:commons-cli:jar:1.2
+        assertContains(results, "org.apache.storm", "flux-core", "1.0.0");
+        assertContains(results, "commons-cli", "commons-cli", "1.2");
+    }
+
+    private void assertContains(List<ArtifactResult> results, String groupId, String artifactId, String version) {
+        for (ArtifactResult result : results) {
+            if (result.getArtifact().getGroupId().equals(groupId) &&
+                    result.getArtifact().getArtifactId().equals(artifactId) &&
+                    result.getArtifact().getVersion().equals(version) &&
+                    result.isResolved()) {
+                return;
+            }
+        }
+
+        throw new AssertionError("Result doesn't contain expected artifact > " + groupId + ":" + artifactId + ":" + version);
+    }
+
+}
\ No newline at end of file


[11/23] storm git commit: STORM-2453 Move non-connectors into the top directory

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestUtils.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestUtils.java b/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestUtils.java
deleted file mode 100644
index 0a3bac6..0000000
--- a/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestUtils.java
+++ /dev/null
@@ -1,584 +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
- *  * <p>
- *  * http://www.apache.org/licenses/LICENSE-2.0
- *  * <p>
- *  * 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.sql;
-
-import org.apache.storm.sql.runtime.ChannelContext;
-import org.apache.storm.sql.runtime.ChannelHandler;
-import org.apache.storm.sql.runtime.DataSource;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.sql.runtime.SimpleSqlTridentConsumer;
-import org.apache.storm.task.IMetricsContext;
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.trident.operation.TridentCollector;
-import org.apache.storm.trident.operation.TridentOperationContext;
-import org.apache.storm.trident.spout.IBatchSpout;
-import org.apache.storm.trident.state.State;
-import org.apache.storm.trident.state.StateFactory;
-import org.apache.storm.trident.state.StateUpdater;
-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.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.PriorityQueue;
-
-public class TestUtils {
-  public static class MyPlus {
-    public static Integer evaluate(Integer x, Integer y) {
-      return x + y;
-    }
-  }
-
-  public static class MyConcat {
-    public static String init() {
-      return "";
-    }
-    public static String add(String accumulator, String val) {
-      return accumulator + val;
-    }
-    public static String result(String accumulator) {
-      return accumulator;
-    }
-  }
-
-
-  public static class TopN {
-    public static PriorityQueue<Integer> init() {
-      return new PriorityQueue<>();
-    }
-    public static PriorityQueue<Integer> add(PriorityQueue<Integer> accumulator, Integer n, Integer val) {
-      if (n <= 0) {
-        return accumulator;
-      }
-      if (accumulator.size() >= n) {
-        if (val > accumulator.peek()) {
-          accumulator.remove();
-          accumulator.add(val);
-        }
-      } else {
-        accumulator.add(val);
-      }
-      return accumulator;
-    }
-    public static List<Integer> result(PriorityQueue<Integer> accumulator) {
-      List<Integer> res = new ArrayList<>(accumulator);
-      Collections.reverse(res);
-      return res;
-    }
-  }
-
-
-  public static class MockDataSource implements DataSource {
-    private final ArrayList<Values> RECORDS = new ArrayList<>();
-
-    public MockDataSource() {
-      for (int i = 0; i < 5; ++i) {
-        RECORDS.add(new Values(i, "x", null));
-      }
-    }
-
-    @Override
-    public void open(ChannelContext ctx) {
-      for (Values v : RECORDS) {
-        ctx.emit(v);
-      }
-      ctx.fireChannelInactive();
-    }
-  }
-
-  public static class MockGroupDataSource implements DataSource {
-    private final ArrayList<Values> RECORDS = new ArrayList<>();
-
-    public MockGroupDataSource() {
-      for (int i = 0; i < 10; ++i) {
-        RECORDS.add(new Values(i/3, i, (i+1)* 0.5, "x", i/2));
-      }
-    }
-
-    @Override
-    public void open(ChannelContext ctx) {
-      for (Values v : RECORDS) {
-        ctx.emit(v);
-      }
-      // force evaluation of the aggregate function on the last group
-      ctx.flush();
-      ctx.fireChannelInactive();
-    }
-  }
-
-  public static class MockEmpDataSource implements DataSource {
-    private final ArrayList<Values> RECORDS = new ArrayList<>();
-
-    public MockEmpDataSource() {
-      RECORDS.add(new Values(1, "emp1", 1));
-      RECORDS.add(new Values(2, "emp2", 1));
-      RECORDS.add(new Values(3, "emp3", 2));
-    }
-
-    @Override
-    public void open(ChannelContext ctx) {
-      for (Values v : RECORDS) {
-        ctx.emit(v);
-      }
-      ctx.flush();
-      ctx.fireChannelInactive();
-    }
-  }
-
-  public static class MockDeptDataSource implements DataSource {
-    private final ArrayList<Values> RECORDS = new ArrayList<>();
-
-    public MockDeptDataSource() {
-      RECORDS.add(new Values(1, "dept1"));
-      RECORDS.add(new Values(2, "dept2"));
-      RECORDS.add(new Values(3, "dept3"));
-    }
-
-    @Override
-    public void open(ChannelContext ctx) {
-      for (Values v : RECORDS) {
-        ctx.emit(v);
-      }
-      ctx.flush();
-      ctx.fireChannelInactive();
-    }
-  }
-
-  public static class MockNestedDataSource implements DataSource {
-    private final ArrayList<Values> RECORDS = new ArrayList<>();
-
-    public MockNestedDataSource() {
-      List<Integer> ints = Arrays.asList(100, 200, 300);
-      for (int i = 0; i < 5; ++i) {
-        Map<String, Integer> map = new HashMap<>();
-        map.put("b", i);
-        map.put("c", i*i);
-        Map<String, Map<String, Integer>> mm = new HashMap<>();
-        mm.put("a", map);
-        RECORDS.add(new Values(i, map, mm, ints));
-      }
-    }
-
-    @Override
-    public void open(ChannelContext ctx) {
-      for (Values v : RECORDS) {
-        ctx.emit(v);
-      }
-      ctx.fireChannelInactive();
-    }
-  }
-
-  public static class MockState implements State {
-    /**
-     * Collect all values in a static variable as the instance will go through serialization and deserialization.
-     * NOTE: This should be cleared before or after running each test.
-     */
-    private transient static final List<List<Object> > VALUES = new ArrayList<>();
-
-    public static List<List<Object>> getCollectedValues() {
-      return VALUES;
-    }
-
-    @Override
-    public void beginCommit(Long txid) {
-      // NOOP
-    }
-
-    @Override
-    public void commit(Long txid) {
-      // NOOP
-    }
-
-    public void updateState(List<TridentTuple> tuples, TridentCollector collector) {
-      for (TridentTuple tuple : tuples) {
-        VALUES.add(tuple.getValues());
-      }
-    }
-  }
-
-  public static class MockStateFactory implements StateFactory {
-
-    @Override
-    public State makeState(Map conf, IMetricsContext metrics, int partitionIndex, int numPartitions) {
-      return new MockState();
-    }
-  }
-
-  public static class MockStateUpdater implements StateUpdater<MockState> {
-
-    @Override
-    public void updateState(MockState state, List<TridentTuple> tuples, TridentCollector collector) {
-      state.updateState(tuples, collector);
-    }
-
-    @Override
-    public void prepare(Map conf, TridentOperationContext context) {
-      // NOOP
-    }
-
-    @Override
-    public void cleanup() {
-      // NOOP
-    }
-  }
-
-  public static class MockSqlTridentDataSource implements ISqlTridentDataSource {
-    @Override
-    public IBatchSpout getProducer() {
-      return new MockSpout();
-    }
-
-    @Override
-    public SqlTridentConsumer getConsumer() {
-      return new SimpleSqlTridentConsumer(new MockStateFactory(), new MockStateUpdater());
-    }
-
-    private static class MockSpout implements IBatchSpout {
-      private final ArrayList<Values> RECORDS = new ArrayList<>();
-      private final Fields OUTPUT_FIELDS = new Fields("ID", "NAME", "ADDR");
-
-      public MockSpout() {
-        RECORDS.add(new Values(0, "a", "y"));
-        RECORDS.add(new Values(1, "ab", "y"));
-        RECORDS.add(new Values(2, "abc", "y"));
-        RECORDS.add(new Values(3, "abcd", "y"));
-        RECORDS.add(new Values(4, "abcde", "y"));
-      }
-
-      private boolean emitted = false;
-
-      @Override
-      public void open(Map conf, TopologyContext context) {
-      }
-
-      @Override
-      public void emitBatch(long batchId, TridentCollector collector) {
-        if (emitted) {
-          return;
-        }
-
-        for (Values r : RECORDS) {
-          collector.emit(r);
-        }
-        emitted = true;
-      }
-
-      @Override
-      public void ack(long batchId) {
-      }
-
-      @Override
-      public void close() {
-      }
-
-      @Override
-      public Map<String, Object> getComponentConfiguration() {
-        return null;
-      }
-
-      @Override
-      public Fields getOutputFields() {
-        return OUTPUT_FIELDS;
-      }
-    }
-  }
-
-  public static class MockSqlTridentGroupedDataSource implements ISqlTridentDataSource {
-    @Override
-    public IBatchSpout getProducer() {
-      return new MockGroupedSpout();
-    }
-
-    @Override
-    public SqlTridentConsumer getConsumer() {
-      return new SimpleSqlTridentConsumer(new MockStateFactory(), new MockStateUpdater());
-    }
-
-    private static class MockGroupedSpout implements IBatchSpout {
-      private final ArrayList<Values> RECORDS = new ArrayList<>();
-      private final Fields OUTPUT_FIELDS = new Fields("ID", "GRPID", "NAME", "ADDR", "AGE", "SCORE");
-
-      public MockGroupedSpout() {
-        for (int i = 0; i < 5; ++i) {
-          RECORDS.add(new Values(i, 0, "x", "y", 5 - i, i * 10));
-        }
-      }
-
-      private boolean emitted = false;
-
-      @Override
-      public void open(Map conf, TopologyContext context) {
-      }
-
-      @Override
-      public void emitBatch(long batchId, TridentCollector collector) {
-        if (emitted) {
-          return;
-        }
-
-        for (Values r : RECORDS) {
-          collector.emit(r);
-        }
-        emitted = true;
-      }
-
-      @Override
-      public void ack(long batchId) {
-      }
-
-      @Override
-      public void close() {
-      }
-
-      @Override
-      public Map<String, Object> getComponentConfiguration() {
-        return null;
-      }
-
-      @Override
-      public Fields getOutputFields() {
-        return OUTPUT_FIELDS;
-      }
-    }
-  }
-
-  public static class MockSqlTridentJoinDataSourceEmp implements ISqlTridentDataSource {
-    @Override
-    public IBatchSpout getProducer() {
-      return new MockSpout();
-    }
-
-    @Override
-    public SqlTridentConsumer getConsumer() {
-      return new SimpleSqlTridentConsumer(new MockStateFactory(), new MockStateUpdater());
-    }
-
-    private static class MockSpout implements IBatchSpout {
-      private final ArrayList<Values> RECORDS = new ArrayList<>();
-      private final Fields OUTPUT_FIELDS = new Fields("EMPID", "EMPNAME", "DEPTID");
-
-      public MockSpout() {
-        for (int i = 0; i < 5; ++i) {
-          RECORDS.add(new Values(i, "emp-" + i, i % 2));
-        }
-        for (int i = 10; i < 15; ++i) {
-          RECORDS.add(new Values(i, "emp-" + i, i));
-        }
-      }
-
-      private boolean emitted = false;
-
-      @Override
-      public void open(Map conf, TopologyContext context) {
-      }
-
-      @Override
-      public void emitBatch(long batchId, TridentCollector collector) {
-        if (emitted) {
-          return;
-        }
-
-        for (Values r : RECORDS) {
-          collector.emit(r);
-        }
-        emitted = true;
-      }
-
-      @Override
-      public void ack(long batchId) {
-      }
-
-      @Override
-      public void close() {
-      }
-
-      @Override
-      public Map<String, Object> getComponentConfiguration() {
-        return null;
-      }
-
-      @Override
-      public Fields getOutputFields() {
-        return OUTPUT_FIELDS;
-      }
-    }
-  }
-
-  public static class MockSqlTridentJoinDataSourceDept implements ISqlTridentDataSource {
-    @Override
-    public IBatchSpout getProducer() {
-      return new MockSpout();
-    }
-
-    @Override
-    public SqlTridentConsumer getConsumer() {
-      return new SimpleSqlTridentConsumer(new MockStateFactory(), new MockStateUpdater());
-    }
-
-    private static class MockSpout implements IBatchSpout {
-      private final ArrayList<Values> RECORDS = new ArrayList<>();
-      private final Fields OUTPUT_FIELDS = new Fields("DEPTID", "DEPTNAME");
-
-      public MockSpout() {
-        for (int i = 0; i < 5; ++i) {
-          RECORDS.add(new Values(i, "dept-" + i));
-        }
-      }
-
-      private boolean emitted = false;
-
-      @Override
-      public void open(Map conf, TopologyContext context) {
-      }
-
-      @Override
-      public void emitBatch(long batchId, TridentCollector collector) {
-        if (emitted) {
-          return;
-        }
-
-        for (Values r : RECORDS) {
-          collector.emit(r);
-        }
-        emitted = true;
-      }
-
-      @Override
-      public void ack(long batchId) {
-      }
-
-      @Override
-      public void close() {
-      }
-
-      @Override
-      public Map<String, Object> getComponentConfiguration() {
-        return null;
-      }
-
-      @Override
-      public Fields getOutputFields() {
-        return OUTPUT_FIELDS;
-      }
-    }
-  }
-
-  public static class MockSqlTridentNestedDataSource implements ISqlTridentDataSource {
-    @Override
-    public IBatchSpout getProducer() {
-      return new MockSpout();
-    }
-
-    @Override
-    public SqlTridentConsumer getConsumer() {
-      return new SimpleSqlTridentConsumer(new MockStateFactory(), new MockStateUpdater());
-    }
-
-    private static class MockSpout implements IBatchSpout {
-      private final ArrayList<Values> RECORDS = new ArrayList<>();
-      private final Fields OUTPUT_FIELDS = new Fields("ID", "MAPFIELD", "NESTEDMAPFIELD", "ARRAYFIELD");
-
-      public MockSpout() {
-        List<Integer> ints = Arrays.asList(100, 200, 300);
-        for (int i = 0; i < 5; ++i) {
-          Map<String, Integer> map = new HashMap<>();
-          map.put("b", i);
-          map.put("c", i*i);
-          Map<String, Map<String, Integer>> mm = new HashMap<>();
-          mm.put("a", map);
-          RECORDS.add(new Values(i, map, mm, ints));
-        }
-      }
-
-      private boolean emitted = false;
-
-      @Override
-      public void open(Map conf, TopologyContext context) {
-      }
-
-      @Override
-      public void emitBatch(long batchId, TridentCollector collector) {
-        if (emitted) {
-          return;
-        }
-
-        for (Values r : RECORDS) {
-          collector.emit(r);
-        }
-        emitted = true;
-      }
-
-      @Override
-      public void ack(long batchId) {
-      }
-
-      @Override
-      public void close() {
-      }
-
-      @Override
-      public Map<String, Object> getComponentConfiguration() {
-        return null;
-      }
-
-      @Override
-      public Fields getOutputFields() {
-        return OUTPUT_FIELDS;
-      }
-    }
-  }
-
-  public static class CollectDataChannelHandler implements ChannelHandler {
-    private final List<Values> values;
-
-    public CollectDataChannelHandler(List<Values> values) {
-      this.values = values;
-    }
-
-    @Override
-    public void dataReceived(ChannelContext ctx, Values data) {
-      values.add(data);
-    }
-
-    @Override
-    public void channelInactive(ChannelContext ctx) {}
-
-    @Override
-    public void exceptionCaught(Throwable cause) {
-      throw new RuntimeException(cause);
-    }
-
-    @Override
-    public void flush(ChannelContext ctx) {}
-
-    @Override
-    public void setSource(ChannelContext ctx, Object source) {}
-  }
-
-  public static long monotonicNow() {
-    final long NANOSECONDS_PER_MILLISECOND = 1000000;
-    return System.nanoTime() / NANOSECONDS_PER_MILLISECOND;
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/runtime/datasource/socket/TestSocketDataSourceProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/runtime/datasource/socket/TestSocketDataSourceProvider.java b/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/runtime/datasource/socket/TestSocketDataSourceProvider.java
deleted file mode 100644
index 1e8edee..0000000
--- a/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/runtime/datasource/socket/TestSocketDataSourceProvider.java
+++ /dev/null
@@ -1,94 +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.sql.runtime.datasource.socket;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import org.apache.storm.sql.runtime.DataSourcesRegistry;
-import org.apache.storm.sql.runtime.FieldInfo;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.sql.runtime.serde.json.JsonSerializer;
-import org.apache.storm.sql.runtime.datasource.socket.trident.SocketState;
-import org.apache.storm.sql.runtime.datasource.socket.trident.SocketStateUpdater;
-import org.apache.storm.trident.state.StateUpdater;
-import org.apache.storm.trident.tuple.TridentTuple;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Properties;
-
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-public class TestSocketDataSourceProvider {
-    private static final List<FieldInfo> FIELDS = ImmutableList.of(
-            new FieldInfo("ID", int.class, true),
-            new FieldInfo("val", String.class, false));
-    private static final List<String> FIELD_NAMES = ImmutableList.of("ID", "val");
-    private static final JsonSerializer SERIALIZER = new JsonSerializer(FIELD_NAMES);
-
-    @Test
-    public void testSocketSink() throws IOException {
-        ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(
-                URI.create("socket://localhost:8888"), null, null, new Properties(), FIELDS);
-        Assert.assertNotNull(ds);
-
-        ISqlTridentDataSource.SqlTridentConsumer consumer = ds.getConsumer();
-
-        Assert.assertEquals(SocketState.Factory.class, consumer.getStateFactory().getClass());
-        Assert.assertEquals(SocketStateUpdater.class, consumer.getStateUpdater().getClass());
-
-        // makeState() fails on creating State so we just mock SocketState anyway
-        SocketState mockState = mock(SocketState.class);
-        StateUpdater stateUpdater = consumer.getStateUpdater();
-
-        List<TridentTuple> tupleList = mockTupleList();
-
-        stateUpdater.updateState(mockState, tupleList, null);
-        for (TridentTuple t : tupleList) {
-            String serializedValue = new String(SERIALIZER.write(t.getValues(), null).array());
-            verify(mockState).write(serializedValue + "\n");
-        }
-    }
-
-    private static List<TridentTuple> mockTupleList() {
-        List<TridentTuple> tupleList = new ArrayList<>();
-        TridentTuple t0 = mock(TridentTuple.class);
-        TridentTuple t1 = mock(TridentTuple.class);
-        when(t0.getValueByField("ID")).thenReturn(1);
-        when(t0.getValueByField("val")).thenReturn("2");
-        doReturn(Lists.<Object>newArrayList(1, "2")).when(t0).getValues();
-        when(t0.size()).thenReturn(2);
-
-        when(t1.getValueByField("ID")).thenReturn(2);
-        when(t1.getValueByField("val")).thenReturn("3");
-        doReturn(Lists.<Object>newArrayList(2, "3")).when(t1).getValues();
-        when(t1.size()).thenReturn(2);
-
-        tupleList.add(t0);
-        tupleList.add(t1);
-        return tupleList;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/storm-submit-tools/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-submit-tools/pom.xml b/external/storm-submit-tools/pom.xml
deleted file mode 100644
index 90e6dd8..0000000
--- a/external/storm-submit-tools/pom.xml
+++ /dev/null
@@ -1,216 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements.  See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License.  You may obtain a copy of the License at
-     http://www.apache.org/licenses/LICENSE-2.0
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <parent>
-        <artifactId>storm</artifactId>
-        <groupId>org.apache.storm</groupId>
-        <version>2.0.0-SNAPSHOT</version>
-        <relativePath>../../pom.xml</relativePath>
-    </parent>
-    <modelVersion>4.0.0</modelVersion>
-
-    <artifactId>storm-submit-tools</artifactId>
-
-    <dependencies>
-        <dependency>
-            <groupId>commons-lang</groupId>
-            <artifactId>commons-lang</artifactId>
-        </dependency>
-
-        <dependency>
-            <groupId>com.googlecode.json-simple</groupId>
-            <artifactId>json-simple</artifactId>
-        </dependency>
-
-        <dependency>
-            <groupId>com.google.guava</groupId>
-            <artifactId>guava</artifactId>
-        </dependency>
-
-        <!-- Aether :: maven dependency resolution -->
-        <dependency>
-            <groupId>org.apache.maven</groupId>
-            <artifactId>maven-plugin-api</artifactId>
-            <version>3.0</version>
-            <exclusions>
-                <exclusion>
-                    <groupId>org.codehaus.plexus</groupId>
-                    <artifactId>plexus-utils</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.sonatype.sisu</groupId>
-                    <artifactId>sisu-inject-plexus</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.apache.maven</groupId>
-                    <artifactId>maven-model</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-
-        <dependency>
-            <groupId>org.sonatype.aether</groupId>
-            <artifactId>aether-api</artifactId>
-            <version>1.12</version>
-        </dependency>
-
-        <dependency>
-            <groupId>org.sonatype.aether</groupId>
-            <artifactId>aether-util</artifactId>
-            <version>1.12</version>
-        </dependency>
-
-        <dependency>
-            <groupId>org.sonatype.aether</groupId>
-            <artifactId>aether-impl</artifactId>
-            <version>1.12</version>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.maven</groupId>
-            <artifactId>maven-aether-provider</artifactId>
-            <version>3.0.3</version>
-            <exclusions>
-                <exclusion>
-                    <groupId>org.sonatype.aether</groupId>
-                    <artifactId>aether-api</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.sonatype.aether</groupId>
-                    <artifactId>aether-spi</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.sonatype.aether</groupId>
-                    <artifactId>aether-util</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.sonatype.aether</groupId>
-                    <artifactId>aether-impl</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.codehaus.plexus</groupId>
-                    <artifactId>plexus-utils</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-
-        <dependency>
-            <groupId>org.sonatype.aether</groupId>
-            <artifactId>aether-connector-file</artifactId>
-            <version>1.12</version>
-        </dependency>
-
-        <dependency>
-            <groupId>org.sonatype.aether</groupId>
-            <artifactId>aether-connector-wagon</artifactId>
-            <version>1.12</version>
-            <exclusions>
-                <exclusion>
-                    <groupId>org.apache.maven.wagon</groupId>
-                    <artifactId>wagon-provider-api</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.maven.wagon</groupId>
-            <artifactId>wagon-provider-api</artifactId>
-            <version>1.0</version>
-            <exclusions>
-                <exclusion>
-                    <groupId>org.codehaus.plexus</groupId>
-                    <artifactId>plexus-utils</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.maven.wagon</groupId>
-            <artifactId>wagon-http-lightweight</artifactId>
-            <version>1.0</version>
-            <exclusions>
-                <exclusion>
-                    <groupId>org.apache.maven.wagon</groupId>
-                    <artifactId>wagon-http-shared</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.maven.wagon</groupId>
-            <artifactId>wagon-http</artifactId>
-            <version>1.0</version>
-            <exclusions>
-            </exclusions>
-        </dependency>
-
-        <!-- storm-core is needed only for test (surefire) -->
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
-            <version>${project.version}</version>
-            <scope>test</scope>
-        </dependency>
-    </dependencies>
-
-    <build>
-        <plugins>
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-shade-plugin</artifactId>
-                <configuration>
-                    <keepDependenciesWithProvidedScope>false</keepDependenciesWithProvidedScope>
-                    <createDependencyReducedPom>true</createDependencyReducedPom>
-                    <filters>
-                        <filter>
-                            <artifact>*:*</artifact>
-                            <excludes>
-                                <exclude>META-INF/*.SF</exclude>
-                                <exclude>META-INF/*.sf</exclude>
-                                <exclude>META-INF/*.DSA</exclude>
-                                <exclude>META-INF/*.dsa</exclude>
-                                <exclude>META-INF/*.RSA</exclude>
-                                <exclude>META-INF/*.rsa</exclude>
-                                <exclude>META-INF/*.EC</exclude>
-                                <exclude>META-INF/*.ec</exclude>
-                                <exclude>META-INF/MSFTSIG.SF</exclude>
-                                <exclude>META-INF/MSFTSIG.RSA</exclude>
-                            </excludes>
-                        </filter>
-                    </filters>
-                </configuration>
-                <executions>
-                    <execution>
-                        <phase>package</phase>
-                        <goals>
-                            <goal>shade</goal>
-                        </goals>
-                        <configuration>
-                            <transformers>
-                                <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
-                                <transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
-                                </transformer>
-                            </transformers>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-        </plugins>
-    </build>
-
-</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/storm-submit-tools/src/main/java/org/apache/storm/submit/command/DependencyResolverMain.java
----------------------------------------------------------------------
diff --git a/external/storm-submit-tools/src/main/java/org/apache/storm/submit/command/DependencyResolverMain.java b/external/storm-submit-tools/src/main/java/org/apache/storm/submit/command/DependencyResolverMain.java
deleted file mode 100644
index ac50110..0000000
--- a/external/storm-submit-tools/src/main/java/org/apache/storm/submit/command/DependencyResolverMain.java
+++ /dev/null
@@ -1,158 +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.submit.command;
-
-import com.google.common.base.Predicate;
-import com.google.common.collect.Iterables;
-import org.apache.commons.lang.StringUtils;
-import org.apache.storm.submit.dependency.AetherUtils;
-import org.apache.storm.submit.dependency.DependencyResolver;
-import org.json.simple.JSONValue;
-import org.sonatype.aether.artifact.Artifact;
-import org.sonatype.aether.graph.Dependency;
-import org.sonatype.aether.repository.RemoteRepository;
-import org.sonatype.aether.resolution.ArtifactResult;
-
-import java.io.File;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-
-public class DependencyResolverMain {
-
-    public static void main(String[] args) {
-        if (args.length < 1) {
-            throw new IllegalArgumentException("artifacts must be presented.");
-        }
-
-        String artifactsArg = args[0];
-
-        // DO NOT CHANGE THIS TO SYSOUT
-        System.err.println("DependencyResolver input - artifacts: " + artifactsArg);
-        List<Dependency> dependencies = parseArtifactArgs(artifactsArg);
-
-        List<RemoteRepository> repositories;
-        if (args.length > 1) {
-            String remoteRepositoryArg = args[1];
-
-            // DO NOT CHANGE THIS TO SYSOUT
-            System.err.println("DependencyResolver input - repositories: " + remoteRepositoryArg);
-
-            repositories = parseRemoteRepositoryArgs(remoteRepositoryArg);
-        } else {
-            repositories = Collections.emptyList();
-        }
-
-        try {
-            String localMavenRepoPath = getOrDefaultLocalMavenRepositoryPath("local-repo");
-            DependencyResolver resolver = new DependencyResolver(localMavenRepoPath, repositories);
-
-            List<ArtifactResult> artifactResults = resolver.resolve(dependencies);
-
-            Iterable<ArtifactResult> missingArtifacts = filterMissingArtifacts(artifactResults);
-            if (missingArtifacts.iterator().hasNext()) {
-                printMissingArtifactsToSysErr(missingArtifacts);
-                throw new RuntimeException("Some artifacts are not resolved");
-            }
-
-            System.out.println(JSONValue.toJSONString(transformArtifactResultToArtifactToPaths(artifactResults)));
-            System.out.flush();
-        } catch (Throwable e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    private static Iterable<ArtifactResult> filterMissingArtifacts(List<ArtifactResult> artifactResults) {
-        return Iterables.filter(artifactResults, new Predicate<ArtifactResult>() {
-            @Override
-            public boolean apply(ArtifactResult artifactResult) {
-                return artifactResult.isMissing();
-            }
-        });
-    }
-
-    private static void printMissingArtifactsToSysErr(Iterable<ArtifactResult> missingArtifacts) {
-        for (ArtifactResult artifactResult : missingArtifacts) {
-            System.err.println("ArtifactResult : " + artifactResult + " / Errors : " + artifactResult.getExceptions());
-        }
-    }
-
-    private static List<Dependency> parseArtifactArgs(String artifactArgs) {
-        List<String> artifacts = Arrays.asList(artifactArgs.split(","));
-        List<Dependency> dependencies = new ArrayList<>(artifacts.size());
-        for (String artifactOpt : artifacts) {
-            if (artifactOpt.trim().isEmpty()) {
-                continue;
-            }
-
-            dependencies.add(AetherUtils.parseDependency(artifactOpt));
-        }
-
-        return dependencies;
-    }
-
-    private static List<RemoteRepository> parseRemoteRepositoryArgs(String remoteRepositoryArg) {
-        List<String> repositories = Arrays.asList(remoteRepositoryArg.split(","));
-        List<RemoteRepository> remoteRepositories = new ArrayList<>(repositories.size());
-        for (String repositoryOpt : repositories) {
-            if (repositoryOpt.trim().isEmpty()) {
-                continue;
-            }
-
-            remoteRepositories.add(AetherUtils.parseRemoteRepository(repositoryOpt));
-        }
-
-        return remoteRepositories;
-    }
-
-    private static Map<String, String> transformArtifactResultToArtifactToPaths(List<ArtifactResult> artifactResults) {
-        Map<String, String> artifactToPath = new LinkedHashMap<>();
-        for (ArtifactResult artifactResult : artifactResults) {
-            Artifact artifact = artifactResult.getArtifact();
-            artifactToPath.put(AetherUtils.artifactToString(artifact), artifact.getFile().getAbsolutePath());
-        }
-        return artifactToPath;
-    }
-
-    private static String getOrDefaultLocalMavenRepositoryPath(String defaultPath) {
-        String localMavenRepoPathStr = getLocalMavenRepositoryPath();
-        if (StringUtils.isNotEmpty(localMavenRepoPathStr)) {
-            Path localMavenRepoPath = new File(localMavenRepoPathStr).toPath();
-            if (Files.exists(localMavenRepoPath) && Files.isDirectory(localMavenRepoPath)) {
-                return localMavenRepoPathStr;
-            }
-        }
-
-        return defaultPath;
-    }
-
-    private static String getLocalMavenRepositoryPath() {
-        String userHome = System.getProperty("user.home");
-        if (StringUtils.isNotEmpty(userHome)) {
-            return userHome + File.separator + ".m2" + File.separator + "repository";
-        }
-
-        return null;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/AetherUtils.java
----------------------------------------------------------------------
diff --git a/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/AetherUtils.java b/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/AetherUtils.java
deleted file mode 100644
index 086be2b..0000000
--- a/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/AetherUtils.java
+++ /dev/null
@@ -1,91 +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.submit.dependency;
-
-import org.sonatype.aether.artifact.Artifact;
-import org.sonatype.aether.graph.Dependency;
-import org.sonatype.aether.graph.Exclusion;
-import org.sonatype.aether.repository.RemoteRepository;
-import org.sonatype.aether.util.artifact.DefaultArtifact;
-import org.sonatype.aether.util.artifact.JavaScopes;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-
-public class AetherUtils {
-    private AetherUtils() {
-    }
-
-    public static Dependency parseDependency(String dependency) {
-        List<String> dependencyAndExclusions = Arrays.asList(dependency.split("\\^"));
-        Collection<Exclusion> exclusions = new ArrayList<>();
-        for (int idx = 1 ; idx < dependencyAndExclusions.size() ; idx++) {
-            exclusions.add(AetherUtils.createExclusion(dependencyAndExclusions.get(idx)));
-        }
-
-        Artifact artifact = new DefaultArtifact(dependencyAndExclusions.get(0));
-        return new Dependency(artifact, JavaScopes.COMPILE, false, exclusions);
-    }
-
-    public static Exclusion createExclusion(String exclusionString) {
-        String[] parts = exclusionString.split(":");
-
-        // length of parts should be greater than 0
-        String groupId = parts[0];
-
-        String artifactId = "*";
-        String classifier = "*";
-        String extension = "*";
-
-        int len = parts.length;
-        if (len > 1) {
-            artifactId = parts[1];
-        }
-        if (len > 2) {
-            classifier = parts[2];
-        }
-        if (len > 3) {
-            extension = parts[3];
-        }
-
-        return new Exclusion(groupId, artifactId, classifier, extension);
-    }
-
-    public static String artifactToString(Artifact artifact) {
-        StringBuilder buffer = new StringBuilder(128);
-        buffer.append(artifact.getGroupId());
-        buffer.append(':').append(artifact.getArtifactId());
-        buffer.append(':').append(artifact.getExtension());
-        if (artifact.getClassifier().length() > 0) {
-            buffer.append(':').append(artifact.getClassifier());
-        }
-        buffer.append(':').append(artifact.getVersion());
-        return buffer.toString();
-    }
-
-    public static RemoteRepository parseRemoteRepository(String repository) {
-        String[] parts = repository.split("\\^");
-        if (parts.length < 2) {
-            throw new IllegalArgumentException("Bad remote repository form: " + repository);
-        }
-
-        return new RemoteRepository(parts[0], "default", parts[1]);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/Booter.java
----------------------------------------------------------------------
diff --git a/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/Booter.java b/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/Booter.java
deleted file mode 100644
index 816e3cc..0000000
--- a/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/Booter.java
+++ /dev/null
@@ -1,51 +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.submit.dependency;
-
-import org.apache.maven.repository.internal.MavenRepositorySystemSession;
-import org.sonatype.aether.RepositorySystem;
-import org.sonatype.aether.RepositorySystemSession;
-import org.sonatype.aether.repository.LocalRepository;
-import org.sonatype.aether.repository.RemoteRepository;
-
-import java.io.File;
-
-/**
- * Manage mvn repository.
- */
-public class Booter {
-    public static RepositorySystem newRepositorySystem() {
-        return RepositorySystemFactory.newRepositorySystem();
-    }
-
-    public static RepositorySystemSession newRepositorySystemSession(
-            RepositorySystem system, String localRepoPath) {
-        MavenRepositorySystemSession session = new MavenRepositorySystemSession();
-
-        LocalRepository localRepo =
-                new LocalRepository(new File(localRepoPath).getAbsolutePath());
-        session.setLocalRepositoryManager(system.newLocalRepositoryManager(localRepo));
-
-        return session;
-    }
-
-    public static RemoteRepository newCentralRepository() {
-        return new RemoteRepository("central", "default", "http://repo1.maven.org/maven2/");
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/DependencyResolver.java
----------------------------------------------------------------------
diff --git a/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/DependencyResolver.java b/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/DependencyResolver.java
deleted file mode 100644
index 4534344..0000000
--- a/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/DependencyResolver.java
+++ /dev/null
@@ -1,98 +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.submit.dependency;
-
-import org.sonatype.aether.RepositorySystem;
-import org.sonatype.aether.RepositorySystemSession;
-import org.sonatype.aether.collection.CollectRequest;
-import org.sonatype.aether.graph.Dependency;
-import org.sonatype.aether.graph.DependencyFilter;
-import org.sonatype.aether.repository.RemoteRepository;
-import org.sonatype.aether.resolution.ArtifactResolutionException;
-import org.sonatype.aether.resolution.ArtifactResult;
-import org.sonatype.aether.resolution.DependencyRequest;
-import org.sonatype.aether.resolution.DependencyResolutionException;
-import org.sonatype.aether.util.artifact.JavaScopes;
-import org.sonatype.aether.util.filter.DependencyFilterUtils;
-
-import java.io.File;
-import java.net.MalformedURLException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-public class DependencyResolver {
-    private final RepositorySystem system = Booter.newRepositorySystem();
-    private final RepositorySystemSession session;
-
-    private final List<RemoteRepository> remoteRepositories;
-
-    public DependencyResolver(String localRepoPath) {
-        this(localRepoPath, Collections.emptyList());
-    }
-
-    public DependencyResolver(String localRepoPath, List<RemoteRepository> repositories) {
-        localRepoPath = handleRelativePath(localRepoPath);
-
-        session = Booter.newRepositorySystemSession(system, localRepoPath);
-
-        remoteRepositories = new ArrayList<>();
-        remoteRepositories.add(Booter.newCentralRepository());
-        remoteRepositories.addAll(repositories);
-    }
-
-    private String handleRelativePath(String localRepoPath) {
-        File repoDir = new File(localRepoPath);
-        if (!repoDir.isAbsolute()) {
-            // find homedir
-            String home = System.getProperty("storm.home");
-            if (home == null) {
-                home = ".";
-            }
-
-            localRepoPath = home + "/" + localRepoPath;
-        }
-        return localRepoPath;
-    }
-
-    public List<ArtifactResult> resolve(List<Dependency> dependencies) throws MalformedURLException,
-            DependencyResolutionException, ArtifactResolutionException {
-
-        DependencyFilter classpathFilter = DependencyFilterUtils
-                .classpathFilter(JavaScopes.COMPILE, JavaScopes.RUNTIME);
-
-        if (dependencies.size() == 0) {
-            return Collections.EMPTY_LIST;
-        }
-
-        CollectRequest collectRequest = new CollectRequest();
-        collectRequest.setRoot(dependencies.get(0));
-        for (int idx = 1; idx < dependencies.size(); idx++) {
-            collectRequest.addDependency(dependencies.get(idx));
-        }
-
-        for (RemoteRepository repository : remoteRepositories) {
-            collectRequest.addRepository(repository);
-        }
-
-        DependencyRequest dependencyRequest = new DependencyRequest(collectRequest, classpathFilter);
-        return system.resolveDependencies(session, dependencyRequest).getArtifactResults();
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/RepositorySystemFactory.java
----------------------------------------------------------------------
diff --git a/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/RepositorySystemFactory.java b/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/RepositorySystemFactory.java
deleted file mode 100644
index f1fa2db..0000000
--- a/external/storm-submit-tools/src/main/java/org/apache/storm/submit/dependency/RepositorySystemFactory.java
+++ /dev/null
@@ -1,67 +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.submit.dependency;
-
-import org.apache.maven.repository.internal.DefaultServiceLocator;
-import org.apache.maven.wagon.Wagon;
-import org.apache.maven.wagon.providers.http.HttpWagon;
-import org.apache.maven.wagon.providers.http.LightweightHttpWagon;
-import org.sonatype.aether.RepositorySystem;
-import org.sonatype.aether.connector.file.FileRepositoryConnectorFactory;
-import org.sonatype.aether.connector.wagon.WagonProvider;
-import org.sonatype.aether.connector.wagon.WagonRepositoryConnectorFactory;
-import org.sonatype.aether.spi.connector.RepositoryConnectorFactory;
-
-/**
- * Get maven repository instance.
- */
-public class RepositorySystemFactory {
-    public static RepositorySystem newRepositorySystem() {
-        DefaultServiceLocator locator = new DefaultServiceLocator();
-        locator.addService(RepositoryConnectorFactory.class, FileRepositoryConnectorFactory.class);
-        locator.addService(RepositoryConnectorFactory.class, WagonRepositoryConnectorFactory.class);
-        locator.setServices(WagonProvider.class, new ManualWagonProvider());
-
-        return locator.getService(RepositorySystem.class);
-    }
-
-    /**
-     * ManualWagonProvider
-     */
-    public static class ManualWagonProvider implements WagonProvider {
-
-        @Override
-        public Wagon lookup(String roleHint) throws Exception {
-            if ("http".equals(roleHint)) {
-                return new LightweightHttpWagon();
-            }
-
-            if ("https".equals(roleHint)) {
-                return new HttpWagon();
-            }
-
-            return null;
-        }
-
-        @Override
-        public void release(Wagon arg0) {
-
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/AetherUtilsTest.java
----------------------------------------------------------------------
diff --git a/external/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/AetherUtilsTest.java b/external/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/AetherUtilsTest.java
deleted file mode 100644
index 9951621..0000000
--- a/external/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/AetherUtilsTest.java
+++ /dev/null
@@ -1,102 +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.submit.dependency;
-
-import com.google.common.collect.Lists;
-import org.junit.Test;
-import org.sonatype.aether.artifact.Artifact;
-import org.sonatype.aether.graph.Dependency;
-import org.sonatype.aether.graph.Exclusion;
-import org.sonatype.aether.util.artifact.DefaultArtifact;
-import org.sonatype.aether.util.artifact.JavaScopes;
-
-import java.util.List;
-
-import static org.junit.Assert.*;
-
-public class AetherUtilsTest {
-    @Test
-    public void parseDependency() throws Exception {
-        String testDependency = "testgroup:testartifact:1.0.0^testgroup:testexcartifact^testgroup:*";
-
-        Dependency dependency = AetherUtils.parseDependency(testDependency);
-
-        assertEquals("testgroup", dependency.getArtifact().getGroupId());
-        assertEquals("testartifact", dependency.getArtifact().getArtifactId());
-        assertEquals("1.0.0", dependency.getArtifact().getVersion());
-        assertEquals(JavaScopes.COMPILE, dependency.getScope());
-
-        assertEquals(2, dependency.getExclusions().size());
-
-        List<Exclusion> exclusions = Lists.newArrayList(dependency.getExclusions());
-
-        Exclusion exclusion = exclusions.get(0);
-        assertEquals("testgroup", exclusion.getGroupId());
-        assertEquals("testexcartifact", exclusion.getArtifactId());
-        assertEquals(JavaScopes.COMPILE, dependency.getScope());
-
-        exclusion = exclusions.get(1);
-        assertEquals("testgroup", exclusion.getGroupId());
-        assertEquals("*", exclusion.getArtifactId());
-        assertEquals(JavaScopes.COMPILE, dependency.getScope());
-    }
-
-    @Test
-    public void createExclusion() throws Exception {
-        String testExclusion = "group";
-        Exclusion exclusion = AetherUtils.createExclusion(testExclusion);
-
-        assertEquals("group", exclusion.getGroupId());
-        assertEquals("*", exclusion.getArtifactId());
-        assertEquals("*", exclusion.getClassifier());
-        assertEquals("*", exclusion.getExtension());
-
-        testExclusion = "group:artifact";
-        exclusion = AetherUtils.createExclusion(testExclusion);
-
-        assertEquals("group", exclusion.getGroupId());
-        assertEquals("artifact", exclusion.getArtifactId());
-        assertEquals("*", exclusion.getClassifier());
-        assertEquals("*", exclusion.getExtension());
-
-        testExclusion = "group:artifact:site";
-        exclusion = AetherUtils.createExclusion(testExclusion);
-
-        assertEquals("group", exclusion.getGroupId());
-        assertEquals("artifact", exclusion.getArtifactId());
-        assertEquals("site", exclusion.getClassifier());
-        assertEquals("*", exclusion.getExtension());
-
-        testExclusion = "group:artifact:site:jar";
-        exclusion = AetherUtils.createExclusion(testExclusion);
-
-        assertEquals("group", exclusion.getGroupId());
-        assertEquals("artifact", exclusion.getArtifactId());
-        assertEquals("site", exclusion.getClassifier());
-        assertEquals("jar", exclusion.getExtension());
-    }
-
-    @Test
-    public void artifactToString() throws Exception {
-        Artifact testArtifact = new DefaultArtifact("org.apache.storm:storm-core:1.0.0");
-
-        String ret = AetherUtils.artifactToString(testArtifact);
-        assertEquals("org.apache.storm:storm-core:jar:1.0.0", ret);
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/DependencyResolverTest.java
----------------------------------------------------------------------
diff --git a/external/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/DependencyResolverTest.java b/external/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/DependencyResolverTest.java
deleted file mode 100644
index 086abad..0000000
--- a/external/storm-submit-tools/src/test/java/org/apache/storm/submit/dependency/DependencyResolverTest.java
+++ /dev/null
@@ -1,83 +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.submit.dependency;
-
-import com.google.common.collect.Lists;
-import org.apache.commons.io.FileUtils;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.sonatype.aether.graph.Dependency;
-import org.sonatype.aether.resolution.ArtifactResult;
-import org.sonatype.aether.util.artifact.DefaultArtifact;
-import org.sonatype.aether.util.artifact.JavaScopes;
-
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.util.List;
-
-import static org.junit.Assert.assertTrue;
-
-public class DependencyResolverTest {
-    private static Path tempDirForTest;
-
-    private DependencyResolver sut;
-
-    @BeforeClass
-    public static void setUpBeforeClass() throws Exception {
-        tempDirForTest = Files.createTempDirectory("dr-test");
-    }
-
-    @AfterClass
-    public static void tearDownAfterClass() throws Exception {
-        FileUtils.deleteQuietly(tempDirForTest.toFile());
-    }
-
-    @Before
-    public void setUp() {
-        sut = new DependencyResolver(tempDirForTest.toAbsolutePath().toString());
-    }
-
-    @Test
-    public void resolveValid() throws Exception {
-        // please pick small artifact which has small transitive dependency
-        // and let's mark as Ignore if we want to run test even without internet or maven central is often not stable
-        Dependency dependency = new Dependency(new DefaultArtifact("org.apache.storm:flux-core:1.0.0"), JavaScopes.COMPILE);
-        List<ArtifactResult> results = sut.resolve(Lists.newArrayList(dependency));
-
-        assertTrue(results.size() > 0);
-        // it should be org.apache.storm:flux-core:jar:1.0.0 and commons-cli:commons-cli:jar:1.2
-        assertContains(results, "org.apache.storm", "flux-core", "1.0.0");
-        assertContains(results, "commons-cli", "commons-cli", "1.2");
-    }
-
-    private void assertContains(List<ArtifactResult> results, String groupId, String artifactId, String version) {
-        for (ArtifactResult result : results) {
-            if (result.getArtifact().getGroupId().equals(groupId) &&
-                    result.getArtifact().getArtifactId().equals(artifactId) &&
-                    result.getArtifact().getVersion().equals(version) &&
-                    result.isResolved()) {
-                return;
-            }
-        }
-
-        throw new AssertionError("Result doesn't contain expected artifact > " + groupId + ":" + artifactId + ":" + version);
-    }
-
-}
\ No newline at end of file


[03/23] storm git commit: STORM-2453 Move non-connectors into the top directory

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-hdfs/src/jvm/org/apache/storm/sql/hdfs/HdfsDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-hdfs/src/jvm/org/apache/storm/sql/hdfs/HdfsDataSourcesProvider.java b/sql/storm-sql-external/storm-sql-hdfs/src/jvm/org/apache/storm/sql/hdfs/HdfsDataSourcesProvider.java
new file mode 100644
index 0000000..38c3fcb
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-hdfs/src/jvm/org/apache/storm/sql/hdfs/HdfsDataSourcesProvider.java
@@ -0,0 +1,135 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.hdfs;
+
+import com.google.common.base.Preconditions;
+import org.apache.storm.hdfs.trident.HdfsState;
+import org.apache.storm.hdfs.trident.HdfsStateFactory;
+import org.apache.storm.hdfs.trident.HdfsUpdater;
+import org.apache.storm.hdfs.trident.format.FileNameFormat;
+import org.apache.storm.hdfs.trident.format.RecordFormat;
+import org.apache.storm.hdfs.trident.format.SimpleFileNameFormat;
+import org.apache.storm.hdfs.trident.rotation.FileRotationPolicy;
+import org.apache.storm.hdfs.trident.rotation.FileSizeRotationPolicy;
+import org.apache.storm.hdfs.trident.rotation.TimedRotationPolicy;
+import org.apache.storm.sql.runtime.DataSource;
+import org.apache.storm.sql.runtime.DataSourcesProvider;
+import org.apache.storm.sql.runtime.FieldInfo;
+import org.apache.storm.sql.runtime.IOutputSerializer;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.sql.runtime.SimpleSqlTridentConsumer;
+import org.apache.storm.sql.runtime.utils.FieldInfoUtils;
+import org.apache.storm.sql.runtime.utils.SerdeUtils;
+import org.apache.storm.trident.spout.ITridentDataSource;
+import org.apache.storm.trident.state.StateFactory;
+import org.apache.storm.trident.state.StateUpdater;
+import org.apache.storm.trident.tuple.TridentTuple;
+
+import java.net.URI;
+import java.util.List;
+import java.util.Properties;
+
+/**
+ * Create a HDFS sink based on the URI and properties. The URI has the format of hdfs://host:port/path-to-file
+ * The properties are in JSON format which specifies the name / path of the hdfs file and etc.
+ */
+public class HdfsDataSourcesProvider implements DataSourcesProvider {
+
+  private static class HdfsTridentDataSource implements ISqlTridentDataSource {
+    private final String url;
+    private final Properties props;
+    private final IOutputSerializer serializer;
+
+    private HdfsTridentDataSource(String url, Properties props, IOutputSerializer serializer) {
+      this.url = url;
+      this.props = props;
+      this.serializer = serializer;
+    }
+
+    @Override
+    public ITridentDataSource getProducer() {
+      throw new UnsupportedOperationException(this.getClass().getName() + " doesn't provide Producer");
+    }
+
+    @Override
+    public SqlTridentConsumer getConsumer() {
+      FileNameFormat fileNameFormat = new SimpleFileNameFormat()
+          .withPath(props.getProperty("hdfs.file.path", "/storm"))
+          .withName(props.getProperty("hdfs.file.name", "$TIME.$NUM.txt"));
+
+      RecordFormat recordFormat = new TridentRecordFormat(serializer);
+
+      FileRotationPolicy rotationPolicy;
+      String size = props.getProperty("hdfs.rotation.size.kb");
+      String interval = props.getProperty("hdfs.rotation.time.seconds");
+      Preconditions.checkArgument(size != null || interval != null, "Hdfs data source must contain file rotation config");
+
+      if (size != null) {
+        rotationPolicy = new FileSizeRotationPolicy(Float.parseFloat(size), FileSizeRotationPolicy.Units.KB);
+      } else {
+        rotationPolicy = new TimedRotationPolicy(Float.parseFloat(interval), TimedRotationPolicy.TimeUnit.SECONDS);
+      }
+
+      HdfsState.Options options = new HdfsState.HdfsFileOptions()
+          .withFileNameFormat(fileNameFormat)
+          .withRecordFormat(recordFormat)
+          .withRotationPolicy(rotationPolicy)
+          .withFsUrl(url);
+
+      StateFactory stateFactory = new HdfsStateFactory().withOptions(options);
+      StateUpdater stateUpdater = new HdfsUpdater();
+
+      return new SimpleSqlTridentConsumer(stateFactory, stateUpdater);
+    }
+  }
+
+  private static class TridentRecordFormat implements RecordFormat {
+    private final IOutputSerializer serializer;
+
+    private TridentRecordFormat(IOutputSerializer serializer) {
+      this.serializer = serializer;
+    }
+
+    @Override
+    public byte[] format(TridentTuple tuple) {
+      //TODO we should handle '\n'. ref DelimitedRecordFormat
+      return serializer.write(tuple.getValues(), null).array();
+    }
+
+  }
+
+  @Override
+  public String scheme() {
+    return "hdfs";
+  }
+
+  @Override
+  public DataSource construct(URI uri, String inputFormatClass, String outputFormatClass,
+                              List<FieldInfo> fields) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
+                                                Properties properties, List<FieldInfo> fields) {
+    List<String> fieldNames = FieldInfoUtils.getFieldNames(fields);
+    IOutputSerializer serializer = SerdeUtils.getSerializer(outputFormatClass, properties, fieldNames);
+    return new HdfsTridentDataSource(uri.toString(), properties, serializer);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-hdfs/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-hdfs/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider b/sql/storm-sql-external/storm-sql-hdfs/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
new file mode 100644
index 0000000..5fac84f
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-hdfs/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
@@ -0,0 +1,16 @@
+# 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.
+
+org.apache.storm.sql.hdfs.HdfsDataSourcesProvider
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-hdfs/src/test/org/apache/storm/sql/hdfs/TestHdfsDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-hdfs/src/test/org/apache/storm/sql/hdfs/TestHdfsDataSourcesProvider.java b/sql/storm-sql-external/storm-sql-hdfs/src/test/org/apache/storm/sql/hdfs/TestHdfsDataSourcesProvider.java
new file mode 100644
index 0000000..1473438
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-hdfs/src/test/org/apache/storm/sql/hdfs/TestHdfsDataSourcesProvider.java
@@ -0,0 +1,129 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.hdfs;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.storm.hdfs.trident.HdfsState;
+import org.apache.storm.hdfs.trident.HdfsStateFactory;
+import org.apache.storm.hdfs.trident.HdfsUpdater;
+import org.apache.storm.sql.runtime.DataSourcesRegistry;
+import org.apache.storm.sql.runtime.FieldInfo;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.trident.state.StateUpdater;
+import org.apache.storm.trident.tuple.TridentTuple;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.internal.util.reflection.Whitebox;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+
+import static org.apache.storm.hdfs.trident.HdfsState.HdfsFileOptions;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+
+public class TestHdfsDataSourcesProvider {
+  private static final List<FieldInfo> FIELDS = ImmutableList.of(
+      new FieldInfo("ID", int.class, true),
+      new FieldInfo("val", String.class, false));
+  private static final Properties TBL_PROPERTIES = new Properties();
+
+  private static String hdfsURI;
+  private static MiniDFSCluster hdfsCluster;
+
+  static {
+    TBL_PROPERTIES.put("hdfs.file.path", "/unittest");
+    TBL_PROPERTIES.put("hdfs.file.name", "test1.txt");
+    TBL_PROPERTIES.put("hdfs.rotation.time.seconds", "120");
+  }
+
+  @Before
+  public void setup() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set("fs.trash.interval", "10");
+    conf.setBoolean("dfs.permissions", true);
+    File baseDir = new File("./target/hdfs/").getAbsoluteFile();
+    FileUtil.fullyDelete(baseDir);
+    conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath());
+
+    MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
+    hdfsCluster = builder.build();
+    hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/";
+  }
+
+  @After
+  public void shutDown() throws IOException {
+    hdfsCluster.shutdown();
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testHdfsSink() {
+    ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(
+            URI.create(hdfsURI), null, null, TBL_PROPERTIES, FIELDS);
+    Assert.assertNotNull(ds);
+
+    ISqlTridentDataSource.SqlTridentConsumer consumer = ds.getConsumer();
+
+    Assert.assertEquals(HdfsStateFactory.class, consumer.getStateFactory().getClass());
+    Assert.assertEquals(HdfsUpdater.class, consumer.getStateUpdater().getClass());
+
+    HdfsState state = (HdfsState) consumer.getStateFactory().makeState(Collections.emptyMap(), null, 0, 1);
+    StateUpdater stateUpdater = consumer.getStateUpdater();
+
+    HdfsFileOptions options = mock(HdfsFileOptions.class);
+    Whitebox.setInternalState(state, "options", options);
+
+    List<TridentTuple> tupleList = mockTupleList();
+
+    for (TridentTuple t : tupleList) {
+      stateUpdater.updateState(state, Collections.singletonList(t), null);
+      try {
+        verify(options).execute(Collections.singletonList(t));
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
+  private static List<TridentTuple> mockTupleList() {
+    List<TridentTuple> tupleList = new ArrayList<>();
+    TridentTuple t0 = mock(TridentTuple.class);
+    TridentTuple t1 = mock(TridentTuple.class);
+    doReturn(1).when(t0).get(0);
+    doReturn(2).when(t1).get(0);
+    doReturn(Lists.<Object>newArrayList(1, "2")).when(t0).getValues();
+    doReturn(Lists.<Object>newArrayList(2, "3")).when(t1).getValues();
+    tupleList.add(t0);
+    tupleList.add(t1);
+    return tupleList;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-kafka/pom.xml
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-kafka/pom.xml b/sql/storm-sql-external/storm-sql-kafka/pom.xml
new file mode 100644
index 0000000..881343d
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-kafka/pom.xml
@@ -0,0 +1,93 @@
+<?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>
+        <relativePath>../../../pom.xml</relativePath>
+    </parent>
+
+    <artifactId>storm-sql-kafka</artifactId>
+
+    <developers>
+        <developer>
+            <id>haohui</id>
+            <name>Haohui Mai</name>
+            <email>ricetons@gmail.com</email>
+        </developer>
+    </developers>
+
+    <dependencies>
+        <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-sql-runtime</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-sql-runtime</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+            <type>test-jar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-kafka</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.kafka</groupId>
+            <artifactId>${storm.kafka.artifact.id}</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.kafka</groupId>
+            <artifactId>kafka-clients</artifactId>
+            <version>${storm.kafka.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-all</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+    <build>
+        <sourceDirectory>src/jvm</sourceDirectory>
+        <testSourceDirectory>src/test</testSourceDirectory>
+        <resources>
+            <resource>
+                <directory>${basedir}/src/resources</directory>
+            </resource>
+        </resources>
+    </build>
+</project>

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/KafkaDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/KafkaDataSourcesProvider.java b/sql/storm-sql-external/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/KafkaDataSourcesProvider.java
new file mode 100644
index 0000000..2aa98ba
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/KafkaDataSourcesProvider.java
@@ -0,0 +1,170 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.kafka;
+
+import com.google.common.base.Preconditions;
+import org.apache.storm.kafka.ZkHosts;
+import org.apache.storm.kafka.trident.OpaqueTridentKafkaSpout;
+import org.apache.storm.kafka.trident.TridentKafkaConfig;
+import org.apache.storm.kafka.trident.TridentKafkaStateFactory;
+import org.apache.storm.kafka.trident.TridentKafkaUpdater;
+import org.apache.storm.kafka.trident.mapper.TridentTupleToKafkaMapper;
+import org.apache.storm.kafka.trident.selector.DefaultTopicSelector;
+import org.apache.storm.spout.Scheme;
+import org.apache.storm.spout.SchemeAsMultiScheme;
+import org.apache.storm.sql.runtime.DataSource;
+import org.apache.storm.sql.runtime.DataSourcesProvider;
+import org.apache.storm.sql.runtime.FieldInfo;
+import org.apache.storm.sql.runtime.IOutputSerializer;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.sql.runtime.SimpleSqlTridentConsumer;
+import org.apache.storm.sql.runtime.utils.SerdeUtils;
+import org.apache.storm.trident.spout.ITridentDataSource;
+import org.apache.storm.trident.tuple.TridentTuple;
+
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+/**
+ * Create a Kafka spout/sink based on the URI and properties. The URI has the format of
+ * kafka://zkhost:port/broker_path?topic=topic. The properties are in JSON format which specifies the producer config
+ * of the Kafka broker.
+ */
+public class KafkaDataSourcesProvider implements DataSourcesProvider {
+  private static final int DEFAULT_ZK_PORT = 2181;
+
+  private static class SqlKafkaMapper implements TridentTupleToKafkaMapper<Object, ByteBuffer> {
+    private final int primaryKeyIndex;
+    private final IOutputSerializer serializer;
+
+    private SqlKafkaMapper(int primaryKeyIndex, IOutputSerializer serializer) {
+      this.primaryKeyIndex = primaryKeyIndex;
+      this.serializer = serializer;
+    }
+
+    @Override
+    public Object getKeyFromTuple(TridentTuple tuple) {
+      return tuple.get(primaryKeyIndex);
+    }
+
+    @Override
+    public ByteBuffer getMessageFromTuple(TridentTuple tuple) {
+      return serializer.write(tuple.getValues(), null);
+    }
+  }
+
+  private static class KafkaTridentDataSource implements ISqlTridentDataSource {
+    private final TridentKafkaConfig conf;
+    private final String topic;
+    private final int primaryKeyIndex;
+    private final Properties props;
+    private final IOutputSerializer serializer;
+    private KafkaTridentDataSource(TridentKafkaConfig conf, String topic, int primaryKeyIndex,
+                                   Properties props, IOutputSerializer serializer) {
+      this.conf = conf;
+      this.topic = topic;
+      this.primaryKeyIndex = primaryKeyIndex;
+      this.props = props;
+      this.serializer = serializer;
+    }
+
+    @Override
+    public ITridentDataSource getProducer() {
+      return new OpaqueTridentKafkaSpout(conf);
+    }
+
+    @Override
+    public SqlTridentConsumer getConsumer() {
+      Preconditions.checkArgument(!props.isEmpty(),
+              "Writable Kafka Table " + topic + " must contain producer config");
+      HashMap<String, Object> producerConfig = (HashMap<String, Object>) props.get("producer");
+      props.putAll(producerConfig);
+      Preconditions.checkState(props.containsKey("bootstrap.servers"),
+              "Writable Kafka Table " + topic + " must contain \"bootstrap.servers\" config");
+
+      SqlKafkaMapper mapper = new SqlKafkaMapper(primaryKeyIndex, serializer);
+
+      TridentKafkaStateFactory stateFactory = new TridentKafkaStateFactory()
+              .withKafkaTopicSelector(new DefaultTopicSelector(topic))
+              .withProducerProperties(props)
+              .withTridentTupleToKafkaMapper(mapper);
+
+      TridentKafkaUpdater stateUpdater = new TridentKafkaUpdater();
+
+      return new SimpleSqlTridentConsumer(stateFactory, stateUpdater);
+    }
+  }
+
+  @Override
+  public String scheme() {
+    return "kafka";
+  }
+
+  @Override
+  public DataSource construct(URI uri, String inputFormatClass, String outputFormatClass,
+                              List<FieldInfo> fields) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
+                                                Properties properties, List<FieldInfo> fields) {
+    int port = uri.getPort() != -1 ? uri.getPort() : DEFAULT_ZK_PORT;
+    ZkHosts zk = new ZkHosts(uri.getHost() + ":" + port, uri.getPath());
+    Map<String, String> values = parseURIParams(uri.getQuery());
+    String topic = values.get("topic");
+    Preconditions.checkNotNull(topic, "No topic of the spout is specified");
+    TridentKafkaConfig conf = new TridentKafkaConfig(zk, topic);
+    List<String> fieldNames = new ArrayList<>();
+    int primaryIndex = -1;
+    for (int i = 0; i < fields.size(); ++i) {
+      FieldInfo f = fields.get(i);
+      fieldNames.add(f.name());
+      if (f.isPrimary()) {
+        primaryIndex = i;
+      }
+    }
+    Preconditions.checkState(primaryIndex != -1, "Kafka stream table must have a primary key");
+    Scheme scheme = SerdeUtils.getScheme(inputFormatClass, properties, fieldNames);
+    conf.scheme = new SchemeAsMultiScheme(scheme);
+    IOutputSerializer serializer = SerdeUtils.getSerializer(outputFormatClass, properties, fieldNames);
+
+    return new KafkaTridentDataSource(conf, topic, primaryIndex, properties, serializer);
+  }
+
+  private static Map<String, String> parseURIParams(String query) {
+    HashMap<String, String> res = new HashMap<>();
+    if (query == null) {
+      return res;
+    }
+
+    String[] params = query.split("&");
+    for (String p : params) {
+      String[] v = p.split("=", 2);
+      if (v.length > 1) {
+        res.put(v[0], v[1]);
+      }
+    }
+    return res;
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-kafka/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-kafka/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider b/sql/storm-sql-external/storm-sql-kafka/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
new file mode 100644
index 0000000..7f687cc
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-kafka/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
@@ -0,0 +1,16 @@
+# 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.
+
+org.apache.storm.sql.kafka.KafkaDataSourcesProvider
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java b/sql/storm-sql-external/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java
new file mode 100644
index 0000000..0cde492
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.kafka;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.storm.kafka.trident.TridentKafkaState;
+import org.apache.storm.kafka.trident.TridentKafkaStateFactory;
+import org.apache.storm.kafka.trident.TridentKafkaUpdater;
+import org.apache.storm.sql.runtime.DataSourcesRegistry;
+import org.apache.storm.sql.runtime.FieldInfo;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.sql.runtime.serde.json.JsonSerializer;
+import org.apache.storm.trident.tuple.TridentTuple;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.ArgumentMatcher;
+import org.mockito.internal.util.reflection.Whitebox;
+
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.*;
+import java.util.concurrent.Future;
+
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.argThat;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+
+public class TestKafkaDataSourcesProvider {
+  private static final List<FieldInfo> FIELDS = ImmutableList.of(
+          new FieldInfo("ID", int.class, true),
+          new FieldInfo("val", String.class, false));
+  private static final List<String> FIELD_NAMES = ImmutableList.of("ID", "val");
+  private static final JsonSerializer SERIALIZER = new JsonSerializer(FIELD_NAMES);
+  private static final Properties TBL_PROPERTIES = new Properties();
+
+  static {
+    Map<String,Object> map = new HashMap<>();
+    map.put("bootstrap.servers", "localhost:9092");
+    map.put("acks", "1");
+    map.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
+    map.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
+    TBL_PROPERTIES.put("producer", map);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testKafkaSink() {
+    ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(
+            URI.create("kafka://mock?topic=foo"), null, null, TBL_PROPERTIES, FIELDS);
+    Assert.assertNotNull(ds);
+
+    ISqlTridentDataSource.SqlTridentConsumer consumer = ds.getConsumer();
+
+    Assert.assertEquals(TridentKafkaStateFactory.class, consumer.getStateFactory().getClass());
+    Assert.assertEquals(TridentKafkaUpdater.class, consumer.getStateUpdater().getClass());
+
+    TridentKafkaState state = (TridentKafkaState) consumer.getStateFactory().makeState(Collections.emptyMap(), null, 0, 1);
+    KafkaProducer producer = mock(KafkaProducer.class);
+    doReturn(mock(Future.class)).when(producer).send(any(ProducerRecord.class));
+    Whitebox.setInternalState(state, "producer", producer);
+
+    List<TridentTuple> tupleList = mockTupleList();
+    for (TridentTuple t : tupleList) {
+      state.updateState(Collections.singletonList(t), null);
+      verify(producer).send(argThat(new KafkaMessageMatcher(t)));
+    }
+    verifyNoMoreInteractions(producer);
+  }
+
+  private static List<TridentTuple> mockTupleList() {
+    List<TridentTuple> tupleList = new ArrayList<>();
+    TridentTuple t0 = mock(TridentTuple.class);
+    TridentTuple t1 = mock(TridentTuple.class);
+    doReturn(1).when(t0).get(0);
+    doReturn(2).when(t1).get(0);
+    doReturn(Lists.<Object>newArrayList(1, "2")).when(t0).getValues();
+    doReturn(Lists.<Object>newArrayList(2, "3")).when(t1).getValues();
+    tupleList.add(t0);
+    tupleList.add(t1);
+    return tupleList;
+  }
+
+  private static class KafkaMessageMatcher extends ArgumentMatcher<ProducerRecord> {
+    private static final int PRIMARY_INDEX = 0;
+    private final TridentTuple tuple;
+
+    private KafkaMessageMatcher(TridentTuple tuple) {
+      this.tuple = tuple;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public boolean matches(Object o) {
+      ProducerRecord<Object, ByteBuffer> m = (ProducerRecord<Object,ByteBuffer>)o;
+      if (m.key() != tuple.get(PRIMARY_INDEX)) {
+        return false;
+      }
+      ByteBuffer buf = m.value();
+      ByteBuffer b = SERIALIZER.write(tuple.getValues(), null);
+      return b.equals(buf);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-mongodb/pom.xml
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-mongodb/pom.xml b/sql/storm-sql-external/storm-sql-mongodb/pom.xml
new file mode 100644
index 0000000..218d89b
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-mongodb/pom.xml
@@ -0,0 +1,84 @@
+<?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>
+        <relativePath>../../../pom.xml</relativePath>
+    </parent>
+
+    <artifactId>storm-sql-mongodb</artifactId>
+
+    <developers>
+        <developer>
+            <id>vesense</id>
+            <name>Xin Wang</name>
+            <email>data.xinwang@gmail.com</email>
+        </developer>
+    </developers>
+
+    <dependencies>
+        <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-sql-runtime</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-sql-runtime</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+            <type>test-jar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-mongodb</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-all</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+    <build>
+        <sourceDirectory>src/jvm</sourceDirectory>
+        <testSourceDirectory>src/test</testSourceDirectory>
+        <resources>
+            <resource>
+                <directory>${basedir}/src/resources</directory>
+            </resource>
+        </resources>
+    </build>
+</project>

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-mongodb/src/jvm/org/apache/storm/sql/mongodb/MongoDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-mongodb/src/jvm/org/apache/storm/sql/mongodb/MongoDataSourcesProvider.java b/sql/storm-sql-external/storm-sql-mongodb/src/jvm/org/apache/storm/sql/mongodb/MongoDataSourcesProvider.java
new file mode 100644
index 0000000..60d52d1
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-mongodb/src/jvm/org/apache/storm/sql/mongodb/MongoDataSourcesProvider.java
@@ -0,0 +1,126 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.mongodb;
+
+import com.google.common.base.Preconditions;
+import org.apache.storm.mongodb.common.mapper.MongoMapper;
+import org.apache.storm.mongodb.trident.state.MongoState;
+import org.apache.storm.mongodb.trident.state.MongoStateFactory;
+import org.apache.storm.mongodb.trident.state.MongoStateUpdater;
+import org.apache.storm.sql.runtime.DataSource;
+import org.apache.storm.sql.runtime.DataSourcesProvider;
+import org.apache.storm.sql.runtime.FieldInfo;
+import org.apache.storm.sql.runtime.IOutputSerializer;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.sql.runtime.SimpleSqlTridentConsumer;
+import org.apache.storm.sql.runtime.utils.FieldInfoUtils;
+import org.apache.storm.sql.runtime.utils.SerdeUtils;
+import org.apache.storm.trident.spout.ITridentDataSource;
+import org.apache.storm.trident.state.StateFactory;
+import org.apache.storm.trident.state.StateUpdater;
+import org.apache.storm.tuple.ITuple;
+import org.bson.Document;
+
+import java.net.URI;
+import java.util.List;
+import java.util.Properties;
+
+/**
+ * Create a MongoDB sink based on the URI and properties. The URI has the format of
+ * mongodb://[username:password@]host1[:port1][,host2[:port2],...[,hostN[:portN]]][/[database][?options]].
+ * The properties are in JSON format which specifies the name of the MongoDB collection and etc.
+ */
+public class MongoDataSourcesProvider implements DataSourcesProvider {
+
+  private static class MongoTridentDataSource implements ISqlTridentDataSource {
+    private final String url;
+    private final Properties props;
+    private final IOutputSerializer serializer;
+
+    private MongoTridentDataSource(String url, Properties props, IOutputSerializer serializer) {
+      this.url = url;
+      this.props = props;
+      this.serializer = serializer;
+    }
+
+    @Override
+    public ITridentDataSource getProducer() {
+      throw new UnsupportedOperationException(this.getClass().getName() + " doesn't provide Producer");
+    }
+
+    @Override
+    public SqlTridentConsumer getConsumer() {
+      Preconditions.checkArgument(!props.isEmpty(), "Writable MongoDB must contain collection config");
+      String serField = props.getProperty("trident.ser.field", "tridentSerField");
+      MongoMapper mapper = new TridentMongoMapper(serField, serializer);
+
+      MongoState.Options options = new MongoState.Options()
+          .withUrl(url)
+          .withCollectionName(props.getProperty("collection.name"))
+          .withMapper(mapper);
+
+      StateFactory stateFactory = new MongoStateFactory(options);
+      StateUpdater stateUpdater = new MongoStateUpdater();
+
+      return new SimpleSqlTridentConsumer(stateFactory, stateUpdater);
+    }
+  }
+
+  private static class TridentMongoMapper implements MongoMapper {
+    private final String serField;
+    private final IOutputSerializer serializer;
+
+    private TridentMongoMapper(String serField, IOutputSerializer serializer) {
+      this.serField = serField;
+      this.serializer = serializer;
+    }
+
+    @Override
+    public Document toDocument(ITuple tuple) {
+      Document document = new Document();
+      byte[] array = serializer.write(tuple.getValues(), null).array();
+      document.append(serField, array);
+      return document;
+    }
+
+    @Override
+    public Document toDocumentByKeys(List<Object> keys) {
+      return null;
+    }
+  }
+
+  @Override
+  public String scheme() {
+    return "mongodb";
+  }
+
+  @Override
+  public DataSource construct(URI uri, String inputFormatClass, String outputFormatClass,
+                              List<FieldInfo> fields) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
+                                                Properties properties, List<FieldInfo> fields) {
+    List<String> fieldNames = FieldInfoUtils.getFieldNames(fields);
+    IOutputSerializer serializer = SerdeUtils.getSerializer(outputFormatClass, properties, fieldNames);
+    return new MongoTridentDataSource(uri.toString(), properties, serializer);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-mongodb/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-mongodb/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider b/sql/storm-sql-external/storm-sql-mongodb/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
new file mode 100644
index 0000000..e46d794
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-mongodb/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
@@ -0,0 +1,16 @@
+# 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.
+
+org.apache.storm.sql.mongodb.MongoDataSourcesProvider
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-mongodb/src/test/org/apache/storm/sql/mongodb/TestMongoDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-mongodb/src/test/org/apache/storm/sql/mongodb/TestMongoDataSourcesProvider.java b/sql/storm-sql-external/storm-sql-mongodb/src/test/org/apache/storm/sql/mongodb/TestMongoDataSourcesProvider.java
new file mode 100644
index 0000000..3b15345
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-mongodb/src/test/org/apache/storm/sql/mongodb/TestMongoDataSourcesProvider.java
@@ -0,0 +1,122 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.mongodb;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import org.apache.storm.mongodb.common.MongoDBClient;
+import org.apache.storm.mongodb.trident.state.MongoState;
+import org.apache.storm.mongodb.trident.state.MongoStateFactory;
+import org.apache.storm.mongodb.trident.state.MongoStateUpdater;
+import org.apache.storm.sql.runtime.DataSourcesRegistry;
+import org.apache.storm.sql.runtime.FieldInfo;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.sql.runtime.serde.json.JsonSerializer;
+import org.apache.storm.trident.state.StateUpdater;
+import org.apache.storm.trident.tuple.TridentTuple;
+import org.bson.Document;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.ArgumentMatcher;
+import org.mockito.internal.util.reflection.Whitebox;
+
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+
+import static org.mockito.Matchers.argThat;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+
+public class TestMongoDataSourcesProvider {
+  private static final List<FieldInfo> FIELDS = ImmutableList.of(
+      new FieldInfo("ID", int.class, true),
+      new FieldInfo("val", String.class, false));
+  private static final List<String> FIELD_NAMES = ImmutableList.of("ID", "val");
+  private static final JsonSerializer SERIALIZER = new JsonSerializer(FIELD_NAMES);
+  private static final Properties TBL_PROPERTIES = new Properties();
+
+  static {
+    TBL_PROPERTIES.put("collection.name", "collection1");
+    TBL_PROPERTIES.put("trident.ser.field", "tridentSerField");
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testMongoSink() {
+    ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(
+            URI.create("mongodb://127.0.0.1:27017/test"), null, null, TBL_PROPERTIES, FIELDS);
+    Assert.assertNotNull(ds);
+
+    ISqlTridentDataSource.SqlTridentConsumer consumer = ds.getConsumer();
+
+    Assert.assertEquals(MongoStateFactory.class, consumer.getStateFactory().getClass());
+    Assert.assertEquals(MongoStateUpdater.class, consumer.getStateUpdater().getClass());
+
+    MongoState state = (MongoState) consumer.getStateFactory().makeState(Collections.emptyMap(), null, 0, 1);
+    StateUpdater stateUpdater = consumer.getStateUpdater();
+
+    MongoDBClient mongoClient = mock(MongoDBClient.class);
+    Whitebox.setInternalState(state, "mongoClient", mongoClient);
+
+    List<TridentTuple> tupleList = mockTupleList();
+
+    for (TridentTuple t : tupleList) {
+      stateUpdater.updateState(state, Collections.singletonList(t), null);
+      verify(mongoClient).insert(argThat(new MongoArgMatcher(t)) , eq(true));
+    }
+
+    verifyNoMoreInteractions(mongoClient);
+  }
+
+  private static List<TridentTuple> mockTupleList() {
+    List<TridentTuple> tupleList = new ArrayList<>();
+    TridentTuple t0 = mock(TridentTuple.class);
+    TridentTuple t1 = mock(TridentTuple.class);
+    doReturn(1).when(t0).get(0);
+    doReturn(2).when(t1).get(0);
+    doReturn(Lists.<Object>newArrayList(1, "2")).when(t0).getValues();
+    doReturn(Lists.<Object>newArrayList(2, "3")).when(t1).getValues();
+    tupleList.add(t0);
+    tupleList.add(t1);
+    return tupleList;
+  }
+
+  private static class MongoArgMatcher extends ArgumentMatcher<List<Document>> {
+    private final TridentTuple tuple;
+
+    private MongoArgMatcher(TridentTuple tuple) {
+      this.tuple = tuple;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public boolean matches(Object o) {
+      Document doc = ((List<Document>)o).get(0);
+      ByteBuffer buf = ByteBuffer.wrap((byte[])doc.get(TBL_PROPERTIES.getProperty("trident.ser.field")));
+      ByteBuffer b = SERIALIZER.write(tuple.getValues(), null);
+      return b.equals(buf);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-redis/pom.xml
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-redis/pom.xml b/sql/storm-sql-external/storm-sql-redis/pom.xml
new file mode 100644
index 0000000..32ce432
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-redis/pom.xml
@@ -0,0 +1,76 @@
+<?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>
+        <relativePath>../../../pom.xml</relativePath>
+    </parent>
+
+    <artifactId>storm-sql-redis</artifactId>
+
+    <dependencies>
+        <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-sql-runtime</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-sql-runtime</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+            <type>test-jar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-redis</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-all</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+    <build>
+        <sourceDirectory>src/jvm</sourceDirectory>
+        <testSourceDirectory>src/test</testSourceDirectory>
+        <resources>
+            <resource>
+                <directory>${basedir}/src/resources</directory>
+            </resource>
+        </resources>
+    </build>
+</project>

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-redis/src/jvm/org/apache/storm/sql/redis/RedisDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-redis/src/jvm/org/apache/storm/sql/redis/RedisDataSourcesProvider.java b/sql/storm-sql-external/storm-sql-redis/src/jvm/org/apache/storm/sql/redis/RedisDataSourcesProvider.java
new file mode 100644
index 0000000..68933b2
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-redis/src/jvm/org/apache/storm/sql/redis/RedisDataSourcesProvider.java
@@ -0,0 +1,228 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.redis;
+
+import com.google.common.base.Preconditions;
+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.apache.storm.redis.trident.state.RedisClusterState;
+import org.apache.storm.redis.trident.state.RedisClusterStateUpdater;
+import org.apache.storm.redis.trident.state.RedisState;
+import org.apache.storm.redis.trident.state.RedisStateUpdater;
+import org.apache.storm.sql.runtime.DataSource;
+import org.apache.storm.sql.runtime.DataSourcesProvider;
+import org.apache.storm.sql.runtime.FieldInfo;
+import org.apache.storm.sql.runtime.IOutputSerializer;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.sql.runtime.SimpleSqlTridentConsumer;
+import org.apache.storm.sql.runtime.utils.SerdeUtils;
+import org.apache.storm.sql.runtime.utils.FieldInfoUtils;
+import org.apache.storm.trident.spout.ITridentDataSource;
+import org.apache.storm.trident.state.StateFactory;
+import org.apache.storm.trident.state.StateUpdater;
+import org.apache.storm.tuple.ITuple;
+import redis.clients.util.JedisURIHelper;
+
+import java.io.Serializable;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+
+/**
+ * Create a Redis sink based on the URI and properties. The URI has the format of
+ * redis://:[password]@[host]:[port]/[dbIdx]. Only host is mandatory and others can be set to default.
+ *
+ * The properties are in JSON format which specifies the config of the Redis data type and etc.
+ * Please note that when "use.redis.cluster" is "true", cluster discovery is only done from given URI.
+ */
+public class RedisDataSourcesProvider implements DataSourcesProvider {
+  private static final int DEFAULT_REDIS_PORT = 6379;
+  private static final int DEFAULT_TIMEOUT = 2000;
+
+  private abstract static class AbstractRedisTridentDataSource implements ISqlTridentDataSource, Serializable {
+    protected abstract StateFactory newStateFactory();
+    protected abstract StateUpdater newStateUpdater(RedisStoreMapper storeMapper);
+
+    private final Properties props;
+    private final List<FieldInfo> fields;
+    private final IOutputSerializer serializer;
+
+    AbstractRedisTridentDataSource(Properties props, List<FieldInfo> fields, IOutputSerializer serializer) {
+      this.props = props;
+      this.fields = fields;
+      this.serializer = serializer;
+    }
+
+    @Override
+    public ITridentDataSource getProducer() {
+      throw new UnsupportedOperationException(this.getClass().getName() + " doesn't provide Producer");
+    }
+
+    @Override
+    public SqlTridentConsumer getConsumer() {
+      RedisDataTypeDescription dataTypeDescription = getDataTypeDesc(props);
+
+      RedisStoreMapper storeMapper = new TridentRedisStoreMapper(dataTypeDescription, fields, serializer);
+
+      StateFactory stateFactory = newStateFactory();
+      StateUpdater stateUpdater = newStateUpdater(storeMapper);
+
+      return new SimpleSqlTridentConsumer(stateFactory, stateUpdater);
+    }
+
+    private RedisDataTypeDescription getDataTypeDesc(Properties props) {
+      Preconditions.checkArgument(props.containsKey("data.type"),
+              "Redis data source must contain \"data.type\" config");
+
+      RedisDataTypeDescription.RedisDataType dataType = RedisDataTypeDescription.RedisDataType.valueOf(props.getProperty("data.type").toUpperCase());
+      String additionalKey = props.getProperty("data.additional.key");
+
+      return new RedisDataTypeDescription(dataType, additionalKey);
+    }
+  }
+
+  private static class RedisClusterTridentDataSource extends AbstractRedisTridentDataSource {
+    private final JedisClusterConfig config;
+
+    RedisClusterTridentDataSource(JedisClusterConfig config, Properties props, List<FieldInfo> fields, IOutputSerializer serializer) {
+      super(props, fields, serializer);
+      this.config = config;
+    }
+
+    @Override
+    protected StateFactory newStateFactory() {
+      return new RedisClusterState.Factory(config);
+    }
+
+    @Override
+    protected StateUpdater newStateUpdater(RedisStoreMapper storeMapper) {
+      return new RedisClusterStateUpdater(storeMapper);
+    }
+  }
+
+  private static class RedisTridentDataSource extends AbstractRedisTridentDataSource {
+    private final JedisPoolConfig config;
+
+    RedisTridentDataSource(JedisPoolConfig config, Properties props, List<FieldInfo> fields, IOutputSerializer serializer) {
+      super(props, fields, serializer);
+      this.config = config;
+    }
+
+    @Override
+    protected StateFactory newStateFactory() {
+      return new RedisState.Factory(config);
+    }
+
+    @Override
+    protected StateUpdater newStateUpdater(RedisStoreMapper storeMapper) {
+      return new RedisStateUpdater(storeMapper);
+    }
+  }
+
+  @Override
+  public String scheme() {
+    return "redis";
+  }
+
+  @Override
+  public DataSource construct(URI uri, String inputFormatClass, String outputFormatClass, List<FieldInfo> fields) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass, Properties props, List<FieldInfo> fields) {
+    Preconditions.checkArgument(JedisURIHelper.isValid(uri), "URI is not valid for Redis: " + uri);
+
+    String host = uri.getHost();
+    int port = uri.getPort() != -1 ? uri.getPort() : DEFAULT_REDIS_PORT;
+    int dbIdx = JedisURIHelper.getDBIndex(uri);
+    String password = JedisURIHelper.getPassword(uri);
+
+    int timeout = Integer.parseInt(props.getProperty("redis.timeout", String.valueOf(DEFAULT_TIMEOUT)));
+
+    boolean clusterMode = Boolean.valueOf(props.getProperty("use.redis.cluster", "false"));
+
+    List<String> fieldNames = FieldInfoUtils.getFieldNames(fields);
+    IOutputSerializer serializer = SerdeUtils.getSerializer(outputFormatClass, props, fieldNames);
+    if (clusterMode) {
+      JedisClusterConfig config = new JedisClusterConfig.Builder()
+              .setNodes(Collections.singleton(new InetSocketAddress(host, port)))
+              .setTimeout(timeout)
+              .build();
+      return new RedisClusterTridentDataSource(config, props, fields, serializer);
+    } else {
+      JedisPoolConfig config = new JedisPoolConfig(host, port, timeout, password, dbIdx);
+      return new RedisTridentDataSource(config, props, fields, serializer);
+    }
+  }
+
+  private static class TridentRedisStoreMapper implements RedisStoreMapper {
+    private final RedisDataTypeDescription dataTypeDescription;
+    private final FieldInfo primaryKeyField;
+    private final IOutputSerializer outputSerializer;
+
+    private TridentRedisStoreMapper(RedisDataTypeDescription dataTypeDescription, List<FieldInfo> fields, IOutputSerializer outputSerializer) {
+      this.dataTypeDescription = dataTypeDescription;
+      this.outputSerializer = outputSerializer;
+
+      // find primary key from constructor
+      FieldInfo pkField = findPrimaryKeyField(fields);
+      Preconditions.checkArgument(pkField != null, "Primary key must be presented to field list");
+
+      this.primaryKeyField = pkField;
+    }
+
+    private FieldInfo findPrimaryKeyField(List<FieldInfo> fields) {
+      FieldInfo pkField = null;
+      for (FieldInfo field : fields) {
+        if (field.isPrimary()) {
+          // TODO: this assumes key is only from the one field
+          // if not we need to have order of fields in PK
+          pkField = field;
+          break;
+        }
+      }
+      return pkField;
+    }
+
+    @Override
+    public RedisDataTypeDescription getDataTypeDescription() {
+      return dataTypeDescription;
+    }
+
+    @Override
+    public String getKeyFromTuple(ITuple tuple) {
+      String keyFieldName = primaryKeyField.name();
+      Object key = tuple.getValueByField(keyFieldName);
+      if (key == null) {
+        throw new NullPointerException("key field " + keyFieldName + " is null");
+      }
+      return String.valueOf(key);
+    }
+
+    @Override
+    public String getValueFromTuple(ITuple tuple) {
+      byte[] array = outputSerializer.write(tuple.getValues(), null).array();
+      return new String(array);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-redis/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-redis/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider b/sql/storm-sql-external/storm-sql-redis/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
new file mode 100644
index 0000000..23b0444
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-redis/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
@@ -0,0 +1,16 @@
+# 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.
+
+org.apache.storm.sql.redis.RedisDataSourcesProvider
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-redis/src/test/org/apache/storm/sql/redis/TestRedisDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-redis/src/test/org/apache/storm/sql/redis/TestRedisDataSourcesProvider.java b/sql/storm-sql-external/storm-sql-redis/src/test/org/apache/storm/sql/redis/TestRedisDataSourcesProvider.java
new file mode 100644
index 0000000..94d4949
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-redis/src/test/org/apache/storm/sql/redis/TestRedisDataSourcesProvider.java
@@ -0,0 +1,158 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.redis;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import org.apache.storm.redis.trident.state.RedisClusterState;
+import org.apache.storm.redis.trident.state.RedisClusterStateUpdater;
+import org.apache.storm.redis.trident.state.RedisState;
+import org.apache.storm.redis.trident.state.RedisStateUpdater;
+import org.apache.storm.sql.runtime.DataSourcesRegistry;
+import org.apache.storm.sql.runtime.FieldInfo;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.sql.runtime.serde.json.JsonSerializer;
+import org.apache.storm.trident.state.StateUpdater;
+import org.apache.storm.trident.tuple.TridentTuple;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.internal.util.reflection.Whitebox;
+import redis.clients.jedis.Jedis;
+import redis.clients.jedis.JedisCluster;
+import redis.clients.jedis.JedisPool;
+import redis.clients.jedis.Pipeline;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class TestRedisDataSourcesProvider {
+  private static final List<FieldInfo> FIELDS = ImmutableList.of(
+      new FieldInfo("ID", int.class, true),
+      new FieldInfo("val", String.class, false));
+  private static final List<String> FIELD_NAMES = ImmutableList.of("ID", "val");
+  private static final String ADDITIONAL_KEY = "hello";
+  private static final JsonSerializer SERIALIZER = new JsonSerializer(FIELD_NAMES);
+  private static final Properties TBL_PROPERTIES = new Properties();
+  private static final Properties CLUSTER_TBL_PROPERTIES = new Properties();
+
+  static {
+    TBL_PROPERTIES.put("data.type", "HASH");
+    TBL_PROPERTIES.put("data.additional.key", ADDITIONAL_KEY);
+    CLUSTER_TBL_PROPERTIES.put("data.type", "HASH");
+    CLUSTER_TBL_PROPERTIES.put("data.additional.key", ADDITIONAL_KEY);
+    CLUSTER_TBL_PROPERTIES.put("use.redis.cluster", "true");
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testRedisSink() {
+    ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(
+        URI.create("redis://:foobared@localhost:6380/2"), null, null, TBL_PROPERTIES, FIELDS);
+    Assert.assertNotNull(ds);
+
+    ISqlTridentDataSource.SqlTridentConsumer consumer = ds.getConsumer();
+
+    Assert.assertEquals(RedisState.Factory.class, consumer.getStateFactory().getClass());
+    Assert.assertEquals(RedisStateUpdater.class, consumer.getStateUpdater().getClass());
+
+    RedisState state = (RedisState) consumer.getStateFactory().makeState(Collections.emptyMap(), null, 0, 1);
+    StateUpdater stateUpdater = consumer.getStateUpdater();
+
+    JedisPool mockJedisPool = mock(JedisPool.class);
+    Jedis mockJedis = mock(Jedis.class);
+    Pipeline mockPipeline = mock(Pipeline.class);
+
+    Whitebox.setInternalState(state, "jedisPool", mockJedisPool);
+    when(mockJedisPool.getResource()).thenReturn(mockJedis);
+    when(mockJedis.pipelined()).thenReturn(mockPipeline);
+
+    List<TridentTuple> tupleList = mockTupleList();
+
+    stateUpdater.updateState(state, tupleList, null);
+    for (TridentTuple t : tupleList) {
+      // PK goes to the key
+      String id = String.valueOf(t.getValueByField("ID"));
+      String serializedValue = new String(SERIALIZER.write(t.getValues(), null).array());
+      verify(mockPipeline).hset(eq(ADDITIONAL_KEY), eq(id), eq(serializedValue));
+    }
+
+    verify(mockPipeline).sync();
+    verify(mockJedis).close();
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testRedisClusterSink() throws IOException {
+    ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(
+        URI.create("redis://localhost:6380"), null, null, CLUSTER_TBL_PROPERTIES, FIELDS);
+    Assert.assertNotNull(ds);
+
+    ISqlTridentDataSource.SqlTridentConsumer consumer = ds.getConsumer();
+
+    Assert.assertEquals(RedisClusterState.Factory.class, consumer.getStateFactory().getClass());
+    Assert.assertEquals(RedisClusterStateUpdater.class, consumer.getStateUpdater().getClass());
+
+    RedisClusterState state = (RedisClusterState) consumer.getStateFactory().makeState(Collections.emptyMap(), null, 0, 1);
+    StateUpdater stateUpdater = consumer.getStateUpdater();
+
+    JedisCluster mockJedisCluster = mock(JedisCluster.class);
+
+    Whitebox.setInternalState(state, "jedisCluster", mockJedisCluster);
+
+    List<TridentTuple> tupleList = mockTupleList();
+
+    stateUpdater.updateState(state, tupleList, null);
+    for (TridentTuple t : tupleList) {
+      // PK goes to the key
+      String id = String.valueOf(t.getValueByField("ID"));
+      String serializedValue = new String(SERIALIZER.write(t.getValues(), null).array());
+      verify(mockJedisCluster).hset(eq(ADDITIONAL_KEY), eq(id), eq(serializedValue));
+    }
+
+    verify(mockJedisCluster, never()).close();
+  }
+
+  private static List<TridentTuple> mockTupleList() {
+    List<TridentTuple> tupleList = new ArrayList<>();
+    TridentTuple t0 = mock(TridentTuple.class);
+    TridentTuple t1 = mock(TridentTuple.class);
+    when(t0.getValueByField("ID")).thenReturn(1);
+    when(t0.getValueByField("val")).thenReturn("2");
+    doReturn(Lists.<Object>newArrayList(1, "2")).when(t0).getValues();
+
+    when(t1.getValueByField("ID")).thenReturn(2);
+    when(t1.getValueByField("val")).thenReturn("3");
+    doReturn(Lists.<Object>newArrayList(2, "3")).when(t1).getValues();
+
+    tupleList.add(t0);
+    tupleList.add(t1);
+    return tupleList;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/pom.xml
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/pom.xml b/sql/storm-sql-runtime/pom.xml
new file mode 100644
index 0000000..ce57cb2
--- /dev/null
+++ b/sql/storm-sql-runtime/pom.xml
@@ -0,0 +1,136 @@
+<?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>
+        <relativePath>../../pom.xml</relativePath>
+    </parent>
+
+    <artifactId>storm-sql-runtime</artifactId>
+
+    <developers>
+        <developer>
+            <id>haohui</id>
+            <name>Haohui Mai</name>
+            <email>ricetons@gmail.com</email>
+        </developer>
+    </developers>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-core</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.calcite</groupId>
+            <artifactId>calcite-core</artifactId>
+            <version>${calcite.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>commons-dbcp</groupId>
+                    <artifactId>commons-dbcp</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>com.google.code.findbugs</groupId>
+                    <artifactId>jsr305</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.pentaho</groupId>
+                    <artifactId>pentaho-aggdesigner-algorithm</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>com.fasterxml.jackson.core</groupId>
+                    <artifactId>jackson-annotations</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.avro</groupId>
+            <artifactId>avro</artifactId>
+            <version>1.7.7</version>
+        </dependency>
+        <dependency>
+            <groupId>commons-lang</groupId>
+            <artifactId>commons-lang</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-csv</artifactId>
+            <version>1.4</version>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-all</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+    <build>
+        <sourceDirectory>src/jvm</sourceDirectory>
+        <testSourceDirectory>src/test</testSourceDirectory>
+        <resources>
+            <resource>
+                <directory>${basedir}/src/resources</directory>
+            </resource>
+        </resources>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>appassembler-maven-plugin</artifactId>
+                <version>1.9</version>
+                <executions>
+                    <execution>
+                        <id>create-repo</id>
+                        <goals>
+                            <goal>create-repository</goal>
+                        </goals>
+                        <configuration>
+                            <assembleDirectory>${project.build.directory}/app-assembler</assembleDirectory>
+                            <repositoryLayout>flat</repositoryLayout>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+</project>

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/calcite/interpreter/StormContext.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/calcite/interpreter/StormContext.java b/sql/storm-sql-runtime/src/jvm/org/apache/calcite/interpreter/StormContext.java
new file mode 100644
index 0000000..aa7e435
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/calcite/interpreter/StormContext.java
@@ -0,0 +1,31 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.calcite.interpreter;
+
+import org.apache.calcite.DataContext;
+
+import java.io.Serializable;
+
+/**
+ * This is a hack to use Calcite Context.
+ */
+public class StormContext extends Context implements Serializable {
+    public StormContext(DataContext root) {
+        super(root);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractChannelHandler.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractChannelHandler.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractChannelHandler.java
new file mode 100644
index 0000000..64be39d
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractChannelHandler.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.runtime;
+
+import org.apache.storm.tuple.Values;
+
+public abstract class AbstractChannelHandler implements ChannelHandler {
+  @Override
+  public abstract void dataReceived(ChannelContext ctx, Values data);
+
+  @Override
+  public void channelInactive(ChannelContext ctx) {
+
+  }
+
+  @Override
+  public void exceptionCaught(Throwable cause) {
+
+  }
+
+  @Override
+  public void flush(ChannelContext ctx) {
+    ctx.flush();
+  }
+
+  @Override
+  public void setSource(ChannelContext ctx, Object source) {
+
+  }
+
+  public static final AbstractChannelHandler PASS_THROUGH = new AbstractChannelHandler() {
+    @Override
+    public void dataReceived(ChannelContext ctx, Values data) {
+      ctx.emit(data);
+    }
+  };
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractValuesProcessor.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractValuesProcessor.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractValuesProcessor.java
new file mode 100644
index 0000000..6a853be
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractValuesProcessor.java
@@ -0,0 +1,44 @@
+/**
+ * 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.sql.runtime;
+
+import org.apache.storm.tuple.Values;
+
+import java.util.Map;
+
+/**
+ * Subclass of AbstractTupleProcessor provides a series of tuple. It
+ * takes a series of iterators of {@link Values} and produces a stream of
+ * tuple.
+ *
+ * The subclass implements the {@see next()} method to provide
+ * the output of the stream. It can choose to return null in {@see next()} to
+ * indicate that this particular iteration is a no-op. SQL processors depend
+ * on this semantic to implement filtering and nullable records.
+ */
+public abstract class AbstractValuesProcessor {
+
+  /**
+   * Initialize the data sources.
+   *
+   * @param data a map from the table name to the iterators of the values.
+   *
+   */
+  public abstract void initialize(Map<String, DataSource> data, ChannelHandler
+      result);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelContext.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelContext.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelContext.java
new file mode 100644
index 0000000..65ad01c
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelContext.java
@@ -0,0 +1,30 @@
+/**
+ * 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.sql.runtime;
+
+import org.apache.storm.tuple.Values;
+
+public interface ChannelContext {
+  /**
+   * Emit data to the next stage of the data pipeline.
+   */
+  void emit(Values data);
+  void fireChannelInactive();
+  void flush();
+  void setSource(Object source);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelHandler.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelHandler.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelHandler.java
new file mode 100644
index 0000000..af02b7e
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelHandler.java
@@ -0,0 +1,41 @@
+/**
+ * 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.sql.runtime;
+
+import org.apache.storm.tuple.Values;
+
+/**
+ * DataListener provides an event-driven interface for the user to process
+ * series of events.
+ */
+public interface ChannelHandler {
+  void dataReceived(ChannelContext ctx, Values data);
+
+  /**
+   * The producer of the data has indicated that the channel is no longer
+   * active.
+   * @param ctx
+   */
+  void channelInactive(ChannelContext ctx);
+
+  void exceptionCaught(Throwable cause);
+
+  void flush(ChannelContext ctx);
+
+  void setSource(ChannelContext ctx, Object source);
+}


[08/23] storm git commit: STORM-2453 Move non-connectors into the top directory

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/existing-topology-reflection.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/existing-topology-reflection.yaml b/flux/flux-core/src/test/resources/configs/existing-topology-reflection.yaml
new file mode 100644
index 0000000..975885b
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/existing-topology-reflection.yaml
@@ -0,0 +1,24 @@
+# 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.
+---
+
+# configuration that uses an existing topology that does not implement TopologySource
+name: "existing-topology"
+topologySource:
+  className: "org.apache.storm.flux.test.SimpleTopology"
+  constructorArgs:
+    - "foo"
+    - "bar"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/existing-topology-trident.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/existing-topology-trident.yaml b/flux/flux-core/src/test/resources/configs/existing-topology-trident.yaml
new file mode 100644
index 0000000..978181b
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/existing-topology-trident.yaml
@@ -0,0 +1,24 @@
+# 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.
+---
+
+name: "existing-topology"
+
+config:
+  topology.workers: 1
+
+topologySource:
+  className: "org.apache.storm.flux.test.TridentTopologySource"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/existing-topology.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/existing-topology.yaml b/flux/flux-core/src/test/resources/configs/existing-topology.yaml
new file mode 100644
index 0000000..e112c0f
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/existing-topology.yaml
@@ -0,0 +1,23 @@
+# 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.
+---
+
+name: "existing-topology"
+topologySource:
+  className: "org.apache.storm.flux.test.SimpleTopologySource"
+  constructorArgs:
+    - "foo"
+    - "bar"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/hdfs_test.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/hdfs_test.yaml b/flux/flux-core/src/test/resources/configs/hdfs_test.yaml
new file mode 100644
index 0000000..2bccb33
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/hdfs_test.yaml
@@ -0,0 +1,97 @@
+# 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.
+
+# Test ability to wire together shell spouts/bolts
+---
+
+# topology definition
+# name to be used when submitting
+name: "hdfs-topology"
+
+# Components
+# Components are analagous to Spring beans. They are meant to be used as constructor,
+# property(setter), and builder arguments.
+#
+# for the time being, components must be declared in the order they are referenced
+components:
+  - id: "syncPolicy"
+    className: "org.apache.storm.hdfs.bolt.sync.CountSyncPolicy"
+    constructorArgs:
+      - 1000
+  - id: "rotationPolicy"
+    className: "org.apache.storm.hdfs.bolt.rotation.FileSizeRotationPolicy"
+    constructorArgs:
+      - 5.0
+      - MB
+
+  - id: "fileNameFormat"
+    className: "org.apache.storm.hdfs.bolt.format.DefaultFileNameFormat"
+    configMethods:
+      - name: "withPath"
+        args: ["/tmp/foo/"]
+      - name: "withExtension"
+        args: [".txt"]
+
+  - id: "recordFormat"
+    className: "org.apache.storm.hdfs.bolt.format.DelimitedRecordFormat"
+    configMethods:
+      - name: "withFieldDelimiter"
+        args: ["|"]
+
+  - id: "rotationAction"
+    className: "org.apache.storm.hdfs.common.rotation.MoveFileAction"
+    configMethods:
+      - name: "toDestination"
+        args: ["/tmp/dest2"]
+
+# spout definitions
+spouts:
+  - id: "spout-1"
+    className: "org.apache.storm.testing.TestWordSpout"
+    parallelism: 1
+    # ...
+
+# bolt definitions
+
+#        HdfsBolt bolt = new HdfsBolt()
+#                .withConfigKey("hdfs.config")
+#                .withFsUrl(args[0])
+#                .withFileNameFormat(fileNameFormat)
+#                .withRecordFormat(format)
+#                .withRotationPolicy(rotationPolicy)
+#                .withSyncPolicy(syncPolicy)
+#                .addRotationAction(new MoveFileAction().toDestination("/tmp/dest2/"));
+bolts:
+  - id: "bolt-1"
+    className: "org.apache.storm.hdfs.bolt.HdfsBolt"
+    configMethods:
+      - name: "withConfigKey"
+        args: ["hdfs.config"]
+      - name: "withFsUrl"
+        args: ["hdfs://hadoop:54310"]
+      - name: "withFileNameFormat"
+        args: [ref: "fileNameFormat"]
+      - name: "withRecordFormat"
+        args: [ref: "recordFormat"]
+      - name: "withRotationPolicy"
+        args: [ref: "rotationPolicy"]
+      - name: "withSyncPolicy"
+        args: [ref: "syncPolicy"]
+      - name: "addRotationAction"
+        args: [ref: "rotationAction"]
+    parallelism: 1
+    # ...
+

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/include_test.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/include_test.yaml b/flux/flux-core/src/test/resources/configs/include_test.yaml
new file mode 100644
index 0000000..702f590
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/include_test.yaml
@@ -0,0 +1,25 @@
+# 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.
+
+# Test includes by defining nothing, and simply override the topology name
+---
+
+name: "include-topology"
+
+includes:
+  - resource: true
+    file: "/configs/shell_test.yaml"
+    override: false #otherwise subsequent includes that define 'name' would override
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/invalid-existing-topology.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/invalid-existing-topology.yaml b/flux/flux-core/src/test/resources/configs/invalid-existing-topology.yaml
new file mode 100644
index 0000000..c2dfac0
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/invalid-existing-topology.yaml
@@ -0,0 +1,33 @@
+# 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.
+
+# This is an invalid config. It defines both a topologySource and a list of spouts.
+---
+
+name: "existing-topology"
+topologySource:
+  className: "org.apache.storm.flux.test.SimpleTopologySource"
+
+spouts:
+  - id: "sentence-spout"
+    className: "org.apache.storm.flux.wrappers.spouts.FluxShellSpout"
+    # shell spout constructor takes 2 arguments: String[], String[]
+    constructorArgs:
+      # command line
+      - ["node", "randomsentence.js"]
+      # output fields
+      - ["word"]
+    parallelism: 1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/kafka_test.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/kafka_test.yaml b/flux/flux-core/src/test/resources/configs/kafka_test.yaml
new file mode 100644
index 0000000..1fb59ca
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/kafka_test.yaml
@@ -0,0 +1,126 @@
+# 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.
+---
+
+# topology definition
+# name to be used when submitting
+name: "kafka-topology"
+
+# Components
+# Components are analagous to Spring beans. They are meant to be used as constructor,
+# property(setter), and builder arguments.
+#
+# for the time being, components must be declared in the order they are referenced
+components:
+  - id: "stringScheme"
+    className: "org.apache.storm.kafka.StringScheme"
+
+  - id: "stringMultiScheme"
+    className: "org.apache.storm.spout.SchemeAsMultiScheme"
+    constructorArgs:
+      - ref: "stringScheme"
+
+  - id: "zkHosts"
+    className: "org.apache.storm.kafka.ZkHosts"
+    constructorArgs:
+      - "localhost:2181"
+
+# Alternative kafka config
+#  - id: "kafkaConfig"
+#    className: "org.apache.storm.kafka.KafkaConfig"
+#    constructorArgs:
+#      # brokerHosts
+#      - ref: "zkHosts"
+#      # topic
+#      - "myKafkaTopic"
+#      # clientId (optional)
+#      - "myKafkaClientId"
+
+  - id: "spoutConfig"
+    className: "org.apache.storm.kafka.SpoutConfig"
+    constructorArgs:
+      # brokerHosts
+      - ref: "zkHosts"
+      # topic
+      - "myKafkaTopic"
+      # zkRoot
+      - "/kafkaSpout"
+      # id
+      - "myId"
+    properties:
+      - name: "ignoreZkOffsets"
+        value: true
+      - name: "scheme"
+        ref: "stringMultiScheme"
+
+# topology configuration
+# this will be passed to the submitter as a map of config options
+#
+config:
+  topology.workers: 1
+  # ...
+
+# spout definitions
+spouts:
+  - id: "kafka-spout"
+    className: "org.apache.storm.kafka.KafkaSpout"
+    constructorArgs:
+      - ref: "spoutConfig"
+
+# bolt definitions
+bolts:
+  - id: "splitsentence"
+    className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
+    constructorArgs:
+      # command line
+      - ["python", "splitsentence.py"]
+      # output fields
+      - ["word"]
+    parallelism: 1
+
+  - id: "log"
+    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+    parallelism: 1
+    # ...
+
+  - id: "count"
+    className: "org.apache.storm.testing.TestWordCounter"
+    parallelism: 1
+
+#stream definitions
+# stream definitions define connections between spouts and bolts.
+# note that such connections can be cyclical
+# custom stream groupings are also supported
+
+streams:
+  - name: "kafka --> split" # name isn't used (placeholder for logging, UI, etc.)
+    from: "kafka-spout"
+    to: "splitsentence"
+    grouping:
+      type: SHUFFLE
+
+  - name: "split --> count"
+    from: "splitsentence"
+    to: "count"
+    grouping:
+      type: FIELDS
+      args: ["word"]
+
+  - name: "count --> log"
+    from: "count"
+    to: "log"
+    grouping:
+      type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/shell_test.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/shell_test.yaml b/flux/flux-core/src/test/resources/configs/shell_test.yaml
new file mode 100644
index 0000000..dfab397
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/shell_test.yaml
@@ -0,0 +1,118 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# Test ability to wire together shell spouts/bolts
+---
+
+# topology definition
+# name to be used when submitting
+name: "shell-topology"
+
+# Components
+# Components are analagous to Spring beans. They are meant to be used as constructor,
+# property(setter), and builder arguments.
+#components:
+#  - id: "myComponent"
+#    className: "com.foo.bar.MyComponent"
+#    constructorArgs:
+#      - ...
+#    properties:
+#      foo: "bar"
+#      bar: "foo"
+
+# NOTE: We may want to consider some level of spring integration. For example, allowing component references
+# to a spring `ApplicationContext`.
+
+# topology configuration
+# this will be passed to the submitter as a map of config options
+#
+config:
+  topology.workers: 1
+  # ...
+
+# spout definitions
+spouts:
+  - id: "sentence-spout"
+    className: "org.apache.storm.flux.wrappers.spouts.FluxShellSpout"
+    # shell spout constructor takes 2 arguments: String[], String[]
+    constructorArgs:
+      # command line
+      - ["node", "randomsentence.js"]
+      # output fields
+      - ["word"]
+    configMethods:
+      - name: "addComponentConfig"
+        args: ["rabbitmq.configfile", "etc/rabbit.yml"]
+      - name: "addComponentConfig"
+        args:
+        - "publisher.data_paths"
+        - ["actions"]
+    parallelism: 1
+    # ...
+
+# bolt definitions
+bolts:
+  - id: "splitsentence"
+    className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
+    constructorArgs:
+      # command line
+      - ["python", "splitsentence.py"]
+      # output fields
+      - ["word"]
+    configMethods:
+      - name: "addComponentConfig"
+        args: ["rabbitmq.configfile", "etc/rabbit.yml"]
+      - name: "addComponentConfig"
+        args:
+        - "publisher.data_paths"
+        - ["actions"]
+    parallelism: 1
+    # ...
+
+  - id: "log"
+    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+    parallelism: 1
+    # ...
+
+  - id: "count"
+    className: "org.apache.storm.testing.TestWordCounter"
+    parallelism: 1
+    # ...
+
+#stream definitions
+# stream definitions define connections between spouts and bolts.
+# note that such connections can be cyclical
+# custom stream groupings are also supported
+
+streams:
+  - name: "spout --> split" # name isn't used (placeholder for logging, UI, etc.)
+    from: "sentence-spout"
+    to: "splitsentence"
+    grouping:
+      type: SHUFFLE
+
+  - name: "split --> count"
+    from: "splitsentence"
+    to: "count"
+    grouping:
+      type: FIELDS
+      args: ["word"]
+
+  - name: "count --> log"
+    from: "count"
+    to: "log"
+    grouping:
+      type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/simple_hbase.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/simple_hbase.yaml b/flux/flux-core/src/test/resources/configs/simple_hbase.yaml
new file mode 100644
index 0000000..b841b53
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/simple_hbase.yaml
@@ -0,0 +1,120 @@
+# 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.
+
+# Test ability to wire together shell spouts/bolts
+---
+
+# topology definition
+# name to be used when submitting
+name: "hbase-wordcount"
+
+# Components
+# Components are analagous to Spring beans. They are meant to be used as constructor,
+# property(setter), and builder arguments.
+#
+# for the time being, components must be declared in the order they are referenced
+
+#        WordSpout spout = new WordSpout();
+#        WordCounter bolt = new WordCounter();
+#
+#        SimpleHBaseMapper mapper = new SimpleHBaseMapper()
+#                .withRowKeyField("word")
+#                .withColumnFields(new Fields("word"))
+#                .withCounterFields(new Fields("count"))
+#                .withColumnFamily("cf");
+#
+#        HBaseBolt hbase = new HBaseBolt("WordCount", mapper)
+#                .withConfigKey("hbase.conf");
+#
+#
+#        // wordSpout ==> countBolt ==> HBaseBolt
+#        TopologyBuilder builder = new TopologyBuilder();
+#
+#        builder.setSpout(WORD_SPOUT, spout, 1);
+#        builder.setBolt(COUNT_BOLT, bolt, 1).shuffleGrouping(WORD_SPOUT);
+#        builder.setBolt(HBASE_BOLT, hbase, 1).fieldsGrouping(COUNT_BOLT, new Fields("word"));
+
+
+
+
+components:
+  - id: "columnFields"
+    className: "org.apache.storm.tuple.Fields"
+    constructorArgs:
+      - ["word"]
+
+  - id: "counterFields"
+    className: "org.apache.storm.tuple.Fields"
+    constructorArgs:
+      - ["count"]
+
+  - id: "mapper"
+    className: "org.apache.storm.hbase.bolt.mapper.SimpleHBaseMapper"
+    configMethods:
+      - name: "withRowKeyField"
+        args: ["word"]
+      - name: "withColumnFields"
+        args: [ref: "columnFields"]
+      - name: "withCounterFields"
+        args: [ref: "counterFields"]
+      - name: "withColumnFamily"
+        args: ["cf"]
+
+# topology configuration
+# this will be passed to the submitter as a map of config options
+#
+config:
+  topology.workers: 1
+  hbase.conf:
+    hbase.rootdir: "hdfs://hadoop:54310/hbase"
+    hbase.zookeeper.quorum: "hadoop"
+
+# spout definitions
+spouts:
+  - id: "word-spout"
+    className: "org.apache.storm.testing.TestWordSpout"
+    parallelism: 1
+
+# bolt definitions
+
+bolts:
+  - id: "count-bolt"
+    className: "org.apache.storm.testing.TestWordCounter"
+
+  - id: "hbase-bolt"
+    className: "org.apache.storm.hbase.bolt.HBaseBolt"
+    constructorArgs:
+      - "WordCount" # HBase table name
+      - ref: "mapper"
+    configMethods:
+      - name: "withConfigKey"
+        args: ["hbase.conf"]
+    parallelism: 1
+
+
+streams:
+  - name: "" # name isn't used (placeholder for logging, UI, etc.)
+    from: "word-spout"
+    to: "count-bolt"
+    grouping:
+      type: SHUFFLE
+
+  - name: "" # name isn't used (placeholder for logging, UI, etc.)
+    from: "count-bolt"
+    to: "hbase-bolt"
+    grouping:
+      type: FIELDS
+      args: ["word"]

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/substitution-test.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/substitution-test.yaml b/flux/flux-core/src/test/resources/configs/substitution-test.yaml
new file mode 100644
index 0000000..ce9e62d
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/substitution-test.yaml
@@ -0,0 +1,106 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# Test ability to wire together shell spouts/bolts
+---
+
+# topology definition
+# name to be used when submitting
+name: "${topology.name}"
+
+# Components
+# Components are analagous to Spring beans. They are meant to be used as constructor,
+# property(setter), and builder arguments.
+#components:
+#  - id: "myComponent"
+#    className: "com.foo.bar.MyComponent"
+#    constructorArgs:
+#      - ...
+#    properties:
+#      foo: "bar"
+#      bar: "foo"
+
+# NOTE: We may want to consider some level of spring integration. For example, allowing component references
+# to a spring `ApplicationContext`.
+
+# topology configuration
+# this will be passed to the submitter as a map of config options
+#
+config:
+  topology.workers: 1
+  # test environent variable substitution
+  test.env.value: ${ENV-PATH}
+  # ...
+
+# spout definitions
+spouts:
+  - id: "sentence-spout"
+    className: "org.apache.storm.flux.wrappers.spouts.FluxShellSpout"
+    # shell spout constructor takes 2 arguments: String[], String[]
+    constructorArgs:
+      # command line
+      - ["node", "randomsentence.js"]
+      # output fields
+      - ["word"]
+    parallelism: 1
+    # ...
+
+# bolt definitions
+bolts:
+  - id: "splitsentence"
+    className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
+    constructorArgs:
+      # command line
+      - ["python", "splitsentence.py"]
+      # output fields
+      - ["word"]
+    parallelism: 1
+    # ...
+
+  - id: "log"
+    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+    parallelism: 1
+    # ...
+
+  - id: "count"
+    className: "org.apache.storm.testing.TestWordCounter"
+    parallelism: 1
+    # ...
+
+#stream definitions
+# stream definitions define connections between spouts and bolts.
+# note that such connections can be cyclical
+# custom stream groupings are also supported
+
+streams:
+  - name: "spout --> split" # name isn't used (placeholder for logging, UI, etc.)
+    from: "sentence-spout"
+    to: "splitsentence"
+    grouping:
+      type: SHUFFLE
+
+  - name: "split --> count"
+    from: "splitsentence"
+    to: "count"
+    grouping:
+      type: FIELDS
+      args: ["word"]
+
+  - name: "count --> log"
+    from: "count"
+    to: "log"
+    grouping:
+      type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/tck.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/tck.yaml b/flux/flux-core/src/test/resources/configs/tck.yaml
new file mode 100644
index 0000000..5d40445
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/tck.yaml
@@ -0,0 +1,95 @@
+# 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.
+
+
+# YAML configuration to serve as a basic smoke test for what is supported.
+#
+# We should support comments, so if we've failed so far, things aren't good.
+
+# we shouldn't choke if we see a document separator...
+---
+
+# topology definition
+# name to be used when submitting
+name: "yaml-topology"
+
+# Components
+# Components are analagous to Spring beans. They are meant to be used as constructor,
+# property(setter), and builder arguments.
+#components:
+#  - id: "myComponent"
+#    className: "com.foo.bar.MyComponent"
+#    properties:
+#      foo: "bar"
+#      bar: "foo"
+
+# NOTE: We may want to consider some level of spring integration. For example, allowing component references
+# to a spring `ApplicationContext`.
+
+# topology configuration
+# this will be passed to the submitter as a map of config options
+#
+config:
+  topology.workers: 1
+  # ...
+
+# spout definitions
+spouts:
+  - id: "spout-1"
+    className: "org.apache.storm.testing.TestWordSpout"
+    parallelism: 1
+    # ...
+
+# bolt definitions
+bolts:
+  - id: "bolt-1"
+    className: "org.apache.storm.testing.TestWordCounter"
+    parallelism: 1
+    # ...
+
+  - id: "bolt-2"
+    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+    parallelism: 1
+    # ...
+
+#stream definitions
+# stream definitions define connections between spouts and bolts.
+# note that such connections can be cyclical
+streams:
+  - name: "spout-1 --> bolt-1" # name isn't used (placeholder for logging, UI, etc.)
+#    id: "connection-1"
+    from: "spout-1"
+    to: "bolt-1"
+    grouping:
+      type: FIELDS
+      args: ["word"]
+
+  - name: "bolt-1 --> bolt2"
+    from: "bolt-1"
+    to: "bolt-2"
+    grouping:
+      type: CUSTOM
+      customClass:
+        className: "org.apache.storm.testing.NGrouping"
+        constructorArgs:
+          - 1
+
+
+
+
+
+
+

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/test.properties
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/test.properties b/flux/flux-core/src/test/resources/configs/test.properties
new file mode 100644
index 0000000..ecd89d9
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/test.properties
@@ -0,0 +1,18 @@
+# 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.
+
+topology.name: substitution-topology
+some.other.property: foo bar
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/log4j2.xml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/log4j2.xml b/flux/flux-core/src/test/resources/log4j2.xml
new file mode 100644
index 0000000..d3333f3
--- /dev/null
+++ b/flux/flux-core/src/test/resources/log4j2.xml
@@ -0,0 +1,34 @@
+<?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.
+-->
+<Configuration scan="true" monitorInterval="30">
+  <Appenders>
+    <Console name="A1" target="SYSTEM_OUT">
+      <PatternLayout>
+        <pattern>%-4r [%t] %-5level %c{1} - %msg%n</pattern>
+      </PatternLayout>
+    </Console>
+  </Appenders>
+  <Loggers>
+    <Logger name="org.apache.storm.curator" level="warn"/>
+    <Logger name="org.apache.storm.flux" level="debug"/>
+    <Logger name="org.apache.storm.zookeeper" level="warn"/>
+    <Root level="debug">
+      <appender-ref ref="A1"/>
+    </Root>
+  </Loggers>
+</Configuration>

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/README.md
----------------------------------------------------------------------
diff --git a/flux/flux-examples/README.md b/flux/flux-examples/README.md
new file mode 100644
index 0000000..3d610b4
--- /dev/null
+++ b/flux/flux-examples/README.md
@@ -0,0 +1,93 @@
+# Flux Examples
+A collection of examples illustrating various capabilities.
+
+## Building From Source and Running
+
+Checkout the projects source and perform a top level Maven build (i.e. from the `flux` directory):
+
+```bash
+git clone https://github.com/apache/storm.git
+cd storm
+mvn install -DskipTests=true
+```
+
+This will create a shaded (i.e. "fat" or "uber") jar in the `external/flux/flux-examples/target` directory that can run/deployed with
+the `storm` command:
+
+```bash
+cd flux-examples
+storm jar ./target/flux-examples-*-SNAPSHOT.jar org.apache.storm.flux.Flux --local ./src/main/resources/simple_wordcount.yaml
+```
+
+The example YAML files are also packaged in the examples jar, so they can also be referenced with Flux's `--resource`
+command line switch:
+
+```bash
+storm jar ./target/flux-examples-*.jar org.apache.storm.flux.Flux --local --resource /simple_wordcount.yaml
+```
+
+## Available Examples
+
+### [simple_wordcount.yaml](src/main/resources/simple_wordcount.yaml)
+
+This is a very basic wordcount example using Java spouts and bolts. It simply logs the running count of each word
+received.
+
+### [multilang.yaml](src/main/resources/multilang.yaml)
+
+Another wordcount example that uses a spout written in JavaScript (node.js), a bolt written in Python, and two bolts
+written in java.
+
+### [kafka_spout.yaml](src/main/resources/kafka_spout.yaml)
+
+This example illustrates how to configure Storm's `storm-kafka` spout using Flux YAML DSL `components`, `references`,
+and `constructor arguments` constructs.
+
+### [simple_hdfs.yaml](src/main/resources/simple_hdfs.yaml)
+
+This example demonstrates using Flux to setup a storm-hdfs bolt to write to an HDFS cluster. It also demonstrates Flux's
+variable substitution/filtering feature.
+
+To run the `simple_hdfs.yaml` example, copy the `hdfs_bolt.properties` file to a convenient location and change, at
+least, the property `hdfs.url` to point to a HDFS cluster. Then you can run the example something like:
+
+```bash
+storm jar ./target/flux-examples-*.jar org.apache.storm.flux.Flux --local ./src/main/resources/simple_hdfs.yaml --filter my_hdfs_bolt.properties
+```
+
+### [simple_hbase.yaml](src/main/resources/simple_hbase.yaml)
+
+This example illustrates how to use Flux to setup a storm-hbase bolt to write to HBase.
+
+To run the `simple_hbase.yaml` example, copy the `hbase_bolt.properties` file to a convenient location and change the properties
+ `hbase.rootdir` and `hbase.zookeeper.quorum`. Then you can run the example something like:
+
+```bash
+storm jar ./target/flux-examples-*.jar org.apache.storm.flux.Flux --local ./src/main/resources/simple_hbase.yaml --filter my_hbase_bolt.properties
+```
+
+### [simple_windowing.yaml](src/main/resources/simple_windowing.yaml)
+
+This example illustrates how to use Flux to set up a storm topology that contains windowing operations.
+
+To run,
+
+```bash
+storm jar ./target/flux-examples-*.jar org.apache.storm.flux.Flux --local ./src/main/resources/simple_windowing.yaml
+```
+
+### [simple_stateful_wordcount.yaml](src/main/resources/simple_stateful_wordcount.yaml)
+
+Flux also supports stateful bolts which is illustrated with this example. It is basically an extension of the basic wordcount example.
+The state is periodically saved (checkpointed) and restored when the topology is restarted.
+
+```bash
+storm jar ./target/flux-examples-*.jar org.apache.storm.flux.Flux --local ./src/main/resources/simple_stateful_wordcount.yaml
+```
+
+By default the state is stored in-memory only. As such you won't see a resumed state unless you configure to use Redis as the state backend.
+Ensure that you have Redis running at `localhost:6379` and that `storm-redis-*.jar` is in the classpath.
+
+```bash
+STORM_EXT_CLASSPATH=../../storm-redis/target storm jar ./target/flux-examples-*.jar -c topology.state.provider=org.apache.storm.redis.state.RedisKeyValueStateProvider org.apache.storm.flux.Flux --local ./src/main/resources/simple_stateful_wordcount.yaml
+```

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/pom.xml
----------------------------------------------------------------------
diff --git a/flux/flux-examples/pom.xml b/flux/flux-examples/pom.xml
new file mode 100644
index 0000000..ef56ebd
--- /dev/null
+++ b/flux/flux-examples/pom.xml
@@ -0,0 +1,146 @@
+<?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>
+        <groupId>org.apache.storm</groupId>
+        <artifactId>flux</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+
+    <artifactId>flux-examples</artifactId>
+    <packaging>jar</packaging>
+
+    <name>flux-examples</name>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>flux-core</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>flux-wrappers</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-hdfs</artifactId>
+            <version>${project.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>commons-beanutils</groupId>
+                    <artifactId>commons-beanutils-core</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>commons-beanutils</groupId>
+                    <artifactId>commons-beanutils</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-yarn-api</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>javax.servlet-api</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-hbase</artifactId>
+            <version>${project.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.mortbay.jetty</groupId>
+                    <artifactId>servlet-api-2.5</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>commons-collections</groupId>
+                    <artifactId>commons-collections</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>javax.servlet</groupId>
+                    <artifactId>servlet-api</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-kafka</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.kafka</groupId>
+            <artifactId>${storm.kafka.artifact.id}</artifactId>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-shade-plugin</artifactId>
+                <configuration>
+                    <createDependencyReducedPom>true</createDependencyReducedPom>
+                    <filters>
+                        <filter>
+                            <artifact>*:*</artifact>
+                            <excludes>
+                                <exclude>META-INF/*.SF</exclude>
+                                <exclude>META-INF/*.sf</exclude>
+                                <exclude>META-INF/*.DSA</exclude>
+                                <exclude>META-INF/*.dsa</exclude>
+                                <exclude>META-INF/*.RSA</exclude>
+                                <exclude>META-INF/*.rsa</exclude>
+                                <exclude>META-INF/*.EC</exclude>
+                                <exclude>META-INF/*.ec</exclude>
+                                <exclude>META-INF/MSFTSIG.SF</exclude>
+                                <exclude>META-INF/MSFTSIG.RSA</exclude>
+                            </excludes>
+                        </filter>
+                    </filters>
+                </configuration>
+                <executions>
+                    <execution>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>shade</goal>
+                        </goals>
+                        <configuration>
+                            <transformers>
+                                <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
+                                <transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
+                                    <mainClass>org.apache.storm.flux.Flux</mainClass>
+                                </transformer>
+                            </transformers>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+</project>

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/StatefulWordCounter.java
----------------------------------------------------------------------
diff --git a/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/StatefulWordCounter.java b/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/StatefulWordCounter.java
new file mode 100644
index 0000000..5534888
--- /dev/null
+++ b/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/StatefulWordCounter.java
@@ -0,0 +1,64 @@
+/*
+ * 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.flux.examples;
+
+import org.apache.storm.state.KeyValueState;
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.base.BaseStatefulBolt;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.Values;
+
+import java.util.Map;
+
+public class StatefulWordCounter extends BaseStatefulBolt<KeyValueState<String, Long>> {
+
+    private KeyValueState<String, Long> wordCounts;
+    private OutputCollector collector;
+
+    @SuppressWarnings("rawtypes")
+    @Override
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        this.collector = collector;
+    }
+
+    @Override
+    public void initState(KeyValueState<String, Long> state) {
+        wordCounts = state;
+    }
+
+    @Override
+    public void execute(Tuple tuple) {
+        String word = tuple.getString(0);
+
+        Long count = wordCounts.get(word, 0L);
+        count++;
+        wordCounts.put(word, count);
+
+        collector.emit(tuple, new Values(word, count));
+        collector.ack(tuple);
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        declarer.declare(new Fields("word", "count"));
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestPrintBolt.java
----------------------------------------------------------------------
diff --git a/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestPrintBolt.java b/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestPrintBolt.java
new file mode 100644
index 0000000..137e354
--- /dev/null
+++ b/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestPrintBolt.java
@@ -0,0 +1,39 @@
+/**
+ * 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.flux.examples;
+
+import org.apache.storm.topology.BasicOutputCollector;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseBasicBolt;
+import org.apache.storm.tuple.Tuple;
+
+/**
+ * Prints the tuples to stdout
+ */
+public class TestPrintBolt extends BaseBasicBolt {
+
+    @Override
+    public void execute(Tuple tuple, BasicOutputCollector collector) {
+        System.out.println(tuple);
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer ofd) {
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestWindowBolt.java
----------------------------------------------------------------------
diff --git a/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestWindowBolt.java b/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestWindowBolt.java
new file mode 100644
index 0000000..8c904d9
--- /dev/null
+++ b/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestWindowBolt.java
@@ -0,0 +1,47 @@
+/*
+ * 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.flux.examples;
+
+import org.apache.storm.task.OutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseWindowedBolt;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.windowing.TupleWindow;
+
+import java.util.Map;
+
+public class TestWindowBolt extends BaseWindowedBolt {
+    private OutputCollector collector;
+
+    @Override
+    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+        this.collector = collector;
+    }
+
+    @Override
+    public void execute(TupleWindow inputWindow) {
+        collector.emit(new Values(inputWindow.get().size()));
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        declarer.declare(new Fields("count"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCountClient.java
----------------------------------------------------------------------
diff --git a/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCountClient.java b/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCountClient.java
new file mode 100644
index 0000000..eb4fb7a
--- /dev/null
+++ b/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCountClient.java
@@ -0,0 +1,74 @@
+/**
+ * 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.flux.examples;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import java.io.FileInputStream;
+import java.util.Properties;
+
+/**
+ * Connects to the 'WordCount' HBase table and prints counts for each word.
+ *
+ * Assumes you have run (or are running) the YAML topology definition in
+ * <code>simple_hbase.yaml</code>
+ *
+ * You will also need to modify `src/main/resources/hbase-site.xml`
+ * to point to your HBase instance, and then repackage with `mvn package`.
+ * This is a known issue.
+ *
+ */
+public class WordCountClient {
+
+    public static void main(String[] args) throws Exception {
+        Configuration config = HBaseConfiguration.create();
+        if(args.length == 1){
+            Properties props = new Properties();
+            props.load(new FileInputStream(args[0]));
+            System.out.println("HBase configuration:");
+            for(Object key : props.keySet()) {
+                System.out.println(key + "=" + props.get(key));
+                config.set((String)key, props.getProperty((String)key));
+            }
+        } else {
+            System.out.println("Usage: WordCountClient <hbase_config.properties>");
+            System.exit(1);
+        }
+
+        HTable table = new HTable(config, "WordCount");
+        String[] words = new String[] {"nathan", "mike", "jackson", "golda", "bertels"};
+
+        for (String word : words) {
+            Get get = new Get(Bytes.toBytes(word));
+            Result result = table.get(get);
+
+            byte[] countBytes = result.getValue(Bytes.toBytes("cf"), Bytes.toBytes("count"));
+            byte[] wordBytes = result.getValue(Bytes.toBytes("cf"), Bytes.toBytes("word"));
+
+            String wordStr = Bytes.toString(wordBytes);
+            long count = Bytes.toLong(countBytes);
+            System.out.println("Word: '" + wordStr + "', Count: " + count);
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCounter.java
----------------------------------------------------------------------
diff --git a/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCounter.java b/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCounter.java
new file mode 100644
index 0000000..7093105
--- /dev/null
+++ b/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCounter.java
@@ -0,0 +1,71 @@
+/**
+ * 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.flux.examples;
+
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.BasicOutputCollector;
+import org.apache.storm.topology.IBasicBolt;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseBasicBolt;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+
+import static org.apache.storm.utils.Utils.tuple;
+
+/**
+ * This bolt is used by the HBase example. It simply emits the first field
+ * found in the incoming tuple as "word", with a "count" of `1`.
+ *
+ * In this case, the downstream HBase bolt handles the counting, so a value
+ * of `1` will just increment the HBase counter by one.
+ */
+public class WordCounter extends BaseBasicBolt {
+    private static final Logger LOG = LoggerFactory.getLogger(WordCounter.class);
+
+
+
+    @SuppressWarnings("rawtypes")
+    public void prepare(Map stormConf, TopologyContext context) {
+    }
+
+    /*
+     * Just output the word value with a count of 1.
+     * The HBaseBolt will handle incrementing the counter.
+     */
+    public void execute(Tuple input, BasicOutputCollector collector) {
+        collector.emit(tuple(input.getValues().get(0), 1));
+    }
+
+    public void cleanup() {
+
+    }
+
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        declarer.declare(new Fields("word", "count"));
+    }
+
+    @Override
+    public Map<String, Object> getComponentConfiguration() {
+        return null;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/src/main/resources/hbase_bolt.properties
----------------------------------------------------------------------
diff --git a/flux/flux-examples/src/main/resources/hbase_bolt.properties b/flux/flux-examples/src/main/resources/hbase_bolt.properties
new file mode 100644
index 0000000..f8ed50c
--- /dev/null
+++ b/flux/flux-examples/src/main/resources/hbase_bolt.properties
@@ -0,0 +1,18 @@
+# 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.
+
+hbase.rootdir=hdfs://hadoop:54310/hbase
+hbase.zookeeper.quorum=hadoop
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/src/main/resources/hdfs_bolt.properties
----------------------------------------------------------------------
diff --git a/flux/flux-examples/src/main/resources/hdfs_bolt.properties b/flux/flux-examples/src/main/resources/hdfs_bolt.properties
new file mode 100644
index 0000000..7bcbe7a
--- /dev/null
+++ b/flux/flux-examples/src/main/resources/hdfs_bolt.properties
@@ -0,0 +1,26 @@
+# 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.
+
+
+# The HDFS url
+hdfs.url=hdfs://hadoop:54310
+
+# The HDFS directory where the bolt will write incoming data
+hdfs.write.dir=/incoming
+
+# The HDFS directory where files will be moved once the bolt has
+# finished writing to it.
+hdfs.dest.dir=/complete
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/src/main/resources/kafka_spout.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-examples/src/main/resources/kafka_spout.yaml b/flux/flux-examples/src/main/resources/kafka_spout.yaml
new file mode 100644
index 0000000..7533ce4
--- /dev/null
+++ b/flux/flux-examples/src/main/resources/kafka_spout.yaml
@@ -0,0 +1,136 @@
+# 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.
+
+
+# Test ability to wire together shell spouts/bolts
+---
+
+# topology definition
+# name to be used when submitting
+name: "kafka-topology"
+
+# Components
+# Components are analagous to Spring beans. They are meant to be used as constructor,
+# property(setter), and builder arguments.
+#
+# for the time being, components must be declared in the order they are referenced
+components:
+  - id: "stringScheme"
+    className: "org.apache.storm.kafka.StringScheme"
+
+  - id: "stringMultiScheme"
+    className: "org.apache.storm.spout.SchemeAsMultiScheme"
+    constructorArgs:
+      - ref: "stringScheme"
+
+  - id: "zkHosts"
+    className: "org.apache.storm.kafka.ZkHosts"
+    constructorArgs:
+      - "localhost:2181"
+
+# Alternative kafka config
+#  - id: "kafkaConfig"
+#    className: "org.apache.storm.kafka.KafkaConfig"
+#    constructorArgs:
+#      # brokerHosts
+#      - ref: "zkHosts"
+#      # topic
+#      - "myKafkaTopic"
+#      # clientId (optional)
+#      - "myKafkaClientId"
+
+  - id: "spoutConfig"
+    className: "org.apache.storm.kafka.SpoutConfig"
+    constructorArgs:
+      # brokerHosts
+      - ref: "zkHosts"
+      # topic
+      - "myKafkaTopic"
+      # zkRoot
+      - "/kafkaSpout"
+      # id
+      - "myId"
+    properties:
+      - name: "ignoreZkOffsets"
+        value: true
+      - name: "scheme"
+        ref: "stringMultiScheme"
+
+
+
+# NOTE: We may want to consider some level of spring integration. For example, allowing component references
+# to a spring `ApplicationContext`.
+
+# topology configuration
+# this will be passed to the submitter as a map of config options
+#
+config:
+  topology.workers: 1
+  # ...
+
+# spout definitions
+spouts:
+  - id: "kafka-spout"
+    className: "org.apache.storm.kafka.KafkaSpout"
+    constructorArgs:
+      - ref: "spoutConfig"
+
+# bolt definitions
+bolts:
+  - id: "splitsentence"
+    className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
+    constructorArgs:
+      # command line
+      - ["python", "splitsentence.py"]
+      # output fields
+      - ["word"]
+    parallelism: 1
+    # ...
+
+  - id: "log"
+    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+    parallelism: 1
+    # ...
+
+  - id: "count"
+    className: "org.apache.storm.testing.TestWordCounter"
+    parallelism: 1
+    # ...
+
+#stream definitions
+# stream definitions define connections between spouts and bolts.
+# note that such connections can be cyclical
+# custom stream groupings are also supported
+
+streams:
+  - name: "kafka --> split" # name isn't used (placeholder for logging, UI, etc.)
+    from: "kafka-spout"
+    to: "splitsentence"
+    grouping:
+      type: SHUFFLE
+
+  - name: "split --> count"
+    from: "splitsentence"
+    to: "count"
+    grouping:
+      type: FIELDS
+      args: ["word"]
+
+  - name: "count --> log"
+    from: "count"
+    to: "log"
+    grouping:
+      type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/src/main/resources/multilang.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-examples/src/main/resources/multilang.yaml b/flux/flux-examples/src/main/resources/multilang.yaml
new file mode 100644
index 0000000..aaab5d3
--- /dev/null
+++ b/flux/flux-examples/src/main/resources/multilang.yaml
@@ -0,0 +1,89 @@
+# 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.
+
+# Test ability to wire together shell spouts/bolts
+---
+
+# topology definition
+# name to be used when submitting
+name: "shell-topology"
+
+# topology configuration
+# this will be passed to the submitter as a map of config options
+#
+config:
+  topology.workers: 1
+  # ...
+
+# spout definitions
+spouts:
+  - id: "sentence-spout"
+    className: "org.apache.storm.flux.wrappers.spouts.FluxShellSpout"
+    # shell spout constructor takes 2 arguments: String[], String[]
+    constructorArgs:
+      # command line
+      - ["node", "randomsentence.js"]
+      # output fields
+      - ["word"]
+    parallelism: 1
+    # ...
+
+# bolt definitions
+bolts:
+  - id: "splitsentence"
+    className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
+    constructorArgs:
+      # command line
+      - ["python", "splitsentence.py"]
+      # output fields
+      - ["word"]
+    parallelism: 1
+    # ...
+
+  - id: "log"
+    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+    parallelism: 1
+    # ...
+
+  - id: "count"
+    className: "org.apache.storm.testing.TestWordCounter"
+    parallelism: 1
+    # ...
+
+#stream definitions
+# stream definitions define connections between spouts and bolts.
+# note that such connections can be cyclical
+# custom stream groupings are also supported
+
+streams:
+  - name: "spout --> split" # name isn't used (placeholder for logging, UI, etc.)
+    from: "sentence-spout"
+    to: "splitsentence"
+    grouping:
+      type: SHUFFLE
+
+  - name: "split --> count"
+    from: "splitsentence"
+    to: "count"
+    grouping:
+      type: FIELDS
+      args: ["word"]
+
+  - name: "count --> log"
+    from: "count"
+    to: "log"
+    grouping:
+      type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/src/main/resources/simple_hbase.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-examples/src/main/resources/simple_hbase.yaml b/flux/flux-examples/src/main/resources/simple_hbase.yaml
new file mode 100644
index 0000000..93a2781
--- /dev/null
+++ b/flux/flux-examples/src/main/resources/simple_hbase.yaml
@@ -0,0 +1,92 @@
+# 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.
+---
+# NOTE: To use this example, you will need to modify `src/main/resources/hbase-site.xml`
+# to point to your HBase instance, and then repackage with `mvn package`.
+# This is a known issue.
+
+# topology definition
+# name to be used when submitting
+name: "hbase-persistent-wordcount"
+
+# Components
+components:
+  - id: "columnFields"
+    className: "org.apache.storm.tuple.Fields"
+    constructorArgs:
+      - ["word"]
+
+  - id: "counterFields"
+    className: "org.apache.storm.tuple.Fields"
+    constructorArgs:
+      - ["count"]
+
+  - id: "mapper"
+    className: "org.apache.storm.hbase.bolt.mapper.SimpleHBaseMapper"
+    configMethods:
+      - name: "withRowKeyField"
+        args: ["word"]
+      - name: "withColumnFields"
+        args: [ref: "columnFields"]
+      - name: "withCounterFields"
+        args: [ref: "counterFields"]
+      - name: "withColumnFamily"
+        args: ["cf"]
+
+# topology configuration
+# this will be passed to the submitter as a map of config options
+config:
+  topology.workers: 1
+  hbase.conf:
+    hbase.rootdir: "${hbase.rootdir}"
+    hbase.zookeeper.quorum: "${hbase.zookeeper.quorum}"
+
+# spout definitions
+spouts:
+  - id: "word-spout"
+    className: "org.apache.storm.testing.TestWordSpout"
+    parallelism: 1
+
+# bolt definitions
+
+bolts:
+  - id: "count-bolt"
+    className: "org.apache.storm.flux.examples.WordCounter"
+    parallelism: 1
+
+  - id: "hbase-bolt"
+    className: "org.apache.storm.hbase.bolt.HBaseBolt"
+    constructorArgs:
+      - "WordCount" # HBase table name
+      - ref: "mapper"
+    configMethods:
+      - name: "withConfigKey"
+        args: ["hbase.conf"]
+    parallelism: 1
+
+streams:
+  - name: "" # name isn't used (placeholder for logging, UI, etc.)
+    from: "word-spout"
+    to: "count-bolt"
+    grouping:
+      type: SHUFFLE
+
+  - name: "" # name isn't used (placeholder for logging, UI, etc.)
+    from: "count-bolt"
+    to: "hbase-bolt"
+    grouping:
+      type: FIELDS
+      args: ["word"]

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/src/main/resources/simple_hdfs.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-examples/src/main/resources/simple_hdfs.yaml b/flux/flux-examples/src/main/resources/simple_hdfs.yaml
new file mode 100644
index 0000000..b8d4020
--- /dev/null
+++ b/flux/flux-examples/src/main/resources/simple_hdfs.yaml
@@ -0,0 +1,105 @@
+# 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.
+
+# Test ability to wire together shell spouts/bolts
+---
+
+# topology definition
+# name to be used when submitting
+name: "hdfs-topology"
+
+# Components
+# Components are analagous to Spring beans. They are meant to be used as constructor,
+# property(setter), and builder arguments.
+#
+# for the time being, components must be declared in the order they are referenced
+components:
+  - id: "syncPolicy"
+    className: "org.apache.storm.hdfs.bolt.sync.CountSyncPolicy"
+    constructorArgs:
+      - 1000
+  - id: "rotationPolicy"
+    className: "org.apache.storm.hdfs.bolt.rotation.TimedRotationPolicy"
+    constructorArgs:
+      - 30
+      - SECONDS
+
+  - id: "fileNameFormat"
+    className: "org.apache.storm.hdfs.bolt.format.DefaultFileNameFormat"
+    configMethods:
+      - name: "withPath"
+        args: ["${hdfs.write.dir}"]
+      - name: "withExtension"
+        args: [".txt"]
+
+  - id: "recordFormat"
+    className: "org.apache.storm.hdfs.bolt.format.DelimitedRecordFormat"
+    configMethods:
+      - name: "withFieldDelimiter"
+        args: ["|"]
+
+  - id: "rotationAction"
+    className: "org.apache.storm.hdfs.common.rotation.MoveFileAction"
+    configMethods:
+      - name: "toDestination"
+        args: ["${hdfs.dest.dir}"]
+
+# spout definitions
+spouts:
+  - id: "spout-1"
+    className: "org.apache.storm.testing.TestWordSpout"
+    parallelism: 1
+    # ...
+
+# bolt definitions
+
+bolts:
+  - id: "bolt-1"
+    className: "org.apache.storm.hdfs.bolt.HdfsBolt"
+    configMethods:
+      - name: "withConfigKey"
+        args: ["hdfs.config"]
+      - name: "withFsUrl"
+        args: ["${hdfs.url}"]
+      - name: "withFileNameFormat"
+        args: [ref: "fileNameFormat"]
+      - name: "withRecordFormat"
+        args: [ref: "recordFormat"]
+      - name: "withRotationPolicy"
+        args: [ref: "rotationPolicy"]
+      - name: "withSyncPolicy"
+        args: [ref: "syncPolicy"]
+      - name: "addRotationAction"
+        args: [ref: "rotationAction"]
+    parallelism: 1
+    # ...
+
+  - id: "bolt-2"
+    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+    parallelism: 1
+
+streams:
+  - name: "" # name isn't used (placeholder for logging, UI, etc.)
+    from: "spout-1"
+    to: "bolt-1"
+    grouping:
+      type: SHUFFLE
+
+  - name: "" # name isn't used (placeholder for logging, UI, etc.)
+    from: "spout-1"
+    to: "bolt-2"
+    grouping:
+      type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/src/main/resources/simple_stateful_wordcount.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-examples/src/main/resources/simple_stateful_wordcount.yaml b/flux/flux-examples/src/main/resources/simple_stateful_wordcount.yaml
new file mode 100644
index 0000000..14b9b3a
--- /dev/null
+++ b/flux/flux-examples/src/main/resources/simple_stateful_wordcount.yaml
@@ -0,0 +1,60 @@
+# 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.
+
+---
+
+# topology definition
+# name to be used when submitting
+name: "stateful-wordcount-topology"
+
+# topology configuration
+# this will be passed to the submitter as a map of config options
+#
+config:
+  topology.workers: 1
+
+# spout definitions
+spouts:
+  - id: "spout-1"
+    className: "org.apache.storm.testing.TestWordSpout"
+    parallelism: 1
+
+# bolt definitions
+bolts:
+  - id: "bolt-1"
+    className: "org.apache.storm.flux.examples.StatefulWordCounter"
+    parallelism: 1
+
+  - id: "bolt-2"
+    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+    parallelism: 1
+
+#stream definitions
+# stream definitions define connections between spouts and bolts.
+# note that such connections can be cyclical
+streams:
+  - name: "spout-1 --> bolt-1" # name isn't used (placeholder for logging, UI, etc.)
+    from: "spout-1"
+    to: "bolt-1"
+    grouping:
+      type: FIELDS
+      args: ["word"]
+
+  - name: "bolt-1 --> bolt2"
+    from: "bolt-1"
+    to: "bolt-2"
+    grouping:
+      type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/src/main/resources/simple_windowing.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-examples/src/main/resources/simple_windowing.yaml b/flux/flux-examples/src/main/resources/simple_windowing.yaml
new file mode 100755
index 0000000..f2f74ff
--- /dev/null
+++ b/flux/flux-examples/src/main/resources/simple_windowing.yaml
@@ -0,0 +1,69 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+---
+
+name: "sliding-window-topology"
+
+components:
+  - id: "windowLength"
+    className: "org.apache.storm.topology.base.BaseWindowedBolt$Count"
+    constructorArgs:
+      - 5
+  - id: "slidingInterval"
+    className: "org.apache.storm.topology.base.BaseWindowedBolt$Count"
+    constructorArgs:
+      - 3
+
+config:
+  topology.workers: 1
+
+# spout definitions
+spouts:
+  - id: "spout-1"
+    className: "org.apache.storm.testing.TestWordSpout"
+    parallelism: 1
+
+# bolt definitions
+bolts:
+  - id: "bolt-1"
+    className: "org.apache.storm.flux.examples.TestWindowBolt"
+    configMethods:
+      - name: "withWindow"
+        args: [ref: "windowLength", ref: "slidingInterval"]
+    parallelism: 1
+  - id: "bolt-2"
+    className: "org.apache.storm.flux.examples.TestPrintBolt"
+    parallelism: 1
+
+
+#stream definitions
+# stream definitions define connections between spouts and bolts.
+# note that such connections can be cyclical
+streams:
+  - name: "spout-1 --> bolt-1" # name isn't used (placeholder for logging, UI, etc.)
+#    id: "connection-1"
+    from: "spout-1"
+    to: "bolt-1"
+    grouping:
+      type: FIELDS
+      args: ["word"]
+  - name: "bolt-1 --> bolt-2" # name isn't used (placeholder for logging, UI, etc.)
+#    id: "connection-1"
+    from: "bolt-1"
+    to: "bolt-2"
+    grouping:
+      type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-examples/src/main/resources/simple_wordcount.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-examples/src/main/resources/simple_wordcount.yaml b/flux/flux-examples/src/main/resources/simple_wordcount.yaml
new file mode 100644
index 0000000..6443a97
--- /dev/null
+++ b/flux/flux-examples/src/main/resources/simple_wordcount.yaml
@@ -0,0 +1,68 @@
+# 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.
+
+---
+
+# topology definition
+# name to be used when submitting
+name: "yaml-topology"
+
+# topology configuration
+# this will be passed to the submitter as a map of config options
+#
+config:
+  topology.workers: 1
+
+# spout definitions
+spouts:
+  - id: "spout-1"
+    className: "org.apache.storm.testing.TestWordSpout"
+    parallelism: 1
+
+# bolt definitions
+bolts:
+  - id: "bolt-1"
+    className: "org.apache.storm.testing.TestWordCounter"
+    parallelism: 1
+
+  - id: "bolt-2"
+    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+    parallelism: 1
+
+#stream definitions
+# stream definitions define connections between spouts and bolts.
+# note that such connections can be cyclical
+streams:
+  - name: "spout-1 --> bolt-1" # name isn't used (placeholder for logging, UI, etc.)
+#    id: "connection-1"
+    from: "spout-1"
+    to: "bolt-1"
+    grouping:
+      type: FIELDS
+      args: ["word"]
+
+  - name: "bolt-1 --> bolt2"
+    from: "bolt-1"
+    to: "bolt-2"
+    grouping:
+      type: SHUFFLE
+
+
+
+
+
+
+

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-ui/README.md
----------------------------------------------------------------------
diff --git a/flux/flux-ui/README.md b/flux/flux-ui/README.md
new file mode 100644
index 0000000..8b6bd5f
--- /dev/null
+++ b/flux/flux-ui/README.md
@@ -0,0 +1,3 @@
+# Flux-UI
+
+Placeholder for Flux GUI
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-wrappers/pom.xml
----------------------------------------------------------------------
diff --git a/flux/flux-wrappers/pom.xml b/flux/flux-wrappers/pom.xml
new file mode 100644
index 0000000..7d97a9f
--- /dev/null
+++ b/flux/flux-wrappers/pom.xml
@@ -0,0 +1,51 @@
+<?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>
+        <groupId>org.apache.storm</groupId>
+        <artifactId>flux</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+
+    <artifactId>flux-wrappers</artifactId>
+    <packaging>jar</packaging>
+
+    <name>flux-wrappers</name>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>multilang-javascript</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>multilang-ruby</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>multilang-python</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+    </dependencies>
+
+</project>


[18/23] storm git commit: STORM-2453 Move non-connectors into the top directory

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/pom.xml
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/pom.xml b/external/flux/flux-examples/pom.xml
deleted file mode 100644
index ef56ebd..0000000
--- a/external/flux/flux-examples/pom.xml
+++ /dev/null
@@ -1,146 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements.  See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License.  You may obtain a copy of the License at
-
-     http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <modelVersion>4.0.0</modelVersion>
-
-    <parent>
-        <groupId>org.apache.storm</groupId>
-        <artifactId>flux</artifactId>
-        <version>2.0.0-SNAPSHOT</version>
-        <relativePath>../pom.xml</relativePath>
-    </parent>
-
-    <artifactId>flux-examples</artifactId>
-    <packaging>jar</packaging>
-
-    <name>flux-examples</name>
-
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>flux-core</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>flux-wrappers</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>storm-hdfs</artifactId>
-            <version>${project.version}</version>
-            <exclusions>
-                <exclusion>
-                    <groupId>commons-beanutils</groupId>
-                    <artifactId>commons-beanutils-core</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>commons-beanutils</groupId>
-                    <artifactId>commons-beanutils</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.apache.hadoop</groupId>
-                    <artifactId>hadoop-yarn-api</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>javax.servlet</groupId>
-                    <artifactId>servlet-api</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>javax.servlet</groupId>
-                    <artifactId>javax.servlet-api</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>storm-hbase</artifactId>
-            <version>${project.version}</version>
-            <exclusions>
-                <exclusion>
-                    <groupId>org.mortbay.jetty</groupId>
-                    <artifactId>servlet-api-2.5</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>commons-collections</groupId>
-                    <artifactId>commons-collections</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>javax.servlet</groupId>
-                    <artifactId>servlet-api</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>storm-kafka</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.kafka</groupId>
-            <artifactId>${storm.kafka.artifact.id}</artifactId>
-        </dependency>
-    </dependencies>
-
-    <build>
-        <plugins>
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-shade-plugin</artifactId>
-                <configuration>
-                    <createDependencyReducedPom>true</createDependencyReducedPom>
-                    <filters>
-                        <filter>
-                            <artifact>*:*</artifact>
-                            <excludes>
-                                <exclude>META-INF/*.SF</exclude>
-                                <exclude>META-INF/*.sf</exclude>
-                                <exclude>META-INF/*.DSA</exclude>
-                                <exclude>META-INF/*.dsa</exclude>
-                                <exclude>META-INF/*.RSA</exclude>
-                                <exclude>META-INF/*.rsa</exclude>
-                                <exclude>META-INF/*.EC</exclude>
-                                <exclude>META-INF/*.ec</exclude>
-                                <exclude>META-INF/MSFTSIG.SF</exclude>
-                                <exclude>META-INF/MSFTSIG.RSA</exclude>
-                            </excludes>
-                        </filter>
-                    </filters>
-                </configuration>
-                <executions>
-                    <execution>
-                        <phase>package</phase>
-                        <goals>
-                            <goal>shade</goal>
-                        </goals>
-                        <configuration>
-                            <transformers>
-                                <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
-                                <transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
-                                    <mainClass>org.apache.storm.flux.Flux</mainClass>
-                                </transformer>
-                            </transformers>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-        </plugins>
-    </build>
-</project>

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/StatefulWordCounter.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/StatefulWordCounter.java b/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/StatefulWordCounter.java
deleted file mode 100644
index 5534888..0000000
--- a/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/StatefulWordCounter.java
+++ /dev/null
@@ -1,64 +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.flux.examples;
-
-import org.apache.storm.state.KeyValueState;
-import org.apache.storm.task.OutputCollector;
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.topology.base.BaseStatefulBolt;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Tuple;
-import org.apache.storm.tuple.Values;
-
-import java.util.Map;
-
-public class StatefulWordCounter extends BaseStatefulBolt<KeyValueState<String, Long>> {
-
-    private KeyValueState<String, Long> wordCounts;
-    private OutputCollector collector;
-
-    @SuppressWarnings("rawtypes")
-    @Override
-    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
-        this.collector = collector;
-    }
-
-    @Override
-    public void initState(KeyValueState<String, Long> state) {
-        wordCounts = state;
-    }
-
-    @Override
-    public void execute(Tuple tuple) {
-        String word = tuple.getString(0);
-
-        Long count = wordCounts.get(word, 0L);
-        count++;
-        wordCounts.put(word, count);
-
-        collector.emit(tuple, new Values(word, count));
-        collector.ack(tuple);
-    }
-
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer declarer) {
-        declarer.declare(new Fields("word", "count"));
-    }
-    
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestPrintBolt.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestPrintBolt.java b/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestPrintBolt.java
deleted file mode 100644
index 137e354..0000000
--- a/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestPrintBolt.java
+++ /dev/null
@@ -1,39 +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.flux.examples;
-
-import org.apache.storm.topology.BasicOutputCollector;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.topology.base.BaseBasicBolt;
-import org.apache.storm.tuple.Tuple;
-
-/**
- * Prints the tuples to stdout
- */
-public class TestPrintBolt extends BaseBasicBolt {
-
-    @Override
-    public void execute(Tuple tuple, BasicOutputCollector collector) {
-        System.out.println(tuple);
-    }
-
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer ofd) {
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestWindowBolt.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestWindowBolt.java b/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestWindowBolt.java
deleted file mode 100644
index 8c904d9..0000000
--- a/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/TestWindowBolt.java
+++ /dev/null
@@ -1,47 +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.flux.examples;
-
-import org.apache.storm.task.OutputCollector;
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.topology.base.BaseWindowedBolt;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Values;
-import org.apache.storm.windowing.TupleWindow;
-
-import java.util.Map;
-
-public class TestWindowBolt extends BaseWindowedBolt {
-    private OutputCollector collector;
-
-    @Override
-    public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
-        this.collector = collector;
-    }
-
-    @Override
-    public void execute(TupleWindow inputWindow) {
-        collector.emit(new Values(inputWindow.get().size()));
-    }
-
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer declarer) {
-        declarer.declare(new Fields("count"));
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCountClient.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCountClient.java b/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCountClient.java
deleted file mode 100644
index eb4fb7a..0000000
--- a/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCountClient.java
+++ /dev/null
@@ -1,74 +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.flux.examples;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.util.Bytes;
-
-import java.io.FileInputStream;
-import java.util.Properties;
-
-/**
- * Connects to the 'WordCount' HBase table and prints counts for each word.
- *
- * Assumes you have run (or are running) the YAML topology definition in
- * <code>simple_hbase.yaml</code>
- *
- * You will also need to modify `src/main/resources/hbase-site.xml`
- * to point to your HBase instance, and then repackage with `mvn package`.
- * This is a known issue.
- *
- */
-public class WordCountClient {
-
-    public static void main(String[] args) throws Exception {
-        Configuration config = HBaseConfiguration.create();
-        if(args.length == 1){
-            Properties props = new Properties();
-            props.load(new FileInputStream(args[0]));
-            System.out.println("HBase configuration:");
-            for(Object key : props.keySet()) {
-                System.out.println(key + "=" + props.get(key));
-                config.set((String)key, props.getProperty((String)key));
-            }
-        } else {
-            System.out.println("Usage: WordCountClient <hbase_config.properties>");
-            System.exit(1);
-        }
-
-        HTable table = new HTable(config, "WordCount");
-        String[] words = new String[] {"nathan", "mike", "jackson", "golda", "bertels"};
-
-        for (String word : words) {
-            Get get = new Get(Bytes.toBytes(word));
-            Result result = table.get(get);
-
-            byte[] countBytes = result.getValue(Bytes.toBytes("cf"), Bytes.toBytes("count"));
-            byte[] wordBytes = result.getValue(Bytes.toBytes("cf"), Bytes.toBytes("word"));
-
-            String wordStr = Bytes.toString(wordBytes);
-            long count = Bytes.toLong(countBytes);
-            System.out.println("Word: '" + wordStr + "', Count: " + count);
-        }
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCounter.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCounter.java b/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCounter.java
deleted file mode 100644
index 7093105..0000000
--- a/external/flux/flux-examples/src/main/java/org/apache/storm/flux/examples/WordCounter.java
+++ /dev/null
@@ -1,71 +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.flux.examples;
-
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.topology.BasicOutputCollector;
-import org.apache.storm.topology.IBasicBolt;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.topology.base.BaseBasicBolt;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Tuple;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Map;
-
-import static org.apache.storm.utils.Utils.tuple;
-
-/**
- * This bolt is used by the HBase example. It simply emits the first field
- * found in the incoming tuple as "word", with a "count" of `1`.
- *
- * In this case, the downstream HBase bolt handles the counting, so a value
- * of `1` will just increment the HBase counter by one.
- */
-public class WordCounter extends BaseBasicBolt {
-    private static final Logger LOG = LoggerFactory.getLogger(WordCounter.class);
-
-
-
-    @SuppressWarnings("rawtypes")
-    public void prepare(Map stormConf, TopologyContext context) {
-    }
-
-    /*
-     * Just output the word value with a count of 1.
-     * The HBaseBolt will handle incrementing the counter.
-     */
-    public void execute(Tuple input, BasicOutputCollector collector) {
-        collector.emit(tuple(input.getValues().get(0), 1));
-    }
-
-    public void cleanup() {
-
-    }
-
-    public void declareOutputFields(OutputFieldsDeclarer declarer) {
-        declarer.declare(new Fields("word", "count"));
-    }
-
-    @Override
-    public Map<String, Object> getComponentConfiguration() {
-        return null;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/src/main/resources/hbase_bolt.properties
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/src/main/resources/hbase_bolt.properties b/external/flux/flux-examples/src/main/resources/hbase_bolt.properties
deleted file mode 100644
index f8ed50c..0000000
--- a/external/flux/flux-examples/src/main/resources/hbase_bolt.properties
+++ /dev/null
@@ -1,18 +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.
-
-hbase.rootdir=hdfs://hadoop:54310/hbase
-hbase.zookeeper.quorum=hadoop
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/src/main/resources/hdfs_bolt.properties
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/src/main/resources/hdfs_bolt.properties b/external/flux/flux-examples/src/main/resources/hdfs_bolt.properties
deleted file mode 100644
index 7bcbe7a..0000000
--- a/external/flux/flux-examples/src/main/resources/hdfs_bolt.properties
+++ /dev/null
@@ -1,26 +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.
-
-
-# The HDFS url
-hdfs.url=hdfs://hadoop:54310
-
-# The HDFS directory where the bolt will write incoming data
-hdfs.write.dir=/incoming
-
-# The HDFS directory where files will be moved once the bolt has
-# finished writing to it.
-hdfs.dest.dir=/complete
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/src/main/resources/kafka_spout.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/src/main/resources/kafka_spout.yaml b/external/flux/flux-examples/src/main/resources/kafka_spout.yaml
deleted file mode 100644
index 7533ce4..0000000
--- a/external/flux/flux-examples/src/main/resources/kafka_spout.yaml
+++ /dev/null
@@ -1,136 +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.
-
-
-# Test ability to wire together shell spouts/bolts
----
-
-# topology definition
-# name to be used when submitting
-name: "kafka-topology"
-
-# Components
-# Components are analagous to Spring beans. They are meant to be used as constructor,
-# property(setter), and builder arguments.
-#
-# for the time being, components must be declared in the order they are referenced
-components:
-  - id: "stringScheme"
-    className: "org.apache.storm.kafka.StringScheme"
-
-  - id: "stringMultiScheme"
-    className: "org.apache.storm.spout.SchemeAsMultiScheme"
-    constructorArgs:
-      - ref: "stringScheme"
-
-  - id: "zkHosts"
-    className: "org.apache.storm.kafka.ZkHosts"
-    constructorArgs:
-      - "localhost:2181"
-
-# Alternative kafka config
-#  - id: "kafkaConfig"
-#    className: "org.apache.storm.kafka.KafkaConfig"
-#    constructorArgs:
-#      # brokerHosts
-#      - ref: "zkHosts"
-#      # topic
-#      - "myKafkaTopic"
-#      # clientId (optional)
-#      - "myKafkaClientId"
-
-  - id: "spoutConfig"
-    className: "org.apache.storm.kafka.SpoutConfig"
-    constructorArgs:
-      # brokerHosts
-      - ref: "zkHosts"
-      # topic
-      - "myKafkaTopic"
-      # zkRoot
-      - "/kafkaSpout"
-      # id
-      - "myId"
-    properties:
-      - name: "ignoreZkOffsets"
-        value: true
-      - name: "scheme"
-        ref: "stringMultiScheme"
-
-
-
-# NOTE: We may want to consider some level of spring integration. For example, allowing component references
-# to a spring `ApplicationContext`.
-
-# topology configuration
-# this will be passed to the submitter as a map of config options
-#
-config:
-  topology.workers: 1
-  # ...
-
-# spout definitions
-spouts:
-  - id: "kafka-spout"
-    className: "org.apache.storm.kafka.KafkaSpout"
-    constructorArgs:
-      - ref: "spoutConfig"
-
-# bolt definitions
-bolts:
-  - id: "splitsentence"
-    className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
-    constructorArgs:
-      # command line
-      - ["python", "splitsentence.py"]
-      # output fields
-      - ["word"]
-    parallelism: 1
-    # ...
-
-  - id: "log"
-    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
-    parallelism: 1
-    # ...
-
-  - id: "count"
-    className: "org.apache.storm.testing.TestWordCounter"
-    parallelism: 1
-    # ...
-
-#stream definitions
-# stream definitions define connections between spouts and bolts.
-# note that such connections can be cyclical
-# custom stream groupings are also supported
-
-streams:
-  - name: "kafka --> split" # name isn't used (placeholder for logging, UI, etc.)
-    from: "kafka-spout"
-    to: "splitsentence"
-    grouping:
-      type: SHUFFLE
-
-  - name: "split --> count"
-    from: "splitsentence"
-    to: "count"
-    grouping:
-      type: FIELDS
-      args: ["word"]
-
-  - name: "count --> log"
-    from: "count"
-    to: "log"
-    grouping:
-      type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/src/main/resources/multilang.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/src/main/resources/multilang.yaml b/external/flux/flux-examples/src/main/resources/multilang.yaml
deleted file mode 100644
index aaab5d3..0000000
--- a/external/flux/flux-examples/src/main/resources/multilang.yaml
+++ /dev/null
@@ -1,89 +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.
-
-# Test ability to wire together shell spouts/bolts
----
-
-# topology definition
-# name to be used when submitting
-name: "shell-topology"
-
-# topology configuration
-# this will be passed to the submitter as a map of config options
-#
-config:
-  topology.workers: 1
-  # ...
-
-# spout definitions
-spouts:
-  - id: "sentence-spout"
-    className: "org.apache.storm.flux.wrappers.spouts.FluxShellSpout"
-    # shell spout constructor takes 2 arguments: String[], String[]
-    constructorArgs:
-      # command line
-      - ["node", "randomsentence.js"]
-      # output fields
-      - ["word"]
-    parallelism: 1
-    # ...
-
-# bolt definitions
-bolts:
-  - id: "splitsentence"
-    className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
-    constructorArgs:
-      # command line
-      - ["python", "splitsentence.py"]
-      # output fields
-      - ["word"]
-    parallelism: 1
-    # ...
-
-  - id: "log"
-    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
-    parallelism: 1
-    # ...
-
-  - id: "count"
-    className: "org.apache.storm.testing.TestWordCounter"
-    parallelism: 1
-    # ...
-
-#stream definitions
-# stream definitions define connections between spouts and bolts.
-# note that such connections can be cyclical
-# custom stream groupings are also supported
-
-streams:
-  - name: "spout --> split" # name isn't used (placeholder for logging, UI, etc.)
-    from: "sentence-spout"
-    to: "splitsentence"
-    grouping:
-      type: SHUFFLE
-
-  - name: "split --> count"
-    from: "splitsentence"
-    to: "count"
-    grouping:
-      type: FIELDS
-      args: ["word"]
-
-  - name: "count --> log"
-    from: "count"
-    to: "log"
-    grouping:
-      type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/src/main/resources/simple_hbase.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/src/main/resources/simple_hbase.yaml b/external/flux/flux-examples/src/main/resources/simple_hbase.yaml
deleted file mode 100644
index 93a2781..0000000
--- a/external/flux/flux-examples/src/main/resources/simple_hbase.yaml
+++ /dev/null
@@ -1,92 +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.
----
-# NOTE: To use this example, you will need to modify `src/main/resources/hbase-site.xml`
-# to point to your HBase instance, and then repackage with `mvn package`.
-# This is a known issue.
-
-# topology definition
-# name to be used when submitting
-name: "hbase-persistent-wordcount"
-
-# Components
-components:
-  - id: "columnFields"
-    className: "org.apache.storm.tuple.Fields"
-    constructorArgs:
-      - ["word"]
-
-  - id: "counterFields"
-    className: "org.apache.storm.tuple.Fields"
-    constructorArgs:
-      - ["count"]
-
-  - id: "mapper"
-    className: "org.apache.storm.hbase.bolt.mapper.SimpleHBaseMapper"
-    configMethods:
-      - name: "withRowKeyField"
-        args: ["word"]
-      - name: "withColumnFields"
-        args: [ref: "columnFields"]
-      - name: "withCounterFields"
-        args: [ref: "counterFields"]
-      - name: "withColumnFamily"
-        args: ["cf"]
-
-# topology configuration
-# this will be passed to the submitter as a map of config options
-config:
-  topology.workers: 1
-  hbase.conf:
-    hbase.rootdir: "${hbase.rootdir}"
-    hbase.zookeeper.quorum: "${hbase.zookeeper.quorum}"
-
-# spout definitions
-spouts:
-  - id: "word-spout"
-    className: "org.apache.storm.testing.TestWordSpout"
-    parallelism: 1
-
-# bolt definitions
-
-bolts:
-  - id: "count-bolt"
-    className: "org.apache.storm.flux.examples.WordCounter"
-    parallelism: 1
-
-  - id: "hbase-bolt"
-    className: "org.apache.storm.hbase.bolt.HBaseBolt"
-    constructorArgs:
-      - "WordCount" # HBase table name
-      - ref: "mapper"
-    configMethods:
-      - name: "withConfigKey"
-        args: ["hbase.conf"]
-    parallelism: 1
-
-streams:
-  - name: "" # name isn't used (placeholder for logging, UI, etc.)
-    from: "word-spout"
-    to: "count-bolt"
-    grouping:
-      type: SHUFFLE
-
-  - name: "" # name isn't used (placeholder for logging, UI, etc.)
-    from: "count-bolt"
-    to: "hbase-bolt"
-    grouping:
-      type: FIELDS
-      args: ["word"]

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/src/main/resources/simple_hdfs.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/src/main/resources/simple_hdfs.yaml b/external/flux/flux-examples/src/main/resources/simple_hdfs.yaml
deleted file mode 100644
index b8d4020..0000000
--- a/external/flux/flux-examples/src/main/resources/simple_hdfs.yaml
+++ /dev/null
@@ -1,105 +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.
-
-# Test ability to wire together shell spouts/bolts
----
-
-# topology definition
-# name to be used when submitting
-name: "hdfs-topology"
-
-# Components
-# Components are analagous to Spring beans. They are meant to be used as constructor,
-# property(setter), and builder arguments.
-#
-# for the time being, components must be declared in the order they are referenced
-components:
-  - id: "syncPolicy"
-    className: "org.apache.storm.hdfs.bolt.sync.CountSyncPolicy"
-    constructorArgs:
-      - 1000
-  - id: "rotationPolicy"
-    className: "org.apache.storm.hdfs.bolt.rotation.TimedRotationPolicy"
-    constructorArgs:
-      - 30
-      - SECONDS
-
-  - id: "fileNameFormat"
-    className: "org.apache.storm.hdfs.bolt.format.DefaultFileNameFormat"
-    configMethods:
-      - name: "withPath"
-        args: ["${hdfs.write.dir}"]
-      - name: "withExtension"
-        args: [".txt"]
-
-  - id: "recordFormat"
-    className: "org.apache.storm.hdfs.bolt.format.DelimitedRecordFormat"
-    configMethods:
-      - name: "withFieldDelimiter"
-        args: ["|"]
-
-  - id: "rotationAction"
-    className: "org.apache.storm.hdfs.common.rotation.MoveFileAction"
-    configMethods:
-      - name: "toDestination"
-        args: ["${hdfs.dest.dir}"]
-
-# spout definitions
-spouts:
-  - id: "spout-1"
-    className: "org.apache.storm.testing.TestWordSpout"
-    parallelism: 1
-    # ...
-
-# bolt definitions
-
-bolts:
-  - id: "bolt-1"
-    className: "org.apache.storm.hdfs.bolt.HdfsBolt"
-    configMethods:
-      - name: "withConfigKey"
-        args: ["hdfs.config"]
-      - name: "withFsUrl"
-        args: ["${hdfs.url}"]
-      - name: "withFileNameFormat"
-        args: [ref: "fileNameFormat"]
-      - name: "withRecordFormat"
-        args: [ref: "recordFormat"]
-      - name: "withRotationPolicy"
-        args: [ref: "rotationPolicy"]
-      - name: "withSyncPolicy"
-        args: [ref: "syncPolicy"]
-      - name: "addRotationAction"
-        args: [ref: "rotationAction"]
-    parallelism: 1
-    # ...
-
-  - id: "bolt-2"
-    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
-    parallelism: 1
-
-streams:
-  - name: "" # name isn't used (placeholder for logging, UI, etc.)
-    from: "spout-1"
-    to: "bolt-1"
-    grouping:
-      type: SHUFFLE
-
-  - name: "" # name isn't used (placeholder for logging, UI, etc.)
-    from: "spout-1"
-    to: "bolt-2"
-    grouping:
-      type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/src/main/resources/simple_stateful_wordcount.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/src/main/resources/simple_stateful_wordcount.yaml b/external/flux/flux-examples/src/main/resources/simple_stateful_wordcount.yaml
deleted file mode 100644
index 14b9b3a..0000000
--- a/external/flux/flux-examples/src/main/resources/simple_stateful_wordcount.yaml
+++ /dev/null
@@ -1,60 +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.
-
----
-
-# topology definition
-# name to be used when submitting
-name: "stateful-wordcount-topology"
-
-# topology configuration
-# this will be passed to the submitter as a map of config options
-#
-config:
-  topology.workers: 1
-
-# spout definitions
-spouts:
-  - id: "spout-1"
-    className: "org.apache.storm.testing.TestWordSpout"
-    parallelism: 1
-
-# bolt definitions
-bolts:
-  - id: "bolt-1"
-    className: "org.apache.storm.flux.examples.StatefulWordCounter"
-    parallelism: 1
-
-  - id: "bolt-2"
-    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
-    parallelism: 1
-
-#stream definitions
-# stream definitions define connections between spouts and bolts.
-# note that such connections can be cyclical
-streams:
-  - name: "spout-1 --> bolt-1" # name isn't used (placeholder for logging, UI, etc.)
-    from: "spout-1"
-    to: "bolt-1"
-    grouping:
-      type: FIELDS
-      args: ["word"]
-
-  - name: "bolt-1 --> bolt2"
-    from: "bolt-1"
-    to: "bolt-2"
-    grouping:
-      type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/src/main/resources/simple_windowing.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/src/main/resources/simple_windowing.yaml b/external/flux/flux-examples/src/main/resources/simple_windowing.yaml
deleted file mode 100755
index f2f74ff..0000000
--- a/external/flux/flux-examples/src/main/resources/simple_windowing.yaml
+++ /dev/null
@@ -1,69 +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.
-
----
-
-name: "sliding-window-topology"
-
-components:
-  - id: "windowLength"
-    className: "org.apache.storm.topology.base.BaseWindowedBolt$Count"
-    constructorArgs:
-      - 5
-  - id: "slidingInterval"
-    className: "org.apache.storm.topology.base.BaseWindowedBolt$Count"
-    constructorArgs:
-      - 3
-
-config:
-  topology.workers: 1
-
-# spout definitions
-spouts:
-  - id: "spout-1"
-    className: "org.apache.storm.testing.TestWordSpout"
-    parallelism: 1
-
-# bolt definitions
-bolts:
-  - id: "bolt-1"
-    className: "org.apache.storm.flux.examples.TestWindowBolt"
-    configMethods:
-      - name: "withWindow"
-        args: [ref: "windowLength", ref: "slidingInterval"]
-    parallelism: 1
-  - id: "bolt-2"
-    className: "org.apache.storm.flux.examples.TestPrintBolt"
-    parallelism: 1
-
-
-#stream definitions
-# stream definitions define connections between spouts and bolts.
-# note that such connections can be cyclical
-streams:
-  - name: "spout-1 --> bolt-1" # name isn't used (placeholder for logging, UI, etc.)
-#    id: "connection-1"
-    from: "spout-1"
-    to: "bolt-1"
-    grouping:
-      type: FIELDS
-      args: ["word"]
-  - name: "bolt-1 --> bolt-2" # name isn't used (placeholder for logging, UI, etc.)
-#    id: "connection-1"
-    from: "bolt-1"
-    to: "bolt-2"
-    grouping:
-      type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/src/main/resources/simple_wordcount.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/src/main/resources/simple_wordcount.yaml b/external/flux/flux-examples/src/main/resources/simple_wordcount.yaml
deleted file mode 100644
index 6443a97..0000000
--- a/external/flux/flux-examples/src/main/resources/simple_wordcount.yaml
+++ /dev/null
@@ -1,68 +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.
-
----
-
-# topology definition
-# name to be used when submitting
-name: "yaml-topology"
-
-# topology configuration
-# this will be passed to the submitter as a map of config options
-#
-config:
-  topology.workers: 1
-
-# spout definitions
-spouts:
-  - id: "spout-1"
-    className: "org.apache.storm.testing.TestWordSpout"
-    parallelism: 1
-
-# bolt definitions
-bolts:
-  - id: "bolt-1"
-    className: "org.apache.storm.testing.TestWordCounter"
-    parallelism: 1
-
-  - id: "bolt-2"
-    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
-    parallelism: 1
-
-#stream definitions
-# stream definitions define connections between spouts and bolts.
-# note that such connections can be cyclical
-streams:
-  - name: "spout-1 --> bolt-1" # name isn't used (placeholder for logging, UI, etc.)
-#    id: "connection-1"
-    from: "spout-1"
-    to: "bolt-1"
-    grouping:
-      type: FIELDS
-      args: ["word"]
-
-  - name: "bolt-1 --> bolt2"
-    from: "bolt-1"
-    to: "bolt-2"
-    grouping:
-      type: SHUFFLE
-
-
-
-
-
-
-

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-ui/README.md
----------------------------------------------------------------------
diff --git a/external/flux/flux-ui/README.md b/external/flux/flux-ui/README.md
deleted file mode 100644
index 8b6bd5f..0000000
--- a/external/flux/flux-ui/README.md
+++ /dev/null
@@ -1,3 +0,0 @@
-# Flux-UI
-
-Placeholder for Flux GUI
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-wrappers/pom.xml
----------------------------------------------------------------------
diff --git a/external/flux/flux-wrappers/pom.xml b/external/flux/flux-wrappers/pom.xml
deleted file mode 100644
index 7d97a9f..0000000
--- a/external/flux/flux-wrappers/pom.xml
+++ /dev/null
@@ -1,51 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements.  See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License.  You may obtain a copy of the License at
-
-     http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <modelVersion>4.0.0</modelVersion>
-
-    <parent>
-        <groupId>org.apache.storm</groupId>
-        <artifactId>flux</artifactId>
-        <version>2.0.0-SNAPSHOT</version>
-        <relativePath>../pom.xml</relativePath>
-    </parent>
-
-    <artifactId>flux-wrappers</artifactId>
-    <packaging>jar</packaging>
-
-    <name>flux-wrappers</name>
-
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>multilang-javascript</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>multilang-ruby</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>multilang-python</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-    </dependencies>
-
-</project>

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/FluxShellBolt.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/FluxShellBolt.java b/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/FluxShellBolt.java
deleted file mode 100644
index 05b8e7a..0000000
--- a/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/FluxShellBolt.java
+++ /dev/null
@@ -1,175 +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.flux.wrappers.bolts;
-
-import org.apache.storm.task.ShellBolt;
-import org.apache.storm.topology.IRichBolt;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.tuple.Fields;
-
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-/**
- * A generic `ShellBolt` implementation that allows you specify output fields
- * and even streams without having to subclass `ShellBolt` to do so.
- *
- */
-public class FluxShellBolt extends ShellBolt implements IRichBolt{
-    private Map<String, String[]> outputFields;
-    private Map<String, Object> componentConfig;
-    
-    /**
-     * Create a ShellBolt with command line arguments
-     * @param command Command line arguments for the bolt
-     */
-    public FluxShellBolt(String[] command){
-        super(command);
-        this.outputFields = new HashMap<String, String[]>();
-    }
-
-    /**
-     * Create a ShellBolt with command line arguments and output fields
-     * 
-     * Keep this constructor for backward compatibility.
-     * 
-     * @param command Command line arguments for the bolt
-     * @param outputFields Names of fields the bolt will emit (if any).
-     */
-    public FluxShellBolt(String[] command, String[] outputFields){
-        this(command);
-        this.setDefaultStream(outputFields);
-    }
-
-    /**
-     * Add configuration for this bolt. This method is called from YAML file:
-     *
-     * ```
-     * className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
-     * constructorArgs:
-     * # command line
-     * - ["python", "splitsentence.py"]
-     * # output fields
-     * - ["word"]
-     * configMethods:
-     * - name: "addComponentConfig"
-     *   args: ["publisher.data_paths", "actions"]
-     * ```
-     *
-     * @param key
-     * @param value
-     */
-    public void addComponentConfig(String key, Object value) {
-        if (this.componentConfig == null) {
-            this.componentConfig = new HashMap<String, Object>();
-        }
-        this.componentConfig.put(key, value);
-    }
-
-    /**
-     * Add configuration for this bolt. This method is called from YAML file:
-     *
-     * ```
-     * className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
-     * constructorArgs:
-     * # command line
-     * - ["python", "splitsentence.py"]
-     * # output fields
-     * - ["word"]
-     * configMethods:
-     * - name: "addComponentConfig"
-     *   args:
-     *   - "publisher.data_paths"
-     *   - ["actions"]
-     * ```
-     *
-     * @param key
-     * @param values
-     */
-    public void addComponentConfig(String key, List<Object> values) {
-        if (this.componentConfig == null) {
-            this.componentConfig = new HashMap<String, Object>();
-        }
-        this.componentConfig.put(key, values);
-    }
-
-    /**
-     * Set default stream outputFields, this method is called from YAML file:
-     * 
-     * ```
-     * bolts:
-     * - className: org.apache.storm.flux.wrappers.bolts.FluxShellBolt
-     *   id: my_bolt
-     *   constructorArgs:
-     *   - [python, my_bolt.py]
-     *   configMethods:
-     *   - name: setDefaultStream
-     *     args:
-     *     - [word, count]
-     * ```
-     * 
-     * @param outputFields Names of fields the bolt will emit (if any) in default stream.
-     */
-    public void setDefaultStream(String[] outputFields) {
-        this.setNamedStream("default", outputFields);
-    }
-
-    /**
-     * Set custom *named* stream outputFields, this method is called from YAML file:
-     * 
-     * ```
-     * bolts:
-     * - className: org.apache.storm.flux.wrappers.bolts.FluxShellBolt
-     *   id: my_bolt
-     *   constructorArgs:
-     *   - [python, my_bolt.py]
-     *   configMethods:
-     *   - name: setNamedStream
-     *     args:
-     *     - first
-     *     - [word, count]
-     * ```
-     * @param name Name of stream the bolt will emit into.
-     * @param outputFields Names of fields the bolt will emit in custom *named* stream.
-     */
-    public void setNamedStream(String name, String[] outputFields) {
-        this.outputFields.put(name, outputFields);
-    }
-
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer declarer) {
-        Iterator it = this.outputFields.entrySet().iterator();
-        while (it.hasNext()) {
-            Map.Entry entryTuple = (Map.Entry)it.next();
-            String key = (String)entryTuple.getKey();
-            String[] value = (String[])entryTuple.getValue();
-            if(key.equals("default")) {
-                declarer.declare(new Fields(value));
-            } else {
-                declarer.declareStream(key, new Fields(value));
-            }
-        }
-    }
-
-    @Override
-    public Map<String, Object> getComponentConfiguration() {
-        return this.componentConfig;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/LogInfoBolt.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/LogInfoBolt.java b/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/LogInfoBolt.java
deleted file mode 100644
index 5f0e84b..0000000
--- a/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/LogInfoBolt.java
+++ /dev/null
@@ -1,44 +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.flux.wrappers.bolts;
-
-import org.apache.storm.topology.BasicOutputCollector;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.topology.base.BaseBasicBolt;
-import org.apache.storm.tuple.Tuple;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Simple bolt that does nothing other than LOG.info() every tuple recieveed.
- *
- */
-public class LogInfoBolt extends BaseBasicBolt {
-    private static final Logger LOG = LoggerFactory.getLogger(LogInfoBolt.class);
-
-    @Override
-    public void execute(Tuple tuple, BasicOutputCollector basicOutputCollector) {
-       LOG.info("{}", tuple);
-    }
-
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer outputFieldsDeclarer) {
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/spouts/FluxShellSpout.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/spouts/FluxShellSpout.java b/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/spouts/FluxShellSpout.java
deleted file mode 100644
index 5fd378d..0000000
--- a/external/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/spouts/FluxShellSpout.java
+++ /dev/null
@@ -1,175 +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.flux.wrappers.spouts;
-
-import org.apache.storm.spout.ShellSpout;
-import org.apache.storm.topology.IRichSpout;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.tuple.Fields;
-
-import java.util.List;
-import java.util.Map;
-import java.util.HashMap;
-import java.util.Iterator;
-
-/**
- * A generic `ShellSpout` implementation that allows you specify output fields
- * and even streams without having to subclass `ShellSpout` to do so.
- *
- */
-public class FluxShellSpout extends ShellSpout implements IRichSpout {
-    private Map<String, String[]> outputFields;
-    private Map<String, Object> componentConfig;
-    
-    /**
-     * Create a ShellSpout with command line arguments
-     * @param command Command line arguments for the bolt
-     */
-    public FluxShellSpout(String[] command){
-        super(command);
-        this.outputFields = new HashMap<String, String[]>();
-    }
-
-    /**
-     * Create a ShellSpout with command line arguments and output fields
-     * 
-     * Keep this constructor for backward compatibility.
-     * 
-     * @param args Command line arguments for the spout
-     * @param outputFields Names of fields the spout will emit.
-     */
-    public FluxShellSpout(String[] args, String[] outputFields){
-        this(args);
-        this.setDefaultStream(outputFields);
-    }
-
-    /**
-     * Add configuration for this spout. This method is called from YAML file:
-     *
-     * ```
-     * className: "org.apache.storm.flux.wrappers.bolts.FluxShellSpout"
-     * constructorArgs:
-     * # command line
-     * - ["python", "splitsentence.py"]
-     * # output fields
-     * - ["word"]
-     * configMethods:
-     * - name: "addComponentConfig"
-     *   args: ["publisher.data_paths", "actions"]
-     * ```
-     *
-     * @param key
-     * @param value
-     */
-    public void addComponentConfig(String key, Object value) {
-        if (this.componentConfig == null) {
-            this.componentConfig = new HashMap<String, Object>();
-        }
-        this.componentConfig.put(key, value);
-    }
-
-    /**
-     * Add configuration for this spout. This method is called from YAML file:
-     *
-     * ```
-     * className: "org.apache.storm.flux.wrappers.bolts.FluxShellSpout"
-     * constructorArgs:
-     * # command line
-     * - ["python", "splitsentence.py"]
-     * # output fields
-     * - ["word"]
-     * configMethods:
-     * - name: "addComponentConfig"
-     *   args:
-     *   - "publisher.data_paths"
-     *   - ["actions"]
-     * ```
-     *
-     * @param key
-     * @param values
-     */
-    public void addComponentConfig(String key, List<Object> values) {
-        if (this.componentConfig == null) {
-            this.componentConfig = new HashMap<String, Object>();
-        }
-        this.componentConfig.put(key, values);
-    }
-
-    /**
-     * Set default stream outputFields, this method is called from YAML file:
-     * 
-     * ```
-     * spouts:
-     * - className: org.apache.storm.flux.wrappers.bolts.FluxShellSpout
-     *   id: my_spout
-     *   constructorArgs:
-     *   - [python, my_spout.py]
-     *   configMethods:
-     *   - name: setDefaultStream
-     *     args:
-     *     - [word, count]
-     * ```
-     * 
-     * @param outputFields Names of fields the spout will emit (if any) in default stream.
-     */
-    public void setDefaultStream(String[] outputFields) {
-        this.setNamedStream("default", outputFields);
-    }
-
-    /**
-     * Set custom *named* stream outputFields, this method is called from YAML file:
-     * 
-     * ```
-     * spouts:
-     * - className: org.apache.storm.flux.wrappers.bolts.FluxShellSpout
-     *   id: my_spout
-     *   constructorArgs:
-     *   - [python, my_spout.py]
-     *   configMethods:
-     *   - name: setNamedStream
-     *     args:
-     *     - first
-     *     - [word, count]
-     * ```
-     * @param name Name of stream the spout will emit into.
-     * @param outputFields Names of fields the spout will emit in custom *named* stream.
-     */
-    public void setNamedStream(String name, String[] outputFields) {
-        this.outputFields.put(name, outputFields);
-    }
-
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer declarer) {
-        Iterator it = this.outputFields.entrySet().iterator();
-        while (it.hasNext()) {
-            Map.Entry entryTuple = (Map.Entry)it.next();
-            String key = (String)entryTuple.getKey();
-            String[] value = (String[])entryTuple.getValue();
-            if(key.equals("default")) {
-                declarer.declare(new Fields(value));
-            } else {
-                declarer.declareStream(key, new Fields(value));
-            }
-        }
-    }
-
-    @Override
-    public Map<String, Object> getComponentConfiguration() {
-        return this.componentConfig;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-wrappers/src/main/resources/resources/randomsentence.js
----------------------------------------------------------------------
diff --git a/external/flux/flux-wrappers/src/main/resources/resources/randomsentence.js b/external/flux/flux-wrappers/src/main/resources/resources/randomsentence.js
deleted file mode 100644
index b121915..0000000
--- a/external/flux/flux-wrappers/src/main/resources/resources/randomsentence.js
+++ /dev/null
@@ -1,93 +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.
- */
-
-/**
- * Example for storm spout. Emits random sentences.
- * The original class in java - org.apache.storm.starter.spout.RandomSentenceSpout.
- *
- */
-
-var storm = require('./storm');
-var Spout = storm.Spout;
-
-
-var SENTENCES = [
-    "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"]
-
-function RandomSentenceSpout(sentences) {
-    Spout.call(this);
-    this.runningTupleId = 0;
-    this.sentences = sentences;
-    this.pending = {};
-};
-
-RandomSentenceSpout.prototype = Object.create(Spout.prototype);
-RandomSentenceSpout.prototype.constructor = RandomSentenceSpout;
-
-RandomSentenceSpout.prototype.getRandomSentence = function() {
-    return this.sentences[getRandomInt(0, this.sentences.length - 1)];
-}
-
-RandomSentenceSpout.prototype.nextTuple = function(done) {
-    var self = this;
-    var sentence = this.getRandomSentence();
-    var tup = [sentence];
-    var id = this.createNextTupleId();
-    this.pending[id] = tup;
-    //This timeout can be removed if TOPOLOGY_SLEEP_SPOUT_WAIT_STRATEGY_TIME_MS is configured to 100
-    setTimeout(function() {
-        self.emit({tuple: tup, id: id}, function(taskIds) {
-            self.log(tup + ' sent to task ids - ' + taskIds);
-        });
-        done();
-    },100);
-}
-
-RandomSentenceSpout.prototype.createNextTupleId = function() {
-    var id = this.runningTupleId;
-    this.runningTupleId++;
-    return id;
-}
-
-RandomSentenceSpout.prototype.ack = function(id, done) {
-    this.log('Received ack for - ' + id);
-    delete this.pending[id];
-    done();
-}
-
-RandomSentenceSpout.prototype.fail = function(id, done) {
-    var self = this;
-    this.log('Received fail for - ' + id + '. Retrying.');
-    this.emit({tuple: this.pending[id], id:id}, function(taskIds) {
-        self.log(self.pending[id] + ' sent to task ids - ' + taskIds);
-    });
-    done();
-}
-
-/**
- * Returns a random integer between min (inclusive) and max (inclusive)
- */
-function getRandomInt(min, max) {
-    return Math.floor(Math.random() * (max - min + 1)) + min;
-}
-
-new RandomSentenceSpout(SENTENCES).run();

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-wrappers/src/main/resources/resources/splitsentence.py
----------------------------------------------------------------------
diff --git a/external/flux/flux-wrappers/src/main/resources/resources/splitsentence.py b/external/flux/flux-wrappers/src/main/resources/resources/splitsentence.py
deleted file mode 100644
index 300105f..0000000
--- a/external/flux/flux-wrappers/src/main/resources/resources/splitsentence.py
+++ /dev/null
@@ -1,24 +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.
-import storm
-
-class SplitSentenceBolt(storm.BasicBolt):
-    def process(self, tup):
-        words = tup.values[0].split(" ")
-        for word in words:
-          storm.emit([word])
-
-SplitSentenceBolt().run()
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/pom.xml
----------------------------------------------------------------------
diff --git a/external/flux/pom.xml b/external/flux/pom.xml
deleted file mode 100644
index b51a505..0000000
--- a/external/flux/pom.xml
+++ /dev/null
@@ -1,85 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements.  See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License.  You may obtain a copy of the License at
-
-     http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <modelVersion>4.0.0</modelVersion>
-
-    <artifactId>flux</artifactId>
-    <packaging>pom</packaging>
-    <name>flux</name>
-
-
-    <parent>
-        <artifactId>storm</artifactId>
-        <groupId>org.apache.storm</groupId>
-        <version>2.0.0-SNAPSHOT</version>
-        <relativePath>../../pom.xml</relativePath>
-    </parent>
-
-    <developers>
-        <developer>
-            <id>ptgoetz</id>
-            <name>P. Taylor Goetz</name>
-            <email>ptgoetz@apache.org</email>
-        </developer>
-    </developers>
-
-    <properties>
-        <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-        <!-- see comment below... This fixes an annoyance with intellij -->
-        <provided.scope>provided</provided.scope>
-    </properties>
-
-    <profiles>
-        <!--
-            Hack to make intellij behave.
-            If you use intellij, enable this profile in your IDE.
-            It should make life easier.
-        -->
-        <profile>
-            <id>intellij</id>
-            <properties>
-                <provided.scope>compile</provided.scope>
-            </properties>
-        </profile>
-    </profiles>
-
-    <modules>
-        <module>flux-wrappers</module>
-        <module>flux-core</module>
-        <module>flux-examples</module>
-    </modules>
-
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
-            <version>${project.version}</version>
-            <scope>${provided.scope}</scope>
-        </dependency>
-        <dependency>
-            <groupId>commons-cli</groupId>
-            <artifactId>commons-cli</artifactId>
-            <version>1.2</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.kafka</groupId>
-            <artifactId>${storm.kafka.artifact.id}</artifactId>
-            <scope>test</scope>
-        </dependency>
-    </dependencies>
-</project>

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/README.md
----------------------------------------------------------------------
diff --git a/external/sql/README.md b/external/sql/README.md
deleted file mode 100644
index a4b44fb..0000000
--- a/external/sql/README.md
+++ /dev/null
@@ -1,207 +0,0 @@
-# Storm SQL
-
-Compile SQL queries to Storm topologies.
-
-## Usage
-
-Run the ``storm sql`` command to compile SQL statements into Trident topology, and submit it to the Storm cluster
-
-```
-$ bin/storm sql <sql-file> <topo-name>
-```
-
-In which `sql-file` contains a list of SQL statements to be executed, and `topo-name` is the name of the topology.
-
-StormSQL activates `explain mode` and shows query plan instead of submitting topology when user specifies `topo-name` as `--explain`.
-Detailed explanation is available from `Showing Query Plan (explain mode)` section.
-
-## Supported Features
-
-The following features are supported in the current repository:
-
-* Streaming from and to external data sources
-* Filtering tuples
-* Projections
-* Aggregations (Grouping)
-* User defined function (scalar and aggregate)
-* Join (Inner, Left outer, Right outer, Full outer)
-
-## Specifying External Data Sources
-
-In StormSQL data is represented by external tables. Users can specify data sources using the `CREATE EXTERNAL TABLE`
-statement. For example, the following statement specifies a Kafka spouts and sink:
-
-```
-CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY) LOCATION 'kafka://localhost:2181/brokers?topic=test' TBLPROPERTIES '{"producer":{"bootstrap.servers":"localhost:9092","acks":"1","key.serializer":"org.apache.storm.kafka.IntSerializer","value.serializer":"org.apache.storm.kafka.ByteBufferSerializer"}}'
-```
-
-The syntax of `CREATE EXTERNAL TABLE` closely follows the one defined in
-[Hive Data Definition Language](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL).
-
-`PARALLELISM` is StormSQL's own keyword which describes parallelism hint for input data source. This is same as providing parallelism hint to Trident Spout.
-Downstream operators are executed with same parallelism before repartition (Aggregation triggers repartition).
-
-Default value is 1, and this option is no effect on output data source. (We might change if needed. Normally repartition is the thing to avoid.)
-
-## Plugging in External Data Sources
-
-Users plug in external data sources through implementing the `ISqlTridentDataSource` interface and registers them using
-the mechanisms of Java's service loader. The external data source will be chosen based on the scheme of the URI of the
-tables. Please refer to the implementation of `storm-sql-kafka` for more details.
-
-## Specifying User Defined Function (UDF)
-
-Users can define user defined function (scalar or aggregate) using `CREATE FUNCTION` statement.
-For example, the following statement defines `MYPLUS` function which uses `org.apache.storm.sql.TestUtils$MyPlus` class.
-
-```
-CREATE FUNCTION MYPLUS AS 'org.apache.storm.sql.TestUtils$MyPlus'
-```
-
-Storm SQL determines whether the function as scalar or aggregate by checking which methods are defined.
-If the class defines `evaluate` method, Storm SQL treats the function as `scalar`,
-and if the class defines `add` method, Storm SQL treats the function as `aggregate`.
-
-Example of class for scalar function is here:
-
-```
-  public class MyPlus {
-    public static Integer evaluate(Integer x, Integer y) {
-      return x + y;
-    }
-  }
-
-```
-
-and class for aggregate function is here:
-
-```
-  public class MyConcat {
-    public static String init() {
-      return "";
-    }
-    public static String add(String accumulator, String val) {
-      return accumulator + val;
-    }
-    public static String result(String accumulator) {
-      return accumulator;
-    }
-  }
-```
-
-If users don't define `result` method, result is the last return value of `add` method.
-Users need to define `result` method only when we need to transform accumulated value.
-
-## Example: Filtering Kafka Stream
-
-Let's say there is a Kafka stream that represents the transactions of orders. Each message in the stream contains the id
-of the order, the unit price of the product and the quantity of the orders. The goal is to filter orders where the
-transactions are significant and to insert these orders into another Kafka stream for further analysis.
-
-The user can specify the following SQL statements in the SQL file:
-
-```
-CREATE EXTERNAL TABLE ORDERS (ID INT PRIMARY KEY, UNIT_PRICE INT, QUANTITY INT) LOCATION 'kafka://localhost:2181/brokers?topic=orders' TBLPROPERTIES '{"producer":{"bootstrap.servers":"localhost:9092","acks":"1","key.serializer":"org.apache.storm.kafka.IntSerializer","value.serializer":"org.apache.storm.kafka.ByteBufferSerializer"}}'
-
-CREATE EXTERNAL TABLE LARGE_ORDERS (ID INT PRIMARY KEY, TOTAL INT) LOCATION 'kafka://localhost:2181/brokers?topic=large_orders' TBLPROPERTIES '{"producer":{"bootstrap.servers":"localhost:9092","acks":"1","key.serializer":"org.apache.storm.kafka.IntSerializer","value.serializer":"org.apache.storm.kafka.ByteBufferSerializer"}}'
-
-INSERT INTO LARGE_ORDERS SELECT ID, UNIT_PRICE * QUANTITY AS TOTAL FROM ORDERS WHERE UNIT_PRICE * QUANTITY > 50
-```
-
-The first statement defines the table `ORDER` which represents the input stream. The `LOCATION` clause specifies the
-ZkHost (`localhost:2181`), the path of the brokers in ZooKeeper (`/brokers`) and the topic (`orders`).
-The `TBLPROPERTIES` clause specifies the configuration of
-[KafkaProducer](http://kafka.apache.org/documentation.html#newproducerconfigs).
-Current implementation of `storm-sql-kafka` requires specifying both `LOCATION` and `TBLPROPERTIES` clauses even though
-the table is read-only or write-only.
-
-Similarly, the second statement specifies the table `LARGE_ORDERS` which represents the output stream. The third
-statement is a `SELECT` statement which defines the topology: it instructs StormSQL to filter all orders in the external
-table `ORDERS`, calculates the total price and inserts matching records into the Kafka stream specified by
-`LARGE_ORDER`.
-
-To run this example, users need to include the data sources (`storm-sql-kafka` in this case) and its dependency in the
-class path. Dependencies for Storm SQL are automatically handled when users run `storm sql`. Users can include data sources at the submission step like below:
-
-```
-$ bin/storm sql order_filtering.sql order_filtering --artifacts "org.apache.storm:storm-sql-kafka:2.0.0-SNAPSHOT,org.apache.storm:storm-kafka:2.0.0-SNAPSHOT,org.apache.kafka:kafka_2.10:0.8.2.2\!org.slf4j:slf4j-log4j12,org.apache.kafka:kafka-clients:0.8.2.2"
-```
-
-Above command submits the SQL statements to StormSQL. Users need to modify each artifacts' version if users are using different version of Storm or Kafka. 
-
-By now you should be able to see the `order_filtering` topology in the Storm UI.
-
-## Showing Query Plan (explain mode)
-
-Like `explain` on SQL statement, StormSQL provides `explain mode` when running Storm SQL Runner. In explain mode, StormSQL analyzes each query statement (only DML) and show plan instead of submitting topology.
-
-In order to run `explain mode`, you need to provide topology name as `--explain` and run `storm sql` as same as submitting.
-
-For example, when you run the example seen above with explain mode:
- 
-```
-$ bin/storm sql order_filtering.sql --explain --artifacts "org.apache.storm:storm-sql-kafka:2.0.0-SNAPSHOT,org.apache.storm:storm-kafka:2.0.0-SNAPSHOT,org.apache.kafka:kafka_2.10:0.8.2.2\!org.slf4j:slf4j-log4j12,org.apache.kafka:kafka-clients:0.8.2.2"
-```
-
-StormSQL prints out like below:
- 
-```
-
-===========================================================
-query>
-CREATE EXTERNAL TABLE ORDERS (ID INT PRIMARY KEY, UNIT_PRICE INT, QUANTITY INT) LOCATION 'kafka://localhost:2181/brokers?topic=orders' TBLPROPERTIES '{"producer":{"bootstrap.servers":"localhost:9092","acks":"1","key.serializer":"org.apache.storm.kafka.IntSerializer","value.serializer":"org.apache.storm.kafka.ByteBufferSerializer"}}'
------------------------------------------------------------
-16:53:43.951 [main] INFO  o.a.s.s.r.DataSourcesRegistry - Registering scheme kafka with org.apache.storm.sql.kafka.KafkaDataSourcesProvider@4d1bf319
-No plan presented on DDL
-===========================================================
-===========================================================
-query>
-CREATE EXTERNAL TABLE LARGE_ORDERS (ID INT PRIMARY KEY, TOTAL INT) LOCATION 'kafka://localhost:2181/brokers?topic=large_orders' TBLPROPERTIES '{"producer":{"bootstrap.servers":"localhost:9092","acks":"1","key.serializer":"org.apache.storm.kafka.IntSerializer","value.serializer":"org.apache.storm.kafka.ByteBufferSerializer"}}'
------------------------------------------------------------
-No plan presented on DDL
-===========================================================
-===========================================================
-query>
-INSERT INTO LARGE_ORDERS SELECT ID, UNIT_PRICE * QUANTITY AS TOTAL FROM ORDERS WHERE UNIT_PRICE * QUANTITY > 50
------------------------------------------------------------
-plan>
-LogicalTableModify(table=[[LARGE_ORDERS]], operation=[INSERT], updateColumnList=[[]], flattened=[true]), id = 8
-  LogicalProject(ID=[$0], TOTAL=[*($1, $2)]), id = 7
-    LogicalFilter(condition=[>(*($1, $2), 50)]), id = 6
-      EnumerableTableScan(table=[[ORDERS]]), id = 5
-
-===========================================================
-
-```
-
-## Current Limitations
-
-- Windowing is yet to be implemented.
-- Only equi-join (single field equality) is supported for joining table.
-- Joining table only applies within each small batch that comes off of the spout.
-  - Not across batches.
-  - Limitation came from `join` feature of Trident.
-  - Please refer this doc: `Trident API Overview` for details.
-
-## License
-
-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.
-
-## Committer Sponsors
- * Sriharsha Chintalapani ([sriharsha@apache.org](mailto:sriharsha@apache.org))
- * P. Taylor Goetz ([ptgoetz@apache.org](mailto:ptgoetz@apache.org))
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/pom.xml
----------------------------------------------------------------------
diff --git a/external/sql/pom.xml b/external/sql/pom.xml
deleted file mode 100644
index 02264f7..0000000
--- a/external/sql/pom.xml
+++ /dev/null
@@ -1,47 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements.  See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License.  You may obtain a copy of the License at
-
-     http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/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>
-        <relativePath>../../pom.xml</relativePath>
-    </parent>
-
-    <artifactId>sql</artifactId>
-    <packaging>pom</packaging>
-
-    <developers>
-        <developer>
-            <id>haohui</id>
-            <name>Haohui Mai</name>
-            <email>ricetons@gmail.com</email>
-        </developer>
-    </developers>
-
-    <modules>
-        <module>storm-sql-core</module>
-        <module>storm-sql-runtime</module>
-        <module>storm-sql-external/storm-sql-kafka</module>
-        <module>storm-sql-external/storm-sql-redis</module>
-        <module>storm-sql-external/storm-sql-mongodb</module>
-        <module>storm-sql-external/storm-sql-hdfs</module>
-    </modules>
-</project>


[06/23] storm git commit: STORM-2453 Move non-connectors into the top directory

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/CompilerUtil.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/CompilerUtil.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/CompilerUtil.java
new file mode 100644
index 0000000..2e237c0
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/CompilerUtil.java
@@ -0,0 +1,183 @@
+/**
+ * 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.sql.compiler;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.*;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlMonotonicity;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
+import org.apache.storm.sql.calcite.ParallelStreamableTable;
+import org.apache.storm.sql.parser.ColumnConstraint;
+
+import java.util.ArrayList;
+
+import static org.apache.calcite.rel.RelFieldCollation.Direction;
+import static org.apache.calcite.rel.RelFieldCollation.Direction.ASCENDING;
+import static org.apache.calcite.rel.RelFieldCollation.Direction.DESCENDING;
+import static org.apache.calcite.rel.RelFieldCollation.NullDirection;
+import static org.apache.calcite.sql.validate.SqlMonotonicity.INCREASING;
+
+public class CompilerUtil {
+  public static String escapeJavaString(String s, boolean nullMeansNull) {
+      if(s == null) {
+        return nullMeansNull ? "null" : "\"\"";
+      } else {
+        String s1 = Util.replace(s, "\\", "\\\\");
+        String s2 = Util.replace(s1, "\"", "\\\"");
+        String s3 = Util.replace(s2, "\n\r", "\\n");
+        String s4 = Util.replace(s3, "\n", "\\n");
+        String s5 = Util.replace(s4, "\r", "\\r");
+        return "\"" + s5 + "\"";
+      }
+  }
+
+  public static class TableBuilderInfo {
+    private final RelDataTypeFactory typeFactory;
+
+    public TableBuilderInfo(RelDataTypeFactory typeFactory) {
+      this.typeFactory = typeFactory;
+    }
+
+    private static class FieldType {
+      private final String name;
+      private final RelDataType relDataType;
+
+      private FieldType(String name, RelDataType relDataType) {
+        this.name = name;
+        this.relDataType = relDataType;
+      }
+
+    }
+
+    private final ArrayList<FieldType> fields = new ArrayList<>();
+    private final ArrayList<Object[]> rows = new ArrayList<>();
+    private int primaryKey = -1;
+    private Integer parallelismHint;
+    private SqlMonotonicity primaryKeyMonotonicity;
+    private Statistic stats;
+
+    public TableBuilderInfo field(String name, SqlTypeName type) {
+      return field(name, typeFactory.createSqlType(type));
+    }
+
+    public TableBuilderInfo field(String name, RelDataType type) {
+      fields.add(new FieldType(name, type));
+      return this;
+    }
+
+    public TableBuilderInfo field(String name, SqlDataTypeSpec type, ColumnConstraint constraint) {
+      RelDataType dataType = type.deriveType(typeFactory);
+      if (constraint instanceof ColumnConstraint.PrimaryKey) {
+        ColumnConstraint.PrimaryKey pk = (ColumnConstraint.PrimaryKey) constraint;
+        Preconditions.checkState(primaryKey == -1, "There are more than one primary key in the table");
+        primaryKey = fields.size();
+        primaryKeyMonotonicity = pk.monotonicity();
+      }
+      fields.add(new FieldType(name, dataType));
+      return this;
+    }
+
+    public TableBuilderInfo statistics(Statistic stats) {
+      this.stats = stats;
+      return this;
+    }
+
+    @VisibleForTesting
+    public TableBuilderInfo rows(Object[] data) {
+      rows.add(data);
+      return this;
+    }
+
+    public TableBuilderInfo parallelismHint(int parallelismHint) {
+      this.parallelismHint = parallelismHint;
+      return this;
+    }
+
+    public StreamableTable build() {
+      final Statistic stat = buildStatistic();
+      final Table tbl = new Table() {
+        @Override
+        public RelDataType getRowType(
+            RelDataTypeFactory relDataTypeFactory) {
+          RelDataTypeFactory.FieldInfoBuilder b = relDataTypeFactory.builder();
+          for (FieldType f : fields) {
+            b.add(f.name, f.relDataType);
+          }
+          return b.build();
+        }
+
+        @Override
+        public Statistic getStatistic() {
+          return stat != null ? stat : Statistics.of(rows.size(),
+                                                     ImmutableList.<ImmutableBitSet>of());
+        }
+
+        @Override
+        public Schema.TableType getJdbcTableType() {
+          return Schema.TableType.STREAM;
+        }
+      };
+
+      return new ParallelStreamableTable() {
+        @Override
+        public Integer parallelismHint() {
+          return parallelismHint;
+        }
+
+        @Override
+        public Table stream() {
+          return tbl;
+        }
+
+        @Override
+        public RelDataType getRowType(RelDataTypeFactory relDataTypeFactory) {
+          return tbl.getRowType(relDataTypeFactory);
+        }
+
+        @Override
+        public Statistic getStatistic() {
+          return tbl.getStatistic();
+        }
+
+        @Override
+        public Schema.TableType getJdbcTableType() {
+          return Schema.TableType.STREAM;
+        }
+      };
+    }
+
+    private Statistic buildStatistic() {
+      if (stats != null || primaryKey == -1) {
+        return stats;
+      }
+      Direction dir = primaryKeyMonotonicity == INCREASING ? ASCENDING : DESCENDING;
+      RelFieldCollation collation = new RelFieldCollation(primaryKey, dir, NullDirection.UNSPECIFIED);
+      return Statistics.of(fields.size(), ImmutableList.of(ImmutableBitSet.of(primaryKey)),
+          ImmutableList.of(RelCollations.of(collation)));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RexNodeToJavaCodeCompiler.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RexNodeToJavaCodeCompiler.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RexNodeToJavaCodeCompiler.java
new file mode 100644
index 0000000..5ac95e0
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RexNodeToJavaCodeCompiler.java
@@ -0,0 +1,231 @@
+/*
+ * 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.sql.compiler;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import org.apache.calcite.adapter.enumerable.JavaRowFormat;
+import org.apache.calcite.adapter.enumerable.PhysType;
+import org.apache.calcite.adapter.enumerable.PhysTypeImpl;
+import org.apache.calcite.adapter.enumerable.RexToLixTranslator;
+import org.apache.calcite.interpreter.Context;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.BlockStatement;
+import org.apache.calcite.linq4j.tree.ClassDeclaration;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.MemberDeclaration;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.linq4j.tree.Types;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Pair;
+import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.lang.reflect.Type;
+import java.util.List;
+
+/**
+ * Compiles a scalar expression ({@link org.apache.calcite.rex.RexNode}) to Java source code String.
+ *
+ * This code is inspired by JaninoRexCompiler in Calcite, but while it is returning {@link org.apache.calcite.interpreter.Scalar} which is executable,
+ * we need to pass the source code to compile and serialize instance so that it can be executed on worker efficiently.
+ */
+public class RexNodeToJavaCodeCompiler {
+  private final RexBuilder rexBuilder;
+
+  public RexNodeToJavaCodeCompiler(RexBuilder rexBuilder) {
+    this.rexBuilder = rexBuilder;
+  }
+
+  public BlockStatement compileToBlock(List<RexNode> nodes, RelDataType inputRowType) {
+    final RexProgramBuilder programBuilder =
+        new RexProgramBuilder(inputRowType, rexBuilder);
+    for (RexNode node : nodes) {
+      programBuilder.addProject(node, null);
+    }
+
+    return compileToBlock(programBuilder.getProgram());
+  }
+
+  public BlockStatement compileToBlock(final RexProgram program) {
+    final ParameterExpression context_ =
+            Expressions.parameter(Context.class, "context");
+    final ParameterExpression outputValues_ =
+            Expressions.parameter(Object[].class, "outputValues");
+
+    return compileToBlock(program, context_, outputValues_).toBlock();
+  }
+
+  public String compile(List<RexNode> nodes, RelDataType inputRowType, String className) {
+    final RexProgramBuilder programBuilder =
+            new RexProgramBuilder(inputRowType, rexBuilder);
+    for (RexNode node : nodes) {
+      programBuilder.addProject(node, null);
+    }
+
+    return compile(programBuilder.getProgram(), className);
+  }
+
+  public String compile(final RexProgram program, String className) {
+    final ParameterExpression context_ =
+            Expressions.parameter(Context.class, "context");
+    final ParameterExpression outputValues_ =
+            Expressions.parameter(Object[].class, "outputValues");
+
+    BlockBuilder builder = compileToBlock(program, context_, outputValues_);
+    return baz(context_, outputValues_, builder.toBlock(), className);
+  }
+
+  private BlockBuilder compileToBlock(final RexProgram program, ParameterExpression context_,
+                                        ParameterExpression outputValues_) {
+    RelDataType inputRowType = program.getInputRowType();
+    final BlockBuilder builder = new BlockBuilder();
+    final JavaTypeFactoryImpl javaTypeFactory =
+            new JavaTypeFactoryImpl(rexBuilder.getTypeFactory().getTypeSystem());
+
+    final RexToLixTranslator.InputGetter inputGetter =
+            new RexToLixTranslator.InputGetterImpl(
+                    ImmutableList.of(
+                            Pair.<Expression, PhysType>of(
+                                    Expressions.field(context_,
+                                            BuiltInMethod.CONTEXT_VALUES.field),
+                                    PhysTypeImpl.of(javaTypeFactory, inputRowType,
+                                            JavaRowFormat.ARRAY, false))));
+    final Function1<String, RexToLixTranslator.InputGetter> correlates =
+            new Function1<String, RexToLixTranslator.InputGetter>() {
+              public RexToLixTranslator.InputGetter apply(String a0) {
+                throw new UnsupportedOperationException();
+              }
+            };
+    final Expression root =
+            Expressions.field(context_, BuiltInMethod.CONTEXT_ROOT.field);
+    final List<Expression> list =
+            RexToLixTranslator.translateProjects(program, javaTypeFactory, builder,
+                    null, root, inputGetter, correlates);
+    for (int i = 0; i < list.size(); i++) {
+      builder.add(
+              Expressions.statement(
+                      Expressions.assign(
+                              Expressions.arrayIndex(outputValues_,
+                                      Expressions.constant(i)),
+                              list.get(i))));
+    }
+
+    return builder;
+  }
+
+  /** Given a method that implements {@link ExecutableExpression#execute(Context, Object[])},
+   * adds a bridge method that implements {@link ExecutableExpression#execute(Context)}, and
+   * compiles. */
+  static String baz(ParameterExpression context_,
+                    ParameterExpression outputValues_, BlockStatement block, String className) {
+    final List<MemberDeclaration> declarations = Lists.newArrayList();
+
+    // public void execute(Context, Object[] outputValues)
+    declarations.add(
+            Expressions.methodDecl(Modifier.PUBLIC, void.class,
+                    StormBuiltInMethod.EXPR_EXECUTE2.method.getName(),
+                    ImmutableList.of(context_, outputValues_), block));
+
+    // public Object execute(Context)
+    final BlockBuilder builder = new BlockBuilder();
+    final Expression values_ = builder.append("values",
+            Expressions.newArrayBounds(Object.class, 1,
+                    Expressions.constant(1)));
+    builder.add(
+            Expressions.statement(
+                    Expressions.call(
+                            Expressions.parameter(ExecutableExpression.class, "this"),
+                            StormBuiltInMethod.EXPR_EXECUTE2.method, context_, values_)));
+    builder.add(
+            Expressions.return_(null,
+                    Expressions.arrayIndex(values_, Expressions.constant(0))));
+    declarations.add(
+            Expressions.methodDecl(Modifier.PUBLIC, Object.class,
+                    StormBuiltInMethod.EXPR_EXECUTE1.method.getName(),
+                    ImmutableList.of(context_), builder.toBlock()));
+
+    final ClassDeclaration classDeclaration =
+            Expressions.classDecl(Modifier.PUBLIC, className, null,
+                    ImmutableList.<Type>of(ExecutableExpression.class), declarations);
+
+    return Expressions.toString(Lists.newArrayList(classDeclaration), "\n", false);
+  }
+
+  enum StormBuiltInMethod {
+    EXPR_EXECUTE1(ExecutableExpression.class, "execute", Context.class),
+    EXPR_EXECUTE2(ExecutableExpression.class, "execute", Context.class, Object[].class);
+
+    public final Method method;
+    public final Constructor constructor;
+    public final Field field;
+
+    public static final ImmutableMap<Method, BuiltInMethod> MAP;
+
+    static {
+      final ImmutableMap.Builder<Method, BuiltInMethod> builder =
+              ImmutableMap.builder();
+      for (BuiltInMethod value : BuiltInMethod.values()) {
+        if (value.method != null) {
+          builder.put(value.method, value);
+        }
+      }
+      MAP = builder.build();
+    }
+
+    private StormBuiltInMethod(Method method, Constructor constructor, Field field) {
+      this.method = method;
+      this.constructor = constructor;
+      this.field = field;
+    }
+
+    /**
+     * Defines a method.
+     */
+    StormBuiltInMethod(Class clazz, String methodName, Class... argumentTypes) {
+      this(Types.lookupMethod(clazz, methodName, argumentTypes), null, null);
+    }
+
+    /**
+     * Defines a constructor.
+     */
+    StormBuiltInMethod(Class clazz, Class... argumentTypes) {
+      this(null, Types.lookupConstructor(clazz, argumentTypes), null);
+    }
+
+    /**
+     * Defines a field.
+     */
+    StormBuiltInMethod(Class clazz, String fieldName, boolean dummy) {
+      this(null, null, Types.lookupField(clazz, fieldName));
+      assert dummy : "dummy value for method overloading must be true";
+    }
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/StormSqlTypeFactoryImpl.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/StormSqlTypeFactoryImpl.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/StormSqlTypeFactoryImpl.java
new file mode 100644
index 0000000..21ca063
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/StormSqlTypeFactoryImpl.java
@@ -0,0 +1,51 @@
+/**
+ * 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.sql.compiler;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Lists;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rel.type.RelRecordType;
+import org.apache.calcite.sql.type.JavaToSqlTypeConversionRules;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+public class StormSqlTypeFactoryImpl extends JavaTypeFactoryImpl {
+
+    public StormSqlTypeFactoryImpl() {
+    }
+
+    public StormSqlTypeFactoryImpl(RelDataTypeSystem typeSystem) {
+        super(typeSystem);
+    }
+
+    @Override
+    public RelDataType toSql(RelDataType type) {
+        if (type instanceof JavaType) {
+            JavaType javaType = (JavaType) type;
+            SqlTypeName sqlTypeName = JavaToSqlTypeConversionRules.instance().lookup(javaType.getJavaClass());
+            if (sqlTypeName == null) {
+                sqlTypeName = SqlTypeName.ANY;
+            }
+            return createTypeWithNullability(createSqlType(sqlTypeName), type.isNullable());
+        }
+        return super.toSql(type);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/BuiltinAggregateFunctions.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/BuiltinAggregateFunctions.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/BuiltinAggregateFunctions.java
new file mode 100644
index 0000000..9dc4ba8
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/BuiltinAggregateFunctions.java
@@ -0,0 +1,238 @@
+/**
+ * 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.sql.compiler.backends.standalone;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.storm.tuple.Values;
+
+import java.lang.reflect.Type;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Built-in implementations for some of the standard aggregation operations.
+ * Aggregations can be implemented as a class with the following methods viz. init, add and result.
+ * The class could contain only static methods, only non-static methods or be generic.
+ */
+public class BuiltinAggregateFunctions {
+    // binds the type information and the class implementing the aggregation
+    public static class TypeClass {
+        public static class GenericType {
+        }
+
+        public final Type ty;
+        public final Class<?> clazz;
+
+        private TypeClass(Type ty, Class<?> clazz) {
+            this.ty = ty;
+            this.clazz = clazz;
+        }
+
+        static TypeClass of(Type ty, Class<?> clazz) {
+            return new TypeClass(ty, clazz);
+        }
+    }
+
+    static final Map<String, List<TypeClass>> TABLE = new HashMap<>();
+
+    public static class ByteSum {
+        public static Byte init() {
+            return 0;
+        }
+
+        public static Byte add(Byte accumulator, Byte val) {
+            return (byte) (accumulator + val);
+        }
+
+        public static Byte result(Byte accumulator) {
+            return accumulator;
+        }
+    }
+
+    public static class ShortSum {
+        public static Short init() {
+            return 0;
+        }
+
+        public static Short add(Short accumulator, Short val) {
+            return (short) (accumulator + val);
+        }
+
+        public static Short result(Short accumulator) {
+            return accumulator;
+        }
+    }
+
+    public static class IntSum {
+        public static Integer init() {
+            return 0;
+        }
+
+        public static Integer add(Integer accumulator, Integer val) {
+            return accumulator + val;
+        }
+
+        public static Integer result(Integer accumulator) {
+            return accumulator;
+        }
+    }
+
+    public static class LongSum {
+        public static Long init() {
+            return 0L;
+        }
+
+        public static Long add(Long accumulator, Long val) {
+            return accumulator + val;
+        }
+
+        public static Long result(Long accumulator) {
+            return accumulator;
+        }
+    }
+
+    public static class FloatSum {
+        public static Float init() {
+            return 0.0f;
+        }
+
+        public static Float add(Float accumulator, Float val) {
+            return accumulator + val;
+        }
+
+        public static Float result(Float accumulator) {
+            return accumulator;
+        }
+    }
+
+    public static class DoubleSum {
+        public static Double init() {
+            return 0.0;
+        }
+
+        public static Double add(Double accumulator, Double val) {
+            return accumulator + val;
+        }
+
+        public static Double result(Double accumulator) {
+            return accumulator;
+        }
+    }
+
+    public static class Max<T extends Comparable<T>> {
+        public T init() {
+            return null;
+        }
+
+        public T add(T accumulator, T val) {
+            return (accumulator == null || accumulator.compareTo(val) < 0) ? val : accumulator;
+        }
+
+        public T result(T accumulator) {
+            return accumulator;
+        }
+    }
+
+    public static class Min<T extends Comparable<T>> {
+        public T init() {
+            return null;
+        }
+
+        public T add(T accumulator, T val) {
+            return (accumulator == null || accumulator.compareTo(val) > 0) ? val : accumulator;
+        }
+
+        public T result(T accumulator) {
+            return accumulator;
+        }
+    }
+
+    public static class IntAvg {
+        private int count;
+
+        public Integer init() {
+            return 0;
+        }
+
+        public Integer add(Integer accumulator, Integer val) {
+            ++count;
+            return accumulator + val;
+        }
+
+        public Integer result(Integer accumulator) {
+            Integer result = accumulator / count;
+            count = 0;
+            return result;
+        }
+    }
+
+    public static class DoubleAvg {
+        private int count;
+
+        public Double init() {
+            return 0.0;
+        }
+
+        public Double add(Double accumulator, Double val) {
+            ++count;
+            return accumulator + val;
+        }
+
+        public Double result(Double accumulator) {
+            Double result = accumulator / count;
+            count = 0;
+            return result;
+        }
+    }
+
+    public static class Count {
+        public static Long init() {
+            return 0L;
+        }
+
+        public static Long add(Long accumulator, Values vals) {
+            for (Object val : vals) {
+                if (val == null) {
+                    return accumulator;
+                }
+            }
+            return accumulator + 1;
+        }
+
+        public static Long result(Long accumulator) {
+            return accumulator;
+        }
+    }
+
+    static {
+        TABLE.put("SUM", ImmutableList.of(
+                TypeClass.of(float.class, FloatSum.class),
+                TypeClass.of(double.class, DoubleSum.class),
+                TypeClass.of(byte.class, ByteSum.class),
+                TypeClass.of(short.class, ShortSum.class),
+                TypeClass.of(long.class, LongSum.class),
+                TypeClass.of(int.class, IntSum.class)));
+        TABLE.put("AVG", ImmutableList.of(
+                TypeClass.of(double.class, DoubleAvg.class),
+                TypeClass.of(int.class, IntAvg.class)));
+        TABLE.put("COUNT", ImmutableList.of(TypeClass.of(long.class, Count.class)));
+        TABLE.put("MAX", ImmutableList.of(TypeClass.of(TypeClass.GenericType.class, Max.class)));
+        TABLE.put("MIN", ImmutableList.of(TypeClass.of(TypeClass.GenericType.class, Min.class)));
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PlanCompiler.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PlanCompiler.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PlanCompiler.java
new file mode 100644
index 0000000..01546ed
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PlanCompiler.java
@@ -0,0 +1,139 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.compiler.backends.standalone;
+
+import com.google.common.base.Joiner;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.storm.sql.compiler.CompilerUtil;
+import org.apache.storm.sql.javac.CompilingClassLoader;
+import org.apache.storm.sql.runtime.AbstractValuesProcessor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.HashSet;
+import java.util.Set;
+
+public class PlanCompiler {
+  private static final Logger LOG = LoggerFactory.getLogger(PlanCompiler.class);
+
+  private static final Joiner NEW_LINE_JOINER = Joiner.on("\n");
+  private static final String PACKAGE_NAME = "org.apache.storm.sql.generated";
+  private static final String PROLOGUE = NEW_LINE_JOINER.join(
+      "// GENERATED CODE", "package " + PACKAGE_NAME + ";", "",
+      "import java.util.Iterator;", "import java.util.Map;", "import java.util.HashMap;",
+      "import java.util.List;", "import java.util.ArrayList;",
+      "import java.util.LinkedHashMap;",
+      "import org.apache.storm.tuple.Values;",
+      "import org.apache.storm.sql.runtime.AbstractChannelHandler;",
+      "import org.apache.storm.sql.runtime.Channels;",
+      "import org.apache.storm.sql.runtime.ChannelContext;",
+      "import org.apache.storm.sql.runtime.ChannelHandler;",
+      "import org.apache.storm.sql.runtime.DataSource;",
+      "import org.apache.storm.sql.runtime.AbstractValuesProcessor;",
+      "import com.google.common.collect.ArrayListMultimap;",
+      "import com.google.common.collect.Multimap;",
+      "import org.apache.calcite.interpreter.Context;",
+      "import org.apache.calcite.interpreter.StormContext;",
+      "import org.apache.calcite.DataContext;",
+      "import org.apache.storm.sql.runtime.calcite.StormDataContext;",
+      "public final class Processor extends AbstractValuesProcessor {",
+      "  public final static DataContext dataContext = new StormDataContext();",
+      "");
+  private static final String INITIALIZER_PROLOGUE = NEW_LINE_JOINER.join(
+      "  @Override",
+      "  public void initialize(Map<String, DataSource> data,",
+      "                         ChannelHandler result) {",
+      "    ChannelContext r = Channels.chain(Channels.voidContext(), result);",
+      ""
+  );
+
+  private final JavaTypeFactory typeFactory;
+
+  public PlanCompiler(JavaTypeFactory typeFactory) {
+    this.typeFactory = typeFactory;
+  }
+
+  private String generateJavaSource(RelNode root) throws Exception {
+    StringWriter sw = new StringWriter();
+    try (PrintWriter pw = new PrintWriter(sw)) {
+      RelNodeCompiler compiler = new RelNodeCompiler(pw, typeFactory);
+      printPrologue(pw);
+      compiler.traverse(root);
+      printMain(pw, root);
+      printEpilogue(pw);
+    }
+    return sw.toString();
+  }
+
+  private void printMain(PrintWriter pw, RelNode root) {
+    Set<TableScan> tables = new HashSet<>();
+    pw.print(INITIALIZER_PROLOGUE);
+    chainOperators(pw, root, tables);
+    for (TableScan n : tables) {
+      String escaped = CompilerUtil.escapeJavaString(
+          Joiner.on('.').join(n.getTable().getQualifiedName()), true);
+      String r = NEW_LINE_JOINER.join(
+          "    if (!data.containsKey(%1$s))",
+          "      throw new RuntimeException(\"Cannot find table \" + %1$s);",
+          "  data.get(%1$s).open(CTX_%2$d);",
+          "");
+      pw.print(String.format(r, escaped, n.getId()));
+    }
+    pw.print("  }\n");
+  }
+
+  private void chainOperators(PrintWriter pw, RelNode root, Set<TableScan> tables) {
+    doChainOperators(pw, root, tables, "r");
+  }
+
+  private void doChainOperators(PrintWriter pw, RelNode node, Set<TableScan> tables, String parentCtx) {
+    pw.print(
+            String.format("    ChannelContext CTX_%d = Channels.chain(%2$s, %3$s);\n",
+                          node.getId(), parentCtx, RelNodeCompiler.getStageName(node)));
+    String currentCtx = String.format("CTX_%d", node.getId());
+    if (node instanceof TableScan) {
+      tables.add((TableScan) node);
+    }
+    for (RelNode i : node.getInputs()) {
+      doChainOperators(pw, i, tables, currentCtx);
+    }
+  }
+
+  public AbstractValuesProcessor compile(RelNode plan) throws Exception {
+    String javaCode = generateJavaSource(plan);
+    LOG.debug("Compiling... source code {}", javaCode);
+    ClassLoader cl = new CompilingClassLoader(getClass().getClassLoader(),
+                                              PACKAGE_NAME + ".Processor",
+                                              javaCode, null);
+    return (AbstractValuesProcessor) cl.loadClass(
+        PACKAGE_NAME + ".Processor").newInstance();
+  }
+
+  private static void printEpilogue(
+      PrintWriter pw) throws Exception {
+    pw.print("}\n");
+  }
+
+  private static void printPrologue(PrintWriter pw) {
+    pw.append(PROLOGUE);
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PostOrderRelNodeVisitor.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PostOrderRelNodeVisitor.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PostOrderRelNodeVisitor.java
new file mode 100644
index 0000000..afed8a9
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PostOrderRelNodeVisitor.java
@@ -0,0 +1,132 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.compiler.backends.standalone;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.*;
+import org.apache.calcite.rel.stream.Delta;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public abstract class PostOrderRelNodeVisitor<T> {
+  public final T traverse(RelNode n) throws Exception {
+    List<T> inputStreams = new ArrayList<>();
+    for (RelNode input : n.getInputs()) {
+      inputStreams.add(traverse(input));
+    }
+
+    if (n instanceof Aggregate) {
+      return visitAggregate((Aggregate) n, inputStreams);
+    } else if (n instanceof Calc) {
+      return visitCalc((Calc) n, inputStreams);
+    } else if (n instanceof Collect) {
+      return visitCollect((Collect) n, inputStreams);
+    } else if (n instanceof Correlate) {
+      return visitCorrelate((Correlate) n, inputStreams);
+    } else if (n instanceof Delta) {
+      return visitDelta((Delta) n, inputStreams);
+    } else if (n instanceof Exchange) {
+      return visitExchange((Exchange) n, inputStreams);
+    } else if (n instanceof Project) {
+      return visitProject((Project) n, inputStreams);
+    } else if (n instanceof Filter) {
+      return visitFilter((Filter) n, inputStreams);
+    } else if (n instanceof Sample) {
+      return visitSample((Sample) n, inputStreams);
+    } else if (n instanceof Sort) {
+      return visitSort((Sort) n, inputStreams);
+    } else if (n instanceof TableModify) {
+      return visitTableModify((TableModify) n, inputStreams);
+    } else if (n instanceof TableScan) {
+      return visitTableScan((TableScan) n, inputStreams);
+    } else if (n instanceof Uncollect) {
+      return visitUncollect((Uncollect) n, inputStreams);
+    } else if (n instanceof Window) {
+      return visitWindow((Window) n, inputStreams);
+    } else if (n instanceof Join) {
+      return visitJoin((Join) n, inputStreams);
+    } else {
+      return defaultValue(n, inputStreams);
+    }
+  }
+
+  public T visitAggregate(Aggregate aggregate, List<T> inputStreams) throws Exception {
+    return defaultValue(aggregate, inputStreams);
+  }
+
+  public T visitCalc(Calc calc, List<T> inputStreams) throws Exception {
+    return defaultValue(calc, inputStreams);
+  }
+
+  public T visitCollect(Collect collect, List<T> inputStreams) throws Exception {
+    return defaultValue(collect, inputStreams);
+  }
+
+  public T visitCorrelate(Correlate correlate, List<T> inputStreams) throws Exception {
+    return defaultValue(correlate, inputStreams);
+  }
+
+  public T visitDelta(Delta delta, List<T> inputStreams) throws Exception {
+    return defaultValue(delta, inputStreams);
+  }
+
+  public T visitExchange(Exchange exchange, List<T> inputStreams) throws Exception {
+    return defaultValue(exchange, inputStreams);
+  }
+
+  public T visitProject(Project project, List<T> inputStreams) throws Exception {
+    return defaultValue(project, inputStreams);
+  }
+
+  public T visitFilter(Filter filter, List<T> inputStreams) throws Exception {
+    return defaultValue(filter, inputStreams);
+  }
+
+  public T visitSample(Sample sample, List<T> inputStreams) throws Exception {
+    return defaultValue(sample, inputStreams);
+  }
+
+  public T visitSort(Sort sort, List<T> inputStreams) throws Exception {
+    return defaultValue(sort, inputStreams);
+  }
+
+  public T visitTableModify(TableModify modify, List<T> inputStreams) throws Exception {
+    return defaultValue(modify, inputStreams);
+  }
+
+  public T visitTableScan(TableScan scan, List<T> inputStreams) throws Exception {
+    return defaultValue(scan, inputStreams);
+  }
+
+  public T visitUncollect(Uncollect uncollect, List<T> inputStreams) throws Exception {
+    return defaultValue(uncollect, inputStreams);
+  }
+
+  public T visitWindow(Window window, List<T> inputStreams) throws Exception {
+    return defaultValue(window, inputStreams);
+  }
+
+  public T visitJoin(Join join, List<T> inputStreams) throws Exception {
+    return defaultValue(join, inputStreams);
+  }
+
+  public T defaultValue(RelNode n, List<T> inputStreams) {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/RelNodeCompiler.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/RelNodeCompiler.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/RelNodeCompiler.java
new file mode 100644
index 0000000..97995c7
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/RelNodeCompiler.java
@@ -0,0 +1,484 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.compiler.backends.standalone;
+
+import com.google.common.base.Joiner;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.stream.Delta;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.schema.AggregateFunction;
+import org.apache.calcite.schema.impl.AggregateFunctionImpl;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.validate.SqlUserDefinedAggFunction;
+import org.apache.storm.sql.compiler.RexNodeToJavaCodeCompiler;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.lang.reflect.Type;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Compile RelNodes into individual functions.
+ */
+class RelNodeCompiler extends PostOrderRelNodeVisitor<Void> {
+  public static Joiner NEW_LINE_JOINER = Joiner.on('\n');
+
+  private final PrintWriter pw;
+  private final JavaTypeFactory typeFactory;
+  private final RexNodeToJavaCodeCompiler rexCompiler;
+
+  private static final String STAGE_PROLOGUE = NEW_LINE_JOINER.join(
+    "  private static final ChannelHandler %1$s = ",
+    "    new AbstractChannelHandler() {",
+    "    @Override",
+    "    public void dataReceived(ChannelContext ctx, Values _data) {",
+    ""
+  );
+
+  private static final String AGGREGATE_STAGE_PROLOGUE = NEW_LINE_JOINER.join(
+          "  private static final ChannelHandler %1$s = ",
+          "    new AbstractChannelHandler() {",
+          "    private final Values EMPTY_VALUES = new Values();",
+          "    private final Map<List<Object>, Map<String, Object>> state = new LinkedHashMap<>();",
+          "    private final int[] groupIndices = new int[] {%2$s};",
+          "    private List<Object> getGroupValues(Values _data) {",
+          "      List<Object> res = new ArrayList<>();",
+          "      for (int i: groupIndices) {",
+          "        res.add(_data.get(i));",
+          "      }",
+          "      return res;",
+          "    }",
+          "",
+          "    @Override",
+          "    public void flush(ChannelContext ctx) {",
+          "      emitAggregateResults(ctx);",
+          "      super.flush(ctx);",
+          "      state.clear();",
+          "    }",
+          "",
+          "    private void emitAggregateResults(ChannelContext ctx) {",
+          "        for (Map.Entry<List<Object>, Map<String, Object>> entry: state.entrySet()) {",
+          "          List<Object> groupValues = entry.getKey();",
+          "          Map<String, Object> accumulators = entry.getValue();",
+          "          %3$s",
+          "        }",
+          "    }",
+          "",
+          "    @Override",
+          "    public void dataReceived(ChannelContext ctx, Values _data) {",
+          ""
+  );
+
+  private static final String JOIN_STAGE_PROLOGUE = NEW_LINE_JOINER.join(
+          "  private static final ChannelHandler %1$s = ",
+          "    new AbstractChannelHandler() {",
+          "      Object left = %2$s;",
+          "      Object right = %3$s;",
+          "      Object source = null;",
+          "      List<Values> leftRows = new ArrayList<>();",
+          "      List<Values> rightRows = new ArrayList<>();",
+          "      boolean leftDone = false;",
+          "      boolean rightDone = false;",
+          "      int[] ordinals = new int[] {%4$s, %5$s};",
+          "",
+          "      Multimap<Object, Values> getJoinTable(List<Values> rows, int joinIndex) {",
+          "         Multimap<Object, Values> m = ArrayListMultimap.create();",
+          "         for(Values v: rows) {",
+          "           m.put(v.get(joinIndex), v);",
+          "         }",
+          "         return m;",
+          "      }",
+          "",
+          "      List<Values> join(Multimap<Object, Values> tab, List<Values> rows, int rowIdx, boolean rev) {",
+          "         List<Values> res = new ArrayList<>();",
+          "         for (Values row: rows) {",
+          "           for (Values mapValue: tab.get(row.get(rowIdx))) {",
+          "             if (mapValue != null) {",
+          "               Values joinedRow = new Values();",
+          "               if(rev) {",
+          "                 joinedRow.addAll(row);",
+          "                 joinedRow.addAll(mapValue);",
+          "               } else {",
+          "                 joinedRow.addAll(mapValue);",
+          "                 joinedRow.addAll(row);",
+          "               }",
+          "               res.add(joinedRow);",
+          "             }",
+          "           }",
+          "         }",
+          "         return res;",
+          "      }",
+          "",
+          "    @Override",
+          "    public void setSource(ChannelContext ctx, Object source) {",
+          "      this.source = source;",
+          "    }",
+          "",
+          "    @Override",
+          "    public void flush(ChannelContext ctx) {",
+          "        if (source == left) {",
+          "            leftDone = true;",
+          "        } else if (source == right) {",
+          "            rightDone = true;",
+          "        }",
+          "        if (leftDone && rightDone) {",
+          "          if (leftRows.size() <= rightRows.size()) {",
+          "            for(Values res: join(getJoinTable(leftRows, ordinals[0]), rightRows, ordinals[1], false)) {",
+          "              ctx.emit(res);",
+          "            }",
+          "          } else {",
+          "            for(Values res: join(getJoinTable(rightRows, ordinals[1]), leftRows, ordinals[0], true)) {",
+          "              ctx.emit(res);",
+          "            }",
+          "          }",
+          "          leftDone = rightDone = false;",
+          "          leftRows.clear();",
+          "          rightRows.clear();",
+          "          super.flush(ctx);",
+          "        }",
+          "    }",
+          "",
+          "    @Override",
+          "    public void dataReceived(ChannelContext ctx, Values _data) {",
+          ""
+  );
+
+  private static final String STAGE_PASSTHROUGH = NEW_LINE_JOINER.join(
+      "  private static final ChannelHandler %1$s = AbstractChannelHandler.PASS_THROUGH;",
+      "");
+
+  private static final String STAGE_ENUMERABLE_TABLE_SCAN = NEW_LINE_JOINER.join(
+          "  private static final ChannelHandler %1$s = new AbstractChannelHandler() {",
+          "    @Override",
+          "    public void flush(ChannelContext ctx) {",
+          "      ctx.setSource(this);",
+          "      super.flush(ctx);",
+          "    }",
+          "",
+          "    @Override",
+          "    public void dataReceived(ChannelContext ctx, Values _data) {",
+          "      ctx.setSource(this);",
+          "      ctx.emit(_data);",
+          "    }",
+          "  };",
+          "");
+
+  private int nameCount;
+  private Map<AggregateCall, String> aggregateCallVarNames = new HashMap<>();
+
+  RelNodeCompiler(PrintWriter pw, JavaTypeFactory typeFactory) {
+    this.pw = pw;
+    this.typeFactory = typeFactory;
+    this.rexCompiler = new RexNodeToJavaCodeCompiler(new RexBuilder(typeFactory));
+  }
+
+  @Override
+  public Void visitDelta(Delta delta, List<Void> inputStreams) throws Exception {
+    pw.print(String.format(STAGE_PASSTHROUGH, getStageName(delta)));
+    return null;
+  }
+
+  @Override
+  public Void visitFilter(Filter filter, List<Void> inputStreams) throws Exception {
+    beginStage(filter);
+
+    List<RexNode> childExps = filter.getChildExps();
+    RelDataType inputRowType = filter.getInput(0).getRowType();
+
+    pw.print("Context context = new StormContext(Processor.dataContext);\n");
+    pw.print("context.values = _data.toArray();\n");
+    pw.print("Object[] outputValues = new Object[1];\n");
+
+    pw.write(rexCompiler.compileToBlock(childExps, inputRowType).toString());
+
+    String r = "((Boolean) outputValues[0])";
+    if (filter.getCondition().getType().isNullable()) {
+      pw.print(String.format("    if (%s != null && %s) { ctx.emit(_data); }\n", r, r));
+    } else {
+      pw.print(String.format("    if (%s) { ctx.emit(_data); }\n", r, r));
+    }
+    endStage();
+    return null;
+  }
+
+  @Override
+  public Void visitProject(Project project, List<Void> inputStreams) throws Exception {
+    beginStage(project);
+
+    List<RexNode> childExps = project.getChildExps();
+    RelDataType inputRowType = project.getInput(0).getRowType();
+    int outputCount = project.getRowType().getFieldCount();
+
+    pw.print("Context context = new StormContext(Processor.dataContext);\n");
+    pw.print("context.values = _data.toArray();\n");
+    pw.print(String.format("Object[] outputValues = new Object[%d];\n", outputCount));
+
+    pw.write(rexCompiler.compileToBlock(childExps, inputRowType).toString());
+
+    pw.print("    ctx.emit(new Values(outputValues));\n");
+    endStage();
+    return null;
+  }
+
+  @Override
+  public Void defaultValue(RelNode n, List<Void> inputStreams) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Void visitTableScan(TableScan scan, List<Void> inputStreams) throws Exception {
+    pw.print(String.format(STAGE_ENUMERABLE_TABLE_SCAN, getStageName(scan)));
+    return null;
+  }
+
+  @Override
+  public Void visitAggregate(Aggregate aggregate, List<Void> inputStreams) throws Exception {
+    beginAggregateStage(aggregate);
+    pw.println("        if (_data != null) {");
+    pw.println("        List<Object> curGroupValues = getGroupValues(_data);");
+    pw.println("        if (!state.containsKey(curGroupValues)) {");
+    pw.println("          state.put(curGroupValues, new HashMap<String, Object>());");
+    pw.println("        }");
+    pw.println("        Map<String, Object> accumulators = state.get(curGroupValues);");
+    for (AggregateCall call : aggregate.getAggCallList()) {
+      aggregate(call);
+    }
+    pw.println("        }");
+    endStage();
+    return null;
+  }
+
+  @Override
+  public Void visitJoin(Join join, List<Void> inputStreams) {
+    beginJoinStage(join);
+    pw.println("        if (source == left) {");
+    pw.println("            leftRows.add(_data);");
+    pw.println("        } else if (source == right) {");
+    pw.println("            rightRows.add(_data);");
+    pw.println("        }");
+    endStage();
+    return null;
+  }
+
+  private String groupValueEmitStr(String var, int n) {
+    int count = 0;
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0; i < n; i++) {
+      if (++count > 1) {
+        sb.append(", ");
+      }
+      sb.append(var).append(".").append("get(").append(i).append(")");
+    }
+    return sb.toString();
+  }
+
+  private String emitAggregateStmts(Aggregate aggregate) {
+    List<String> res = new ArrayList<>();
+    StringWriter sw = new StringWriter();
+    for (AggregateCall call : aggregate.getAggCallList()) {
+      res.add(aggregateResult(call, new PrintWriter(sw)));
+    }
+    return NEW_LINE_JOINER.join(sw.toString(),
+                                String.format("          ctx.emit(new Values(%s, %s));",
+                                              groupValueEmitStr("groupValues", aggregate.getGroupSet().cardinality()),
+                                              Joiner.on(", ").join(res)));
+  }
+
+  private String aggregateResult(AggregateCall call, PrintWriter pw) {
+    SqlAggFunction aggFunction = call.getAggregation();
+    String aggregationName = call.getAggregation().getName();
+    Type ty = typeFactory.getJavaClass(call.getType());
+    String result;
+    if (aggFunction instanceof SqlUserDefinedAggFunction) {
+      AggregateFunction aggregateFunction = ((SqlUserDefinedAggFunction) aggFunction).function;
+      result = doAggregateResult((AggregateFunctionImpl) aggregateFunction, reserveAggVarName(call), ty, pw);
+    } else {
+      List<BuiltinAggregateFunctions.TypeClass> typeClasses = BuiltinAggregateFunctions.TABLE.get(aggregationName);
+      if (typeClasses == null) {
+        throw new UnsupportedOperationException(aggregationName + " Not implemented");
+      }
+      result = doAggregateResult(AggregateFunctionImpl.create(findMatchingClass(aggregationName, typeClasses, ty)),
+                                 reserveAggVarName(call), ty, pw);
+    }
+    return result;
+  }
+
+  private String doAggregateResult(AggregateFunctionImpl aggFn, String varName, Type ty, PrintWriter pw) {
+    String resultName = varName + "_result";
+    Class<?> accumulatorType = aggFn.accumulatorType;
+    Class<?> resultType = aggFn.resultType;
+    List<String> args = new ArrayList<>();
+    if (!aggFn.isStatic) {
+      String aggObjName = String.format("%s_obj", varName);
+      String aggObjClassName = aggFn.initMethod.getDeclaringClass().getCanonicalName();
+      pw.println("          @SuppressWarnings(\"unchecked\")");
+      pw.println(String.format("          final %1$s %2$s = (%1$s) accumulators.get(\"%2$s\");", aggObjClassName,
+              aggObjName));
+      args.add(aggObjName);
+    }
+    args.add(String.format("(%s)accumulators.get(\"%s\")", accumulatorType.getCanonicalName(), varName));
+    pw.println(String.format("          final %s %s = %s;", resultType.getCanonicalName(),
+                             resultName, printMethodCall(aggFn.resultMethod, args)));
+
+    return resultName;
+  }
+
+  private void aggregate(AggregateCall call) {
+    SqlAggFunction aggFunction = call.getAggregation();
+    String aggregationName = call.getAggregation().getName();
+    Type ty = typeFactory.getJavaClass(call.getType());
+    if (call.getArgList().size() != 1) {
+      if (aggregationName.equals("COUNT")) {
+        if (call.getArgList().size() != 0) {
+          throw new UnsupportedOperationException("Count with nullable fields");
+        }
+      }
+    }
+    if (aggFunction instanceof SqlUserDefinedAggFunction) {
+      AggregateFunction aggregateFunction = ((SqlUserDefinedAggFunction) aggFunction).function;
+      doAggregate((AggregateFunctionImpl) aggregateFunction, reserveAggVarName(call), ty, call.getArgList());
+    } else {
+      List<BuiltinAggregateFunctions.TypeClass> typeClasses = BuiltinAggregateFunctions.TABLE.get(aggregationName);
+      if (typeClasses == null) {
+        throw new UnsupportedOperationException(aggregationName + " Not implemented");
+      }
+      doAggregate(AggregateFunctionImpl.create(findMatchingClass(aggregationName, typeClasses, ty)),
+                  reserveAggVarName(call), ty, call.getArgList());
+    }
+  }
+
+  private Class<?> findMatchingClass(String aggregationName, List<BuiltinAggregateFunctions.TypeClass> typeClasses, Type ty) {
+    for (BuiltinAggregateFunctions.TypeClass typeClass : typeClasses) {
+      if (typeClass.ty.equals(BuiltinAggregateFunctions.TypeClass.GenericType.class) || typeClass.ty.equals(ty)) {
+        return typeClass.clazz;
+      }
+    }
+    throw new UnsupportedOperationException(aggregationName + " Not implemeted for type '" + ty + "'");
+  }
+
+  private void doAggregate(AggregateFunctionImpl aggFn, String varName, Type ty, List<Integer> argList) {
+    List<String> args = new ArrayList<>();
+    Class<?> accumulatorType = aggFn.accumulatorType;
+    if (!aggFn.isStatic) {
+      String aggObjName = String.format("%s_obj", varName);
+      String aggObjClassName = aggFn.initMethod.getDeclaringClass().getCanonicalName();
+      pw.println(String.format("          if (!accumulators.containsKey(\"%s\")) { ", aggObjName));
+      pw.println(String.format("            accumulators.put(\"%s\", new %s());", aggObjName, aggObjClassName));
+      pw.println("          }");
+      pw.println("          @SuppressWarnings(\"unchecked\")");
+      pw.println(String.format("          final %1$s %2$s = (%1$s) accumulators.get(\"%2$s\");", aggObjClassName,
+              aggObjName));
+      args.add(aggObjName);
+    }
+    args.add(String.format("%1$s == null ? %2$s : (%3$s) %1$s",
+                           "accumulators.get(\"" + varName + "\")",
+                           printMethodCall(aggFn.initMethod, args),
+                           accumulatorType.getCanonicalName()));
+    if (argList.isEmpty()) {
+      args.add("EMPTY_VALUES");
+    } else {
+      for (int i = 0; i < aggFn.valueTypes.size(); i++) {
+        args.add(String.format("(%s) %s", aggFn.valueTypes.get(i).getCanonicalName(), "_data.get(" + argList.get(i) + ")"));
+      }
+    }
+    pw.print(String.format("          accumulators.put(\"%s\", %s);\n",
+                           varName,
+                           printMethodCall(aggFn.addMethod, args)));
+  }
+
+  private String reserveAggVarName(AggregateCall call) {
+    String varName;
+    if ((varName = aggregateCallVarNames.get(call)) == null) {
+      varName = call.getAggregation().getName() + ++nameCount;
+      aggregateCallVarNames.put(call, varName);
+    }
+    return varName;
+  }
+
+  private void beginStage(RelNode n) {
+    pw.print(String.format(STAGE_PROLOGUE, getStageName(n)));
+  }
+
+  private void beginAggregateStage(Aggregate n) {
+    pw.print(String.format(AGGREGATE_STAGE_PROLOGUE, getStageName(n), getGroupByIndices(n), emitAggregateStmts(n)));
+  }
+
+  private void beginJoinStage(Join join) {
+    int[] ordinals = new int[2];
+    if (!RelOptUtil.analyzeSimpleEquiJoin((LogicalJoin) join, ordinals)) {
+      throw new UnsupportedOperationException("Only simple equi joins are supported");
+    }
+
+    pw.print(String.format(JOIN_STAGE_PROLOGUE, getStageName(join),
+                           getStageName(join.getLeft()),
+                           getStageName(join.getRight()),
+                           ordinals[0],
+                           ordinals[1]));
+  }
+
+  private void endStage() {
+    pw.print("  }\n  };\n");
+  }
+
+  static String getStageName(RelNode n) {
+    return n.getClass().getSimpleName().toUpperCase() + "_" + n.getId();
+  }
+
+  private String getGroupByIndices(Aggregate n) {
+    StringBuilder res = new StringBuilder();
+    int count = 0;
+    for (int i : n.getGroupSet()) {
+      if (++count > 1) {
+        res.append(", ");
+      }
+      res.append(i);
+    }
+    return res.toString();
+  }
+
+  public static String printMethodCall(Method method, List<String> args) {
+    return printMethodCall(method.getDeclaringClass(), method.getName(),
+            Modifier.isStatic(method.getModifiers()), args);
+  }
+
+  private static String printMethodCall(Class<?> clazz, String method, boolean isStatic, List<String> args) {
+    if (isStatic) {
+      return String.format("%s.%s(%s)", clazz.getCanonicalName(), method, Joiner.on(',').join(args));
+    } else {
+      return String.format("%s.%s(%s)", args.get(0), method,
+              Joiner.on(',').join(args.subList(1, args.size())));
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/javac/CompilingClassLoader.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/javac/CompilingClassLoader.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/javac/CompilingClassLoader.java
new file mode 100644
index 0000000..0b7c053
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/javac/CompilingClassLoader.java
@@ -0,0 +1,225 @@
+/*
+ * Copyright (C) 2010 Google, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.javac;
+
+
+import javax.tools.DiagnosticListener;
+import javax.tools.FileObject;
+import javax.tools.ForwardingJavaFileManager;
+import javax.tools.JavaCompiler;
+import javax.tools.JavaFileManager;
+import javax.tools.JavaFileObject;
+import javax.tools.SimpleJavaFileObject;
+import javax.tools.ToolProvider;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import static java.util.Collections.singleton;
+
+/**
+ * This is a Java ClassLoader that will attempt to load a class from a string of source code.
+ *
+ * <h3>Example</h3>
+ *
+ * <pre>
+ * String className = "com.foo.MyClass";
+ * String classSource =
+ *   "package com.foo;\n" +
+ *   "public class MyClass implements Runnable {\n" +
+ *   "  @Override public void run() {\n" +
+ *   "   log(\"Hello world\");\n" +
+ *   "  }\n" +
+ *   "}";
+ *
+ * // Load class from source.
+ * ClassLoader classLoader = new CompilingClassLoader(
+ *     parentClassLoader, className, classSource);
+ * Class myClass = classLoader.loadClass(className);
+ *
+ * // Use it.
+ * Runnable instance = (Runnable)myClass.newInstance();
+ * instance.run();
+ * </pre>
+ *
+ * Only one chunk of source can be compiled per instance of CompilingClassLoader. If you need to
+ * compile more, create multiple CompilingClassLoader instances.
+ *
+ * Uses Java 1.6's in built compiler API.
+ *
+ * If the class cannot be compiled, loadClass() will throw a ClassNotFoundException and log the
+ * compile errors to System.err. If you don't want the messages logged, or want to explicitly handle
+ * the messages you can provide your own {@link javax.tools.DiagnosticListener} through
+ * {#setDiagnosticListener()}.
+ *
+ * @see java.lang.ClassLoader
+ * @see javax.tools.JavaCompiler
+ */
+public class CompilingClassLoader extends ClassLoader {
+
+  /**
+   * Thrown when code cannot be compiled.
+   */
+  public static class CompilerException extends Exception {
+    private static final long serialVersionUID = -2936958840023603270L;
+
+    public CompilerException(String message) {
+      super(message);
+    }
+  }
+
+  private final Map<String, ByteArrayOutputStream> byteCodeForClasses = new HashMap<>();
+
+  private static final URI EMPTY_URI;
+
+  static {
+    try {
+      // Needed to keep SimpleFileObject constructor happy.
+      EMPTY_URI = new URI("");
+    } catch (URISyntaxException e) {
+      throw new Error(e);
+    }
+  }
+
+  /**
+   * @param parent Parent classloader to resolve dependencies from.
+   * @param className Name of class to compile. eg. "com.foo.MyClass".
+   * @param sourceCode Java source for class. e.g. "package com.foo; class MyClass { ... }".
+   * @param diagnosticListener Notified of compiler errors (may be null).
+   */
+  public CompilingClassLoader(
+      ClassLoader parent,
+      String className,
+      String sourceCode,
+      DiagnosticListener<JavaFileObject> diagnosticListener)
+      throws CompilerException {
+    super(parent);
+    if (!compileSourceCodeToByteCode(className, sourceCode, diagnosticListener)) {
+      throw new CompilerException("Could not compile " + className);
+    }
+  }
+
+  public Map<String, ByteArrayOutputStream> getClasses() {
+    return byteCodeForClasses;
+  }
+
+  /**
+   * Override ClassLoader's class resolving method. Don't call this directly, instead use
+   * {@link ClassLoader#loadClass(String)}.
+   */
+  @Override
+  public Class<?> findClass(String name) throws ClassNotFoundException {
+    ByteArrayOutputStream byteCode = byteCodeForClasses.get(name);
+    if (byteCode == null) {
+      throw new ClassNotFoundException(name);
+    }
+    return defineClass(name, byteCode.toByteArray(), 0, byteCode.size());
+  }
+
+  /**
+   * @return Whether compilation was successful.
+   */
+  private boolean compileSourceCodeToByteCode(
+      String className, String sourceCode, DiagnosticListener<JavaFileObject> diagnosticListener) {
+    JavaCompiler javaCompiler = ToolProvider.getSystemJavaCompiler();
+
+    // Set up the in-memory filesystem.
+    InMemoryFileManager fileManager =
+        new InMemoryFileManager(javaCompiler.getStandardFileManager(null, null, null));
+    JavaFileObject javaFile = new InMemoryJavaFile(className, sourceCode);
+
+    // Javac option: remove these when the javac zip impl is fixed
+    // (http://b/issue?id=1822932)
+    System.setProperty("useJavaUtilZip", "true"); // setting value to any non-null string
+    List<String> options = new LinkedList<>();
+    // this is ignored by javac currently but useJavaUtilZip should be
+    // a valid javac XD option, which is another bug
+    options.add("-XDuseJavaUtilZip");
+
+    // Now compile!
+    JavaCompiler.CompilationTask compilationTask =
+        javaCompiler.getTask(
+            null, // Null: log any unhandled errors to stderr.
+            fileManager,
+            diagnosticListener,
+            options,
+            null,
+            singleton(javaFile));
+    return compilationTask.call();
+  }
+
+  /**
+   * Provides an in-memory representation of JavaFileManager abstraction, so we do not need to write
+   * any files to disk.
+   *
+   * When files are written to, rather than putting the bytes on disk, they are appended to buffers
+   * in byteCodeForClasses.
+   *
+   * @see javax.tools.JavaFileManager
+   */
+  private class InMemoryFileManager extends ForwardingJavaFileManager<JavaFileManager> {
+    public InMemoryFileManager(JavaFileManager fileManager) {
+      super(fileManager);
+    }
+
+    @Override
+    public JavaFileObject getJavaFileForOutput(
+        Location location, final String className, JavaFileObject.Kind kind, FileObject sibling)
+        throws IOException {
+      return new SimpleJavaFileObject(EMPTY_URI, kind) {
+        @Override
+        public OutputStream openOutputStream() throws IOException {
+          ByteArrayOutputStream outputStream = byteCodeForClasses.get(className);
+          if (outputStream != null) {
+            throw new IllegalStateException("Cannot write more than once");
+          }
+          // Reasonable size for a simple .class.
+          outputStream = new ByteArrayOutputStream(256);
+          byteCodeForClasses.put(className, outputStream);
+          return outputStream;
+        }
+      };
+    }
+  }
+
+  private static class InMemoryJavaFile extends SimpleJavaFileObject {
+    private final String sourceCode;
+
+    public InMemoryJavaFile(String className, String sourceCode) {
+      super(makeUri(className), Kind.SOURCE);
+      this.sourceCode = sourceCode;
+    }
+
+    private static URI makeUri(String className) {
+      try {
+        return new URI(className.replaceAll("\\.", "/") + Kind.SOURCE.extension);
+      } catch (URISyntaxException e) {
+        throw new RuntimeException(e); // Not sure what could cause this.
+      }
+    }
+
+    @Override
+    public CharSequence getCharContent(boolean ignoreEncodingErrors) throws IOException {
+      return sourceCode;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnConstraint.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnConstraint.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnConstraint.java
new file mode 100644
index 0000000..c67d8e7
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnConstraint.java
@@ -0,0 +1,42 @@
+/**
+ * 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.sql.parser;
+
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlMonotonicity;
+
+public class ColumnConstraint extends SqlLiteral {
+  private ColumnConstraint(
+      Object value, SqlTypeName typeName, SqlParserPos pos) {
+    super(value, typeName, pos);
+  }
+
+  public static class PrimaryKey extends ColumnConstraint {
+    private final SqlMonotonicity monotonicity;
+    public PrimaryKey(SqlMonotonicity monotonicity, SqlParserPos pos) {
+      super(SqlDDLKeywords.PRIMARY, SqlTypeName.SYMBOL, pos);
+      this.monotonicity = monotonicity;
+    }
+    public SqlMonotonicity monotonicity() {
+      return monotonicity;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnDefinition.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnDefinition.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnDefinition.java
new file mode 100644
index 0000000..3520b86
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnDefinition.java
@@ -0,0 +1,44 @@
+/**
+ * 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.sql.parser;
+
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.parser.SqlParserPos;
+
+import java.util.Arrays;
+
+public class ColumnDefinition extends SqlNodeList {
+  public ColumnDefinition(
+      SqlIdentifier name, SqlDataTypeSpec type, ColumnConstraint constraint, SqlParserPos pos) {
+    super(Arrays.asList(name, type, constraint), pos);
+  }
+
+  public String name() {
+    return get(0).toString();
+  }
+
+  public SqlDataTypeSpec type() {
+    return (SqlDataTypeSpec) get(1);
+  }
+
+  public ColumnConstraint constraint() {
+    return (ColumnConstraint) get(2);
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateFunction.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateFunction.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateFunction.java
new file mode 100644
index 0000000..a53802c
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateFunction.java
@@ -0,0 +1,94 @@
+/**
+ * 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.sql.parser;
+
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.util.ImmutableNullableList;
+import org.apache.calcite.util.NlsString;
+
+import java.util.List;
+
+public class SqlCreateFunction extends SqlCall {
+    public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator(
+            "CREATE_FUNCTION", SqlKind.OTHER) {
+        @Override
+        public SqlCall createCall(
+                SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... o) {
+            assert functionQualifier == null;
+            return new SqlCreateFunction(pos, (SqlIdentifier) o[0], o[1], o[2]);
+        }
+
+        @Override
+        public void unparse(
+                SqlWriter writer, SqlCall call, int leftPrec, int rightPrec) {
+            SqlCreateFunction t = (SqlCreateFunction) call;
+            UnparseUtil u = new UnparseUtil(writer, leftPrec, rightPrec);
+            u.keyword("CREATE", "FUNCTION").node(t.functionName).keyword("AS").node(t.className);
+            if (t.jarName != null) {
+                u.keyword("USING", "JAR").node(t.jarName);
+            }
+        }
+    };
+
+    private final SqlIdentifier functionName;
+    private final SqlNode className;
+    private final SqlNode jarName;
+
+    public SqlCreateFunction(SqlParserPos pos, SqlIdentifier functionName, SqlNode className, SqlNode jarName) {
+        super(pos);
+        this.functionName = functionName;
+        this.className = className;
+        this.jarName = jarName;
+    }
+
+    @Override
+    public SqlOperator getOperator() {
+        return OPERATOR;
+    }
+
+    @Override
+    public List<SqlNode> getOperandList() {
+        return ImmutableNullableList.of(functionName, className);
+    }
+
+
+    @Override
+    public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
+        getOperator().unparse(writer, this, leftPrec, rightPrec);
+    }
+
+    public String functionName() {
+        return functionName.toString();
+    }
+
+    public String className() {
+        return ((NlsString)SqlLiteral.value(className)).getValue();
+    }
+
+    public String jarName() {
+        return jarName == null ? null : ((NlsString)SqlLiteral.value(jarName)).getValue();
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateTable.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateTable.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateTable.java
new file mode 100644
index 0000000..670eedb
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateTable.java
@@ -0,0 +1,166 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.parser;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.util.ImmutableNullableList;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Properties;
+
+public class SqlCreateTable extends SqlCall {
+  private static final int DEFAULT_PARALLELISM = 1;
+
+  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator(
+      "CREATE_TABLE", SqlKind.OTHER) {
+    @Override
+    public SqlCall createCall(
+        SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... o) {
+      assert functionQualifier == null;
+      return new SqlCreateTable(pos, (SqlIdentifier) o[0], (SqlNodeList) o[1],
+                                o[2], o[3], o[4], o[5], o[6], o[7]);
+    }
+
+    @Override
+    public void unparse(
+        SqlWriter writer, SqlCall call, int leftPrec, int rightPrec) {
+      SqlCreateTable t = (SqlCreateTable) call;
+      UnparseUtil u = new UnparseUtil(writer, leftPrec, rightPrec);
+      u.keyword("CREATE", "EXTERNAL", "TABLE").node(t.tblName).nodeList(
+          t.fieldList);
+      if (t.inputFormatClass != null && t.outputFormatClass != null) {
+        u.keyword("STORED", "AS", "INPUTFORMAT").node(
+            t.inputFormatClass).keyword("OUTPUTFORMAT").node(
+            t.outputFormatClass);
+      }
+      u.keyword("LOCATION").node(t.location);
+      if (t.parallelism != null) {
+        u.keyword("PARALLELISM").node(t.parallelism);
+      }
+      if (t.properties != null) {
+        u.keyword("TBLPROPERTIES").node(t.properties);
+      }
+      if (t.query != null) {
+        u.keyword("AS").node(t.query);
+      }
+    }
+  };
+
+  private final SqlIdentifier tblName;
+  private final SqlNodeList fieldList;
+  private final SqlNode inputFormatClass;
+  private final SqlNode outputFormatClass;
+  private final SqlNode location;
+  private final SqlNode parallelism;
+  private final SqlNode properties;
+  private final SqlNode query;
+
+  public SqlCreateTable(
+          SqlParserPos pos, SqlIdentifier tblName, SqlNodeList fieldList,
+          SqlNode inputFormatClass, SqlNode outputFormatClass, SqlNode location,
+          SqlNode parallelism, SqlNode properties, SqlNode query) {
+    super(pos);
+    this.tblName = tblName;
+    this.fieldList = fieldList;
+    this.inputFormatClass = inputFormatClass;
+    this.outputFormatClass = outputFormatClass;
+    this.location = location;
+    this.parallelism = parallelism;
+    this.properties = properties;
+    this.query = query;
+  }
+
+  @Override
+  public SqlOperator getOperator() {
+    return OPERATOR;
+  }
+
+  @Override
+  public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
+    getOperator().unparse(writer, this, leftPrec, rightPrec);
+  }
+
+  @Override
+  public List<SqlNode> getOperandList() {
+    return ImmutableNullableList.of(tblName, fieldList, inputFormatClass,
+                                    outputFormatClass, location, properties,
+                                    query);
+  }
+
+  public String tableName() {
+    return tblName.toString();
+  }
+
+  public URI location() {
+    return URI.create(getString(location));
+  }
+
+  public Integer parallelism() {
+    String parallelismStr = getString(parallelism);
+    if (parallelismStr != null) {
+      return Integer.parseInt(parallelismStr);
+    } else {
+      return DEFAULT_PARALLELISM;
+    }
+  }
+
+  public String inputFormatClass() {
+    return getString(inputFormatClass);
+  }
+
+  public String outputFormatClass() {
+    return getString(outputFormatClass);
+  }
+
+  public Properties properties() {
+    Properties props = new Properties();
+    if (properties != null) {
+      try {
+        ObjectMapper mapper = new ObjectMapper();
+        HashMap<String, Object> map = mapper.readValue(getString(properties), HashMap.class);
+        props.putAll(map);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+    return props;
+  }
+
+  private String getString(SqlNode n) {
+    return n == null ? null : SqlLiteral.stringValue(n);
+  }
+
+  @SuppressWarnings("unchecked")
+  public List<ColumnDefinition> fieldList() {
+    return (List<ColumnDefinition>)((List<? extends SqlNode>)fieldList.getList());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlDDLKeywords.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlDDLKeywords.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlDDLKeywords.java
new file mode 100644
index 0000000..3112e53
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlDDLKeywords.java
@@ -0,0 +1,27 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.parser;
+
+import org.apache.calcite.sql.SqlLiteral;
+
+/**
+ * Define the keywords that can occur in a CREATE TABLE statement
+ */
+public enum SqlDDLKeywords implements SqlLiteral.SqlSymbol {
+  PRIMARY
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/StormParser.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/StormParser.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/StormParser.java
new file mode 100644
index 0000000..8444e1e
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/StormParser.java
@@ -0,0 +1,47 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.parser;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.calcite.config.Lex;
+import org.apache.storm.sql.parser.impl.StormParserImpl;
+
+import java.io.StringReader;
+
+public class StormParser {
+  public static final int DEFAULT_IDENTIFIER_MAX_LENGTH = 128;
+  private final StormParserImpl impl;
+
+  public StormParser(String s) {
+    this.impl = new StormParserImpl(new StringReader(s));
+    this.impl.setTabSize(1);
+    this.impl.setQuotedCasing(Lex.ORACLE.quotedCasing);
+    this.impl.setUnquotedCasing(Lex.ORACLE.unquotedCasing);
+    this.impl.setIdentifierMaxLength(DEFAULT_IDENTIFIER_MAX_LENGTH);
+    /*
+     *  By default parser uses [ ] for quoting identifiers. Switching to DQID (double quoted identifiers)
+     *  is needed for array and map access (m['x'] = 1 or arr[2] = 10 etc) to work.
+     */
+    this.impl.switchTo("DQID");
+  }
+
+  @VisibleForTesting
+  public StormParserImpl impl() {
+    return impl;
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/UnparseUtil.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/UnparseUtil.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/UnparseUtil.java
new file mode 100644
index 0000000..8e0a1d9
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/UnparseUtil.java
@@ -0,0 +1,59 @@
+/**
+ * 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.sql.parser;
+
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlWriter;
+
+class UnparseUtil {
+  private final SqlWriter writer;
+  private final int leftPrec;
+  private final int rightPrec;
+
+  UnparseUtil(SqlWriter writer, int leftPrec, int rightPrec) {
+    this.writer = writer;
+    this.leftPrec = leftPrec;
+    this.rightPrec = rightPrec;
+  }
+
+  UnparseUtil keyword(String... keywords) {
+    for (String k : keywords) {
+      writer.keyword(k);
+    }
+    return this;
+  }
+
+  UnparseUtil node(SqlNode n) {
+    n.unparse(writer, leftPrec, rightPrec);
+    return this;
+  }
+
+  UnparseUtil nodeList(SqlNodeList l) {
+    writer.keyword("(");
+    if (l.size() > 0) {
+      l.get(0).unparse(writer, leftPrec, rightPrec);
+      for (int i = 1; i < l.size(); ++i) {
+        writer.keyword(",");
+        l.get(i).unparse(writer, leftPrec, rightPrec);
+      }
+    }
+    writer.keyword(")");
+    return this;
+  }
+}


[10/23] storm git commit: STORM-2453 Move non-connectors into the top directory

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/README.md
----------------------------------------------------------------------
diff --git a/flux/README.md b/flux/README.md
new file mode 100644
index 0000000..206ae22
--- /dev/null
+++ b/flux/README.md
@@ -0,0 +1,872 @@
+# flux
+A framework for creating and deploying Apache Storm streaming computations with less friction.
+
+## Definition
+**flux** |fl\u0259ks| _noun_
+
+1. The action or process of flowing or flowing out
+2. Continuous change
+3. In physics, the rate of flow of a fluid, radiant energy, or particles across a given area
+4. A substance mixed with a solid to lower its melting point
+
+## Rationale
+Bad things happen when configuration is hard-coded. No one should have to recompile or repackage an application in
+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,
+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.
+
+## Features
+
+ * Easily configure and deploy Storm topologies (Both Storm core and Microbatch API) without embedding configuration
+   in your topology code
+ * Support for existing topology code (see below)
+ * Define Storm Core API (Spouts/Bolts) using a flexible YAML DSL
+ * YAML DSL support for most Storm components (storm-kafka, storm-hdfs, storm-hbase, etc.)
+ * Convenient support for multi-lang components
+ * External property substitution/filtering for easily switching between configurations/environments (similar to Maven-style
+   `${variable.name}` substitution)
+
+## Usage
+
+To use Flux, add it as a dependency and package all your Storm components in a fat jar, then create a YAML document
+to define your topology (see below for YAML configuration options).
+
+### Building from Source
+The easiest way to use Flux, is to add it as a Maven dependency in you project as described below.
+
+If you would like to build Flux from source and run the unit/integration tests, you will need the following installed
+on your system:
+
+* Python 2.6.x or later
+* Node.js 0.10.x or later
+
+#### Building with unit tests enabled:
+
+```
+mvn clean install
+```
+
+#### Building with unit tests disabled:
+If you would like to build Flux without installing Python or Node.js you can simply skip the unit tests:
+
+```
+mvn clean install -DskipTests=true
+```
+
+Note that if you plan on using Flux to deploy topologies to a remote cluster, you will still need to have Python
+installed since it is required by Apache Storm.
+
+
+#### Building with integration tests enabled:
+
+```
+mvn clean install -DskipIntegration=false
+```
+
+
+### Packaging with Maven
+To enable Flux for your Storm components, you need to add it as a dependency such that it's included in the Storm
+topology jar. This can be accomplished with the Maven shade plugin (preferred) or the Maven assembly plugin (not
+recommended).
+
+#### Flux Maven Dependency
+The current version of Flux is available in Maven Central at the following coordinates:
+```xml
+<dependency>
+    <groupId>org.apache.storm</groupId>
+    <artifactId>flux-core</artifactId>
+    <version>${storm.version}</version>
+</dependency>
+```
+
+Using shell spouts and bolts requires additional Flux Wrappers library:
+```xml
+<dependency>
+    <groupId>org.apache.storm</groupId>
+    <artifactId>flux-wrappers</artifactId>
+    <version>${storm.version}</version>
+</dependency>
+```
+
+#### Creating a Flux-Enabled Topology JAR
+The example below illustrates Flux usage with the Maven shade plugin:
+
+ ```xml
+<!-- include Flux and user dependencies in the shaded jar -->
+<dependencies>
+    <!-- Flux include -->
+    <dependency>
+        <groupId>org.apache.storm</groupId>
+        <artifactId>flux-core</artifactId>
+        <version>${storm.version}</version>
+    </dependency>
+    <!-- Flux Wrappers include -->
+    <dependency>
+        <groupId>org.apache.storm</groupId>
+        <artifactId>flux-wrappers</artifactId>
+        <version>${storm.version}</version>
+    </dependency>
+
+    <!-- add user dependencies here... -->
+
+</dependencies>
+<!-- create a fat jar that includes all dependencies -->
+<build>
+    <plugins>
+        <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-shade-plugin</artifactId>
+            <version>1.4</version>
+            <configuration>
+                <createDependencyReducedPom>true</createDependencyReducedPom>
+            </configuration>
+            <executions>
+                <execution>
+                    <phase>package</phase>
+                    <goals>
+                        <goal>shade</goal>
+                    </goals>
+                    <configuration>
+                        <transformers>
+                            <transformer
+                                    implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
+                            <transformer
+                                    implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
+                                <mainClass>org.apache.storm.flux.Flux</mainClass>
+                            </transformer>
+                        </transformers>
+                    </configuration>
+                </execution>
+            </executions>
+        </plugin>
+    </plugins>
+</build>
+ ```
+
+### Deploying and Running a Flux Topology
+Once your topology components are packaged with the Flux dependency, you can run different topologies either locally
+or remotely using the `storm jar` command. For example, if your fat jar is named `myTopology-0.1.0-SNAPSHOT.jar` you
+could run it locally with the command:
+
+
+```bash
+storm jar myTopology-0.1.0-SNAPSHOT.jar org.apache.storm.flux.Flux --local my_config.yaml
+
+```
+
+### Command line options
+```
+usage: storm jar <my_topology_uber_jar.jar> org.apache.storm.flux.Flux
+             [options] <topology-config.yaml>
+ -d,--dry-run                 Do not run or deploy the topology. Just
+                              build, validate, and print information about
+                              the topology.
+ -e,--env-filter              Perform environment variable substitution.
+                              Replace keys identified with `${ENV-[NAME]}`
+                              will be replaced with the corresponding
+                              `NAME` environment value
+ -f,--filter <file>           Perform property substitution. Use the
+                              specified file as a source of properties,
+                              and replace keys identified with {$[property
+                              name]} with the value defined in the
+                              properties file.
+ -i,--inactive                Deploy the topology, but do not activate it.
+ -l,--local                   Run the topology in local mode.
+ -n,--no-splash               Suppress the printing of the splash screen.
+ -q,--no-detail               Suppress the printing of topology details.
+ -r,--remote                  Deploy the topology to a remote cluster.
+ -R,--resource                Treat the supplied path as a classpath
+                              resource instead of a file.
+ -s,--sleep <ms>              When running locally, the amount of time to
+                              sleep (in ms.) before killing the topology
+                              and shutting down the local cluster.
+ -z,--zookeeper <host:port>   When running in local mode, use the
+                              ZooKeeper at the specified <host>:<port>
+                              instead of the in-process ZooKeeper.
+                              (requires Storm 0.9.3 or later)
+```
+
+**NOTE:** Flux tries to avoid command line switch collision with the `storm` command, and allows any other command line
+switches to pass through to the `storm` command.
+
+For example, you can use the `storm` command switch `-c` to override a topology configuration property. The following
+example command will run Flux and override the `nimus.host` configuration:
+
+```bash
+storm jar myTopology-0.1.0-SNAPSHOT.jar org.apache.storm.flux.Flux --remote my_config.yaml -c nimbus.host=localhost
+```
+
+### Sample output
+```
+\u2588\u2588\u2588\u2588\u2588\u2588\u2588\u2557\u2588\u2588\u2557     \u2588\u2588\u2557   \u2588\u2588\u2557\u2588\u2588\u2557  \u2588\u2588\u2557
+\u2588\u2588\u2554\u2550\u2550\u2550\u2550\u255d\u2588\u2588\u2551     \u2588\u2588\u2551   \u2588\u2588\u2551\u255a\u2588\u2588\u2557\u2588\u2588\u2554\u255d
+\u2588\u2588\u2588\u2588\u2588\u2557  \u2588\u2588\u2551     \u2588\u2588\u2551   \u2588\u2588\u2551 \u255a\u2588\u2588\u2588\u2554\u255d
+\u2588\u2588\u2554\u2550\u2550\u255d  \u2588\u2588\u2551     \u2588\u2588\u2551   \u2588\u2588\u2551 \u2588\u2588\u2554\u2588\u2588\u2557
+\u2588\u2588\u2551     \u2588\u2588\u2588\u2588\u2588\u2588\u2588\u2557\u255a\u2588\u2588\u2588\u2588\u2588\u2588\u2554\u255d\u2588\u2588\u2554\u255d \u2588\u2588\u2557
+\u255a\u2550\u255d     \u255a\u2550\u2550\u2550\u2550\u2550\u2550\u255d \u255a\u2550\u2550\u2550\u2550\u2550\u255d \u255a\u2550\u255d  \u255a\u2550\u255d
++-         Apache Storm        -+
++-  data FLow User eXperience  -+
+Version: 0.3.0
+Parsing file: /Users/hsimpson/Projects/donut_domination/storm/shell_test.yaml
+---------- TOPOLOGY DETAILS ----------
+Name: shell-topology
+--------------- SPOUTS ---------------
+sentence-spout[1](org.apache.storm.flux.wrappers.spouts.FluxShellSpout)
+---------------- BOLTS ---------------
+splitsentence[1](org.apache.storm.flux.wrappers.bolts.FluxShellBolt)
+log[1](org.apache.storm.flux.wrappers.bolts.LogInfoBolt)
+count[1](org.apache.storm.testing.TestWordCounter)
+--------------- STREAMS ---------------
+sentence-spout --SHUFFLE--> splitsentence
+splitsentence --FIELDS--> count
+count --SHUFFLE--> log
+--------------------------------------
+Submitting topology: 'shell-topology' to remote cluster...
+```
+
+## YAML Configuration
+Flux topologies are defined in a YAML file that describes a topology. A Flux topology
+definition consists of the following:
+
+  1. A topology name
+  2. A list of topology "components" (named Java objects that will be made available in the environment)
+  3. **EITHER** (A DSL topology definition):
+      * A list of spouts, each identified by a unique ID
+      * A list of bolts, each identified by a unique ID
+      * A list of "stream" objects representing a flow of tuples between spouts and bolts
+  4. **OR** (A JVM class that can produce a `org.apache.storm.generated.StormTopology` instance:
+      * A `topologySource` definition.
+
+
+
+For example, here is a simple definition of a wordcount topology using the YAML DSL:
+
+```yaml
+name: "yaml-topology"
+config:
+  topology.workers: 1
+
+# spout definitions
+spouts:
+  - id: "spout-1"
+    className: "org.apache.storm.testing.TestWordSpout"
+    parallelism: 1
+
+# bolt definitions
+bolts:
+  - id: "bolt-1"
+    className: "org.apache.storm.testing.TestWordCounter"
+    parallelism: 1
+  - id: "bolt-2"
+    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+    parallelism: 1
+
+#stream definitions
+streams:
+  - name: "spout-1 --> bolt-1" # name isn't used (placeholder for logging, UI, etc.)
+    from: "spout-1"
+    to: "bolt-1"
+    grouping:
+      type: FIELDS
+      args: ["word"]
+
+  - name: "bolt-1 --> bolt2"
+    from: "bolt-1"
+    to: "bolt-2"
+    grouping:
+      type: SHUFFLE
+
+
+```
+## Property Substitution/Filtering
+It's common for developers to want to easily switch between configurations, for example switching deployment between
+a development environment and a production environment. This can be accomplished by using separate YAML configuration
+files, but that approach would lead to unnecessary duplication, especially in situations where the Storm topology
+does not change, but configuration settings such as host names, ports, and parallelism paramters do.
+
+For this case, Flux offers properties filtering to allow you two externalize values to a `.properties` file and have
+them substituted before the `.yaml` file is parsed.
+
+To enable property filtering, use the `--filter` command line option and specify a `.properties` file. For example,
+if you invoked flux like so:
+
+```bash
+storm jar myTopology-0.1.0-SNAPSHOT.jar org.apache.storm.flux.Flux --local my_config.yaml --filter dev.properties
+```
+With the following `dev.properties` file:
+
+```properties
+kafka.zookeeper.hosts: localhost:2181
+```
+
+You would then be able to reference those properties by key in your `.yaml` file using `${}` syntax:
+
+```yaml
+  - id: "zkHosts"
+    className: "org.apache.storm.kafka.ZkHosts"
+    constructorArgs:
+      - "${kafka.zookeeper.hosts}"
+```
+
+In this case, Flux would replace `${kafka.zookeeper.hosts}` with `localhost:2181` before parsing the YAML contents.
+
+### Environment Variable Substitution/Filtering
+Flux also allows environment variable substitution. For example, if an environment variable named `ZK_HOSTS` if defined,
+you can reference it in a Flux YAML file with the following syntax:
+
+```
+${ENV-ZK_HOSTS}
+```
+
+## Components
+Components are essentially named object instances that are made available as configuration options for spouts and
+bolts. If you are familiar with the Spring framework, components are roughly analagous to Spring beans.
+
+Every component is identified, at a minimum, by a unique identifier (String) and a class name (String). For example,
+the following will make an instance of the `org.apache.storm.kafka.StringScheme` class available as a reference under the key
+`"stringScheme"` . This assumes the `org.apache.storm.kafka.StringScheme` has a default constructor.
+
+```yaml
+components:
+  - id: "stringScheme"
+    className: "org.apache.storm.kafka.StringScheme"
+```
+
+### Contructor Arguments, References, Properties and Configuration Methods
+
+####Constructor Arguments
+Arguments to a class constructor can be configured by adding a `contructorArgs` element to a components.
+`constructorArgs` is a list of objects that will be passed to the class' constructor. The following example creates an
+object by calling the constructor that takes a single string as an argument:
+
+```yaml
+  - id: "zkHosts"
+    className: "org.apache.storm.kafka.ZkHosts"
+    constructorArgs:
+      - "localhost:2181"
+      - true
+```
+
+####References
+Each component instance is identified by a unique id that allows it to be used/reused by other components. To
+reference an existing component, you specify the id of the component with the `ref` tag.
+
+In the following example, a component with the id `"stringScheme"` is created, and later referenced, as a an argument
+to another component's constructor:
+
+```yaml
+components:
+  - id: "stringScheme"
+    className: "org.apache.storm.kafka.StringScheme"
+
+  - id: "stringMultiScheme"
+    className: "org.apache.storm.spout.SchemeAsMultiScheme"
+    constructorArgs:
+      - ref: "stringScheme" # component with id "stringScheme" must be declared above.
+```
+
+You can also reference existing components in list via specifying the id of the components with the `reflist` tag.
+The type of the reflist will be `List<Object>`, but Flux can automatically convert List to Array (also varargs), 
+so you can use reflist on argument which type is `List<Type>`, or `Type[]`, or `Type...`.
+
+Please note that all components in the list must be same type.
+
+```yaml
+components:
+  - id: "boundCQLStatementMapperBuilder"
+    className: "org.apache.storm.cassandra.query.builder.BoundCQLStatementMapperBuilder"
+    constructorArgs:
+    - "INSERT INTO sink_cassandra (eventKey, driverId, truckId, driverName) VALUES (?, ?, ?, ?)"
+    configMethods:
+    - name: "bind"
+      args:
+      - reflist: ["FieldSelector-1", "FieldSelector-2", "FieldSelector-3", "FieldSelector-4"]
+```
+
+**N.B.:** References can only be used after (below) the object they point to has been declared.
+
+####Properties
+In addition to calling constructors with different arguments, Flux also allows you to configure components using
+JavaBean-like setter methods and fields declared as `public`:
+
+```yaml
+  - id: "spoutConfig"
+    className: "org.apache.storm.kafka.SpoutConfig"
+    constructorArgs:
+      # brokerHosts
+      - ref: "zkHosts"
+      # topic
+      - "myKafkaTopic"
+      # zkRoot
+      - "/kafkaSpout"
+      # id
+      - "myId"
+    properties:
+      - name: "ignoreZkOffsets"
+        value: true
+      - name: "scheme"
+        ref: "stringMultiScheme"
+```
+
+In the example above, the `properties` declaration will cause Flux to look for a public method in the `SpoutConfig` with
+the signature `setForceFromStart(boolean b)` and attempt to invoke it. If a setter method is not found, Flux will then
+look for a public instance variable with the name `ignoreZkOffsets` and attempt to set its value.
+
+References may also be used as property values.
+
+####Configuration Methods
+Conceptually, configuration methods are similar to Properties and Constructor Args -- they allow you to invoke an
+arbitrary method on an object after it is constructed. Configuration methods are useful for working with classes that
+don't expose JavaBean methods or have constructors that can fully configure the object. Common examples include classes
+that use the builder pattern for configuration/composition.
+
+The following YAML example creates a bolt and configures it by calling several methods:
+
+```yaml
+bolts:
+  - id: "bolt-1"
+    className: "org.apache.storm.flux.test.TestBolt"
+    parallelism: 1
+    configMethods:
+      - name: "withFoo"
+        args:
+          - "foo"
+      - name: "withNone"
+      # no args needed, so no "args" line
+      - name: "withBar"
+        args:
+          - "bar"
+      - name: "withFooBar"
+        args:
+          - "foo"
+          - "bar"
+```
+
+The signatures of the corresponding methods are as follows:
+
+```java
+    public void withFoo(String foo);
+    public void withNone(); // method with zero arguments
+    public void withBar(String bar);
+    public void withFooBar(String foo, String bar);
+```
+
+Arguments passed to configuration methods work much the same way as constructor arguments, and support references as
+well.
+
+### Using Java `enum`s in Contructor Arguments, References, Properties and Configuration Methods
+You can easily use Java `enum` values as arguments in a Flux YAML file, simply by referencing the name of the `enum`.
+
+For example, [Storm's HDFS module]() includes the following `enum` definition (simplified for brevity):
+
+```java
+public static enum Units {
+    KB, MB, GB, TB
+}
+```
+
+And the `org.apache.storm.hdfs.bolt.rotation.FileSizeRotationPolicy` class has the following constructor:
+
+```java
+public FileSizeRotationPolicy(float count, Units units)
+
+```
+The following Flux `component` definition could be used to call the constructor:
+
+```yaml
+  - id: "rotationPolicy"
+    className: "org.apache.storm.hdfs.bolt.rotation.FileSizeRotationPolicy"
+    constructorArgs:
+      - 5.0
+      - MB
+```
+
+The above definition is functionally equivalent to the following Java code:
+
+```java
+// rotate files when they reach 5MB
+FileRotationPolicy rotationPolicy = new FileSizeRotationPolicy(5.0f, Units.MB);
+```
+
+## Topology Config
+The `config` section is simply a map of Storm topology configuration parameters that will be passed to the
+`org.apache.storm.StormSubmitter` as an instance of the `org.apache.storm.Config` class:
+
+```yaml
+config:
+  topology.workers: 4
+  topology.max.spout.pending: 1000
+  topology.message.timeout.secs: 30
+```
+
+# Existing Topologies
+If you have existing Storm topologies, you can still use Flux to deploy/run/test them. This feature allows you to
+leverage Flux Constructor Arguments, References, Properties, and Topology Config declarations for existing topology
+classes.
+
+The easiest way to use an existing topology class is to define
+a `getTopology()` instance method with one of the following signatures:
+
+```java
+public StormTopology getTopology(Map<String, Object> config)
+```
+or:
+
+```java
+public StormTopology getTopology(Config config)
+```
+
+You could then use the following YAML to configure your topology:
+
+```yaml
+name: "existing-topology"
+topologySource:
+  className: "org.apache.storm.flux.test.SimpleTopology"
+```
+
+If the class you would like to use as a topology source has a different method name (i.e. not `getTopology`), you can
+override it:
+
+```yaml
+name: "existing-topology"
+topologySource:
+  className: "org.apache.storm.flux.test.SimpleTopology"
+  methodName: "getTopologyWithDifferentMethodName"
+```
+
+__N.B.:__ The specified method must accept a single argument of type `java.util.Map<String, Object>` or
+`org.apache.storm.Config`, and return a `org.apache.storm.generated.StormTopology` object.
+
+# YAML DSL
+## Spouts and Bolts
+Spout and Bolts are configured in their own respective section of the YAML configuration. Spout and Bolt definitions
+are extensions to the `component` definition that add a `parallelism` parameter that sets the parallelism  for a
+component when the topology is deployed.
+
+Because spout and bolt definitions extend `component` they support constructor arguments, references, and properties as
+well.
+
+Shell spout example:
+
+```yaml
+spouts:
+  - id: "sentence-spout"
+    className: "org.apache.storm.flux.wrappers.spouts.FluxShellSpout"
+    # shell spout constructor takes 2 arguments: String[], String[]
+    constructorArgs:
+      # command line
+      - ["node", "randomsentence.js"]
+      # output fields
+      - ["word"]
+    parallelism: 1
+```
+
+Kafka spout example:
+
+```yaml
+components:
+  - id: "stringScheme"
+    className: "org.apache.storm.kafka.StringScheme"
+
+  - id: "stringMultiScheme"
+    className: "org.apache.storm.spout.SchemeAsMultiScheme"
+    constructorArgs:
+      - ref: "stringScheme"
+
+  - id: "zkHosts"
+    className: "org.apache.storm.kafka.ZkHosts"
+    constructorArgs:
+      - "localhost:2181"
+
+# Alternative kafka config
+#  - id: "kafkaConfig"
+#    className: "org.apache.storm.kafka.KafkaConfig"
+#    constructorArgs:
+#      # brokerHosts
+#      - ref: "zkHosts"
+#      # topic
+#      - "myKafkaTopic"
+#      # clientId (optional)
+#      - "myKafkaClientId"
+
+  - id: "spoutConfig"
+    className: "org.apache.storm.kafka.SpoutConfig"
+    constructorArgs:
+      # brokerHosts
+      - ref: "zkHosts"
+      # topic
+      - "myKafkaTopic"
+      # zkRoot
+      - "/kafkaSpout"
+      # id
+      - "myId"
+    properties:
+      - name: "ignoreZkOffsets"
+        value: true
+      - name: "scheme"
+        ref: "stringMultiScheme"
+
+config:
+  topology.workers: 1
+
+# spout definitions
+spouts:
+  - id: "kafka-spout"
+    className: "org.apache.storm.kafka.KafkaSpout"
+    constructorArgs:
+      - ref: "spoutConfig"
+
+```
+
+Bolt Examples:
+
+```yaml
+# bolt definitions
+bolts:
+  - id: "splitsentence"
+    className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
+    constructorArgs:
+      # command line
+      - ["python", "splitsentence.py"]
+      # output fields
+      - ["word"]
+    parallelism: 1
+    # ...
+
+  - id: "log"
+    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+    parallelism: 1
+    # ...
+
+  - id: "count"
+    className: "org.apache.storm.testing.TestWordCounter"
+    parallelism: 1
+    # ...
+```
+## Streams and Stream Groupings
+Streams in Flux are represented as a list of connections (Graph edges, data flow, etc.) between the Spouts and Bolts in
+a topology, with an associated Grouping definition.
+
+A Stream definition has the following properties:
+
+**`name`:** A name for the connection (optional, currently unused)
+
+**`from`:** The `id` of a Spout or Bolt that is the source (publisher)
+
+**`to`:** The `id` of a Spout or Bolt that is the destination (subscriber)
+
+**`grouping`:** The stream grouping definition for the Stream
+
+A Grouping definition has the following properties:
+
+**`type`:** The type of grouping. One of `ALL`,`CUSTOM`,`DIRECT`,`SHUFFLE`,`LOCAL_OR_SHUFFLE`,`FIELDS`,`GLOBAL`, or `NONE`.
+
+**`streamId`:** The Storm stream ID (Optional. If unspecified will use the default stream)
+
+**`args`:** For the `FIELDS` grouping, a list of field names.
+
+**`customClass`** For the `CUSTOM` grouping, a definition of custom grouping class instance
+
+The `streams` definition example below sets up a topology with the following wiring:
+
+```
+    kafka-spout --> splitsentence --> count --> log
+```
+
+
+```yaml
+#stream definitions
+# stream definitions define connections between spouts and bolts.
+# note that such connections can be cyclical
+# custom stream groupings are also supported
+
+streams:
+  - name: "kafka --> split" # name isn't used (placeholder for logging, UI, etc.)
+    from: "kafka-spout"
+    to: "splitsentence"
+    grouping:
+      type: SHUFFLE
+
+  - name: "split --> count"
+    from: "splitsentence"
+    to: "count"
+    grouping:
+      type: FIELDS
+      args: ["word"]
+
+  - name: "count --> log"
+    from: "count"
+    to: "log"
+    grouping:
+      type: SHUFFLE
+```
+
+### Custom Stream Groupings
+Custom stream groupings are defined by setting the grouping type to `CUSTOM` and defining a `customClass` parameter
+that tells Flux how to instantiate the custom class. The `customClass` definition extends `component`, so it supports
+constructor arguments, references, and properties as well.
+
+The example below creates a Stream with an instance of the `org.apache.storm.testing.NGrouping` custom stream grouping
+class.
+
+```yaml
+  - name: "bolt-1 --> bolt2"
+    from: "bolt-1"
+    to: "bolt-2"
+    grouping:
+      type: CUSTOM
+      customClass:
+        className: "org.apache.storm.testing.NGrouping"
+        constructorArgs:
+          - 1
+```
+
+## Includes and Overrides
+Flux allows you to include the contents of other YAML files, and have them treated as though they were defined in the
+same file. Includes may be either files, or classpath resources.
+
+Includes are specified as a list of maps:
+
+```yaml
+includes:
+  - resource: false
+    file: "src/test/resources/configs/shell_test.yaml"
+    override: false
+```
+
+If the `resource` property is set to `true`, the include will be loaded as a classpath resource from the value of the
+`file` attribute, otherwise it will be treated as a regular file.
+
+The `override` property controls how includes affect the values defined in the current file. If `override` is set to
+`true`, values in the included file will replace values in the current file being parsed. If `override` is set to
+`false`, values in the current file being parsed will take precedence, and the parser will refuse to replace them.
+
+**N.B.:** Includes are not yet recursive. Includes from included files will be ignored.
+
+
+## Basic Word Count Example
+
+This example uses a spout implemented in JavaScript, a bolt implemented in Python, and a bolt implemented in Java
+
+Topology YAML config:
+
+```yaml
+---
+name: "shell-topology"
+config:
+  topology.workers: 1
+
+# spout definitions
+spouts:
+  - id: "sentence-spout"
+    className: "org.apache.storm.flux.wrappers.spouts.FluxShellSpout"
+    # shell spout constructor takes 2 arguments: String[], String[]
+    constructorArgs:
+      # command line
+      - ["node", "randomsentence.js"]
+      # output fields
+      - ["word"]
+    parallelism: 1
+
+# bolt definitions
+bolts:
+  - id: "splitsentence"
+    className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
+    constructorArgs:
+      # command line
+      - ["python", "splitsentence.py"]
+      # output fields
+      - ["word"]
+    parallelism: 1
+
+  - id: "log"
+    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+    parallelism: 1
+
+  - id: "count"
+    className: "org.apache.storm.testing.TestWordCounter"
+    parallelism: 1
+
+#stream definitions
+# stream definitions define connections between spouts and bolts.
+# note that such connections can be cyclical
+# custom stream groupings are also supported
+
+streams:
+  - name: "spout --> split" # name isn't used (placeholder for logging, UI, etc.)
+    from: "sentence-spout"
+    to: "splitsentence"
+    grouping:
+      type: SHUFFLE
+
+  - name: "split --> count"
+    from: "splitsentence"
+    to: "count"
+    grouping:
+      type: FIELDS
+      args: ["word"]
+
+  - name: "count --> log"
+    from: "count"
+    to: "log"
+    grouping:
+      type: SHUFFLE
+```
+
+
+## Micro-Batching (Trident) API Support
+Currenty, the Flux YAML DSL only supports the Core Storm API, but support for Storm's micro-batching API is planned.
+
+To use Flux with a Trident topology, define a topology getter method and reference it in your YAML config:
+
+```yaml
+name: "my-trident-topology"
+
+config:
+  topology.workers: 1
+
+topologySource:
+  className: "org.apache.storm.flux.test.TridentTopologySource"
+  # Flux will look for "getTopology", this will override that.
+  methodName: "getTopologyWithDifferentMethodName"
+```
+
+## Committer Sponsors
+
+ * P. Taylor Goetz ([ptgoetz@apache.org](mailto:ptgoetz@apache.org))

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/pom.xml
----------------------------------------------------------------------
diff --git a/flux/flux-core/pom.xml b/flux/flux-core/pom.xml
new file mode 100644
index 0000000..1cf4a9b
--- /dev/null
+++ b/flux/flux-core/pom.xml
@@ -0,0 +1,112 @@
+<?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>
+        <groupId>org.apache.storm</groupId>
+        <artifactId>flux</artifactId>
+        <version>2.0.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+
+    <artifactId>flux-core</artifactId>
+    <packaging>jar</packaging>
+
+    <name>flux-core</name>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>flux-wrappers</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-kafka</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-hdfs</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-hbase</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.yaml</groupId>
+            <artifactId>snakeyaml</artifactId>
+        </dependency>
+    </dependencies>
+    <build>
+        <resources>
+            <resource>
+                <directory>src/main/resources</directory>
+                <filtering>true</filtering>
+            </resource>
+        </resources>
+        <plugins>
+        <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-shade-plugin</artifactId>
+            <configuration>
+                <createDependencyReducedPom>true</createDependencyReducedPom>
+                <filters>
+                    <filter>
+                        <artifact>*:*</artifact>
+                        <excludes>
+                            <exclude>META-INF/*.SF</exclude>
+                            <exclude>META-INF/*.sf</exclude>
+                            <exclude>META-INF/*.DSA</exclude>
+                            <exclude>META-INF/*.dsa</exclude>
+                            <exclude>META-INF/*.RSA</exclude>
+                            <exclude>META-INF/*.rsa</exclude>
+                            <exclude>META-INF/*.EC</exclude>
+                            <exclude>META-INF/*.ec</exclude>
+                            <exclude>META-INF/MSFTSIG.SF</exclude>
+                            <exclude>META-INF/MSFTSIG.RSA</exclude>
+                        </excludes>
+                    </filter>
+                </filters>
+            </configuration>
+            <executions>
+                <execution>
+                    <phase>package</phase>
+                    <goals>
+                        <goal>shade</goal>
+                    </goals>
+                    <configuration>
+                        <transformers>
+                            <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
+                            <transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
+                                <mainClass>org.apache.storm.flux.Flux</mainClass>
+                            </transformer>
+                        </transformers>
+                    </configuration>
+                </execution>
+            </executions>
+        </plugin>
+        </plugins>
+    </build>
+</project>

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/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
new file mode 100644
index 0000000..222bf2d
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/Flux.java
@@ -0,0 +1,259 @@
+/*
+ * 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.flux;
+
+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.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.
+ *
+ */
+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";
+    private static final String OPTION_SLEEP = "sleep";
+    private static final String OPTION_DRY_RUN = "dry-run";
+    private static final String OPTION_NO_DETAIL = "no-detail";
+    private static final String OPTION_NO_SPLASH = "no-splash";
+    private static final String OPTION_INACTIVE = "inactive";
+    private static final String OPTION_ZOOKEEPER = "zookeeper";
+    private static final String OPTION_FILTER = "filter";
+    private static final String OPTION_ENV_FILTER = "env-filter";
+
+    public static void main(String[] args) throws Exception {
+        Options options = new Options();
+
+        options.addOption(option(0, "l", OPTION_LOCAL, "Run the topology in local mode."));
+
+        options.addOption(option(0, "r", OPTION_REMOTE, "Deploy the topology to a remote cluster."));
+
+        options.addOption(option(0, "R", OPTION_RESOURCE, "Treat the supplied path as a classpath resource instead of a file."));
+
+        options.addOption(option(1, "s", OPTION_SLEEP, "ms", "When running locally, the amount of time to sleep (in ms.) " +
+                "before killing the topology and shutting down the local cluster."));
+
+        options.addOption(option(0, "d", OPTION_DRY_RUN, "Do not run or deploy the topology. Just build, validate, " +
+                "and print information about the topology."));
+
+        options.addOption(option(0, "q", OPTION_NO_DETAIL, "Suppress the printing of topology details."));
+
+        options.addOption(option(0, "n", OPTION_NO_SPLASH, "Suppress the printing of the splash screen."));
+
+        options.addOption(option(0, "i", OPTION_INACTIVE, "Deploy the topology, but do not activate it."));
+
+        options.addOption(option(1, "z", OPTION_ZOOKEEPER, "host:port", "When running in local mode, use the ZooKeeper at the " +
+                "specified <host>:<port> instead of the in-process ZooKeeper. (requires Storm 0.9.3 or later)"));
+
+        options.addOption(option(1, "f", OPTION_FILTER, "file", "Perform property substitution. Use the specified file " +
+                "as a source of properties, and replace keys identified with {$[property name]} with the value defined " +
+                "in the properties file."));
+
+        options.addOption(option(0, "e", OPTION_ENV_FILTER, "Perform environment variable substitution. Replace keys" +
+                "identified with `${ENV-[NAME]}` will be replaced with the corresponding `NAME` environment value"));
+
+        CommandLineParser parser = new BasicParser();
+        CommandLine cmd = parser.parse(options, args);
+
+        if (cmd.getArgs().length != 1) {
+            usage(options);
+            System.exit(1);
+        }
+        runCli(cmd);
+    }
+
+    private static Option option(int argCount, String shortName, String longName, String description){
+       return option(argCount, shortName, longName, longName, description);
+    }
+
+    private static Option option(int argCount, String shortName, String longName, String argName, String description){
+        Option option = OptionBuilder.hasArgs(argCount)
+                .withArgName(argName)
+                .withLongOpt(longName)
+                .withDescription(description)
+                .create(shortName);
+        return option;
+    }
+
+    private static void usage(Options options) {
+        HelpFormatter formatter = new HelpFormatter();
+        formatter.printHelp("storm jar <my_topology_uber_jar.jar> " +
+                Flux.class.getName() +
+                " [options] <topology-config.yaml>", options);
+    }
+
+    private static void runCli(CommandLine cmd)throws Exception {
+        if(!cmd.hasOption(OPTION_NO_SPLASH)) {
+            printSplash();
+        }
+
+        boolean dumpYaml = cmd.hasOption("dump-yaml");
+
+        TopologyDef topologyDef = null;
+        String filePath = (String)cmd.getArgList().get(0);
+
+        // TODO conditionally load properties from a file our resource
+        String filterProps = null;
+        if(cmd.hasOption(OPTION_FILTER)){
+            filterProps = cmd.getOptionValue(OPTION_FILTER);
+        }
+
+
+        boolean envFilter = cmd.hasOption(OPTION_ENV_FILTER);
+        if(cmd.hasOption(OPTION_RESOURCE)){
+            printf("Parsing classpath resource: %s", filePath);
+            topologyDef = FluxParser.parseResource(filePath, dumpYaml, true, filterProps, envFilter);
+        } else {
+            printf("Parsing file: %s",
+                    new File(filePath).getAbsolutePath());
+            topologyDef = FluxParser.parseFile(filePath, dumpYaml, true, filterProps, envFilter);
+        }
+
+
+        String topologyName = topologyDef.getName();
+        // merge contents of `config` into topology config
+        Config conf = FluxBuilder.buildConfig(topologyDef);
+        ExecutionContext context = new ExecutionContext(topologyDef, conf);
+        StormTopology topology = FluxBuilder.buildTopology(context);
+
+        if(!cmd.hasOption(OPTION_NO_DETAIL)){
+            printTopologyInfo(context);
+        }
+
+        if(!cmd.hasOption(OPTION_DRY_RUN)) {
+            if (cmd.hasOption(OPTION_REMOTE)) {
+                LOG.info("Running remotely...");
+                // should the topology be active or inactive
+                SubmitOptions submitOptions = null;
+                if(cmd.hasOption(OPTION_INACTIVE)){
+                    LOG.info("Deploying topology in an INACTIVE state...");
+                    submitOptions = new SubmitOptions(TopologyInitialStatus.INACTIVE);
+                } else {
+                    LOG.info("Deploying topology in an ACTIVE state...");
+                    submitOptions = new SubmitOptions(TopologyInitialStatus.ACTIVE);
+                }
+                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();
+                }
+            }
+        }
+    }
+
+    static void printTopologyInfo(ExecutionContext ctx){
+        TopologyDef t = ctx.getTopologyDef();
+        if(t.isDslTopology()) {
+            print("---------- TOPOLOGY DETAILS ----------");
+
+            printf("Topology Name: %s", t.getName());
+            print("--------------- SPOUTS ---------------");
+            for (SpoutDef s : t.getSpouts()) {
+                printf("%s [%d] (%s)", s.getId(), s.getParallelism(), s.getClassName());
+            }
+            print("---------------- BOLTS ---------------");
+            for (BoltDef b : t.getBolts()) {
+                printf("%s [%d] (%s)", b.getId(), b.getParallelism(), b.getClassName());
+            }
+
+            print("--------------- STREAMS ---------------");
+            for (StreamDef sd : t.getStreams()) {
+                printf("%s --%s--> %s", sd.getFrom(), sd.getGrouping().getType(), sd.getTo());
+            }
+            print("--------------------------------------");
+        }
+    }
+
+    // save a little typing
+    private static void printf(String format, Object... args){
+        print(String.format(format, args));
+    }
+
+    private static void print(String string){
+        System.out.println(string);
+    }
+
+    private static void printSplash() throws IOException {
+        // banner
+        InputStream is = Flux.class.getResourceAsStream("/splash.txt");
+        if(is != null){
+            InputStreamReader isr = new InputStreamReader(is, "UTF-8");
+            BufferedReader br = new BufferedReader(isr);
+            String line = null;
+            while((line = br.readLine()) != null){
+                System.out.println(line);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/FluxBuilder.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/FluxBuilder.java b/flux/flux-core/src/main/java/org/apache/storm/flux/FluxBuilder.java
new file mode 100644
index 0000000..e79dfb7
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/FluxBuilder.java
@@ -0,0 +1,630 @@
+/*
+ * 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.flux;
+
+import org.apache.storm.Config;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.grouping.CustomStreamGrouping;
+import org.apache.storm.topology.*;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.flux.model.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.*;
+import java.util.*;
+
+public class FluxBuilder {
+    private static Logger LOG = LoggerFactory.getLogger(FluxBuilder.class);
+
+    /**
+     * Given a topology definition, return a populated `org.apache.storm.Config` instance.
+     *
+     * @param topologyDef
+     * @return
+     */
+    public static Config buildConfig(TopologyDef topologyDef) {
+        // merge contents of `config` into topology config
+        Config conf = new Config();
+        conf.putAll(topologyDef.getConfig());
+        return conf;
+    }
+
+    /**
+     * Given a topology definition, return a Storm topology that can be run either locally or remotely.
+     *
+     * @param context
+     * @return
+     * @throws IllegalAccessException
+     * @throws InstantiationException
+     * @throws ClassNotFoundException
+     * @throws NoSuchMethodException
+     * @throws InvocationTargetException
+     */
+    public static StormTopology buildTopology(ExecutionContext context) throws IllegalAccessException,
+            InstantiationException, ClassNotFoundException, NoSuchMethodException, InvocationTargetException, NoSuchFieldException {
+
+        StormTopology topology = null;
+        TopologyDef topologyDef = context.getTopologyDef();
+
+        if(!topologyDef.validate()){
+            throw new IllegalArgumentException("Invalid topology config. Spouts, bolts and streams cannot be " +
+                    "defined in the same configuration as a topologySource.");
+        }
+
+        // build components that may be referenced by spouts, bolts, etc.
+        // the map will be a String --> Object where the object is a fully
+        // constructed class instance
+        buildComponents(context);
+
+        if(topologyDef.isDslTopology()) {
+            // This is a DSL (YAML, etc.) topology...
+            LOG.info("Detected DSL topology...");
+
+            TopologyBuilder builder = new TopologyBuilder();
+
+            // create spouts
+            buildSpouts(context, builder);
+
+            // we need to be able to lookup bolts by id, then switch based
+            // on whether they are IBasicBolt or IRichBolt instances
+            buildBolts(context);
+
+            // process stream definitions
+            buildStreamDefinitions(context, builder);
+
+            topology = builder.createTopology();
+        } else {
+            // user class supplied...
+            // this also provides a bridge to Trident...
+            LOG.info("A topology source has been specified...");
+            ObjectDef def = topologyDef.getTopologySource();
+            topology = buildExternalTopology(def, context);
+        }
+        return topology;
+    }
+
+    /**
+     * Given a `java.lang.Object` instance and a method name, attempt to find a method that matches the input
+     * parameter: `java.util.Map` or `org.apache.storm.Config`.
+     *
+     * @param topologySource object to inspect for the specified method
+     * @param methodName name of the method to look for
+     * @return
+     * @throws NoSuchMethodException
+     */
+    private static Method findGetTopologyMethod(Object topologySource, String methodName) throws NoSuchMethodException {
+        Class clazz = topologySource.getClass();
+        Method[] methods =  clazz.getMethods();
+        ArrayList<Method> candidates = new ArrayList<Method>();
+        for(Method method : methods){
+            if(!method.getName().equals(methodName)){
+                continue;
+            }
+            if(!method.getReturnType().equals(StormTopology.class)){
+                continue;
+            }
+            Class[] paramTypes = method.getParameterTypes();
+            if(paramTypes.length != 1){
+                continue;
+            }
+            if(paramTypes[0].isAssignableFrom(Map.class) || paramTypes[0].isAssignableFrom(Config.class)){
+                candidates.add(method);
+            }
+        }
+
+        if(candidates.size() == 0){
+            throw new IllegalArgumentException("Unable to find method '" + methodName + "' method in class: " + clazz.getName());
+        } else if (candidates.size() > 1){
+            LOG.warn("Found multiple candidate methods in class '" + clazz.getName() + "'. Using the first one found");
+        }
+
+        return candidates.get(0);
+    }
+
+    /**
+     * @param context
+     * @param builder
+     */
+    private static void buildStreamDefinitions(ExecutionContext context, TopologyBuilder builder)
+            throws ClassNotFoundException, NoSuchMethodException, InvocationTargetException, InstantiationException,
+            IllegalAccessException, NoSuchFieldException {
+        TopologyDef topologyDef = context.getTopologyDef();
+        // process stream definitions
+        HashMap<String, BoltDeclarer> declarers = new HashMap<String, BoltDeclarer>();
+        for (StreamDef stream : topologyDef.getStreams()) {
+            Object boltObj = context.getBolt(stream.getTo());
+            BoltDeclarer declarer = declarers.get(stream.getTo());
+            if (boltObj instanceof IRichBolt) {
+                if(declarer == null) {
+                    declarer = builder.setBolt(stream.getTo(),
+                            (IRichBolt) boltObj,
+                            topologyDef.parallelismForBolt(stream.getTo()));
+                    declarers.put(stream.getTo(), declarer);
+                }
+            } else if (boltObj instanceof IBasicBolt) {
+                if(declarer == null) {
+                    declarer = builder.setBolt(
+                            stream.getTo(),
+                            (IBasicBolt) boltObj,
+                            topologyDef.parallelismForBolt(stream.getTo()));
+                    declarers.put(stream.getTo(), declarer);
+                }
+            } else if (boltObj instanceof IWindowedBolt) {
+                if(declarer == null) {
+                    declarer = builder.setBolt(
+                            stream.getTo(),
+                            (IWindowedBolt) boltObj,
+                            topologyDef.parallelismForBolt(stream.getTo()));
+                    declarers.put(stream.getTo(), declarer);
+                }
+            } else if (boltObj instanceof IStatefulBolt) {
+                if(declarer == null) {
+                    declarer = builder.setBolt(
+                            stream.getTo(),
+                            (IStatefulBolt) boltObj,
+                            topologyDef.parallelismForBolt(stream.getTo()));
+                    declarers.put(stream.getTo(), declarer);
+                }
+            } else {
+                throw new IllegalArgumentException("Class does not appear to be a bolt: " +
+                        boltObj.getClass().getName());
+            }
+
+            GroupingDef grouping = stream.getGrouping();
+            // if the streamId is defined, use it for the grouping, otherwise assume storm's default stream
+            String streamId = (grouping.getStreamId() == null ? Utils.DEFAULT_STREAM_ID : grouping.getStreamId());
+
+
+            switch (grouping.getType()) {
+                case SHUFFLE:
+                    declarer.shuffleGrouping(stream.getFrom(), streamId);
+                    break;
+                case FIELDS:
+                    //TODO check for null grouping args
+                    declarer.fieldsGrouping(stream.getFrom(), streamId, new Fields(grouping.getArgs()));
+                    break;
+                case ALL:
+                    declarer.allGrouping(stream.getFrom(), streamId);
+                    break;
+                case DIRECT:
+                    declarer.directGrouping(stream.getFrom(), streamId);
+                    break;
+                case GLOBAL:
+                    declarer.globalGrouping(stream.getFrom(), streamId);
+                    break;
+                case LOCAL_OR_SHUFFLE:
+                    declarer.localOrShuffleGrouping(stream.getFrom(), streamId);
+                    break;
+                case NONE:
+                    declarer.noneGrouping(stream.getFrom(), streamId);
+                    break;
+                case CUSTOM:
+                    declarer.customGrouping(stream.getFrom(), streamId,
+                            buildCustomStreamGrouping(stream.getGrouping().getCustomClass(), context));
+                    break;
+                default:
+                    throw new UnsupportedOperationException("unsupported grouping type: " + grouping);
+            }
+        }
+    }
+
+    private static void applyProperties(ObjectDef bean, Object instance, ExecutionContext context) throws
+            IllegalAccessException, InvocationTargetException, NoSuchFieldException {
+        List<PropertyDef> props = bean.getProperties();
+        Class clazz = instance.getClass();
+        if (props != null) {
+            for (PropertyDef prop : props) {
+                Object value = prop.isReference() ? context.getComponent(prop.getRef()) : prop.getValue();
+                Method setter = findSetter(clazz, prop.getName(), value);
+                if (setter != null) {
+                    LOG.debug("found setter, attempting to invoke");
+                    // invoke setter
+                    setter.invoke(instance, new Object[]{value});
+                } else {
+                    // look for a public instance variable
+                    LOG.debug("no setter found. Looking for a public instance variable...");
+                    Field field = findPublicField(clazz, prop.getName(), value);
+                    if (field != null) {
+                        field.set(instance, value);
+                    }
+                }
+            }
+        }
+    }
+
+    private static Field findPublicField(Class clazz, String property, Object arg) throws NoSuchFieldException {
+        Field field = clazz.getField(property);
+        return field;
+    }
+
+    private static Method findSetter(Class clazz, String property, Object arg) {
+        String setterName = toSetterName(property);
+        Method retval = null;
+        Method[] methods = clazz.getMethods();
+        for (Method method : methods) {
+            if (setterName.equals(method.getName())) {
+                LOG.debug("Found setter method: " + method.getName());
+                retval = method;
+            }
+        }
+        return retval;
+    }
+
+    private static String toSetterName(String name) {
+        return "set" + name.substring(0, 1).toUpperCase() + name.substring(1, name.length());
+    }
+
+    private static List<Object> resolveReferences(List<Object> args, ExecutionContext context) {
+        LOG.debug("Checking arguments for references.");
+        List<Object> cArgs = new ArrayList<Object>();
+        // resolve references
+        for (Object arg : args) {
+            if (arg instanceof BeanReference) {
+                cArgs.add(context.getComponent(((BeanReference) arg).getId()));
+            } else if (arg instanceof BeanListReference) {
+                List<Object> components = new ArrayList<>();
+                BeanListReference ref = (BeanListReference) arg;
+                for (String id : ref.getIds()) {
+                    components.add(context.getComponent(id));
+                }
+
+                LOG.debug("BeanListReference resolved as {}", components);
+                cArgs.add(components);
+            } else {
+                cArgs.add(arg);
+            }
+        }
+        return cArgs;
+    }
+
+    private static Object buildObject(ObjectDef def, ExecutionContext context) throws ClassNotFoundException,
+            IllegalAccessException, InstantiationException, NoSuchMethodException, InvocationTargetException, NoSuchFieldException {
+        Class clazz = Class.forName(def.getClassName());
+        Object obj = null;
+        if (def.hasConstructorArgs()) {
+            LOG.debug("Found constructor arguments in definition: " + def.getConstructorArgs().getClass().getName());
+            List<Object> cArgs = def.getConstructorArgs();
+            if(def.hasReferences()){
+                cArgs = resolveReferences(cArgs, context);
+            }
+            Constructor con = findCompatibleConstructor(cArgs, clazz);
+            if (con != null) {
+                LOG.debug("Found something seemingly compatible, attempting invocation...");
+                obj = con.newInstance(getArgsWithListCoercian(cArgs, con.getParameterTypes()));
+            } else {
+                String msg = String.format("Couldn't find a suitable constructor for class '%s' with arguments '%s'.",
+                        clazz.getName(),
+                        cArgs);
+                throw new IllegalArgumentException(msg);
+            }
+        } else {
+            obj = clazz.newInstance();
+        }
+        applyProperties(def, obj, context);
+        invokeConfigMethods(def, obj, context);
+        return obj;
+    }
+
+    private static StormTopology buildExternalTopology(ObjectDef def, ExecutionContext context)
+            throws ClassNotFoundException, IllegalAccessException, InstantiationException, NoSuchMethodException,
+            InvocationTargetException, NoSuchFieldException {
+
+        Object topologySource = buildObject(def, context);
+
+        String methodName = context.getTopologyDef().getTopologySource().getMethodName();
+        Method getTopology = findGetTopologyMethod(topologySource, methodName);
+        if(getTopology.getParameterTypes()[0].equals(Config.class)){
+            Config config = new Config();
+            config.putAll(context.getTopologyDef().getConfig());
+            return (StormTopology) getTopology.invoke(topologySource, config);
+        } else {
+            return (StormTopology) getTopology.invoke(topologySource, context.getTopologyDef().getConfig());
+        }
+    }
+
+    private static CustomStreamGrouping buildCustomStreamGrouping(ObjectDef def, ExecutionContext context)
+            throws ClassNotFoundException,
+            IllegalAccessException, InstantiationException, NoSuchMethodException, InvocationTargetException, NoSuchFieldException {
+        Object grouping = buildObject(def, context);
+        return (CustomStreamGrouping)grouping;
+    }
+
+    /**
+     * Given a topology definition, resolve and instantiate all components found and return a map
+     * keyed by the component id.
+     */
+    private static void buildComponents(ExecutionContext context) throws ClassNotFoundException, NoSuchMethodException,
+            IllegalAccessException, InvocationTargetException, InstantiationException, NoSuchFieldException {
+        Collection<BeanDef> cDefs = context.getTopologyDef().getComponents();
+        if (cDefs != null) {
+            for (BeanDef bean : cDefs) {
+                Object obj = buildObject(bean, context);
+                context.addComponent(bean.getId(), obj);
+            }
+        }
+    }
+
+
+    private static void buildSpouts(ExecutionContext context, TopologyBuilder builder) throws ClassNotFoundException,
+            NoSuchMethodException, InvocationTargetException, InstantiationException, IllegalAccessException, NoSuchFieldException {
+        for (SpoutDef sd : context.getTopologyDef().getSpouts()) {
+            IRichSpout spout = buildSpout(sd, context);
+            builder.setSpout(sd.getId(), spout, sd.getParallelism());
+            context.addSpout(sd.getId(), spout);
+        }
+    }
+
+    /**
+     * Given a spout definition, return a Storm spout implementation by attempting to find a matching constructor
+     * in the given spout class. Perform list to array conversion as necessary.
+     */
+    private static IRichSpout buildSpout(SpoutDef def, ExecutionContext context) throws ClassNotFoundException,
+            IllegalAccessException, InstantiationException, NoSuchMethodException, InvocationTargetException, NoSuchFieldException {
+        return (IRichSpout)buildObject(def, context);
+    }
+
+    /**
+     * Given a list of bolt definitions, build a map of Storm bolts with the bolt definition id as the key.
+     * Attempt to coerce the given constructor arguments to a matching bolt constructor as much as possible.
+     */
+    private static void buildBolts(ExecutionContext context) throws ClassNotFoundException, IllegalAccessException,
+            InstantiationException, NoSuchMethodException, InvocationTargetException, NoSuchFieldException {
+        for (BoltDef def : context.getTopologyDef().getBolts()) {
+            Class clazz = Class.forName(def.getClassName());
+            Object bolt = buildObject(def, context);
+            context.addBolt(def.getId(), bolt);
+        }
+    }
+
+    /**
+     * Given a list of constructor arguments, and a target class, attempt to find a suitable constructor.
+     *
+     */
+    private static Constructor findCompatibleConstructor(List<Object> args, Class target) throws NoSuchMethodException {
+        Constructor retval = null;
+        int eligibleCount = 0;
+
+        LOG.debug("Target class: {}, constructor args: {}", target.getName(), args);
+        Constructor[] cons = target.getDeclaredConstructors();
+
+        for (Constructor con : cons) {
+            Class[] paramClasses = con.getParameterTypes();
+            if (paramClasses.length == args.size()) {
+                LOG.debug("found constructor with same number of args..");
+                boolean invokable = canInvokeWithArgs(args, con.getParameterTypes());
+                if (invokable) {
+                    retval = con;
+                    eligibleCount++;
+                }
+                LOG.debug("** invokable --> {}", invokable);
+            } else {
+                LOG.debug("Skipping constructor with wrong number of arguments.");
+            }
+        }
+        if (eligibleCount > 1) {
+            LOG.warn("Found multiple invokable constructors for class {}, given arguments {}. Using the last one found.",
+                    target, args);
+        }
+        return retval;
+    }
+
+
+    public static void invokeConfigMethods(ObjectDef bean, Object instance, ExecutionContext context)
+            throws InvocationTargetException, IllegalAccessException {
+
+        List<ConfigMethodDef> methodDefs = bean.getConfigMethods();
+        if(methodDefs == null || methodDefs.size() == 0){
+            return;
+        }
+        Class clazz = instance.getClass();
+        for(ConfigMethodDef methodDef : methodDefs){
+            List<Object> args = methodDef.getArgs();
+            if (args == null){
+                args = new ArrayList();
+            }
+            if(methodDef.hasReferences()){
+                args = resolveReferences(args, context);
+            }
+            String methodName = methodDef.getName();
+            Method method = findCompatibleMethod(args, clazz, methodName);
+            if(method != null) {
+                Object[] methodArgs = getArgsWithListCoercian(args, method.getParameterTypes());
+                method.invoke(instance, methodArgs);
+            } else {
+                String msg = String.format("Unable to find configuration method '%s' in class '%s' with arguments %s.",
+                        new Object[]{methodName, clazz.getName(), args});
+                throw new IllegalArgumentException(msg);
+            }
+        }
+    }
+
+    private static Method findCompatibleMethod(List<Object> args, Class target, String methodName){
+        Method retval = null;
+        int eligibleCount = 0;
+
+        LOG.debug("Target class: {}, methodName: {}, args: {}", target.getName(), methodName, args);
+        Method[] methods = target.getMethods();
+
+        for (Method method : methods) {
+            Class[] paramClasses = method.getParameterTypes();
+            if (paramClasses.length == args.size() && method.getName().equals(methodName)) {
+                LOG.debug("found constructor with same number of args..");
+                boolean invokable = false;
+                if (args.size() == 0){
+                    // it's a method with zero args
+                    invokable = true;
+                } else {
+                    invokable = canInvokeWithArgs(args, method.getParameterTypes());
+                }
+                if (invokable) {
+                    retval = method;
+                    eligibleCount++;
+                }
+                LOG.debug("** invokable --> {}", invokable);
+            } else {
+                LOG.debug("Skipping method with wrong number of arguments.");
+            }
+        }
+        if (eligibleCount > 1) {
+            LOG.warn("Found multiple invokable methods for class {}, method {}, given arguments {}. " +
+                            "Using the last one found.",
+                            new Object[]{target, methodName, args});
+        }
+        return retval;
+    }
+
+    /**
+     * Given a java.util.List of contructor/method arguments, and a list of parameter types, attempt to convert the
+     * list to an java.lang.Object array that can be used to invoke the constructor. If an argument needs
+     * to be coerced from a List to an Array, do so.
+     */
+    private static Object[] getArgsWithListCoercian(List<Object> args, Class[] parameterTypes) {
+//        Class[] parameterTypes = constructor.getParameterTypes();
+        if (parameterTypes.length != args.size()) {
+            throw new IllegalArgumentException("Contructor parameter count does not egual argument size.");
+        }
+        Object[] constructorParams = new Object[args.size()];
+
+        // loop through the arguments, if we hit a list that has to be convered to an array,
+        // perform the conversion
+        for (int i = 0; i < args.size(); i++) {
+            Object obj = args.get(i);
+            Class paramType = parameterTypes[i];
+            Class objectType = obj.getClass();
+            LOG.debug("Comparing parameter class {} to object class {} to see if assignment is possible.",
+                    paramType, objectType);
+            if (paramType.equals(objectType)) {
+                LOG.debug("They are the same class.");
+                constructorParams[i] = args.get(i);
+                continue;
+            }
+            if (paramType.isAssignableFrom(objectType)) {
+                LOG.debug("Assignment is possible.");
+                constructorParams[i] = args.get(i);
+                continue;
+            }
+            if (isPrimitiveBoolean(paramType) && Boolean.class.isAssignableFrom(objectType)){
+                LOG.debug("Its a primitive boolean.");
+                Boolean bool = (Boolean)args.get(i);
+                constructorParams[i] = bool.booleanValue();
+                continue;
+            }
+            if(isPrimitiveNumber(paramType) && Number.class.isAssignableFrom(objectType)){
+                LOG.debug("Its a primitive number.");
+                Number num = (Number)args.get(i);
+                if(paramType == Float.TYPE){
+                    constructorParams[i] = num.floatValue();
+                } else if (paramType == Double.TYPE) {
+                    constructorParams[i] = num.doubleValue();
+                } else if (paramType == Long.TYPE) {
+                    constructorParams[i] = num.longValue();
+                } else if (paramType == Integer.TYPE) {
+                    constructorParams[i] = num.intValue();
+                } else if (paramType == Short.TYPE) {
+                    constructorParams[i] = num.shortValue();
+                } else if (paramType == Byte.TYPE) {
+                    constructorParams[i] = num.byteValue();
+                } else {
+                    constructorParams[i] = args.get(i);
+                }
+                continue;
+            }
+
+            // enum conversion
+            if(paramType.isEnum() && objectType.equals(String.class)){
+                LOG.debug("Yes, will convert a String to enum");
+                constructorParams[i] = Enum.valueOf(paramType, (String)args.get(i));
+                continue;
+            }
+
+            // List to array conversion
+            if (paramType.isArray() && List.class.isAssignableFrom(objectType)) {
+                // TODO more collection content type checking
+                LOG.debug("Conversion appears possible...");
+                List list = (List) obj;
+                LOG.debug("Array Type: {}, List type: {}", paramType.getComponentType(), list.get(0).getClass());
+
+                // create an array of the right type
+                Object newArrayObj = Array.newInstance(paramType.getComponentType(), list.size());
+                for (int j = 0; j < list.size(); j++) {
+                    Array.set(newArrayObj, j, list.get(j));
+
+                }
+                constructorParams[i] = newArrayObj;
+                LOG.debug("After conversion: {}", constructorParams[i]);
+            }
+        }
+        return constructorParams;
+    }
+
+
+    /**
+     * Determine if the given constructor/method parameter types are compatible given arguments List. Consider if
+     * list coercian can make it possible.
+     *
+     * @param args
+     * @param parameterTypes
+     * @return
+     */
+    private static boolean canInvokeWithArgs(List<Object> args, Class[] parameterTypes) {
+        if (parameterTypes.length != args.size()) {
+            LOG.warn("parameter types were the wrong size");
+            return false;
+        }
+
+        for (int i = 0; i < args.size(); i++) {
+            Object obj = args.get(i);
+            if (obj == null) {
+                throw new IllegalArgumentException("argument shouldn't be null - index: " + i);
+            }
+            Class paramType = parameterTypes[i];
+            Class objectType = obj.getClass();
+            LOG.debug("Comparing parameter class {} to object class {} to see if assignment is possible.",
+                    paramType, objectType);
+            if (paramType.equals(objectType)) {
+                LOG.debug("Yes, they are the same class.");
+            } else if (paramType.isAssignableFrom(objectType)) {
+                LOG.debug("Yes, assignment is possible.");
+            } else if (isPrimitiveBoolean(paramType) && Boolean.class.isAssignableFrom(objectType)){
+                LOG.debug("Yes, assignment is possible.");
+            } else if(isPrimitiveNumber(paramType) && Number.class.isAssignableFrom(objectType)){
+                LOG.debug("Yes, assignment is possible.");
+            } else if(paramType.isEnum() && objectType.equals(String.class)){
+                LOG.debug("Yes, will convert a String to enum");
+            } else if (paramType.isArray() && List.class.isAssignableFrom(objectType)) {
+                // TODO more collection content type checking
+                LOG.debug("Assignment is possible if we convert a List to an array.");
+                LOG.debug("Array Type: {}, List type: {}", paramType.getComponentType(), ((List) obj).get(0).getClass());
+            } else {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    public static boolean isPrimitiveNumber(Class clazz){
+        return clazz.isPrimitive() && !clazz.equals(boolean.class);
+    }
+
+    public static boolean isPrimitiveBoolean(Class clazz){
+        return clazz.isPrimitive() && clazz.equals(boolean.class);
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/api/TopologySource.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/api/TopologySource.java b/flux/flux-core/src/main/java/org/apache/storm/flux/api/TopologySource.java
new file mode 100644
index 0000000..2777854
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/api/TopologySource.java
@@ -0,0 +1,39 @@
+/*
+ * 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.flux.api;
+
+
+import org.apache.storm.generated.StormTopology;
+
+import java.util.Map;
+
+/**
+ * Marker interface for objects that can produce `StormTopology` objects.
+ *
+ * If a `topology-source` class implements the `getTopology()` method, Flux will
+ * call that method. Otherwise, it will introspect the given class and look for a
+ * similar method that produces a `StormTopology` instance.
+ *
+ * Note that it is not strictly necessary for a class to implement this interface.
+ * If a class defines a method with a similar signature, Flux should be able to find
+ * and invoke it.
+ *
+ */
+public interface TopologySource {
+    public StormTopology getTopology(Map<String, Object> config);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanDef.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanDef.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanDef.java
new file mode 100644
index 0000000..f0247ed
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanDef.java
@@ -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.
+ */
+package org.apache.storm.flux.model;
+
+/**
+ * A representation of a Java object that is uniquely identifyable, and given a className, constructor arguments,
+ * and properties, can be instantiated.
+ */
+public class BeanDef extends ObjectDef {
+    private String id;
+
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanListReference.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanListReference.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanListReference.java
new file mode 100644
index 0000000..652210c
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanListReference.java
@@ -0,0 +1,37 @@
+/*
+ * 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.flux.model;
+
+import java.util.List;
+
+/**
+ * A bean list reference is a list of bean reference.
+ */
+public class BeanListReference {
+    public List<String> ids;
+
+    public BeanListReference(){}
+
+    public BeanListReference(List<String> ids){
+        this.ids = ids;
+    }
+
+    public List<String> getIds() {
+        return ids;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanReference.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanReference.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanReference.java
new file mode 100644
index 0000000..bd236f1
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanReference.java
@@ -0,0 +1,39 @@
+/*
+ * 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.flux.model;
+
+/**
+ * A bean reference is simply a string pointer to another id.
+ */
+public class BeanReference {
+    public String id;
+
+    public BeanReference(){}
+
+    public BeanReference(String id){
+        this.id = id;
+    }
+
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/BoltDef.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/BoltDef.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/BoltDef.java
new file mode 100644
index 0000000..362abf1
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/BoltDef.java
@@ -0,0 +1,24 @@
+/*
+ * 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.flux.model;
+
+/**
+ * Bean representation of a Storm bolt.
+ */
+public class BoltDef extends VertexDef {
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/ConfigMethodDef.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/ConfigMethodDef.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/ConfigMethodDef.java
new file mode 100644
index 0000000..69cabc3
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/ConfigMethodDef.java
@@ -0,0 +1,67 @@
+/*
+ * 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.flux.model;
+
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+public class ConfigMethodDef {
+    private String name;
+    private List<Object> args;
+    private boolean hasReferences = false;
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public List<Object> getArgs() {
+        return args;
+    }
+
+    public void setArgs(List<Object> args) {
+
+        List<Object> newVal = new ArrayList<Object>();
+        for(Object obj : args){
+            if(obj instanceof LinkedHashMap){
+                Map map = (Map)obj;
+                if(map.containsKey("ref") && map.size() == 1){
+                    newVal.add(new BeanReference((String)map.get("ref")));
+                    this.hasReferences = true;
+                } else if (map.containsKey("reflist") && map.size() == 1) {
+                    newVal.add(new BeanListReference((List<String>) map.get("reflist")));
+                    this.hasReferences = true;
+                } else {
+                    newVal.add(obj);
+                }
+            } else {
+                newVal.add(obj);
+            }
+        }
+        this.args = newVal;
+    }
+
+    public boolean hasReferences(){
+        return this.hasReferences;
+    }
+}


[16/23] storm git commit: STORM-2453 Move non-connectors into the top directory

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/RelNodeCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/RelNodeCompiler.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/RelNodeCompiler.java
deleted file mode 100644
index 97995c7..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/RelNodeCompiler.java
+++ /dev/null
@@ -1,484 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.compiler.backends.standalone;
-
-import com.google.common.base.Joiner;
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.plan.RelOptUtil;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Aggregate;
-import org.apache.calcite.rel.core.AggregateCall;
-import org.apache.calcite.rel.core.Filter;
-import org.apache.calcite.rel.core.Join;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.core.TableScan;
-import org.apache.calcite.rel.logical.LogicalJoin;
-import org.apache.calcite.rel.stream.Delta;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.schema.AggregateFunction;
-import org.apache.calcite.schema.impl.AggregateFunctionImpl;
-import org.apache.calcite.sql.SqlAggFunction;
-import org.apache.calcite.sql.validate.SqlUserDefinedAggFunction;
-import org.apache.storm.sql.compiler.RexNodeToJavaCodeCompiler;
-
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.lang.reflect.Method;
-import java.lang.reflect.Modifier;
-import java.lang.reflect.Type;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Compile RelNodes into individual functions.
- */
-class RelNodeCompiler extends PostOrderRelNodeVisitor<Void> {
-  public static Joiner NEW_LINE_JOINER = Joiner.on('\n');
-
-  private final PrintWriter pw;
-  private final JavaTypeFactory typeFactory;
-  private final RexNodeToJavaCodeCompiler rexCompiler;
-
-  private static final String STAGE_PROLOGUE = NEW_LINE_JOINER.join(
-    "  private static final ChannelHandler %1$s = ",
-    "    new AbstractChannelHandler() {",
-    "    @Override",
-    "    public void dataReceived(ChannelContext ctx, Values _data) {",
-    ""
-  );
-
-  private static final String AGGREGATE_STAGE_PROLOGUE = NEW_LINE_JOINER.join(
-          "  private static final ChannelHandler %1$s = ",
-          "    new AbstractChannelHandler() {",
-          "    private final Values EMPTY_VALUES = new Values();",
-          "    private final Map<List<Object>, Map<String, Object>> state = new LinkedHashMap<>();",
-          "    private final int[] groupIndices = new int[] {%2$s};",
-          "    private List<Object> getGroupValues(Values _data) {",
-          "      List<Object> res = new ArrayList<>();",
-          "      for (int i: groupIndices) {",
-          "        res.add(_data.get(i));",
-          "      }",
-          "      return res;",
-          "    }",
-          "",
-          "    @Override",
-          "    public void flush(ChannelContext ctx) {",
-          "      emitAggregateResults(ctx);",
-          "      super.flush(ctx);",
-          "      state.clear();",
-          "    }",
-          "",
-          "    private void emitAggregateResults(ChannelContext ctx) {",
-          "        for (Map.Entry<List<Object>, Map<String, Object>> entry: state.entrySet()) {",
-          "          List<Object> groupValues = entry.getKey();",
-          "          Map<String, Object> accumulators = entry.getValue();",
-          "          %3$s",
-          "        }",
-          "    }",
-          "",
-          "    @Override",
-          "    public void dataReceived(ChannelContext ctx, Values _data) {",
-          ""
-  );
-
-  private static final String JOIN_STAGE_PROLOGUE = NEW_LINE_JOINER.join(
-          "  private static final ChannelHandler %1$s = ",
-          "    new AbstractChannelHandler() {",
-          "      Object left = %2$s;",
-          "      Object right = %3$s;",
-          "      Object source = null;",
-          "      List<Values> leftRows = new ArrayList<>();",
-          "      List<Values> rightRows = new ArrayList<>();",
-          "      boolean leftDone = false;",
-          "      boolean rightDone = false;",
-          "      int[] ordinals = new int[] {%4$s, %5$s};",
-          "",
-          "      Multimap<Object, Values> getJoinTable(List<Values> rows, int joinIndex) {",
-          "         Multimap<Object, Values> m = ArrayListMultimap.create();",
-          "         for(Values v: rows) {",
-          "           m.put(v.get(joinIndex), v);",
-          "         }",
-          "         return m;",
-          "      }",
-          "",
-          "      List<Values> join(Multimap<Object, Values> tab, List<Values> rows, int rowIdx, boolean rev) {",
-          "         List<Values> res = new ArrayList<>();",
-          "         for (Values row: rows) {",
-          "           for (Values mapValue: tab.get(row.get(rowIdx))) {",
-          "             if (mapValue != null) {",
-          "               Values joinedRow = new Values();",
-          "               if(rev) {",
-          "                 joinedRow.addAll(row);",
-          "                 joinedRow.addAll(mapValue);",
-          "               } else {",
-          "                 joinedRow.addAll(mapValue);",
-          "                 joinedRow.addAll(row);",
-          "               }",
-          "               res.add(joinedRow);",
-          "             }",
-          "           }",
-          "         }",
-          "         return res;",
-          "      }",
-          "",
-          "    @Override",
-          "    public void setSource(ChannelContext ctx, Object source) {",
-          "      this.source = source;",
-          "    }",
-          "",
-          "    @Override",
-          "    public void flush(ChannelContext ctx) {",
-          "        if (source == left) {",
-          "            leftDone = true;",
-          "        } else if (source == right) {",
-          "            rightDone = true;",
-          "        }",
-          "        if (leftDone && rightDone) {",
-          "          if (leftRows.size() <= rightRows.size()) {",
-          "            for(Values res: join(getJoinTable(leftRows, ordinals[0]), rightRows, ordinals[1], false)) {",
-          "              ctx.emit(res);",
-          "            }",
-          "          } else {",
-          "            for(Values res: join(getJoinTable(rightRows, ordinals[1]), leftRows, ordinals[0], true)) {",
-          "              ctx.emit(res);",
-          "            }",
-          "          }",
-          "          leftDone = rightDone = false;",
-          "          leftRows.clear();",
-          "          rightRows.clear();",
-          "          super.flush(ctx);",
-          "        }",
-          "    }",
-          "",
-          "    @Override",
-          "    public void dataReceived(ChannelContext ctx, Values _data) {",
-          ""
-  );
-
-  private static final String STAGE_PASSTHROUGH = NEW_LINE_JOINER.join(
-      "  private static final ChannelHandler %1$s = AbstractChannelHandler.PASS_THROUGH;",
-      "");
-
-  private static final String STAGE_ENUMERABLE_TABLE_SCAN = NEW_LINE_JOINER.join(
-          "  private static final ChannelHandler %1$s = new AbstractChannelHandler() {",
-          "    @Override",
-          "    public void flush(ChannelContext ctx) {",
-          "      ctx.setSource(this);",
-          "      super.flush(ctx);",
-          "    }",
-          "",
-          "    @Override",
-          "    public void dataReceived(ChannelContext ctx, Values _data) {",
-          "      ctx.setSource(this);",
-          "      ctx.emit(_data);",
-          "    }",
-          "  };",
-          "");
-
-  private int nameCount;
-  private Map<AggregateCall, String> aggregateCallVarNames = new HashMap<>();
-
-  RelNodeCompiler(PrintWriter pw, JavaTypeFactory typeFactory) {
-    this.pw = pw;
-    this.typeFactory = typeFactory;
-    this.rexCompiler = new RexNodeToJavaCodeCompiler(new RexBuilder(typeFactory));
-  }
-
-  @Override
-  public Void visitDelta(Delta delta, List<Void> inputStreams) throws Exception {
-    pw.print(String.format(STAGE_PASSTHROUGH, getStageName(delta)));
-    return null;
-  }
-
-  @Override
-  public Void visitFilter(Filter filter, List<Void> inputStreams) throws Exception {
-    beginStage(filter);
-
-    List<RexNode> childExps = filter.getChildExps();
-    RelDataType inputRowType = filter.getInput(0).getRowType();
-
-    pw.print("Context context = new StormContext(Processor.dataContext);\n");
-    pw.print("context.values = _data.toArray();\n");
-    pw.print("Object[] outputValues = new Object[1];\n");
-
-    pw.write(rexCompiler.compileToBlock(childExps, inputRowType).toString());
-
-    String r = "((Boolean) outputValues[0])";
-    if (filter.getCondition().getType().isNullable()) {
-      pw.print(String.format("    if (%s != null && %s) { ctx.emit(_data); }\n", r, r));
-    } else {
-      pw.print(String.format("    if (%s) { ctx.emit(_data); }\n", r, r));
-    }
-    endStage();
-    return null;
-  }
-
-  @Override
-  public Void visitProject(Project project, List<Void> inputStreams) throws Exception {
-    beginStage(project);
-
-    List<RexNode> childExps = project.getChildExps();
-    RelDataType inputRowType = project.getInput(0).getRowType();
-    int outputCount = project.getRowType().getFieldCount();
-
-    pw.print("Context context = new StormContext(Processor.dataContext);\n");
-    pw.print("context.values = _data.toArray();\n");
-    pw.print(String.format("Object[] outputValues = new Object[%d];\n", outputCount));
-
-    pw.write(rexCompiler.compileToBlock(childExps, inputRowType).toString());
-
-    pw.print("    ctx.emit(new Values(outputValues));\n");
-    endStage();
-    return null;
-  }
-
-  @Override
-  public Void defaultValue(RelNode n, List<Void> inputStreams) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public Void visitTableScan(TableScan scan, List<Void> inputStreams) throws Exception {
-    pw.print(String.format(STAGE_ENUMERABLE_TABLE_SCAN, getStageName(scan)));
-    return null;
-  }
-
-  @Override
-  public Void visitAggregate(Aggregate aggregate, List<Void> inputStreams) throws Exception {
-    beginAggregateStage(aggregate);
-    pw.println("        if (_data != null) {");
-    pw.println("        List<Object> curGroupValues = getGroupValues(_data);");
-    pw.println("        if (!state.containsKey(curGroupValues)) {");
-    pw.println("          state.put(curGroupValues, new HashMap<String, Object>());");
-    pw.println("        }");
-    pw.println("        Map<String, Object> accumulators = state.get(curGroupValues);");
-    for (AggregateCall call : aggregate.getAggCallList()) {
-      aggregate(call);
-    }
-    pw.println("        }");
-    endStage();
-    return null;
-  }
-
-  @Override
-  public Void visitJoin(Join join, List<Void> inputStreams) {
-    beginJoinStage(join);
-    pw.println("        if (source == left) {");
-    pw.println("            leftRows.add(_data);");
-    pw.println("        } else if (source == right) {");
-    pw.println("            rightRows.add(_data);");
-    pw.println("        }");
-    endStage();
-    return null;
-  }
-
-  private String groupValueEmitStr(String var, int n) {
-    int count = 0;
-    StringBuilder sb = new StringBuilder();
-    for (int i = 0; i < n; i++) {
-      if (++count > 1) {
-        sb.append(", ");
-      }
-      sb.append(var).append(".").append("get(").append(i).append(")");
-    }
-    return sb.toString();
-  }
-
-  private String emitAggregateStmts(Aggregate aggregate) {
-    List<String> res = new ArrayList<>();
-    StringWriter sw = new StringWriter();
-    for (AggregateCall call : aggregate.getAggCallList()) {
-      res.add(aggregateResult(call, new PrintWriter(sw)));
-    }
-    return NEW_LINE_JOINER.join(sw.toString(),
-                                String.format("          ctx.emit(new Values(%s, %s));",
-                                              groupValueEmitStr("groupValues", aggregate.getGroupSet().cardinality()),
-                                              Joiner.on(", ").join(res)));
-  }
-
-  private String aggregateResult(AggregateCall call, PrintWriter pw) {
-    SqlAggFunction aggFunction = call.getAggregation();
-    String aggregationName = call.getAggregation().getName();
-    Type ty = typeFactory.getJavaClass(call.getType());
-    String result;
-    if (aggFunction instanceof SqlUserDefinedAggFunction) {
-      AggregateFunction aggregateFunction = ((SqlUserDefinedAggFunction) aggFunction).function;
-      result = doAggregateResult((AggregateFunctionImpl) aggregateFunction, reserveAggVarName(call), ty, pw);
-    } else {
-      List<BuiltinAggregateFunctions.TypeClass> typeClasses = BuiltinAggregateFunctions.TABLE.get(aggregationName);
-      if (typeClasses == null) {
-        throw new UnsupportedOperationException(aggregationName + " Not implemented");
-      }
-      result = doAggregateResult(AggregateFunctionImpl.create(findMatchingClass(aggregationName, typeClasses, ty)),
-                                 reserveAggVarName(call), ty, pw);
-    }
-    return result;
-  }
-
-  private String doAggregateResult(AggregateFunctionImpl aggFn, String varName, Type ty, PrintWriter pw) {
-    String resultName = varName + "_result";
-    Class<?> accumulatorType = aggFn.accumulatorType;
-    Class<?> resultType = aggFn.resultType;
-    List<String> args = new ArrayList<>();
-    if (!aggFn.isStatic) {
-      String aggObjName = String.format("%s_obj", varName);
-      String aggObjClassName = aggFn.initMethod.getDeclaringClass().getCanonicalName();
-      pw.println("          @SuppressWarnings(\"unchecked\")");
-      pw.println(String.format("          final %1$s %2$s = (%1$s) accumulators.get(\"%2$s\");", aggObjClassName,
-              aggObjName));
-      args.add(aggObjName);
-    }
-    args.add(String.format("(%s)accumulators.get(\"%s\")", accumulatorType.getCanonicalName(), varName));
-    pw.println(String.format("          final %s %s = %s;", resultType.getCanonicalName(),
-                             resultName, printMethodCall(aggFn.resultMethod, args)));
-
-    return resultName;
-  }
-
-  private void aggregate(AggregateCall call) {
-    SqlAggFunction aggFunction = call.getAggregation();
-    String aggregationName = call.getAggregation().getName();
-    Type ty = typeFactory.getJavaClass(call.getType());
-    if (call.getArgList().size() != 1) {
-      if (aggregationName.equals("COUNT")) {
-        if (call.getArgList().size() != 0) {
-          throw new UnsupportedOperationException("Count with nullable fields");
-        }
-      }
-    }
-    if (aggFunction instanceof SqlUserDefinedAggFunction) {
-      AggregateFunction aggregateFunction = ((SqlUserDefinedAggFunction) aggFunction).function;
-      doAggregate((AggregateFunctionImpl) aggregateFunction, reserveAggVarName(call), ty, call.getArgList());
-    } else {
-      List<BuiltinAggregateFunctions.TypeClass> typeClasses = BuiltinAggregateFunctions.TABLE.get(aggregationName);
-      if (typeClasses == null) {
-        throw new UnsupportedOperationException(aggregationName + " Not implemented");
-      }
-      doAggregate(AggregateFunctionImpl.create(findMatchingClass(aggregationName, typeClasses, ty)),
-                  reserveAggVarName(call), ty, call.getArgList());
-    }
-  }
-
-  private Class<?> findMatchingClass(String aggregationName, List<BuiltinAggregateFunctions.TypeClass> typeClasses, Type ty) {
-    for (BuiltinAggregateFunctions.TypeClass typeClass : typeClasses) {
-      if (typeClass.ty.equals(BuiltinAggregateFunctions.TypeClass.GenericType.class) || typeClass.ty.equals(ty)) {
-        return typeClass.clazz;
-      }
-    }
-    throw new UnsupportedOperationException(aggregationName + " Not implemeted for type '" + ty + "'");
-  }
-
-  private void doAggregate(AggregateFunctionImpl aggFn, String varName, Type ty, List<Integer> argList) {
-    List<String> args = new ArrayList<>();
-    Class<?> accumulatorType = aggFn.accumulatorType;
-    if (!aggFn.isStatic) {
-      String aggObjName = String.format("%s_obj", varName);
-      String aggObjClassName = aggFn.initMethod.getDeclaringClass().getCanonicalName();
-      pw.println(String.format("          if (!accumulators.containsKey(\"%s\")) { ", aggObjName));
-      pw.println(String.format("            accumulators.put(\"%s\", new %s());", aggObjName, aggObjClassName));
-      pw.println("          }");
-      pw.println("          @SuppressWarnings(\"unchecked\")");
-      pw.println(String.format("          final %1$s %2$s = (%1$s) accumulators.get(\"%2$s\");", aggObjClassName,
-              aggObjName));
-      args.add(aggObjName);
-    }
-    args.add(String.format("%1$s == null ? %2$s : (%3$s) %1$s",
-                           "accumulators.get(\"" + varName + "\")",
-                           printMethodCall(aggFn.initMethod, args),
-                           accumulatorType.getCanonicalName()));
-    if (argList.isEmpty()) {
-      args.add("EMPTY_VALUES");
-    } else {
-      for (int i = 0; i < aggFn.valueTypes.size(); i++) {
-        args.add(String.format("(%s) %s", aggFn.valueTypes.get(i).getCanonicalName(), "_data.get(" + argList.get(i) + ")"));
-      }
-    }
-    pw.print(String.format("          accumulators.put(\"%s\", %s);\n",
-                           varName,
-                           printMethodCall(aggFn.addMethod, args)));
-  }
-
-  private String reserveAggVarName(AggregateCall call) {
-    String varName;
-    if ((varName = aggregateCallVarNames.get(call)) == null) {
-      varName = call.getAggregation().getName() + ++nameCount;
-      aggregateCallVarNames.put(call, varName);
-    }
-    return varName;
-  }
-
-  private void beginStage(RelNode n) {
-    pw.print(String.format(STAGE_PROLOGUE, getStageName(n)));
-  }
-
-  private void beginAggregateStage(Aggregate n) {
-    pw.print(String.format(AGGREGATE_STAGE_PROLOGUE, getStageName(n), getGroupByIndices(n), emitAggregateStmts(n)));
-  }
-
-  private void beginJoinStage(Join join) {
-    int[] ordinals = new int[2];
-    if (!RelOptUtil.analyzeSimpleEquiJoin((LogicalJoin) join, ordinals)) {
-      throw new UnsupportedOperationException("Only simple equi joins are supported");
-    }
-
-    pw.print(String.format(JOIN_STAGE_PROLOGUE, getStageName(join),
-                           getStageName(join.getLeft()),
-                           getStageName(join.getRight()),
-                           ordinals[0],
-                           ordinals[1]));
-  }
-
-  private void endStage() {
-    pw.print("  }\n  };\n");
-  }
-
-  static String getStageName(RelNode n) {
-    return n.getClass().getSimpleName().toUpperCase() + "_" + n.getId();
-  }
-
-  private String getGroupByIndices(Aggregate n) {
-    StringBuilder res = new StringBuilder();
-    int count = 0;
-    for (int i : n.getGroupSet()) {
-      if (++count > 1) {
-        res.append(", ");
-      }
-      res.append(i);
-    }
-    return res.toString();
-  }
-
-  public static String printMethodCall(Method method, List<String> args) {
-    return printMethodCall(method.getDeclaringClass(), method.getName(),
-            Modifier.isStatic(method.getModifiers()), args);
-  }
-
-  private static String printMethodCall(Class<?> clazz, String method, boolean isStatic, List<String> args) {
-    if (isStatic) {
-      return String.format("%s.%s(%s)", clazz.getCanonicalName(), method, Joiner.on(',').join(args));
-    } else {
-      return String.format("%s.%s(%s)", args.get(0), method,
-              Joiner.on(',').join(args.subList(1, args.size())));
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/javac/CompilingClassLoader.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/javac/CompilingClassLoader.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/javac/CompilingClassLoader.java
deleted file mode 100644
index 0b7c053..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/javac/CompilingClassLoader.java
+++ /dev/null
@@ -1,225 +0,0 @@
-/*
- * Copyright (C) 2010 Google, Inc.
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.javac;
-
-
-import javax.tools.DiagnosticListener;
-import javax.tools.FileObject;
-import javax.tools.ForwardingJavaFileManager;
-import javax.tools.JavaCompiler;
-import javax.tools.JavaFileManager;
-import javax.tools.JavaFileObject;
-import javax.tools.SimpleJavaFileObject;
-import javax.tools.ToolProvider;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-import static java.util.Collections.singleton;
-
-/**
- * This is a Java ClassLoader that will attempt to load a class from a string of source code.
- *
- * <h3>Example</h3>
- *
- * <pre>
- * String className = "com.foo.MyClass";
- * String classSource =
- *   "package com.foo;\n" +
- *   "public class MyClass implements Runnable {\n" +
- *   "  @Override public void run() {\n" +
- *   "   log(\"Hello world\");\n" +
- *   "  }\n" +
- *   "}";
- *
- * // Load class from source.
- * ClassLoader classLoader = new CompilingClassLoader(
- *     parentClassLoader, className, classSource);
- * Class myClass = classLoader.loadClass(className);
- *
- * // Use it.
- * Runnable instance = (Runnable)myClass.newInstance();
- * instance.run();
- * </pre>
- *
- * Only one chunk of source can be compiled per instance of CompilingClassLoader. If you need to
- * compile more, create multiple CompilingClassLoader instances.
- *
- * Uses Java 1.6's in built compiler API.
- *
- * If the class cannot be compiled, loadClass() will throw a ClassNotFoundException and log the
- * compile errors to System.err. If you don't want the messages logged, or want to explicitly handle
- * the messages you can provide your own {@link javax.tools.DiagnosticListener} through
- * {#setDiagnosticListener()}.
- *
- * @see java.lang.ClassLoader
- * @see javax.tools.JavaCompiler
- */
-public class CompilingClassLoader extends ClassLoader {
-
-  /**
-   * Thrown when code cannot be compiled.
-   */
-  public static class CompilerException extends Exception {
-    private static final long serialVersionUID = -2936958840023603270L;
-
-    public CompilerException(String message) {
-      super(message);
-    }
-  }
-
-  private final Map<String, ByteArrayOutputStream> byteCodeForClasses = new HashMap<>();
-
-  private static final URI EMPTY_URI;
-
-  static {
-    try {
-      // Needed to keep SimpleFileObject constructor happy.
-      EMPTY_URI = new URI("");
-    } catch (URISyntaxException e) {
-      throw new Error(e);
-    }
-  }
-
-  /**
-   * @param parent Parent classloader to resolve dependencies from.
-   * @param className Name of class to compile. eg. "com.foo.MyClass".
-   * @param sourceCode Java source for class. e.g. "package com.foo; class MyClass { ... }".
-   * @param diagnosticListener Notified of compiler errors (may be null).
-   */
-  public CompilingClassLoader(
-      ClassLoader parent,
-      String className,
-      String sourceCode,
-      DiagnosticListener<JavaFileObject> diagnosticListener)
-      throws CompilerException {
-    super(parent);
-    if (!compileSourceCodeToByteCode(className, sourceCode, diagnosticListener)) {
-      throw new CompilerException("Could not compile " + className);
-    }
-  }
-
-  public Map<String, ByteArrayOutputStream> getClasses() {
-    return byteCodeForClasses;
-  }
-
-  /**
-   * Override ClassLoader's class resolving method. Don't call this directly, instead use
-   * {@link ClassLoader#loadClass(String)}.
-   */
-  @Override
-  public Class<?> findClass(String name) throws ClassNotFoundException {
-    ByteArrayOutputStream byteCode = byteCodeForClasses.get(name);
-    if (byteCode == null) {
-      throw new ClassNotFoundException(name);
-    }
-    return defineClass(name, byteCode.toByteArray(), 0, byteCode.size());
-  }
-
-  /**
-   * @return Whether compilation was successful.
-   */
-  private boolean compileSourceCodeToByteCode(
-      String className, String sourceCode, DiagnosticListener<JavaFileObject> diagnosticListener) {
-    JavaCompiler javaCompiler = ToolProvider.getSystemJavaCompiler();
-
-    // Set up the in-memory filesystem.
-    InMemoryFileManager fileManager =
-        new InMemoryFileManager(javaCompiler.getStandardFileManager(null, null, null));
-    JavaFileObject javaFile = new InMemoryJavaFile(className, sourceCode);
-
-    // Javac option: remove these when the javac zip impl is fixed
-    // (http://b/issue?id=1822932)
-    System.setProperty("useJavaUtilZip", "true"); // setting value to any non-null string
-    List<String> options = new LinkedList<>();
-    // this is ignored by javac currently but useJavaUtilZip should be
-    // a valid javac XD option, which is another bug
-    options.add("-XDuseJavaUtilZip");
-
-    // Now compile!
-    JavaCompiler.CompilationTask compilationTask =
-        javaCompiler.getTask(
-            null, // Null: log any unhandled errors to stderr.
-            fileManager,
-            diagnosticListener,
-            options,
-            null,
-            singleton(javaFile));
-    return compilationTask.call();
-  }
-
-  /**
-   * Provides an in-memory representation of JavaFileManager abstraction, so we do not need to write
-   * any files to disk.
-   *
-   * When files are written to, rather than putting the bytes on disk, they are appended to buffers
-   * in byteCodeForClasses.
-   *
-   * @see javax.tools.JavaFileManager
-   */
-  private class InMemoryFileManager extends ForwardingJavaFileManager<JavaFileManager> {
-    public InMemoryFileManager(JavaFileManager fileManager) {
-      super(fileManager);
-    }
-
-    @Override
-    public JavaFileObject getJavaFileForOutput(
-        Location location, final String className, JavaFileObject.Kind kind, FileObject sibling)
-        throws IOException {
-      return new SimpleJavaFileObject(EMPTY_URI, kind) {
-        @Override
-        public OutputStream openOutputStream() throws IOException {
-          ByteArrayOutputStream outputStream = byteCodeForClasses.get(className);
-          if (outputStream != null) {
-            throw new IllegalStateException("Cannot write more than once");
-          }
-          // Reasonable size for a simple .class.
-          outputStream = new ByteArrayOutputStream(256);
-          byteCodeForClasses.put(className, outputStream);
-          return outputStream;
-        }
-      };
-    }
-  }
-
-  private static class InMemoryJavaFile extends SimpleJavaFileObject {
-    private final String sourceCode;
-
-    public InMemoryJavaFile(String className, String sourceCode) {
-      super(makeUri(className), Kind.SOURCE);
-      this.sourceCode = sourceCode;
-    }
-
-    private static URI makeUri(String className) {
-      try {
-        return new URI(className.replaceAll("\\.", "/") + Kind.SOURCE.extension);
-      } catch (URISyntaxException e) {
-        throw new RuntimeException(e); // Not sure what could cause this.
-      }
-    }
-
-    @Override
-    public CharSequence getCharContent(boolean ignoreEncodingErrors) throws IOException {
-      return sourceCode;
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnConstraint.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnConstraint.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnConstraint.java
deleted file mode 100644
index c67d8e7..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnConstraint.java
+++ /dev/null
@@ -1,42 +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.sql.parser;
-
-import org.apache.calcite.sql.SqlLiteral;
-import org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.sql.validate.SqlMonotonicity;
-
-public class ColumnConstraint extends SqlLiteral {
-  private ColumnConstraint(
-      Object value, SqlTypeName typeName, SqlParserPos pos) {
-    super(value, typeName, pos);
-  }
-
-  public static class PrimaryKey extends ColumnConstraint {
-    private final SqlMonotonicity monotonicity;
-    public PrimaryKey(SqlMonotonicity monotonicity, SqlParserPos pos) {
-      super(SqlDDLKeywords.PRIMARY, SqlTypeName.SYMBOL, pos);
-      this.monotonicity = monotonicity;
-    }
-    public SqlMonotonicity monotonicity() {
-      return monotonicity;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnDefinition.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnDefinition.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnDefinition.java
deleted file mode 100644
index 3520b86..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnDefinition.java
+++ /dev/null
@@ -1,44 +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.sql.parser;
-
-import org.apache.calcite.sql.SqlDataTypeSpec;
-import org.apache.calcite.sql.SqlIdentifier;
-import org.apache.calcite.sql.SqlNodeList;
-import org.apache.calcite.sql.parser.SqlParserPos;
-
-import java.util.Arrays;
-
-public class ColumnDefinition extends SqlNodeList {
-  public ColumnDefinition(
-      SqlIdentifier name, SqlDataTypeSpec type, ColumnConstraint constraint, SqlParserPos pos) {
-    super(Arrays.asList(name, type, constraint), pos);
-  }
-
-  public String name() {
-    return get(0).toString();
-  }
-
-  public SqlDataTypeSpec type() {
-    return (SqlDataTypeSpec) get(1);
-  }
-
-  public ColumnConstraint constraint() {
-    return (ColumnConstraint) get(2);
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateFunction.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateFunction.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateFunction.java
deleted file mode 100644
index a53802c..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateFunction.java
+++ /dev/null
@@ -1,94 +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.sql.parser;
-
-import org.apache.calcite.sql.SqlCall;
-import org.apache.calcite.sql.SqlIdentifier;
-import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.SqlLiteral;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.sql.SqlSpecialOperator;
-import org.apache.calcite.sql.SqlWriter;
-import org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.calcite.util.ImmutableNullableList;
-import org.apache.calcite.util.NlsString;
-
-import java.util.List;
-
-public class SqlCreateFunction extends SqlCall {
-    public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator(
-            "CREATE_FUNCTION", SqlKind.OTHER) {
-        @Override
-        public SqlCall createCall(
-                SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... o) {
-            assert functionQualifier == null;
-            return new SqlCreateFunction(pos, (SqlIdentifier) o[0], o[1], o[2]);
-        }
-
-        @Override
-        public void unparse(
-                SqlWriter writer, SqlCall call, int leftPrec, int rightPrec) {
-            SqlCreateFunction t = (SqlCreateFunction) call;
-            UnparseUtil u = new UnparseUtil(writer, leftPrec, rightPrec);
-            u.keyword("CREATE", "FUNCTION").node(t.functionName).keyword("AS").node(t.className);
-            if (t.jarName != null) {
-                u.keyword("USING", "JAR").node(t.jarName);
-            }
-        }
-    };
-
-    private final SqlIdentifier functionName;
-    private final SqlNode className;
-    private final SqlNode jarName;
-
-    public SqlCreateFunction(SqlParserPos pos, SqlIdentifier functionName, SqlNode className, SqlNode jarName) {
-        super(pos);
-        this.functionName = functionName;
-        this.className = className;
-        this.jarName = jarName;
-    }
-
-    @Override
-    public SqlOperator getOperator() {
-        return OPERATOR;
-    }
-
-    @Override
-    public List<SqlNode> getOperandList() {
-        return ImmutableNullableList.of(functionName, className);
-    }
-
-
-    @Override
-    public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
-        getOperator().unparse(writer, this, leftPrec, rightPrec);
-    }
-
-    public String functionName() {
-        return functionName.toString();
-    }
-
-    public String className() {
-        return ((NlsString)SqlLiteral.value(className)).getValue();
-    }
-
-    public String jarName() {
-        return jarName == null ? null : ((NlsString)SqlLiteral.value(jarName)).getValue();
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateTable.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateTable.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateTable.java
deleted file mode 100644
index 670eedb..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateTable.java
+++ /dev/null
@@ -1,166 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.parser;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.calcite.sql.SqlCall;
-import org.apache.calcite.sql.SqlIdentifier;
-import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.SqlLiteral;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlNodeList;
-import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.sql.SqlSpecialOperator;
-import org.apache.calcite.sql.SqlWriter;
-import org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.calcite.util.ImmutableNullableList;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Properties;
-
-public class SqlCreateTable extends SqlCall {
-  private static final int DEFAULT_PARALLELISM = 1;
-
-  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator(
-      "CREATE_TABLE", SqlKind.OTHER) {
-    @Override
-    public SqlCall createCall(
-        SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... o) {
-      assert functionQualifier == null;
-      return new SqlCreateTable(pos, (SqlIdentifier) o[0], (SqlNodeList) o[1],
-                                o[2], o[3], o[4], o[5], o[6], o[7]);
-    }
-
-    @Override
-    public void unparse(
-        SqlWriter writer, SqlCall call, int leftPrec, int rightPrec) {
-      SqlCreateTable t = (SqlCreateTable) call;
-      UnparseUtil u = new UnparseUtil(writer, leftPrec, rightPrec);
-      u.keyword("CREATE", "EXTERNAL", "TABLE").node(t.tblName).nodeList(
-          t.fieldList);
-      if (t.inputFormatClass != null && t.outputFormatClass != null) {
-        u.keyword("STORED", "AS", "INPUTFORMAT").node(
-            t.inputFormatClass).keyword("OUTPUTFORMAT").node(
-            t.outputFormatClass);
-      }
-      u.keyword("LOCATION").node(t.location);
-      if (t.parallelism != null) {
-        u.keyword("PARALLELISM").node(t.parallelism);
-      }
-      if (t.properties != null) {
-        u.keyword("TBLPROPERTIES").node(t.properties);
-      }
-      if (t.query != null) {
-        u.keyword("AS").node(t.query);
-      }
-    }
-  };
-
-  private final SqlIdentifier tblName;
-  private final SqlNodeList fieldList;
-  private final SqlNode inputFormatClass;
-  private final SqlNode outputFormatClass;
-  private final SqlNode location;
-  private final SqlNode parallelism;
-  private final SqlNode properties;
-  private final SqlNode query;
-
-  public SqlCreateTable(
-          SqlParserPos pos, SqlIdentifier tblName, SqlNodeList fieldList,
-          SqlNode inputFormatClass, SqlNode outputFormatClass, SqlNode location,
-          SqlNode parallelism, SqlNode properties, SqlNode query) {
-    super(pos);
-    this.tblName = tblName;
-    this.fieldList = fieldList;
-    this.inputFormatClass = inputFormatClass;
-    this.outputFormatClass = outputFormatClass;
-    this.location = location;
-    this.parallelism = parallelism;
-    this.properties = properties;
-    this.query = query;
-  }
-
-  @Override
-  public SqlOperator getOperator() {
-    return OPERATOR;
-  }
-
-  @Override
-  public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
-    getOperator().unparse(writer, this, leftPrec, rightPrec);
-  }
-
-  @Override
-  public List<SqlNode> getOperandList() {
-    return ImmutableNullableList.of(tblName, fieldList, inputFormatClass,
-                                    outputFormatClass, location, properties,
-                                    query);
-  }
-
-  public String tableName() {
-    return tblName.toString();
-  }
-
-  public URI location() {
-    return URI.create(getString(location));
-  }
-
-  public Integer parallelism() {
-    String parallelismStr = getString(parallelism);
-    if (parallelismStr != null) {
-      return Integer.parseInt(parallelismStr);
-    } else {
-      return DEFAULT_PARALLELISM;
-    }
-  }
-
-  public String inputFormatClass() {
-    return getString(inputFormatClass);
-  }
-
-  public String outputFormatClass() {
-    return getString(outputFormatClass);
-  }
-
-  public Properties properties() {
-    Properties props = new Properties();
-    if (properties != null) {
-      try {
-        ObjectMapper mapper = new ObjectMapper();
-        HashMap<String, Object> map = mapper.readValue(getString(properties), HashMap.class);
-        props.putAll(map);
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-    }
-    return props;
-  }
-
-  private String getString(SqlNode n) {
-    return n == null ? null : SqlLiteral.stringValue(n);
-  }
-
-  @SuppressWarnings("unchecked")
-  public List<ColumnDefinition> fieldList() {
-    return (List<ColumnDefinition>)((List<? extends SqlNode>)fieldList.getList());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlDDLKeywords.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlDDLKeywords.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlDDLKeywords.java
deleted file mode 100644
index 3112e53..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlDDLKeywords.java
+++ /dev/null
@@ -1,27 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.parser;
-
-import org.apache.calcite.sql.SqlLiteral;
-
-/**
- * Define the keywords that can occur in a CREATE TABLE statement
- */
-public enum SqlDDLKeywords implements SqlLiteral.SqlSymbol {
-  PRIMARY
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/StormParser.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/StormParser.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/StormParser.java
deleted file mode 100644
index 8444e1e..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/StormParser.java
+++ /dev/null
@@ -1,47 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.parser;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.calcite.config.Lex;
-import org.apache.storm.sql.parser.impl.StormParserImpl;
-
-import java.io.StringReader;
-
-public class StormParser {
-  public static final int DEFAULT_IDENTIFIER_MAX_LENGTH = 128;
-  private final StormParserImpl impl;
-
-  public StormParser(String s) {
-    this.impl = new StormParserImpl(new StringReader(s));
-    this.impl.setTabSize(1);
-    this.impl.setQuotedCasing(Lex.ORACLE.quotedCasing);
-    this.impl.setUnquotedCasing(Lex.ORACLE.unquotedCasing);
-    this.impl.setIdentifierMaxLength(DEFAULT_IDENTIFIER_MAX_LENGTH);
-    /*
-     *  By default parser uses [ ] for quoting identifiers. Switching to DQID (double quoted identifiers)
-     *  is needed for array and map access (m['x'] = 1 or arr[2] = 10 etc) to work.
-     */
-    this.impl.switchTo("DQID");
-  }
-
-  @VisibleForTesting
-  public StormParserImpl impl() {
-    return impl;
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/UnparseUtil.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/UnparseUtil.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/UnparseUtil.java
deleted file mode 100644
index 834fe7c..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/UnparseUtil.java
+++ /dev/null
@@ -1,60 +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.sql.parser;
-
-import org.apache.calcite.sql.SqlIdentifier;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlNodeList;
-import org.apache.calcite.sql.SqlWriter;
-
-class UnparseUtil {
-  private final SqlWriter writer;
-  private final int leftPrec;
-  private final int rightPrec;
-
-  UnparseUtil(SqlWriter writer, int leftPrec, int rightPrec) {
-    this.writer = writer;
-    this.leftPrec = leftPrec;
-    this.rightPrec = rightPrec;
-  }
-
-  UnparseUtil keyword(String... keywords) {
-    for (String k : keywords) {
-      writer.keyword(k);
-    }
-    return this;
-  }
-
-  UnparseUtil node(SqlNode n) {
-    n.unparse(writer, leftPrec, rightPrec);
-    return this;
-  }
-
-  UnparseUtil nodeList(SqlNodeList l) {
-    writer.keyword("(");
-    if (l.size() > 0) {
-      l.get(0).unparse(writer, leftPrec, rightPrec);
-      for (int i = 1; i < l.size(); ++i) {
-        writer.keyword(",");
-        l.get(i).unparse(writer, leftPrec, rightPrec);
-      }
-    }
-    writer.keyword(")");
-    return this;
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelDataTypeSystem.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelDataTypeSystem.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelDataTypeSystem.java
deleted file mode 100644
index 437877c..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelDataTypeSystem.java
+++ /dev/null
@@ -1,37 +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.sql.planner;
-
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.calcite.rel.type.RelDataTypeSystemImpl;
-
-public class StormRelDataTypeSystem extends RelDataTypeSystemImpl {
-    public static final RelDataTypeSystem STORM_REL_DATATYPE_SYSTEM = new StormRelDataTypeSystem();
-
-    @Override
-    public int getMaxNumericScale() {
-        return 38;
-    }
-
-    @Override
-    public int getMaxNumericPrecision() {
-        return 38;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelUtils.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelUtils.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelUtils.java
deleted file mode 100644
index 40bbacd..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelUtils.java
+++ /dev/null
@@ -1,68 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.planner;
-
-import org.apache.calcite.plan.RelOptUtil;
-import org.apache.calcite.plan.volcano.RelSubset;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.sql.SqlExplainLevel;
-import org.apache.storm.sql.planner.trident.rel.TridentRel;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.atomic.AtomicInteger;
-
-public class StormRelUtils {
-    private static final Logger LOG = LoggerFactory.getLogger(StormRelUtils.class);
-
-    private static final AtomicInteger sequence = new AtomicInteger(0);
-    private static final AtomicInteger classSequence = new AtomicInteger(0);
-
-    public static String getStageName(TridentRel relNode) {
-        return relNode.getClass().getSimpleName().toUpperCase() + "_" + relNode.getId() + "_" + sequence.getAndIncrement();
-    }
-
-    public static String getClassName(TridentRel relNode) {
-        return "Generated_" + relNode.getClass().getSimpleName().toUpperCase() + "_" + relNode.getId() + "_" +
-                classSequence.getAndIncrement();
-    }
-
-    public static TridentRel getStormRelInput(RelNode input) {
-        if (input instanceof RelSubset) {
-            // go with known best input
-            input = ((RelSubset) input).getBest();
-        }
-        return (TridentRel) input;
-    }
-
-    public static String explain(final RelNode rel) {
-        return explain(rel, SqlExplainLevel.EXPPLAN_ATTRIBUTES);
-    }
-
-    public static String explain(final RelNode rel, SqlExplainLevel detailLevel) {
-        String explain = "";
-        try {
-            explain = RelOptUtil.toString(rel);
-        } catch (StackOverflowError e) {
-            LOG.error("StackOverflowError occurred while extracting plan. Please report it to the dev@ mailing list.");
-            LOG.error("RelNode " + rel + " ExplainLevel " + detailLevel, e);
-            LOG.error("Forcing plan to empty string and continue... SQL Runner may not working properly after.");
-        }
-        return explain;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/UnsupportedOperatorsVisitor.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/UnsupportedOperatorsVisitor.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/UnsupportedOperatorsVisitor.java
deleted file mode 100644
index 258fe72..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/UnsupportedOperatorsVisitor.java
+++ /dev/null
@@ -1,24 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.planner;
-
-import org.apache.calcite.sql.util.SqlShuttle;
-
-public class UnsupportedOperatorsVisitor extends SqlShuttle {
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormCalcRelBase.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormCalcRelBase.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormCalcRelBase.java
deleted file mode 100644
index 1ea7912..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormCalcRelBase.java
+++ /dev/null
@@ -1,32 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.planner.rel;
-
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Calc;
-import org.apache.calcite.rex.RexProgram;
-import org.apache.storm.sql.planner.rel.StormRelNode;
-
-public abstract class StormCalcRelBase extends Calc implements StormRelNode {
-    protected StormCalcRelBase(RelOptCluster cluster, RelTraitSet traits, RelNode child, RexProgram program) {
-        super(cluster, traits, child, program);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormFilterRelBase.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormFilterRelBase.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormFilterRelBase.java
deleted file mode 100644
index 4e28460..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormFilterRelBase.java
+++ /dev/null
@@ -1,32 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.planner.rel;
-
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Filter;
-import org.apache.calcite.rex.RexNode;
-import org.apache.storm.sql.planner.rel.StormRelNode;
-
-public abstract class StormFilterRelBase extends Filter implements StormRelNode {
-    protected StormFilterRelBase(RelOptCluster cluster, RelTraitSet traits, RelNode child, RexNode condition) {
-        super(cluster, traits, child, condition);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormJoinRelBase.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormJoinRelBase.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormJoinRelBase.java
deleted file mode 100644
index 1458ce7..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormJoinRelBase.java
+++ /dev/null
@@ -1,36 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.planner.rel;
-
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.CorrelationId;
-import org.apache.calcite.rel.core.Join;
-import org.apache.calcite.rel.core.JoinRelType;
-import org.apache.calcite.rex.RexNode;
-import org.apache.storm.sql.planner.rel.StormRelNode;
-
-import java.util.Set;
-
-public abstract class StormJoinRelBase extends Join implements StormRelNode {
-    protected StormJoinRelBase(RelOptCluster cluster, RelTraitSet traitSet, RelNode left, RelNode right, RexNode condition, Set<CorrelationId> variablesSet, JoinRelType joinType) {
-        super(cluster, traitSet, left, right, condition, variablesSet, joinType);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormProjectRelBase.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormProjectRelBase.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormProjectRelBase.java
deleted file mode 100644
index f4f23dc..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormProjectRelBase.java
+++ /dev/null
@@ -1,35 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.planner.rel;
-
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexNode;
-import org.apache.storm.sql.planner.rel.StormRelNode;
-
-import java.util.List;
-
-public abstract class StormProjectRelBase extends Project implements StormRelNode {
-    protected StormProjectRelBase(RelOptCluster cluster, RelTraitSet traits, RelNode input, List<? extends RexNode> projects, RelDataType rowType) {
-        super(cluster, traits, input, projects, rowType);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormRelNode.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormRelNode.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormRelNode.java
deleted file mode 100644
index 9327868..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormRelNode.java
+++ /dev/null
@@ -1,24 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.planner.rel;
-
-import org.apache.calcite.rel.RelNode;
-
-public interface StormRelNode extends RelNode {
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamInsertRelBase.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamInsertRelBase.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamInsertRelBase.java
deleted file mode 100644
index df47f3f..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamInsertRelBase.java
+++ /dev/null
@@ -1,36 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.planner.rel;
-
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.prepare.Prepare;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.TableModify;
-import org.apache.calcite.rex.RexNode;
-import org.apache.storm.sql.planner.rel.StormRelNode;
-
-import java.util.List;
-
-public abstract class StormStreamInsertRelBase extends TableModify implements StormRelNode {
-    protected StormStreamInsertRelBase(RelOptCluster cluster, RelTraitSet traits, RelOptTable table, Prepare.CatalogReader catalogReader, RelNode child, Operation operation, List<String> updateColumnList, List<RexNode> sourceExpressionList, boolean flattened) {
-        super(cluster, traits, table, catalogReader, child, operation, updateColumnList, sourceExpressionList, flattened);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamScanRelBase.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamScanRelBase.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamScanRelBase.java
deleted file mode 100644
index 36c62b2..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamScanRelBase.java
+++ /dev/null
@@ -1,34 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.planner.rel;
-
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.core.TableScan;
-import org.apache.storm.sql.planner.rel.StormRelNode;
-
-public abstract class StormStreamScanRelBase extends TableScan implements StormRelNode {
-
-    // FIXME: define Table class and table.unwrap() to get it
-
-    protected StormStreamScanRelBase(RelOptCluster cluster, RelTraitSet traitSet, RelOptTable table) {
-        super(cluster, traitSet, table);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/QueryPlanner.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/QueryPlanner.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/QueryPlanner.java
deleted file mode 100644
index f98fb02..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/QueryPlanner.java
+++ /dev/null
@@ -1,156 +0,0 @@
-/*
- *
- *  * Licensed to the Apache Software Foundation (ASF) under one
- *  * or more contributor license agreements.  See the NOTICE file
- *  * distributed with this work for additional information
- *  * regarding copyright ownership.  The ASF licenses this file
- *  * to you under the Apache License, Version 2.0 (the
- *  * "License"); you may not use this file except in compliance
- *  * with the License.  You may obtain a copy of the License at
- *  *
- *  * http://www.apache.org/licenses/LICENSE-2.0
- *  *
- *  * Unless required by applicable law or agreed to in writing, software
- *  * distributed under the License is distributed on an "AS IS" BASIS,
- *  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  * See the License for the specific language governing permissions and
- *  * limitations under the License.
- *
- */
-package org.apache.storm.sql.planner.trident;
-
-import org.apache.calcite.DataContext;
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.calcite.plan.Contexts;
-import org.apache.calcite.plan.ConventionTraitDef;
-import org.apache.calcite.plan.RelTraitDef;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.prepare.CalciteCatalogReader;
-import org.apache.calcite.rel.RelCollationTraitDef;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlOperatorTable;
-import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.sql.parser.SqlParseException;
-import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
-import org.apache.calcite.tools.FrameworkConfig;
-import org.apache.calcite.tools.Frameworks;
-import org.apache.calcite.tools.Planner;
-import org.apache.calcite.tools.RelConversionException;
-import org.apache.calcite.tools.ValidationException;
-import org.apache.storm.sql.javac.CompilingClassLoader;
-import org.apache.storm.sql.planner.StormRelDataTypeSystem;
-import org.apache.storm.sql.planner.UnsupportedOperatorsVisitor;
-import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
-import org.apache.storm.sql.planner.trident.rel.TridentRel;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.sql.AbstractTridentProcessor;
-import org.apache.storm.trident.Stream;
-import org.apache.storm.trident.TridentTopology;
-import org.apache.storm.trident.fluent.IAggregatableStream;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-public class QueryPlanner {
-
-    public static final int STORM_REL_CONVERSION_RULES = 1;
-
-    private final Planner planner;
-
-    private final JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(
-            RelDataTypeSystem.DEFAULT);
-
-    public QueryPlanner(SchemaPlus schema) {
-        final List<RelTraitDef> traitDefs = new ArrayList<RelTraitDef>();
-
-        traitDefs.add(ConventionTraitDef.INSTANCE);
-        traitDefs.add(RelCollationTraitDef.INSTANCE);
-
-        List<SqlOperatorTable> sqlOperatorTables = new ArrayList<>();
-        sqlOperatorTables.add(SqlStdOperatorTable.instance());
-        sqlOperatorTables.add(new CalciteCatalogReader(CalciteSchema.from(schema),
-                false,
-                Collections.<String>emptyList(), typeFactory));
-
-        FrameworkConfig config = Frameworks.newConfigBuilder()
-                .defaultSchema(schema)
-                .operatorTable(new ChainedSqlOperatorTable(sqlOperatorTables))
-                .traitDefs(traitDefs)
-                .context(Contexts.EMPTY_CONTEXT)
-                .ruleSets(TridentStormRuleSets.getRuleSets())
-                .costFactory(null)
-                .typeSystem(StormRelDataTypeSystem.STORM_REL_DATATYPE_SYSTEM)
-                .build();
-        this.planner = Frameworks.getPlanner(config);
-    }
-
-    public AbstractTridentProcessor compile(Map<String, ISqlTridentDataSource> sources, String query) throws Exception {
-        TridentRel relNode = getPlan(query);
-
-        TridentPlanCreator tridentPlanCreator = new TridentPlanCreator(sources, new RexBuilder(typeFactory));
-        relNode.tridentPlan(tridentPlanCreator);
-
-        final TridentTopology topology = tridentPlanCreator.getTopology();
-        final IAggregatableStream lastStream = tridentPlanCreator.pop();
-        final DataContext dc = tridentPlanCreator.getDataContext();
-        final List<CompilingClassLoader> cls = tridentPlanCreator.getClassLoaders();
-
-        return new AbstractTridentProcessor() {
-            @Override
-            public TridentTopology build() {
-                return topology;
-            }
-
-            @Override
-            public Stream outputStream() {
-                return lastStream.toStream();
-            }
-
-            @Override
-            public DataContext getDataContext() {
-                return dc;
-            }
-
-            @Override
-            public List<CompilingClassLoader> getClassLoaders() {
-                return cls;
-            }
-        };
-    }
-
-    public TridentRel getPlan(String query) throws ValidationException, RelConversionException, SqlParseException {
-        return (TridentRel) validateAndConvert(planner.parse(query));
-    }
-
-    private RelNode validateAndConvert(SqlNode sqlNode) throws ValidationException, RelConversionException {
-        SqlNode validated = validateNode(sqlNode);
-        RelNode relNode = convertToRelNode(validated);
-        return convertToStormRel(relNode);
-    }
-
-    private RelNode convertToStormRel(RelNode relNode) throws RelConversionException {
-        RelTraitSet traitSet = relNode.getTraitSet();
-        traitSet = traitSet.simplify();
-
-        // PlannerImpl.transform() optimizes RelNode with ruleset
-        return planner.transform(STORM_REL_CONVERSION_RULES, traitSet.plus(TridentLogicalConvention.INSTANCE), relNode);
-    }
-
-    private RelNode convertToRelNode(SqlNode sqlNode) throws RelConversionException {
-        return planner.rel(sqlNode).rel;
-    }
-
-    private SqlNode validateNode(SqlNode sqlNode) throws ValidationException {
-        SqlNode validatedSqlNode = planner.validate(sqlNode);
-        validatedSqlNode.accept(new UnsupportedOperatorsVisitor());
-        return validatedSqlNode;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentPlanCreator.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentPlanCreator.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentPlanCreator.java
deleted file mode 100644
index aa30552..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentPlanCreator.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.planner.trident;
-
-import org.apache.calcite.DataContext;
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.interpreter.Scalar;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexProgram;
-import org.apache.storm.sql.compiler.RexNodeToJavaCodeCompiler;
-import org.apache.storm.sql.javac.CompilingClassLoader;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.sql.runtime.calcite.DebuggableExecutableExpression;
-import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
-import org.apache.storm.sql.runtime.calcite.StormDataContext;
-import org.apache.storm.trident.TridentTopology;
-import org.apache.storm.trident.fluent.IAggregatableStream;
-
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Deque;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-public class TridentPlanCreator {
-    private final Map<String, ISqlTridentDataSource> sources;
-    private final JavaTypeFactory typeFactory;
-    private final RexNodeToJavaCodeCompiler rexCompiler;
-    private final DataContext dataContext;
-    private final TridentTopology topology;
-
-    private final Deque<IAggregatableStream> streamStack = new ArrayDeque<>();
-    private final List<CompilingClassLoader> classLoaders = new ArrayList<>();
-
-    public TridentPlanCreator(Map<String, ISqlTridentDataSource> sources, RexBuilder rexBuilder) {
-        this.sources = sources;
-        this.rexCompiler = new RexNodeToJavaCodeCompiler(rexBuilder);
-        this.typeFactory = (JavaTypeFactory) rexBuilder.getTypeFactory();
-
-        this.topology = new TridentTopology();
-        this.dataContext = new StormDataContext();
-    }
-
-    public void addStream(IAggregatableStream stream) throws Exception {
-        push(stream);
-    }
-
-    public IAggregatableStream pop() {
-        return streamStack.pop();
-    }
-
-    public Map<String, ISqlTridentDataSource> getSources() {
-        return sources;
-    }
-
-    public DataContext getDataContext() {
-        return dataContext;
-    }
-
-    public JavaTypeFactory getTypeFactory() {
-        return typeFactory;
-    }
-
-    public TridentTopology getTopology() {
-        return topology;
-    }
-
-    public ExecutableExpression createScalarInstance(List<RexNode> nodes, RelDataType inputRowType, String className)
-            throws CompilingClassLoader.CompilerException, ClassNotFoundException, IllegalAccessException, InstantiationException {
-        String expr = rexCompiler.compile(nodes, inputRowType, className);
-        CompilingClassLoader classLoader = new CompilingClassLoader(
-                getLastClassLoader(), className, expr, null);
-        ExecutableExpression instance = (ExecutableExpression) classLoader.loadClass(className).newInstance();
-        addClassLoader(classLoader);
-        return new DebuggableExecutableExpression(instance, expr);
-    }
-
-    public ExecutableExpression createScalarInstance(RexProgram program, String className)
-            throws CompilingClassLoader.CompilerException, ClassNotFoundException, IllegalAccessException, InstantiationException {
-        String expr = rexCompiler.compile(program, className);
-        CompilingClassLoader classLoader = new CompilingClassLoader(
-                getLastClassLoader(), className, expr, null);
-        ExecutableExpression instance = (ExecutableExpression) classLoader.loadClass(className).newInstance();
-        addClassLoader(classLoader);
-        return new DebuggableExecutableExpression(instance, expr);
-    }
-
-    private void push(IAggregatableStream stream) {
-        streamStack.push(stream);
-    }
-
-    public void addClassLoader(CompilingClassLoader compilingClassLoader) {
-        this.classLoaders.add(compilingClassLoader);
-    }
-
-    public ClassLoader getLastClassLoader() {
-        if (this.classLoaders.size() > 0) {
-            return this.classLoaders.get(this.classLoaders.size() - 1);
-        } else {
-            return this.getClass().getClassLoader();
-        }
-    }
-
-    public List<CompilingClassLoader> getClassLoaders() {
-        return classLoaders;
-    }
-}


[20/23] storm git commit: STORM-2453 Move non-connectors into the top directory

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/FluxBuilder.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/FluxBuilder.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/FluxBuilder.java
deleted file mode 100644
index e79dfb7..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/FluxBuilder.java
+++ /dev/null
@@ -1,630 +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.flux;
-
-import org.apache.storm.Config;
-import org.apache.storm.generated.StormTopology;
-import org.apache.storm.grouping.CustomStreamGrouping;
-import org.apache.storm.topology.*;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.utils.Utils;
-import org.apache.storm.flux.model.*;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.lang.reflect.*;
-import java.util.*;
-
-public class FluxBuilder {
-    private static Logger LOG = LoggerFactory.getLogger(FluxBuilder.class);
-
-    /**
-     * Given a topology definition, return a populated `org.apache.storm.Config` instance.
-     *
-     * @param topologyDef
-     * @return
-     */
-    public static Config buildConfig(TopologyDef topologyDef) {
-        // merge contents of `config` into topology config
-        Config conf = new Config();
-        conf.putAll(topologyDef.getConfig());
-        return conf;
-    }
-
-    /**
-     * Given a topology definition, return a Storm topology that can be run either locally or remotely.
-     *
-     * @param context
-     * @return
-     * @throws IllegalAccessException
-     * @throws InstantiationException
-     * @throws ClassNotFoundException
-     * @throws NoSuchMethodException
-     * @throws InvocationTargetException
-     */
-    public static StormTopology buildTopology(ExecutionContext context) throws IllegalAccessException,
-            InstantiationException, ClassNotFoundException, NoSuchMethodException, InvocationTargetException, NoSuchFieldException {
-
-        StormTopology topology = null;
-        TopologyDef topologyDef = context.getTopologyDef();
-
-        if(!topologyDef.validate()){
-            throw new IllegalArgumentException("Invalid topology config. Spouts, bolts and streams cannot be " +
-                    "defined in the same configuration as a topologySource.");
-        }
-
-        // build components that may be referenced by spouts, bolts, etc.
-        // the map will be a String --> Object where the object is a fully
-        // constructed class instance
-        buildComponents(context);
-
-        if(topologyDef.isDslTopology()) {
-            // This is a DSL (YAML, etc.) topology...
-            LOG.info("Detected DSL topology...");
-
-            TopologyBuilder builder = new TopologyBuilder();
-
-            // create spouts
-            buildSpouts(context, builder);
-
-            // we need to be able to lookup bolts by id, then switch based
-            // on whether they are IBasicBolt or IRichBolt instances
-            buildBolts(context);
-
-            // process stream definitions
-            buildStreamDefinitions(context, builder);
-
-            topology = builder.createTopology();
-        } else {
-            // user class supplied...
-            // this also provides a bridge to Trident...
-            LOG.info("A topology source has been specified...");
-            ObjectDef def = topologyDef.getTopologySource();
-            topology = buildExternalTopology(def, context);
-        }
-        return topology;
-    }
-
-    /**
-     * Given a `java.lang.Object` instance and a method name, attempt to find a method that matches the input
-     * parameter: `java.util.Map` or `org.apache.storm.Config`.
-     *
-     * @param topologySource object to inspect for the specified method
-     * @param methodName name of the method to look for
-     * @return
-     * @throws NoSuchMethodException
-     */
-    private static Method findGetTopologyMethod(Object topologySource, String methodName) throws NoSuchMethodException {
-        Class clazz = topologySource.getClass();
-        Method[] methods =  clazz.getMethods();
-        ArrayList<Method> candidates = new ArrayList<Method>();
-        for(Method method : methods){
-            if(!method.getName().equals(methodName)){
-                continue;
-            }
-            if(!method.getReturnType().equals(StormTopology.class)){
-                continue;
-            }
-            Class[] paramTypes = method.getParameterTypes();
-            if(paramTypes.length != 1){
-                continue;
-            }
-            if(paramTypes[0].isAssignableFrom(Map.class) || paramTypes[0].isAssignableFrom(Config.class)){
-                candidates.add(method);
-            }
-        }
-
-        if(candidates.size() == 0){
-            throw new IllegalArgumentException("Unable to find method '" + methodName + "' method in class: " + clazz.getName());
-        } else if (candidates.size() > 1){
-            LOG.warn("Found multiple candidate methods in class '" + clazz.getName() + "'. Using the first one found");
-        }
-
-        return candidates.get(0);
-    }
-
-    /**
-     * @param context
-     * @param builder
-     */
-    private static void buildStreamDefinitions(ExecutionContext context, TopologyBuilder builder)
-            throws ClassNotFoundException, NoSuchMethodException, InvocationTargetException, InstantiationException,
-            IllegalAccessException, NoSuchFieldException {
-        TopologyDef topologyDef = context.getTopologyDef();
-        // process stream definitions
-        HashMap<String, BoltDeclarer> declarers = new HashMap<String, BoltDeclarer>();
-        for (StreamDef stream : topologyDef.getStreams()) {
-            Object boltObj = context.getBolt(stream.getTo());
-            BoltDeclarer declarer = declarers.get(stream.getTo());
-            if (boltObj instanceof IRichBolt) {
-                if(declarer == null) {
-                    declarer = builder.setBolt(stream.getTo(),
-                            (IRichBolt) boltObj,
-                            topologyDef.parallelismForBolt(stream.getTo()));
-                    declarers.put(stream.getTo(), declarer);
-                }
-            } else if (boltObj instanceof IBasicBolt) {
-                if(declarer == null) {
-                    declarer = builder.setBolt(
-                            stream.getTo(),
-                            (IBasicBolt) boltObj,
-                            topologyDef.parallelismForBolt(stream.getTo()));
-                    declarers.put(stream.getTo(), declarer);
-                }
-            } else if (boltObj instanceof IWindowedBolt) {
-                if(declarer == null) {
-                    declarer = builder.setBolt(
-                            stream.getTo(),
-                            (IWindowedBolt) boltObj,
-                            topologyDef.parallelismForBolt(stream.getTo()));
-                    declarers.put(stream.getTo(), declarer);
-                }
-            } else if (boltObj instanceof IStatefulBolt) {
-                if(declarer == null) {
-                    declarer = builder.setBolt(
-                            stream.getTo(),
-                            (IStatefulBolt) boltObj,
-                            topologyDef.parallelismForBolt(stream.getTo()));
-                    declarers.put(stream.getTo(), declarer);
-                }
-            } else {
-                throw new IllegalArgumentException("Class does not appear to be a bolt: " +
-                        boltObj.getClass().getName());
-            }
-
-            GroupingDef grouping = stream.getGrouping();
-            // if the streamId is defined, use it for the grouping, otherwise assume storm's default stream
-            String streamId = (grouping.getStreamId() == null ? Utils.DEFAULT_STREAM_ID : grouping.getStreamId());
-
-
-            switch (grouping.getType()) {
-                case SHUFFLE:
-                    declarer.shuffleGrouping(stream.getFrom(), streamId);
-                    break;
-                case FIELDS:
-                    //TODO check for null grouping args
-                    declarer.fieldsGrouping(stream.getFrom(), streamId, new Fields(grouping.getArgs()));
-                    break;
-                case ALL:
-                    declarer.allGrouping(stream.getFrom(), streamId);
-                    break;
-                case DIRECT:
-                    declarer.directGrouping(stream.getFrom(), streamId);
-                    break;
-                case GLOBAL:
-                    declarer.globalGrouping(stream.getFrom(), streamId);
-                    break;
-                case LOCAL_OR_SHUFFLE:
-                    declarer.localOrShuffleGrouping(stream.getFrom(), streamId);
-                    break;
-                case NONE:
-                    declarer.noneGrouping(stream.getFrom(), streamId);
-                    break;
-                case CUSTOM:
-                    declarer.customGrouping(stream.getFrom(), streamId,
-                            buildCustomStreamGrouping(stream.getGrouping().getCustomClass(), context));
-                    break;
-                default:
-                    throw new UnsupportedOperationException("unsupported grouping type: " + grouping);
-            }
-        }
-    }
-
-    private static void applyProperties(ObjectDef bean, Object instance, ExecutionContext context) throws
-            IllegalAccessException, InvocationTargetException, NoSuchFieldException {
-        List<PropertyDef> props = bean.getProperties();
-        Class clazz = instance.getClass();
-        if (props != null) {
-            for (PropertyDef prop : props) {
-                Object value = prop.isReference() ? context.getComponent(prop.getRef()) : prop.getValue();
-                Method setter = findSetter(clazz, prop.getName(), value);
-                if (setter != null) {
-                    LOG.debug("found setter, attempting to invoke");
-                    // invoke setter
-                    setter.invoke(instance, new Object[]{value});
-                } else {
-                    // look for a public instance variable
-                    LOG.debug("no setter found. Looking for a public instance variable...");
-                    Field field = findPublicField(clazz, prop.getName(), value);
-                    if (field != null) {
-                        field.set(instance, value);
-                    }
-                }
-            }
-        }
-    }
-
-    private static Field findPublicField(Class clazz, String property, Object arg) throws NoSuchFieldException {
-        Field field = clazz.getField(property);
-        return field;
-    }
-
-    private static Method findSetter(Class clazz, String property, Object arg) {
-        String setterName = toSetterName(property);
-        Method retval = null;
-        Method[] methods = clazz.getMethods();
-        for (Method method : methods) {
-            if (setterName.equals(method.getName())) {
-                LOG.debug("Found setter method: " + method.getName());
-                retval = method;
-            }
-        }
-        return retval;
-    }
-
-    private static String toSetterName(String name) {
-        return "set" + name.substring(0, 1).toUpperCase() + name.substring(1, name.length());
-    }
-
-    private static List<Object> resolveReferences(List<Object> args, ExecutionContext context) {
-        LOG.debug("Checking arguments for references.");
-        List<Object> cArgs = new ArrayList<Object>();
-        // resolve references
-        for (Object arg : args) {
-            if (arg instanceof BeanReference) {
-                cArgs.add(context.getComponent(((BeanReference) arg).getId()));
-            } else if (arg instanceof BeanListReference) {
-                List<Object> components = new ArrayList<>();
-                BeanListReference ref = (BeanListReference) arg;
-                for (String id : ref.getIds()) {
-                    components.add(context.getComponent(id));
-                }
-
-                LOG.debug("BeanListReference resolved as {}", components);
-                cArgs.add(components);
-            } else {
-                cArgs.add(arg);
-            }
-        }
-        return cArgs;
-    }
-
-    private static Object buildObject(ObjectDef def, ExecutionContext context) throws ClassNotFoundException,
-            IllegalAccessException, InstantiationException, NoSuchMethodException, InvocationTargetException, NoSuchFieldException {
-        Class clazz = Class.forName(def.getClassName());
-        Object obj = null;
-        if (def.hasConstructorArgs()) {
-            LOG.debug("Found constructor arguments in definition: " + def.getConstructorArgs().getClass().getName());
-            List<Object> cArgs = def.getConstructorArgs();
-            if(def.hasReferences()){
-                cArgs = resolveReferences(cArgs, context);
-            }
-            Constructor con = findCompatibleConstructor(cArgs, clazz);
-            if (con != null) {
-                LOG.debug("Found something seemingly compatible, attempting invocation...");
-                obj = con.newInstance(getArgsWithListCoercian(cArgs, con.getParameterTypes()));
-            } else {
-                String msg = String.format("Couldn't find a suitable constructor for class '%s' with arguments '%s'.",
-                        clazz.getName(),
-                        cArgs);
-                throw new IllegalArgumentException(msg);
-            }
-        } else {
-            obj = clazz.newInstance();
-        }
-        applyProperties(def, obj, context);
-        invokeConfigMethods(def, obj, context);
-        return obj;
-    }
-
-    private static StormTopology buildExternalTopology(ObjectDef def, ExecutionContext context)
-            throws ClassNotFoundException, IllegalAccessException, InstantiationException, NoSuchMethodException,
-            InvocationTargetException, NoSuchFieldException {
-
-        Object topologySource = buildObject(def, context);
-
-        String methodName = context.getTopologyDef().getTopologySource().getMethodName();
-        Method getTopology = findGetTopologyMethod(topologySource, methodName);
-        if(getTopology.getParameterTypes()[0].equals(Config.class)){
-            Config config = new Config();
-            config.putAll(context.getTopologyDef().getConfig());
-            return (StormTopology) getTopology.invoke(topologySource, config);
-        } else {
-            return (StormTopology) getTopology.invoke(topologySource, context.getTopologyDef().getConfig());
-        }
-    }
-
-    private static CustomStreamGrouping buildCustomStreamGrouping(ObjectDef def, ExecutionContext context)
-            throws ClassNotFoundException,
-            IllegalAccessException, InstantiationException, NoSuchMethodException, InvocationTargetException, NoSuchFieldException {
-        Object grouping = buildObject(def, context);
-        return (CustomStreamGrouping)grouping;
-    }
-
-    /**
-     * Given a topology definition, resolve and instantiate all components found and return a map
-     * keyed by the component id.
-     */
-    private static void buildComponents(ExecutionContext context) throws ClassNotFoundException, NoSuchMethodException,
-            IllegalAccessException, InvocationTargetException, InstantiationException, NoSuchFieldException {
-        Collection<BeanDef> cDefs = context.getTopologyDef().getComponents();
-        if (cDefs != null) {
-            for (BeanDef bean : cDefs) {
-                Object obj = buildObject(bean, context);
-                context.addComponent(bean.getId(), obj);
-            }
-        }
-    }
-
-
-    private static void buildSpouts(ExecutionContext context, TopologyBuilder builder) throws ClassNotFoundException,
-            NoSuchMethodException, InvocationTargetException, InstantiationException, IllegalAccessException, NoSuchFieldException {
-        for (SpoutDef sd : context.getTopologyDef().getSpouts()) {
-            IRichSpout spout = buildSpout(sd, context);
-            builder.setSpout(sd.getId(), spout, sd.getParallelism());
-            context.addSpout(sd.getId(), spout);
-        }
-    }
-
-    /**
-     * Given a spout definition, return a Storm spout implementation by attempting to find a matching constructor
-     * in the given spout class. Perform list to array conversion as necessary.
-     */
-    private static IRichSpout buildSpout(SpoutDef def, ExecutionContext context) throws ClassNotFoundException,
-            IllegalAccessException, InstantiationException, NoSuchMethodException, InvocationTargetException, NoSuchFieldException {
-        return (IRichSpout)buildObject(def, context);
-    }
-
-    /**
-     * Given a list of bolt definitions, build a map of Storm bolts with the bolt definition id as the key.
-     * Attempt to coerce the given constructor arguments to a matching bolt constructor as much as possible.
-     */
-    private static void buildBolts(ExecutionContext context) throws ClassNotFoundException, IllegalAccessException,
-            InstantiationException, NoSuchMethodException, InvocationTargetException, NoSuchFieldException {
-        for (BoltDef def : context.getTopologyDef().getBolts()) {
-            Class clazz = Class.forName(def.getClassName());
-            Object bolt = buildObject(def, context);
-            context.addBolt(def.getId(), bolt);
-        }
-    }
-
-    /**
-     * Given a list of constructor arguments, and a target class, attempt to find a suitable constructor.
-     *
-     */
-    private static Constructor findCompatibleConstructor(List<Object> args, Class target) throws NoSuchMethodException {
-        Constructor retval = null;
-        int eligibleCount = 0;
-
-        LOG.debug("Target class: {}, constructor args: {}", target.getName(), args);
-        Constructor[] cons = target.getDeclaredConstructors();
-
-        for (Constructor con : cons) {
-            Class[] paramClasses = con.getParameterTypes();
-            if (paramClasses.length == args.size()) {
-                LOG.debug("found constructor with same number of args..");
-                boolean invokable = canInvokeWithArgs(args, con.getParameterTypes());
-                if (invokable) {
-                    retval = con;
-                    eligibleCount++;
-                }
-                LOG.debug("** invokable --> {}", invokable);
-            } else {
-                LOG.debug("Skipping constructor with wrong number of arguments.");
-            }
-        }
-        if (eligibleCount > 1) {
-            LOG.warn("Found multiple invokable constructors for class {}, given arguments {}. Using the last one found.",
-                    target, args);
-        }
-        return retval;
-    }
-
-
-    public static void invokeConfigMethods(ObjectDef bean, Object instance, ExecutionContext context)
-            throws InvocationTargetException, IllegalAccessException {
-
-        List<ConfigMethodDef> methodDefs = bean.getConfigMethods();
-        if(methodDefs == null || methodDefs.size() == 0){
-            return;
-        }
-        Class clazz = instance.getClass();
-        for(ConfigMethodDef methodDef : methodDefs){
-            List<Object> args = methodDef.getArgs();
-            if (args == null){
-                args = new ArrayList();
-            }
-            if(methodDef.hasReferences()){
-                args = resolveReferences(args, context);
-            }
-            String methodName = methodDef.getName();
-            Method method = findCompatibleMethod(args, clazz, methodName);
-            if(method != null) {
-                Object[] methodArgs = getArgsWithListCoercian(args, method.getParameterTypes());
-                method.invoke(instance, methodArgs);
-            } else {
-                String msg = String.format("Unable to find configuration method '%s' in class '%s' with arguments %s.",
-                        new Object[]{methodName, clazz.getName(), args});
-                throw new IllegalArgumentException(msg);
-            }
-        }
-    }
-
-    private static Method findCompatibleMethod(List<Object> args, Class target, String methodName){
-        Method retval = null;
-        int eligibleCount = 0;
-
-        LOG.debug("Target class: {}, methodName: {}, args: {}", target.getName(), methodName, args);
-        Method[] methods = target.getMethods();
-
-        for (Method method : methods) {
-            Class[] paramClasses = method.getParameterTypes();
-            if (paramClasses.length == args.size() && method.getName().equals(methodName)) {
-                LOG.debug("found constructor with same number of args..");
-                boolean invokable = false;
-                if (args.size() == 0){
-                    // it's a method with zero args
-                    invokable = true;
-                } else {
-                    invokable = canInvokeWithArgs(args, method.getParameterTypes());
-                }
-                if (invokable) {
-                    retval = method;
-                    eligibleCount++;
-                }
-                LOG.debug("** invokable --> {}", invokable);
-            } else {
-                LOG.debug("Skipping method with wrong number of arguments.");
-            }
-        }
-        if (eligibleCount > 1) {
-            LOG.warn("Found multiple invokable methods for class {}, method {}, given arguments {}. " +
-                            "Using the last one found.",
-                            new Object[]{target, methodName, args});
-        }
-        return retval;
-    }
-
-    /**
-     * Given a java.util.List of contructor/method arguments, and a list of parameter types, attempt to convert the
-     * list to an java.lang.Object array that can be used to invoke the constructor. If an argument needs
-     * to be coerced from a List to an Array, do so.
-     */
-    private static Object[] getArgsWithListCoercian(List<Object> args, Class[] parameterTypes) {
-//        Class[] parameterTypes = constructor.getParameterTypes();
-        if (parameterTypes.length != args.size()) {
-            throw new IllegalArgumentException("Contructor parameter count does not egual argument size.");
-        }
-        Object[] constructorParams = new Object[args.size()];
-
-        // loop through the arguments, if we hit a list that has to be convered to an array,
-        // perform the conversion
-        for (int i = 0; i < args.size(); i++) {
-            Object obj = args.get(i);
-            Class paramType = parameterTypes[i];
-            Class objectType = obj.getClass();
-            LOG.debug("Comparing parameter class {} to object class {} to see if assignment is possible.",
-                    paramType, objectType);
-            if (paramType.equals(objectType)) {
-                LOG.debug("They are the same class.");
-                constructorParams[i] = args.get(i);
-                continue;
-            }
-            if (paramType.isAssignableFrom(objectType)) {
-                LOG.debug("Assignment is possible.");
-                constructorParams[i] = args.get(i);
-                continue;
-            }
-            if (isPrimitiveBoolean(paramType) && Boolean.class.isAssignableFrom(objectType)){
-                LOG.debug("Its a primitive boolean.");
-                Boolean bool = (Boolean)args.get(i);
-                constructorParams[i] = bool.booleanValue();
-                continue;
-            }
-            if(isPrimitiveNumber(paramType) && Number.class.isAssignableFrom(objectType)){
-                LOG.debug("Its a primitive number.");
-                Number num = (Number)args.get(i);
-                if(paramType == Float.TYPE){
-                    constructorParams[i] = num.floatValue();
-                } else if (paramType == Double.TYPE) {
-                    constructorParams[i] = num.doubleValue();
-                } else if (paramType == Long.TYPE) {
-                    constructorParams[i] = num.longValue();
-                } else if (paramType == Integer.TYPE) {
-                    constructorParams[i] = num.intValue();
-                } else if (paramType == Short.TYPE) {
-                    constructorParams[i] = num.shortValue();
-                } else if (paramType == Byte.TYPE) {
-                    constructorParams[i] = num.byteValue();
-                } else {
-                    constructorParams[i] = args.get(i);
-                }
-                continue;
-            }
-
-            // enum conversion
-            if(paramType.isEnum() && objectType.equals(String.class)){
-                LOG.debug("Yes, will convert a String to enum");
-                constructorParams[i] = Enum.valueOf(paramType, (String)args.get(i));
-                continue;
-            }
-
-            // List to array conversion
-            if (paramType.isArray() && List.class.isAssignableFrom(objectType)) {
-                // TODO more collection content type checking
-                LOG.debug("Conversion appears possible...");
-                List list = (List) obj;
-                LOG.debug("Array Type: {}, List type: {}", paramType.getComponentType(), list.get(0).getClass());
-
-                // create an array of the right type
-                Object newArrayObj = Array.newInstance(paramType.getComponentType(), list.size());
-                for (int j = 0; j < list.size(); j++) {
-                    Array.set(newArrayObj, j, list.get(j));
-
-                }
-                constructorParams[i] = newArrayObj;
-                LOG.debug("After conversion: {}", constructorParams[i]);
-            }
-        }
-        return constructorParams;
-    }
-
-
-    /**
-     * Determine if the given constructor/method parameter types are compatible given arguments List. Consider if
-     * list coercian can make it possible.
-     *
-     * @param args
-     * @param parameterTypes
-     * @return
-     */
-    private static boolean canInvokeWithArgs(List<Object> args, Class[] parameterTypes) {
-        if (parameterTypes.length != args.size()) {
-            LOG.warn("parameter types were the wrong size");
-            return false;
-        }
-
-        for (int i = 0; i < args.size(); i++) {
-            Object obj = args.get(i);
-            if (obj == null) {
-                throw new IllegalArgumentException("argument shouldn't be null - index: " + i);
-            }
-            Class paramType = parameterTypes[i];
-            Class objectType = obj.getClass();
-            LOG.debug("Comparing parameter class {} to object class {} to see if assignment is possible.",
-                    paramType, objectType);
-            if (paramType.equals(objectType)) {
-                LOG.debug("Yes, they are the same class.");
-            } else if (paramType.isAssignableFrom(objectType)) {
-                LOG.debug("Yes, assignment is possible.");
-            } else if (isPrimitiveBoolean(paramType) && Boolean.class.isAssignableFrom(objectType)){
-                LOG.debug("Yes, assignment is possible.");
-            } else if(isPrimitiveNumber(paramType) && Number.class.isAssignableFrom(objectType)){
-                LOG.debug("Yes, assignment is possible.");
-            } else if(paramType.isEnum() && objectType.equals(String.class)){
-                LOG.debug("Yes, will convert a String to enum");
-            } else if (paramType.isArray() && List.class.isAssignableFrom(objectType)) {
-                // TODO more collection content type checking
-                LOG.debug("Assignment is possible if we convert a List to an array.");
-                LOG.debug("Array Type: {}, List type: {}", paramType.getComponentType(), ((List) obj).get(0).getClass());
-            } else {
-                return false;
-            }
-        }
-        return true;
-    }
-
-    public static boolean isPrimitiveNumber(Class clazz){
-        return clazz.isPrimitive() && !clazz.equals(boolean.class);
-    }
-
-    public static boolean isPrimitiveBoolean(Class clazz){
-        return clazz.isPrimitive() && clazz.equals(boolean.class);
-    }
-}
-

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/api/TopologySource.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/api/TopologySource.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/api/TopologySource.java
deleted file mode 100644
index 2777854..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/api/TopologySource.java
+++ /dev/null
@@ -1,39 +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.flux.api;
-
-
-import org.apache.storm.generated.StormTopology;
-
-import java.util.Map;
-
-/**
- * Marker interface for objects that can produce `StormTopology` objects.
- *
- * If a `topology-source` class implements the `getTopology()` method, Flux will
- * call that method. Otherwise, it will introspect the given class and look for a
- * similar method that produces a `StormTopology` instance.
- *
- * Note that it is not strictly necessary for a class to implement this interface.
- * If a class defines a method with a similar signature, Flux should be able to find
- * and invoke it.
- *
- */
-public interface TopologySource {
-    public StormTopology getTopology(Map<String, Object> config);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanDef.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanDef.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanDef.java
deleted file mode 100644
index 72ca5ae..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanDef.java
+++ /dev/null
@@ -1,39 +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.flux.model;
-
-import java.util.ArrayList;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * A representation of a Java object that is uniquely identifyable, and given a className, constructor arguments,
- * and properties, can be instantiated.
- */
-public class BeanDef extends ObjectDef {
-    private String id;
-
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanListReference.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanListReference.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanListReference.java
deleted file mode 100644
index 652210c..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanListReference.java
+++ /dev/null
@@ -1,37 +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.flux.model;
-
-import java.util.List;
-
-/**
- * A bean list reference is a list of bean reference.
- */
-public class BeanListReference {
-    public List<String> ids;
-
-    public BeanListReference(){}
-
-    public BeanListReference(List<String> ids){
-        this.ids = ids;
-    }
-
-    public List<String> getIds() {
-        return ids;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanReference.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanReference.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanReference.java
deleted file mode 100644
index bd236f1..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BeanReference.java
+++ /dev/null
@@ -1,39 +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.flux.model;
-
-/**
- * A bean reference is simply a string pointer to another id.
- */
-public class BeanReference {
-    public String id;
-
-    public BeanReference(){}
-
-    public BeanReference(String id){
-        this.id = id;
-    }
-
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BoltDef.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BoltDef.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BoltDef.java
deleted file mode 100644
index 362abf1..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/BoltDef.java
+++ /dev/null
@@ -1,24 +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.flux.model;
-
-/**
- * Bean representation of a Storm bolt.
- */
-public class BoltDef extends VertexDef {
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/ConfigMethodDef.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/ConfigMethodDef.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/ConfigMethodDef.java
deleted file mode 100644
index 69cabc3..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/ConfigMethodDef.java
+++ /dev/null
@@ -1,67 +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.flux.model;
-
-import java.util.ArrayList;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-
-public class ConfigMethodDef {
-    private String name;
-    private List<Object> args;
-    private boolean hasReferences = false;
-
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    public List<Object> getArgs() {
-        return args;
-    }
-
-    public void setArgs(List<Object> args) {
-
-        List<Object> newVal = new ArrayList<Object>();
-        for(Object obj : args){
-            if(obj instanceof LinkedHashMap){
-                Map map = (Map)obj;
-                if(map.containsKey("ref") && map.size() == 1){
-                    newVal.add(new BeanReference((String)map.get("ref")));
-                    this.hasReferences = true;
-                } else if (map.containsKey("reflist") && map.size() == 1) {
-                    newVal.add(new BeanListReference((List<String>) map.get("reflist")));
-                    this.hasReferences = true;
-                } else {
-                    newVal.add(obj);
-                }
-            } else {
-                newVal.add(obj);
-            }
-        }
-        this.args = newVal;
-    }
-
-    public boolean hasReferences(){
-        return this.hasReferences;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/ExecutionContext.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/ExecutionContext.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/ExecutionContext.java
deleted file mode 100644
index 1520006..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/ExecutionContext.java
+++ /dev/null
@@ -1,77 +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.flux.model;
-
-import org.apache.storm.Config;
-import org.apache.storm.task.IBolt;
-import org.apache.storm.topology.IRichSpout;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Container for all the objects required to instantiate a topology.
- */
-public class ExecutionContext {
-    // parsed Topology definition
-    TopologyDef topologyDef;
-
-    // Storm config
-    private Config config;
-
-    // components
-    private List<Object> compontents;
-    // indexed by id
-    private Map<String, Object> componentMap = new HashMap<String, Object>();
-
-    private Map<String, IRichSpout> spoutMap = new HashMap<String, IRichSpout>();
-
-    private List<IBolt> bolts;
-    private Map<String, Object> boltMap = new HashMap<String, Object>();
-
-    public ExecutionContext(TopologyDef topologyDef, Config config){
-        this.topologyDef = topologyDef;
-        this.config = config;
-    }
-
-    public TopologyDef getTopologyDef(){
-        return this.topologyDef;
-    }
-
-    public void addSpout(String id, IRichSpout spout){
-        this.spoutMap.put(id, spout);
-    }
-
-    public void addBolt(String id, Object bolt){
-        this.boltMap.put(id, bolt);
-    }
-
-    public Object getBolt(String id){
-        return this.boltMap.get(id);
-    }
-
-    public void addComponent(String id, Object value){
-        this.componentMap.put(id, value);
-    }
-
-    public Object getComponent(String id){
-        return this.componentMap.get(id);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/GroupingDef.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/GroupingDef.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/GroupingDef.java
deleted file mode 100644
index e4fac8e..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/GroupingDef.java
+++ /dev/null
@@ -1,77 +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.flux.model;
-
-import java.util.List;
-
-/**
- * Bean representation of a Storm stream grouping.
- */
-public class GroupingDef {
-
-    /**
-     * Types of stream groupings Storm allows
-     */
-    public static enum Type {
-        ALL,
-        CUSTOM,
-        DIRECT,
-        SHUFFLE,
-        LOCAL_OR_SHUFFLE,
-        FIELDS,
-        GLOBAL,
-        NONE
-    }
-
-    private Type type;
-    private String streamId;
-    private List<String> args;
-    private ObjectDef customClass;
-
-    public List<String> getArgs() {
-        return args;
-    }
-
-    public void setArgs(List<String> args) {
-        this.args = args;
-    }
-
-    public Type getType() {
-        return type;
-    }
-
-    public void setType(Type type) {
-        this.type = type;
-    }
-
-    public String getStreamId() {
-        return streamId;
-    }
-
-    public void setStreamId(String streamId) {
-        this.streamId = streamId;
-    }
-
-    public ObjectDef getCustomClass() {
-        return customClass;
-    }
-
-    public void setCustomClass(ObjectDef customClass) {
-        this.customClass = customClass;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/IncludeDef.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/IncludeDef.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/IncludeDef.java
deleted file mode 100644
index 23fd9d2..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/IncludeDef.java
+++ /dev/null
@@ -1,54 +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.flux.model;
-
-/**
- * Represents an include. Includes can be either a file or a classpath resource.
- *
- * If an include is marked as `override=true` then existing properties will be replaced.
- *
- */
-public class IncludeDef {
-    private boolean resource = false;
-    boolean override = false;
-    private String file;
-
-    public boolean isResource() {
-        return resource;
-    }
-
-    public void setResource(boolean resource) {
-        this.resource = resource;
-    }
-
-    public String getFile() {
-        return file;
-    }
-
-    public void setFile(String file) {
-        this.file = file;
-    }
-
-    public boolean isOverride() {
-        return override;
-    }
-
-    public void setOverride(boolean override) {
-        this.override = override;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/ObjectDef.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/ObjectDef.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/ObjectDef.java
deleted file mode 100644
index 04a7e8a..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/ObjectDef.java
+++ /dev/null
@@ -1,95 +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.flux.model;
-
-import org.apache.storm.Config;
-
-import java.util.ArrayList;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * A representation of a Java object that given a className, constructor arguments,
- * and properties, can be instantiated.
- */
-public class ObjectDef {
-    private String className;
-    private List<Object> constructorArgs;
-    private boolean hasReferences;
-    private List<PropertyDef> properties;
-    private List<ConfigMethodDef> configMethods;
-
-    public String getClassName() {
-        return className;
-    }
-
-    public void setClassName(String className) {
-        this.className = className;
-    }
-
-    public List<Object> getConstructorArgs() {
-        return constructorArgs;
-    }
-
-    public void setConstructorArgs(List<Object> constructorArgs) {
-
-        List<Object> newVal = new ArrayList<Object>();
-        for(Object obj : constructorArgs){
-            if(obj instanceof LinkedHashMap){
-                Map map = (Map)obj;
-                if(map.containsKey("ref") && map.size() == 1) {
-                    newVal.add(new BeanReference((String) map.get("ref")));
-                    this.hasReferences = true;
-                } else if (map.containsKey("reflist") && map.size() == 1) {
-                    newVal.add(new BeanListReference((List<String>) map.get("reflist")));
-                    this.hasReferences = true;
-                } else {
-                    newVal.add(obj);
-                }
-            } else {
-                newVal.add(obj);
-            }
-        }
-        this.constructorArgs = newVal;
-    }
-
-    public boolean hasConstructorArgs(){
-        return this.constructorArgs != null && this.constructorArgs.size() > 0;
-    }
-
-    public boolean hasReferences(){
-        return this.hasReferences;
-    }
-
-    public List<PropertyDef> getProperties() {
-        return properties;
-    }
-
-    public void setProperties(List<PropertyDef> properties) {
-        this.properties = properties;
-    }
-
-    public List<ConfigMethodDef> getConfigMethods() {
-        return configMethods;
-    }
-
-    public void setConfigMethods(List<ConfigMethodDef> configMethods) {
-        this.configMethods = configMethods;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/PropertyDef.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/PropertyDef.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/PropertyDef.java
deleted file mode 100644
index f3d7704..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/PropertyDef.java
+++ /dev/null
@@ -1,58 +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.flux.model;
-
-public class PropertyDef {
-    private String name;
-    private Object value;
-    private String ref;
-
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    public Object getValue() {
-        return value;
-    }
-
-    public void setValue(Object value) {
-        if(this.ref != null){
-            throw new IllegalStateException("A property can only have a value OR a reference, not both.");
-        }
-        this.value = value;
-    }
-
-    public String getRef() {
-        return ref;
-    }
-
-    public void setRef(String ref) {
-        if(this.value != null){
-            throw new IllegalStateException("A property can only have a value OR a reference, not both.");
-        }
-        this.ref = ref;
-    }
-
-    public boolean isReference(){
-        return this.ref != null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/SpoutDef.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/SpoutDef.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/SpoutDef.java
deleted file mode 100644
index 277c601..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/SpoutDef.java
+++ /dev/null
@@ -1,24 +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.flux.model;
-
-/**
- * Bean representation of a Storm spout.
- */
-public class SpoutDef extends VertexDef {
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/StreamDef.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/StreamDef.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/StreamDef.java
deleted file mode 100644
index da80f1c..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/StreamDef.java
+++ /dev/null
@@ -1,64 +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.flux.model;
-
-/**
- * Represents a stream of tuples from one Storm component (Spout or Bolt) to another (an edge in the topology DAG).
- *
- * Required fields are `from` and `to`, which define the source and destination, and the stream `grouping`.
- *
- */
-public class StreamDef {
-
-    private String name; // not used, placeholder for GUI, etc.
-    private String from;
-    private String to;
-    private GroupingDef grouping;
-
-    public String getTo() {
-        return to;
-    }
-
-    public void setTo(String to) {
-        this.to = to;
-    }
-
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    public String getFrom() {
-        return from;
-    }
-
-    public void setFrom(String from) {
-        this.from = from;
-    }
-
-    public GroupingDef getGrouping() {
-        return grouping;
-    }
-
-    public void setGrouping(GroupingDef grouping) {
-        this.grouping = grouping;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologyDef.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologyDef.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologyDef.java
deleted file mode 100644
index 86614f1..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologyDef.java
+++ /dev/null
@@ -1,216 +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.flux.model;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.*;
-
-/**
- * Bean represenation of a topology.
- *
- * It consists of the following:
- *   1. The topology name
- *   2. A `java.util.Map` representing the `org.apache.storm.config` for the topology
- *   3. A list of spout definitions
- *   4. A list of bolt definitions
- *   5. A list of stream definitions that define the flow between spouts and bolts.
- *
- */
-public class TopologyDef {
-    private static Logger LOG = LoggerFactory.getLogger(TopologyDef.class);
-
-    private String name;
-    private Map<String, BeanDef> componentMap = new LinkedHashMap<String, BeanDef>(); // not required
-    private List<IncludeDef> includes; // not required
-    private Map<String, Object> config = new HashMap<String, Object>();
-
-    // a "topology source" is a class that can produce a `StormTopology` thrift object.
-    private TopologySourceDef topologySource;
-
-    // the following are required if we're defining a core storm topology DAG in YAML, etc.
-    private Map<String, BoltDef> boltMap = new LinkedHashMap<String, BoltDef>();
-    private Map<String, SpoutDef> spoutMap = new LinkedHashMap<String, SpoutDef>();
-    private List<StreamDef> streams = new ArrayList<StreamDef>();
-
-
-    public String getName() {
-        return name;
-    }
-
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    public void setName(String name, boolean override){
-        if(this.name == null || override){
-            this.name = name;
-        } else {
-            LOG.warn("Ignoring attempt to set property 'name' with override == false.");
-        }
-    }
-
-    public List<SpoutDef> getSpouts() {
-        ArrayList<SpoutDef> retval = new ArrayList<SpoutDef>();
-        retval.addAll(this.spoutMap.values());
-        return retval;
-    }
-
-    public void setSpouts(List<SpoutDef> spouts) {
-        this.spoutMap = new LinkedHashMap<String, SpoutDef>();
-        for(SpoutDef spout : spouts){
-            this.spoutMap.put(spout.getId(), spout);
-        }
-    }
-
-    public List<BoltDef> getBolts() {
-        ArrayList<BoltDef> retval = new ArrayList<BoltDef>();
-        retval.addAll(this.boltMap.values());
-        return retval;
-    }
-
-    public void setBolts(List<BoltDef> bolts) {
-        this.boltMap = new LinkedHashMap<String, BoltDef>();
-        for(BoltDef bolt : bolts){
-            this.boltMap.put(bolt.getId(), bolt);
-        }
-    }
-
-    public List<StreamDef> getStreams() {
-        return streams;
-    }
-
-    public void setStreams(List<StreamDef> streams) {
-        this.streams = streams;
-    }
-
-    public Map<String, Object> getConfig() {
-        return config;
-    }
-
-    public void setConfig(Map<String, Object> config) {
-        this.config = config;
-    }
-
-    public List<BeanDef> getComponents() {
-        ArrayList<BeanDef> retval = new ArrayList<BeanDef>();
-        retval.addAll(this.componentMap.values());
-        return retval;
-    }
-
-    public void setComponents(List<BeanDef> components) {
-        this.componentMap = new LinkedHashMap<String, BeanDef>();
-        for(BeanDef component : components){
-            this.componentMap.put(component.getId(), component);
-        }
-    }
-
-    public List<IncludeDef> getIncludes() {
-        return includes;
-    }
-
-    public void setIncludes(List<IncludeDef> includes) {
-        this.includes = includes;
-    }
-
-    // utility methods
-    public int parallelismForBolt(String boltId){
-        return this.boltMap.get(boltId).getParallelism();
-    }
-
-    public BoltDef getBoltDef(String id){
-        return this.boltMap.get(id);
-    }
-
-    public SpoutDef getSpoutDef(String id){
-        return this.spoutMap.get(id);
-    }
-
-    public BeanDef getComponent(String id){
-        return this.componentMap.get(id);
-    }
-
-    // used by includes implementation
-    public void addAllBolts(List<BoltDef> bolts, boolean override){
-        for(BoltDef bolt : bolts){
-            String id = bolt.getId();
-            if(this.boltMap.get(id) == null || override) {
-                this.boltMap.put(bolt.getId(), bolt);
-            } else {
-                LOG.warn("Ignoring attempt to create bolt '{}' with override == false.", id);
-            }
-        }
-    }
-
-    public void addAllSpouts(List<SpoutDef> spouts, boolean override){
-        for(SpoutDef spout : spouts){
-            String id = spout.getId();
-            if(this.spoutMap.get(id) == null || override) {
-                this.spoutMap.put(spout.getId(), spout);
-            } else {
-                LOG.warn("Ignoring attempt to create spout '{}' with override == false.", id);
-            }
-        }
-    }
-
-    public void addAllComponents(List<BeanDef> components, boolean override) {
-        for(BeanDef bean : components){
-            String id = bean.getId();
-            if(this.componentMap.get(id) == null || override) {
-                this.componentMap.put(bean.getId(), bean);
-            } else {
-                LOG.warn("Ignoring attempt to create component '{}' with override == false.", id);
-            }
-        }
-    }
-
-    public void addAllStreams(List<StreamDef> streams, boolean override) {
-        //TODO figure out how we want to deal with overrides. Users may want to add streams even when overriding other
-        // properties. For now we just add them blindly which could lead to a potentially invalid topology.
-        this.streams.addAll(streams);
-    }
-
-    public TopologySourceDef getTopologySource() {
-        return topologySource;
-    }
-
-    public void setTopologySource(TopologySourceDef topologySource) {
-        this.topologySource = topologySource;
-    }
-
-    public boolean isDslTopology(){
-        return this.topologySource == null;
-    }
-
-
-    public boolean validate(){
-        boolean hasSpouts = this.spoutMap != null && this.spoutMap.size() > 0;
-        boolean hasBolts = this.boltMap != null && this.boltMap.size() > 0;
-        boolean hasStreams = this.streams != null && this.streams.size() > 0;
-        boolean hasSpoutsBoltsStreams = hasStreams && hasBolts && hasSpouts;
-        // you cant define a topologySource and a DSL topology at the same time...
-        if (!isDslTopology() && ((hasSpouts || hasBolts || hasStreams))) {
-            return false;
-        }
-        if(isDslTopology() && (hasSpouts && hasBolts && hasStreams)) {
-            return true;
-        }
-        return true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologySourceDef.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologySourceDef.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologySourceDef.java
deleted file mode 100644
index d6a2f57..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologySourceDef.java
+++ /dev/null
@@ -1,36 +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.flux.model;
-
-public class TopologySourceDef extends ObjectDef {
-    public static final String DEFAULT_METHOD_NAME = "getTopology";
-
-    private String methodName;
-
-    public TopologySourceDef(){
-        this.methodName = DEFAULT_METHOD_NAME;
-    }
-
-    public String getMethodName() {
-        return methodName;
-    }
-
-    public void setMethodName(String methodName) {
-        this.methodName = methodName;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/VertexDef.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/VertexDef.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/VertexDef.java
deleted file mode 100644
index e71bcc2..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/model/VertexDef.java
+++ /dev/null
@@ -1,36 +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.flux.model;
-
-/**
- * Abstract parent class of component definitions
- * (spouts/bolts)
- */
-public abstract class VertexDef extends BeanDef {
-
-    // default parallelism to 1 so if it's ommitted, the topology will still function.
-    private int parallelism = 1;
-
-    public int getParallelism() {
-        return parallelism;
-    }
-
-    public void setParallelism(int parallelism) {
-        this.parallelism = parallelism;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/parser/FluxParser.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/parser/FluxParser.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/parser/FluxParser.java
deleted file mode 100644
index 2a18474..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/parser/FluxParser.java
+++ /dev/null
@@ -1,208 +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.flux.parser;
-
-import org.apache.storm.flux.api.TopologySource;
-import org.apache.storm.flux.model.BoltDef;
-import org.apache.storm.flux.model.IncludeDef;
-import org.apache.storm.flux.model.SpoutDef;
-import org.apache.storm.flux.model.TopologyDef;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.yaml.snakeyaml.TypeDescription;
-import org.yaml.snakeyaml.Yaml;
-import org.yaml.snakeyaml.constructor.Constructor;
-
-import java.io.ByteArrayOutputStream;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.ByteBuffer;
-import java.util.Map;
-import java.util.Properties;
-
-public class FluxParser {
-    private static final Logger LOG = LoggerFactory.getLogger(FluxParser.class);
-
-    private FluxParser(){}
-
-    // TODO refactor input stream processing (see parseResource() method).
-    public static TopologyDef parseFile(String inputFile, boolean dumpYaml, boolean processIncludes,
-    	String propertiesFile, boolean envSub) throws IOException {
-   
-        FileInputStream in = new FileInputStream(inputFile);
-        TopologyDef topology = parseInputStream(in, dumpYaml, processIncludes, propertiesFile, envSub);
-        in.close();
-        
-        return topology;
-    }
-
-    public static TopologyDef parseResource(String resource, boolean dumpYaml, boolean processIncludes,
-    	String propertiesFile, boolean envSub) throws IOException {
-        
-        InputStream in = FluxParser.class.getResourceAsStream(resource);
-        TopologyDef topology = parseInputStream(in, dumpYaml, processIncludes, propertiesFile, envSub);
-        in.close();
-        
-        return topology;
-    }
-    
-    public static TopologyDef parseInputStream(InputStream inputStream, boolean dumpYaml, boolean processIncludes,
-    	String propertiesFile, boolean envSub) throws IOException {
-		
-	Yaml yaml = yaml();
-    	
-	if (inputStream == null) {
-		LOG.error("Unable to load input stream");
-		System.exit(1);
-	}
-		
-	TopologyDef topology = loadYaml(yaml, inputStream, propertiesFile, envSub);
-		
-	if (dumpYaml) {
-		dumpYaml(topology, yaml);
-	}
-	
-	if (processIncludes) {
-		return processIncludes(yaml, topology, propertiesFile, envSub);
-	} else {
-		return topology;
-	}
-    }
-
-    private static TopologyDef loadYaml(Yaml yaml, InputStream in, String propsFile, boolean envSubstitution) throws IOException {
-        ByteArrayOutputStream bos = new ByteArrayOutputStream();
-        LOG.info("loading YAML from input stream...");
-        int b = -1;
-        while((b = in.read()) != -1){
-            bos.write(b);
-        }
-
-        // TODO substitution implementation is not exactly efficient or kind to memory...
-        String str = bos.toString();
-        // properties file substitution
-        if(propsFile != null){
-            LOG.info("Performing property substitution.");
-            InputStream propsIn = new FileInputStream(propsFile);
-            Properties props = new Properties();
-            props.load(propsIn);
-            for(Object key : props.keySet()){
-                str = str.replace("${" + key + "}", props.getProperty((String)key));
-            }
-        } else {
-            LOG.info("Not performing property substitution.");
-        }
-
-        // environment variable substitution
-        if(envSubstitution){
-            LOG.info("Performing environment variable substitution...");
-            Map<String, String> envs = System.getenv();
-            for(String key : envs.keySet()){
-                str = str.replace("${ENV-" + key + "}", envs.get(key));
-            }
-        } else {
-            LOG.info("Not performing environment variable substitution.");
-        }
-        return (TopologyDef)yaml.load(str);
-    }
-
-    private static void dumpYaml(TopologyDef topology, Yaml yaml){
-        System.out.println("Configuration (interpreted): \n" + yaml.dump(topology));
-    }
-
-    private static Yaml yaml(){
-        Constructor constructor = new Constructor(TopologyDef.class);
-
-        TypeDescription topologyDescription = new TypeDescription(TopologyDef.class);
-        topologyDescription.putListPropertyType("spouts", SpoutDef.class);
-        topologyDescription.putListPropertyType("bolts", BoltDef.class);
-        topologyDescription.putListPropertyType("includes", IncludeDef.class);
-        constructor.addTypeDescription(topologyDescription);
-
-        Yaml  yaml = new Yaml(constructor);
-        return yaml;
-    }
-
-    /**
-     *
-     * @param yaml the yaml parser for parsing the include file(s)
-     * @param topologyDef the topology definition containing (possibly zero) includes
-     * @return The TopologyDef with includes resolved.
-     */
-    private static TopologyDef processIncludes(Yaml yaml, TopologyDef topologyDef, String propsFile, boolean envSub)
-            throws IOException {
-        //TODO support multiple levels of includes
-        if(topologyDef.getIncludes() != null) {
-            for (IncludeDef include : topologyDef.getIncludes()){
-                TopologyDef includeTopologyDef = null;
-                if (include.isResource()) {
-                    LOG.info("Loading includes from resource: {}", include.getFile());
-                    includeTopologyDef = parseResource(include.getFile(), true, false, propsFile, envSub);
-                } else {
-                    LOG.info("Loading includes from file: {}", include.getFile());
-                    includeTopologyDef = parseFile(include.getFile(), true, false, propsFile, envSub);
-                }
-
-                // if overrides are disabled, we won't replace anything that already exists
-                boolean override = include.isOverride();
-                // name
-                if(includeTopologyDef.getName() != null){
-                    topologyDef.setName(includeTopologyDef.getName(), override);
-                }
-
-                // config
-                if(includeTopologyDef.getConfig() != null) {
-                    //TODO move this logic to the model class
-                    Map<String, Object> config = topologyDef.getConfig();
-                    Map<String, Object> includeConfig = includeTopologyDef.getConfig();
-                    if(override) {
-                        config.putAll(includeTopologyDef.getConfig());
-                    } else {
-                        for(String key : includeConfig.keySet()){
-                            if(config.containsKey(key)){
-                                LOG.warn("Ignoring attempt to set topology config property '{}' with override == false", key);
-                            }
-                            else {
-                                config.put(key, includeConfig.get(key));
-                            }
-                        }
-                    }
-                }
-
-                //component overrides
-                if(includeTopologyDef.getComponents() != null){
-                    topologyDef.addAllComponents(includeTopologyDef.getComponents(), override);
-                }
-                //bolt overrides
-                if(includeTopologyDef.getBolts() != null){
-                    topologyDef.addAllBolts(includeTopologyDef.getBolts(), override);
-                }
-                //spout overrides
-                if(includeTopologyDef.getSpouts() != null) {
-                    topologyDef.addAllSpouts(includeTopologyDef.getSpouts(), override);
-                }
-                //stream overrides
-                //TODO streams should be uniquely identifiable
-                if(includeTopologyDef.getStreams() != null) {
-                    topologyDef.addAllStreams(includeTopologyDef.getStreams(), override);
-                }
-            } // end include processing
-        }
-        return topologyDef;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/resources/splash.txt
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/resources/splash.txt b/external/flux/flux-core/src/main/resources/splash.txt
deleted file mode 100644
index 337931a..0000000
--- a/external/flux/flux-core/src/main/resources/splash.txt
+++ /dev/null
@@ -1,9 +0,0 @@
-\u2588\u2588\u2588\u2588\u2588\u2588\u2588\u2557\u2588\u2588\u2557     \u2588\u2588\u2557   \u2588\u2588\u2557\u2588\u2588\u2557  \u2588\u2588\u2557
-\u2588\u2588\u2554\u2550\u2550\u2550\u2550\u255d\u2588\u2588\u2551     \u2588\u2588\u2551   \u2588\u2588\u2551\u255a\u2588\u2588\u2557\u2588\u2588\u2554\u255d
-\u2588\u2588\u2588\u2588\u2588\u2557  \u2588\u2588\u2551     \u2588\u2588\u2551   \u2588\u2588\u2551 \u255a\u2588\u2588\u2588\u2554\u255d
-\u2588\u2588\u2554\u2550\u2550\u255d  \u2588\u2588\u2551     \u2588\u2588\u2551   \u2588\u2588\u2551 \u2588\u2588\u2554\u2588\u2588\u2557
-\u2588\u2588\u2551     \u2588\u2588\u2588\u2588\u2588\u2588\u2588\u2557\u255a\u2588\u2588\u2588\u2588\u2588\u2588\u2554\u255d\u2588\u2588\u2554\u255d \u2588\u2588\u2557
-\u255a\u2550\u255d     \u255a\u2550\u2550\u2550\u2550\u2550\u2550\u255d \u255a\u2550\u2550\u2550\u2550\u2550\u255d \u255a\u2550\u255d  \u255a\u2550\u255d
-+-         Apache Storm        -+
-+-  data FLow User eXperience  -+
-Version: ${project.version}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/java/org/apache/storm/flux/FluxBuilderTest.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/java/org/apache/storm/flux/FluxBuilderTest.java b/external/flux/flux-core/src/test/java/org/apache/storm/flux/FluxBuilderTest.java
deleted file mode 100644
index ff67867..0000000
--- a/external/flux/flux-core/src/test/java/org/apache/storm/flux/FluxBuilderTest.java
+++ /dev/null
@@ -1,31 +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.flux;
-
-import org.junit.Test;
-import static org.junit.Assert.*;
-
-public class FluxBuilderTest {
-
-    @Test
-    public void testIsPrimitiveNumber() throws Exception {
-        assertTrue(FluxBuilder.isPrimitiveNumber(int.class));
-        assertFalse(FluxBuilder.isPrimitiveNumber(boolean.class));
-        assertFalse(FluxBuilder.isPrimitiveNumber(String.class));
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/java/org/apache/storm/flux/IntegrationTest.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/java/org/apache/storm/flux/IntegrationTest.java b/external/flux/flux-core/src/test/java/org/apache/storm/flux/IntegrationTest.java
deleted file mode 100644
index c5807f8..0000000
--- a/external/flux/flux-core/src/test/java/org/apache/storm/flux/IntegrationTest.java
+++ /dev/null
@@ -1,39 +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.flux;
-
-import org.junit.Test;
-
-public class IntegrationTest {
-
-    private static boolean skipTest = true;
-
-    static {
-        String skipStr = System.getProperty("skipIntegration");
-        if(skipStr != null && skipStr.equalsIgnoreCase("false")){
-            skipTest = false;
-        }
-    }
-
-    @Test
-    public void testRunTopologySource() throws Exception {
-        if(!skipTest) {
-            Flux.main(new String[]{"-s", "30000", "src/test/resources/configs/existing-topology.yaml"});
-        }
-    }
-}


[12/23] storm git commit: STORM-2453 Move non-connectors into the top directory

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/StormSqlFunctions.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/StormSqlFunctions.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/StormSqlFunctions.java
deleted file mode 100644
index a373483..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/StormSqlFunctions.java
+++ /dev/null
@@ -1,34 +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.sql.runtime;
-
-public class StormSqlFunctions {
-  public static Boolean eq(Object b0, Object b1) {
-    if (b0 == null || b1 == null) {
-      return null;
-    }
-    return b0.equals(b1);
-  }
-
-  public static Boolean ne(Object b0, Object b1) {
-    if (b0 == null || b1 == null) {
-      return null;
-    }
-    return !b0.equals(b1);
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/DebuggableExecutableExpression.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/DebuggableExecutableExpression.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/DebuggableExecutableExpression.java
deleted file mode 100644
index e78f354..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/DebuggableExecutableExpression.java
+++ /dev/null
@@ -1,45 +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.sql.runtime.calcite;
-
-import org.apache.calcite.interpreter.Context;
-
-public class DebuggableExecutableExpression implements ExecutableExpression {
-    private ExecutableExpression delegate;
-    private String delegateCode;
-
-    public DebuggableExecutableExpression(ExecutableExpression delegate, String delegateCode) {
-        this.delegate = delegate;
-        this.delegateCode = delegateCode;
-    }
-
-    @Override
-    public Object execute(Context context) {
-        return delegate.execute(context);
-    }
-
-    @Override
-    public void execute(Context context, Object[] results) {
-        delegate.execute(context, results);
-    }
-
-    public String getDelegateCode() {
-        return delegateCode;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/ExecutableExpression.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/ExecutableExpression.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/ExecutableExpression.java
deleted file mode 100644
index 8416945..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/ExecutableExpression.java
+++ /dev/null
@@ -1,31 +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.sql.runtime.calcite;
-
-import org.apache.calcite.interpreter.Context;
-
-import java.io.Serializable;
-
-/**
- * Compiled executable expression.
- */
-public interface ExecutableExpression extends Serializable {
-    Object execute(Context context);
-    void execute(Context context, Object[] results);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/StormDataContext.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/StormDataContext.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/StormDataContext.java
deleted file mode 100644
index 4861b43..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/StormDataContext.java
+++ /dev/null
@@ -1,79 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.runtime.calcite;
-
-import com.google.common.collect.ImmutableMap;
-import org.apache.calcite.DataContext;
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.linq4j.QueryProvider;
-import org.apache.calcite.runtime.Hook;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.util.Holder;
-
-import java.io.Serializable;
-import java.util.Calendar;
-import java.util.TimeZone;
-
-/**
- * This is based on SlimDataContext in Calcite, and borrow some from DataContextImpl in Calcite.
- */
-public class StormDataContext implements DataContext, Serializable {
-    private final ImmutableMap<Object, Object> map;
-
-    public StormDataContext() {
-        // Store the time at which the query started executing. The SQL
-        // standard says that functions such as CURRENT_TIMESTAMP return the
-        // same value throughout the query.
-
-        final Holder<Long> timeHolder = Holder.of(System.currentTimeMillis());
-
-        // Give a hook chance to alter the clock.
-        Hook.CURRENT_TIME.run(timeHolder);
-        final long time = timeHolder.get();
-        final TimeZone timeZone = Calendar.getInstance().getTimeZone();
-        final long localOffset = timeZone.getOffset(time);
-        final long currentOffset = localOffset;
-
-        ImmutableMap.Builder<Object, Object> builder = ImmutableMap.builder();
-        builder.put(Variable.UTC_TIMESTAMP.camelName, time)
-                .put(Variable.CURRENT_TIMESTAMP.camelName, time + currentOffset)
-                .put(Variable.LOCAL_TIMESTAMP.camelName, time + localOffset)
-                .put(Variable.TIME_ZONE.camelName, timeZone);
-        map = builder.build();
-    }
-
-    @Override
-    public SchemaPlus getRootSchema() {
-        return null;
-    }
-
-    @Override
-    public JavaTypeFactory getTypeFactory() {
-        return null;
-    }
-
-    @Override
-    public QueryProvider getQueryProvider() {
-        return null;
-    }
-
-    @Override
-    public Object get(String name) {
-        return map.get(name);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/SocketDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/SocketDataSourcesProvider.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/SocketDataSourcesProvider.java
deleted file mode 100644
index d81e772..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/SocketDataSourcesProvider.java
+++ /dev/null
@@ -1,101 +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.sql.runtime.datasource.socket;
-
-import org.apache.storm.spout.Scheme;
-import org.apache.storm.sql.runtime.DataSource;
-import org.apache.storm.sql.runtime.DataSourcesProvider;
-import org.apache.storm.sql.runtime.FieldInfo;
-import org.apache.storm.sql.runtime.IOutputSerializer;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.sql.runtime.SimpleSqlTridentConsumer;
-import org.apache.storm.sql.runtime.datasource.socket.trident.SocketState;
-import org.apache.storm.sql.runtime.datasource.socket.trident.SocketStateUpdater;
-import org.apache.storm.sql.runtime.datasource.socket.trident.TridentSocketSpout;
-import org.apache.storm.sql.runtime.serde.json.JsonSerializer;
-import org.apache.storm.sql.runtime.utils.FieldInfoUtils;
-import org.apache.storm.sql.runtime.utils.SerdeUtils;
-import org.apache.storm.trident.spout.ITridentDataSource;
-import org.apache.storm.trident.state.StateFactory;
-import org.apache.storm.trident.state.StateUpdater;
-
-import java.net.URI;
-import java.util.List;
-import java.util.Properties;
-
-/**
- * Create a Socket data source based on the URI and properties. The URI has the format of
- * socket://[host]:[port]. Both of host and port are mandatory.
- *
- * Note that it connects to given host and port, and receive the message if it's used for input source,
- * and send the message if it's used for output data source.
- */
-public class SocketDataSourcesProvider implements DataSourcesProvider {
-    @Override
-    public String scheme() {
-        return "socket";
-    }
-
-    private static class SocketTridentDataSource implements ISqlTridentDataSource {
-
-        private final String host;
-        private final int port;
-        private final Scheme scheme;
-        private final IOutputSerializer serializer;
-
-        SocketTridentDataSource(Scheme scheme, IOutputSerializer serializer, String host, int port) {
-            this.scheme = scheme;
-            this.serializer = serializer;
-            this.host = host;
-            this.port = port;
-        }
-
-        @Override
-        public ITridentDataSource getProducer() {
-            return new TridentSocketSpout(scheme, host, port);
-        }
-
-        @Override
-        public SqlTridentConsumer getConsumer() {
-            StateFactory stateFactory = new SocketState.Factory(host, port);
-            StateUpdater<SocketState> stateUpdater = new SocketStateUpdater(serializer);
-            return new SimpleSqlTridentConsumer(stateFactory, stateUpdater);
-        }
-    }
-
-    @Override
-    public DataSource construct(URI uri, String inputFormatClass, String outputFormatClass, List<FieldInfo> fields) {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass, Properties properties, List<FieldInfo> fields) {
-        String host = uri.getHost();
-        int port = uri.getPort();
-        if (port == -1) {
-            throw new RuntimeException("Port information is not available. URI: " + uri);
-        }
-
-        List<String> fieldNames = FieldInfoUtils.getFieldNames(fields);
-        Scheme scheme = SerdeUtils.getScheme(inputFormatClass, properties, fieldNames);
-        IOutputSerializer serializer = SerdeUtils.getSerializer(outputFormatClass, properties, fieldNames);
-
-        return new SocketTridentDataSource(scheme, serializer, host, port);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketState.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketState.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketState.java
deleted file mode 100644
index 3f85756..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketState.java
+++ /dev/null
@@ -1,89 +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.sql.runtime.datasource.socket.trident;
-
-import org.apache.storm.task.IMetricsContext;
-import org.apache.storm.trident.state.State;
-import org.apache.storm.trident.state.StateFactory;
-
-import java.io.BufferedWriter;
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-import java.net.Socket;
-import java.util.Map;
-
-/**
- * Trident State implementation of Socket. It only supports writing.
- */
-public class SocketState implements State {
-    /**
-     * {@inheritDoc}
-     */
-    @Override
-    public void beginCommit(Long txid) {
-        // no-op
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override
-    public void commit(Long txid) {
-        // no-op
-    }
-
-    public static class Factory implements StateFactory {
-        private final String host;
-        private final int port;
-
-        public Factory(String host, int port) {
-            this.host = host;
-            this.port = port;
-        }
-
-        @Override
-        public State makeState(Map conf, IMetricsContext metrics, int partitionIndex, int numPartitions) {
-            BufferedWriter out;
-            try {
-                Socket socket = new Socket(host, port);
-                out = new BufferedWriter(new OutputStreamWriter(socket.getOutputStream()));
-            } catch (IOException e) {
-                throw new RuntimeException("Exception while initializing socket for State. host " +
-                        host + " port " + port, e);
-            }
-
-            // State doesn't have close() and Storm actually doesn't guarantee so we can't release socket resource anyway
-            return new SocketState(out);
-        }
-    }
-
-    private BufferedWriter out;
-
-    private SocketState(BufferedWriter out) {
-        this.out = out;
-    }
-
-    public void write(String str) throws IOException {
-        out.write(str);
-    }
-
-    public void flush() throws IOException {
-        out.flush();
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketStateUpdater.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketStateUpdater.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketStateUpdater.java
deleted file mode 100644
index 3062a90..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketStateUpdater.java
+++ /dev/null
@@ -1,59 +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.sql.runtime.datasource.socket.trident;
-
-import org.apache.storm.sql.runtime.IOutputSerializer;
-import org.apache.storm.trident.operation.TridentCollector;
-import org.apache.storm.trident.state.BaseStateUpdater;
-import org.apache.storm.trident.tuple.TridentTuple;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.List;
-
-/**
- * StateUpdater for SocketState. Serializes tuple one by one and writes to socket, and finally flushes.
- */
-public class SocketStateUpdater extends BaseStateUpdater<SocketState> {
-    private static final Logger LOG = LoggerFactory.getLogger(SocketStateUpdater.class);
-
-    private final IOutputSerializer outputSerializer;
-
-    public SocketStateUpdater(IOutputSerializer outputSerializer) {
-        this.outputSerializer = outputSerializer;
-    }
-
-    @Override
-    public void updateState(SocketState state, List<TridentTuple> tuples, TridentCollector collector) {
-        try {
-            for (TridentTuple tuple : tuples) {
-                byte[] array = outputSerializer.write(tuple.getValues(), null).array();
-                String data = new String(array);
-                state.write(data + "\n");
-            }
-            state.flush();
-        } catch (IOException e) {
-            LOG.error("Error while updating state.", e);
-            collector.reportError(e);
-            throw new RuntimeException("Error while updating state.", e);
-        }
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/TridentSocketSpout.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/TridentSocketSpout.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/TridentSocketSpout.java
deleted file mode 100644
index 97f63a7..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/TridentSocketSpout.java
+++ /dev/null
@@ -1,177 +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.sql.runtime.datasource.socket.trident;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.storm.Config;
-import org.apache.storm.spout.Scheme;
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.trident.operation.TridentCollector;
-import org.apache.storm.trident.spout.IBatchSpout;
-import org.apache.storm.tuple.Fields;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.BufferedReader;
-import java.io.Closeable;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.net.Socket;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.BlockingDeque;
-import java.util.concurrent.LinkedBlockingDeque;
-
-/**
- * Trident Spout for Socket data. Only available for Storm SQL, and only use for test purposes.
- */
-public class TridentSocketSpout implements IBatchSpout {
-    private static final Logger LOG = LoggerFactory.getLogger(TridentSocketSpout.class);
-
-    private final String host;
-    private final int port;
-    private final Scheme scheme;
-
-    private volatile boolean _running = true;
-
-    private BlockingDeque<String> queue;
-    private Socket socket;
-    private Thread readerThread;
-    private BufferedReader in;
-    private ObjectMapper objectMapper;
-
-    private Map<Long, List<List<Object>>> batches;
-
-    public TridentSocketSpout(Scheme scheme, String host, int port) {
-        this.scheme = scheme;
-        this.host = host;
-        this.port = port;
-    }
-
-    @Override
-    public void open(Map conf, TopologyContext context) {
-        this.queue = new LinkedBlockingDeque<>();
-        this.objectMapper = new ObjectMapper();
-        this.batches = new HashMap<>();
-
-        try {
-            socket = new Socket(host, port);
-            in = new BufferedReader(new InputStreamReader(socket.getInputStream()));
-        } catch (IOException e) {
-            throw new RuntimeException("Error opening socket: host " + host + " port " + port);
-        }
-
-        readerThread = new Thread(new SocketReaderRunnable());
-        readerThread.start();
-    }
-
-    @Override
-    public void emitBatch(long batchId, TridentCollector collector) {
-        // read line and parse it to json
-        List<List<Object>> batch = this.batches.get(batchId);
-        if (batch == null) {
-            batch = new ArrayList<>();
-
-            while(queue.peek() != null) {
-                String line = queue.poll();
-                List<Object> values = convertLineToTuple(line);
-                if (values == null) {
-                    continue;
-                }
-
-                batch.add(values);
-            }
-
-            this.batches.put(batchId, batch);
-        }
-
-        for (List<Object> list : batch) {
-            collector.emit(list);
-        }
-    }
-
-    private List<Object> convertLineToTuple(String line) {
-        return scheme.deserialize(ByteBuffer.wrap(line.getBytes()));
-    }
-
-    @Override
-    public void ack(long batchId) {
-        this.batches.remove(batchId);
-    }
-
-    @Override
-    public void close() {
-        _running = false;
-        readerThread.interrupt();
-        queue.clear();
-
-        closeQuietly(in);
-        closeQuietly(socket);
-    }
-
-    @Override
-    public Map<String, Object> getComponentConfiguration() {
-        Config conf = new Config();
-        conf.setMaxTaskParallelism(1);
-        return conf;
-    }
-
-    @Override
-    public Fields getOutputFields() {
-        return scheme.getOutputFields();
-    }
-
-    private class SocketReaderRunnable implements Runnable {
-        public void run() {
-            while (_running) {
-                try {
-                    String line = in.readLine();
-                    if (line == null) {
-                        throw new RuntimeException("EOF reached from the socket. We can't read the data any more.");
-                    }
-
-                    queue.push(line.trim());
-                } catch (Throwable t) {
-                    // This spout is added to test purpose, so just failing fast doesn't hurt much
-                    die(t);
-                }
-            }
-        }
-    }
-
-    private void die(Throwable t) {
-        LOG.error("Halting process: TridentSocketSpout died.", t);
-        if (_running || (t instanceof Error)) { //don't exit if not running, unless it is an Error
-            System.exit(11);
-        }
-    }
-
-    private void closeQuietly(final Closeable closeable) {
-        try {
-            if (closeable != null) {
-                closeable.close();
-            }
-        } catch (final IOException ioe) {
-            // ignore
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroScheme.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroScheme.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroScheme.java
deleted file mode 100644
index 3bf1a23..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroScheme.java
+++ /dev/null
@@ -1,74 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.runtime.serde.avro;
-
-import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericDatumReader;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.io.BinaryDecoder;
-import org.apache.avro.io.DatumReader;
-import org.apache.avro.io.DecoderFactory;
-import org.apache.storm.spout.Scheme;
-import org.apache.storm.sql.runtime.utils.SerdeUtils;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.utils.Utils;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * AvroScheme uses generic(without code generation) instead of specific(with code generation) readers.
- */
-public class AvroScheme implements Scheme {
-  private final String schemaString;
-  private final List<String> fieldNames;
-  private final CachedSchemas schemas;
-
-  public AvroScheme(String schemaString, List<String> fieldNames) {
-    this.schemaString = schemaString;
-    this.fieldNames = fieldNames;
-    this.schemas = new CachedSchemas();
-  }
-
-  @Override
-  public List<Object> deserialize(ByteBuffer ser) {
-    try {
-      Schema schema = schemas.getSchema(schemaString);
-      DatumReader<GenericRecord> reader = new GenericDatumReader<>(schema);
-      BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(Utils.toByteArray(ser), null);
-      GenericRecord record = reader.read(null, decoder);
-
-      ArrayList<Object> list = new ArrayList<>(fieldNames.size());
-      for (String field : fieldNames) {
-        Object value = record.get(field);
-        // Avro strings are stored using a special Avro Utf8 type instead of using Java primitives
-        list.add(SerdeUtils.convertAvroUtf8(value));
-      }
-      return list;
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  @Override
-  public Fields getOutputFields() {
-    return new Fields(fieldNames);
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroSerializer.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroSerializer.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroSerializer.java
deleted file mode 100644
index 5dc3393..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroSerializer.java
+++ /dev/null
@@ -1,72 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.runtime.serde.avro;
-
-import com.google.common.base.Preconditions;
-import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericData;
-import org.apache.avro.generic.GenericDatumWriter;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.io.DatumWriter;
-import org.apache.avro.io.Encoder;
-import org.apache.avro.io.EncoderFactory;
-import org.apache.storm.sql.runtime.IOutputSerializer;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.Serializable;
-import java.nio.ByteBuffer;
-import java.util.List;
-
-/**
- * AvroSerializer uses generic(without code generation) instead of specific(with code generation) writers.
- */
-public class AvroSerializer implements IOutputSerializer, Serializable {
-  private final String schemaString;
-  private final List<String> fieldNames;
-  private final CachedSchemas schemas;
-
-  public AvroSerializer(String schemaString, List<String> fieldNames) {
-    this.schemaString = schemaString;
-    this.fieldNames = fieldNames;
-    this.schemas = new CachedSchemas();
-  }
-
-  @Override
-  public ByteBuffer write(List<Object> data, ByteBuffer buffer) {
-    Preconditions.checkArgument(data != null && data.size() == fieldNames.size(), "Invalid schemas");
-    try {
-      Schema schema = schemas.getSchema(schemaString);
-      GenericRecord record = new GenericData.Record(schema);
-      for (int i = 0; i < fieldNames.size(); i++) {
-        record.put(fieldNames.get(i), data.get(i));
-      }
-
-      ByteArrayOutputStream out = new ByteArrayOutputStream();
-      DatumWriter<GenericRecord> writer = new GenericDatumWriter<>(record.getSchema());
-      Encoder encoder = EncoderFactory.get().directBinaryEncoder(out, null);
-      writer.write(record, encoder);
-      encoder.flush();
-      byte[] bytes = out.toByteArray();
-      out.close();
-      return ByteBuffer.wrap(bytes);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/CachedSchemas.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/CachedSchemas.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/CachedSchemas.java
deleted file mode 100644
index 4f0e747..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/CachedSchemas.java
+++ /dev/null
@@ -1,41 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.runtime.serde.avro;
-
-import org.apache.avro.Schema;
-
-import java.io.Serializable;
-import java.util.HashMap;
-import java.util.Map;
-
-// TODO this class is reserved for supporting messages with different schemas.
-// current only one schema in the cache
-public class CachedSchemas implements Serializable{
-
-    private final Map<String, Schema> cache = new HashMap<>();
-
-    public Schema getSchema(String schemaString) {
-        Schema schema = cache.get(schemaString);
-        if (schema == null) {
-            schema = new Schema.Parser().parse(schemaString);
-            cache.put(schemaString, schema);
-        }
-        return schema;
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvScheme.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvScheme.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvScheme.java
deleted file mode 100644
index 34fb1bb..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvScheme.java
+++ /dev/null
@@ -1,70 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.runtime.serde.csv;
-
-import com.google.common.base.Preconditions;
-import org.apache.commons.csv.CSVFormat;
-import org.apache.commons.csv.CSVParser;
-import org.apache.commons.csv.CSVRecord;
-import org.apache.storm.spout.Scheme;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.utils.Utils;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * CsvScheme uses the standard RFC4180 CSV Parser
- * One of the difference from Tsv format is that fields with embedded commas will be quoted.
- * eg: a,"b,c",d is allowed.
- *
- * @see <a href="https://tools.ietf.org/html/rfc4180">RFC4180</a>
- */
-public class CsvScheme implements Scheme {
-  private final List<String> fieldNames;
-
-  public CsvScheme(List<String> fieldNames) {
-    this.fieldNames = fieldNames;
-  }
-
-  @Override
-  public List<Object> deserialize(ByteBuffer ser) {
-    try {
-      String data = new String(Utils.toByteArray(ser), StandardCharsets.UTF_8);
-      CSVParser parser = CSVParser.parse(data, CSVFormat.RFC4180);
-      CSVRecord record = parser.getRecords().get(0);
-      Preconditions.checkArgument(record.size() == fieldNames.size(), "Invalid schema");
-
-      ArrayList<Object> list = new ArrayList<>(fieldNames.size());
-      for (int i = 0; i < record.size(); i++) {
-        list.add(record.get(i));
-      }
-      return list;
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  @Override
-  public Fields getOutputFields() {
-    return new Fields(fieldNames);
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvSerializer.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvSerializer.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvSerializer.java
deleted file mode 100644
index 0d3bd74..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvSerializer.java
+++ /dev/null
@@ -1,59 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.runtime.serde.csv;
-
-import org.apache.commons.csv.CSVFormat;
-import org.apache.commons.csv.CSVPrinter;
-import org.apache.storm.sql.runtime.IOutputSerializer;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.io.StringWriter;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.util.List;
-
-/**
- * CsvSerializer uses the standard RFC4180 CSV Parser
- * One of the difference from Tsv format is that fields with embedded commas will be quoted.
- * eg: a,"b,c",d is allowed.
- *
- * @see <a href="https://tools.ietf.org/html/rfc4180">RFC4180</a>
- */
-public class CsvSerializer implements IOutputSerializer, Serializable {
-  private final List<String> fields; //reserved for future
-
-  public CsvSerializer(List<String> fields) {
-        this.fields = fields;
-    }
-
-  @Override
-  public ByteBuffer write(List<Object> data, ByteBuffer buffer) {
-    try {
-      StringWriter writer = new StringWriter();
-      CSVPrinter printer = new CSVPrinter(writer, CSVFormat.RFC4180);
-      for (Object o : data) {
-        printer.print(o);
-      }
-      //since using StringWriter, we do not need to close it.
-      return ByteBuffer.wrap(writer.getBuffer().toString().getBytes(StandardCharsets.UTF_8));
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonScheme.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonScheme.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonScheme.java
deleted file mode 100644
index d288fa1..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonScheme.java
+++ /dev/null
@@ -1,58 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.runtime.serde.json;
-
-import org.apache.storm.spout.Scheme;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.utils.Utils;
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-
-public class JsonScheme implements Scheme {
-  private final List<String> fields;
-
-  public JsonScheme(List<String> fields) {
-    this.fields = fields;
-  }
-
-  @Override
-  public List<Object> deserialize(ByteBuffer ser) {
-    ObjectMapper mapper = new ObjectMapper();
-    try {
-      @SuppressWarnings("unchecked")
-      HashMap<String, Object> map = mapper.readValue(Utils.toByteArray(ser), HashMap.class);
-      ArrayList<Object> list = new ArrayList<>(fields.size());
-      for (String f : fields) {
-        list.add(map.get(f));
-      }
-      return list;
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  @Override
-  public Fields getOutputFields() {
-    return new Fields(fields);
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonSerializer.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonSerializer.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonSerializer.java
deleted file mode 100644
index 1e825c4..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonSerializer.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.runtime.serde.json;
-
-import com.fasterxml.jackson.core.JsonFactory;
-import com.fasterxml.jackson.core.JsonGenerator;
-import com.google.common.base.Preconditions;
-import org.apache.storm.sql.runtime.IOutputSerializer;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.io.StringWriter;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.util.List;
-
-public class JsonSerializer implements IOutputSerializer, Serializable {
-  private final List<String> fieldNames;
-  private final JsonFactory jsonFactory;
-
-  public JsonSerializer(List<String> fieldNames) {
-    this.fieldNames = fieldNames;
-    jsonFactory = new JsonFactory();
-  }
-
-  @Override
-  public ByteBuffer write(List<Object> data, ByteBuffer buffer) {
-    Preconditions.checkArgument(data != null && data.size() == fieldNames.size(), "Invalid schema");
-    StringWriter sw = new StringWriter();
-    try (JsonGenerator jg = jsonFactory.createGenerator(sw)) {
-      jg.writeStartObject();
-      for (int i = 0; i < fieldNames.size(); ++i) {
-        jg.writeFieldName(fieldNames.get(i));
-        jg.writeObject(data.get(i));
-      }
-      jg.writeEndObject();
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-    return ByteBuffer.wrap(sw.toString().getBytes(StandardCharsets.UTF_8));
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvScheme.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvScheme.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvScheme.java
deleted file mode 100644
index 310494c..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvScheme.java
+++ /dev/null
@@ -1,58 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.runtime.serde.tsv;
-
-import com.google.common.base.Preconditions;
-import org.apache.storm.spout.Scheme;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.utils.Utils;
-
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * TsvScheme uses a simple delimited format implemention by splitting string,
- * and it supports user defined delimiter.
- */
-public class TsvScheme implements Scheme {
-  private final List<String> fieldNames;
-  private final char delimiter;
-
-  public TsvScheme(List<String> fieldNames, char delimiter) {
-    this.fieldNames = fieldNames;
-    this.delimiter = delimiter;
-  }
-
-  @Override
-  public List<Object> deserialize(ByteBuffer ser) {
-    String data = new String(Utils.toByteArray(ser), StandardCharsets.UTF_8);
-    List<String> parts = org.apache.storm.sql.runtime.utils.Utils.split(data, delimiter);
-    Preconditions.checkArgument(parts.size() == fieldNames.size(), "Invalid schema");
-
-    ArrayList<Object> list = new ArrayList<>(fieldNames.size());
-    list.addAll(parts);
-    return list;
-  }
-
-  @Override
-  public Fields getOutputFields() {
-    return new Fields(fieldNames);
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvSerializer.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvSerializer.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvSerializer.java
deleted file mode 100644
index 1cf1c76..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvSerializer.java
+++ /dev/null
@@ -1,54 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.runtime.serde.tsv;
-
-import org.apache.storm.sql.runtime.IOutputSerializer;
-
-import java.io.Serializable;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.util.List;
-
-/**
- * TsvSerializer uses a simple delimited format implemention by splitting string,
- * and it supports user defined delimiter.
- */
-public class TsvSerializer implements IOutputSerializer, Serializable {
-  private final List<String> fields; //reserved for future
-  private final char delimiter;
-
-  public TsvSerializer(List<String> fields, char delimiter) {
-    this.fields = fields;
-    this.delimiter = delimiter;
-    }
-
-  @Override
-  public ByteBuffer write(List<Object> data, ByteBuffer buffer) {
-    StringBuilder sb = new StringBuilder(512); // 512: for most scenes to avoid inner array resizing
-    for (int i = 0; i < data.size(); i++) {
-      Object o = data.get(i);
-      if (i == 0) {
-        sb.append(o);
-      } else {
-        sb.append(delimiter);
-        sb.append(o);
-      }
-    }
-    return ByteBuffer.wrap(sb.toString().getBytes(StandardCharsets.UTF_8));
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationCalc.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationCalc.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationCalc.java
deleted file mode 100644
index 6c76481..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationCalc.java
+++ /dev/null
@@ -1,86 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * 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.sql.runtime.trident.functions;
-
-import org.apache.calcite.DataContext;
-import org.apache.calcite.interpreter.Context;
-import org.apache.calcite.interpreter.StormContext;
-import org.apache.storm.sql.runtime.calcite.DebuggableExecutableExpression;
-import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
-import org.apache.storm.trident.operation.OperationAwareFlatMapFunction;
-import org.apache.storm.trident.operation.TridentOperationContext;
-import org.apache.storm.trident.tuple.TridentTuple;
-import org.apache.storm.tuple.Values;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Collections;
-import java.util.Map;
-
-public class EvaluationCalc implements OperationAwareFlatMapFunction {
-    private static final Logger LOG = LoggerFactory.getLogger(EvaluationCalc.class);
-
-    private final ExecutableExpression filterInstance;
-    private final ExecutableExpression projectionInstance;
-    private final Object[] outputValues;
-    private final DataContext dataContext;
-
-    public EvaluationCalc(ExecutableExpression filterInstance, ExecutableExpression projectionInstance, int outputCount, DataContext dataContext) {
-        this.filterInstance = filterInstance;
-        this.projectionInstance = projectionInstance;
-        this.outputValues = new Object[outputCount];
-        this.dataContext = dataContext;
-    }
-
-    @Override
-    public void prepare(Map conf, TridentOperationContext context) {
-        if (projectionInstance != null && projectionInstance instanceof DebuggableExecutableExpression) {
-            LOG.info("Expression code for projection: \n{}", ((DebuggableExecutableExpression) projectionInstance).getDelegateCode());
-        }
-        if (filterInstance != null && filterInstance instanceof DebuggableExecutableExpression) {
-            LOG.info("Expression code for filter: \n{}", ((DebuggableExecutableExpression) filterInstance).getDelegateCode());
-        }
-    }
-
-    @Override
-    public void cleanup() {
-
-    }
-
-    @Override
-    public Iterable<Values> execute(TridentTuple input) {
-        Context calciteContext = new StormContext(dataContext);
-        calciteContext.values = input.getValues().toArray();
-
-        if (filterInstance != null) {
-            filterInstance.execute(calciteContext, outputValues);
-            // filtered out
-            if (outputValues[0] == null || !((Boolean) outputValues[0])) {
-                return Collections.emptyList();
-            }
-        }
-
-        if (projectionInstance != null) {
-            projectionInstance.execute(calciteContext, outputValues);
-            return Collections.singletonList(new Values(outputValues));
-        } else {
-            return Collections.singletonList(new Values(input.getValues()));
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFilter.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFilter.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFilter.java
deleted file mode 100644
index 9314852..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFilter.java
+++ /dev/null
@@ -1,61 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * 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.sql.runtime.trident.functions;
-
-import org.apache.calcite.DataContext;
-import org.apache.calcite.interpreter.Context;
-import org.apache.calcite.interpreter.StormContext;
-import org.apache.storm.sql.runtime.calcite.DebuggableExecutableExpression;
-import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
-import org.apache.storm.trident.operation.BaseFilter;
-import org.apache.storm.trident.operation.TridentOperationContext;
-import org.apache.storm.trident.tuple.TridentTuple;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Map;
-
-public class EvaluationFilter extends BaseFilter {
-    private static final Logger LOG = LoggerFactory.getLogger(EvaluationFilter.class);
-
-    private final ExecutableExpression filterInstance;
-    private final DataContext dataContext;
-    private final Object[] outputValues;
-
-    public EvaluationFilter(ExecutableExpression filterInstance, DataContext dataContext) {
-        this.filterInstance = filterInstance;
-        this.dataContext = dataContext;
-        this.outputValues = new Object[1];
-    }
-
-    @Override
-    public void prepare(Map conf, TridentOperationContext context) {
-        if (filterInstance != null && filterInstance instanceof DebuggableExecutableExpression) {
-            LOG.info("Expression code for filter: \n{}", ((DebuggableExecutableExpression) filterInstance).getDelegateCode());
-        }
-    }
-
-    @Override
-    public boolean isKeep(TridentTuple tuple) {
-        Context calciteContext = new StormContext(dataContext);
-        calciteContext.values = tuple.getValues().toArray();
-        filterInstance.execute(calciteContext, outputValues);
-        return (outputValues[0] != null && (boolean) outputValues[0]);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFunction.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFunction.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFunction.java
deleted file mode 100644
index 2608104..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFunction.java
+++ /dev/null
@@ -1,67 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * 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.sql.runtime.trident.functions;
-
-import org.apache.calcite.DataContext;
-import org.apache.calcite.interpreter.Context;
-import org.apache.calcite.interpreter.StormContext;
-import org.apache.storm.sql.runtime.calcite.DebuggableExecutableExpression;
-import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
-import org.apache.storm.trident.operation.OperationAwareMapFunction;
-import org.apache.storm.trident.operation.TridentOperationContext;
-import org.apache.storm.trident.tuple.TridentTuple;
-import org.apache.storm.tuple.Values;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Map;
-
-public class EvaluationFunction implements OperationAwareMapFunction {
-    private static final Logger LOG = LoggerFactory.getLogger(EvaluationFunction.class);
-
-    private final ExecutableExpression projectionInstance;
-    private final Object[] outputValues;
-    private final DataContext dataContext;
-
-    public EvaluationFunction(ExecutableExpression projectionInstance, int outputCount, DataContext dataContext) {
-        this.projectionInstance = projectionInstance;
-        this.outputValues = new Object[outputCount];
-        this.dataContext = dataContext;
-    }
-
-    @Override
-    public void prepare(Map conf, TridentOperationContext context) {
-        if (projectionInstance instanceof DebuggableExecutableExpression) {
-            LOG.info("Expression code: {}", ((DebuggableExecutableExpression) projectionInstance).getDelegateCode());
-        }
-    }
-
-    @Override
-    public void cleanup() {
-
-    }
-
-    @Override
-    public Values execute(TridentTuple input) {
-        Context calciteContext = new StormContext(dataContext);
-        calciteContext.values = input.getValues().toArray();
-        projectionInstance.execute(calciteContext, outputValues);
-        return new Values(outputValues);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/ForwardFunction.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/ForwardFunction.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/ForwardFunction.java
deleted file mode 100644
index 4c3a266..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/ForwardFunction.java
+++ /dev/null
@@ -1,30 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * 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.sql.runtime.trident.functions;
-
-import org.apache.storm.trident.operation.BaseFunction;
-import org.apache.storm.trident.operation.TridentCollector;
-import org.apache.storm.trident.tuple.TridentTuple;
-
-public class ForwardFunction extends BaseFunction {
-    @Override
-    public void execute(TridentTuple tuple, TridentCollector collector) {
-        collector.emit(tuple.getValues());
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/FieldInfoUtils.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/FieldInfoUtils.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/FieldInfoUtils.java
deleted file mode 100644
index efd5d25..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/FieldInfoUtils.java
+++ /dev/null
@@ -1,39 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.runtime.utils;
-
-import com.google.common.base.Function;
-import com.google.common.collect.Lists;
-import org.apache.storm.sql.runtime.FieldInfo;
-
-import java.io.Serializable;
-import java.util.List;
-
-public final class FieldInfoUtils {
-
-    public static List<String> getFieldNames(List<FieldInfo> fields) {
-        return Lists.transform(fields, new FieldNameExtractor());
-    }
-
-    private static class FieldNameExtractor implements Function<FieldInfo, String>, Serializable {
-        @Override
-        public String apply(FieldInfo fieldInfo) {
-            return fieldInfo.name();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/SerdeUtils.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/SerdeUtils.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/SerdeUtils.java
deleted file mode 100644
index 6b3dfc9..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/SerdeUtils.java
+++ /dev/null
@@ -1,123 +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.sql.runtime.utils;
-
-import static org.apache.commons.lang.StringUtils.isNotEmpty;
-
-import com.google.common.base.Preconditions;
-import org.apache.avro.generic.GenericData;
-import org.apache.avro.util.Utf8;
-import org.apache.storm.spout.Scheme;
-import org.apache.storm.sql.runtime.IOutputSerializer;
-import org.apache.storm.sql.runtime.serde.avro.AvroScheme;
-import org.apache.storm.sql.runtime.serde.avro.AvroSerializer;
-import org.apache.storm.sql.runtime.serde.csv.CsvScheme;
-import org.apache.storm.sql.runtime.serde.csv.CsvSerializer;
-import org.apache.storm.sql.runtime.serde.json.JsonScheme;
-import org.apache.storm.sql.runtime.serde.json.JsonSerializer;
-import org.apache.storm.sql.runtime.serde.tsv.TsvScheme;
-import org.apache.storm.sql.runtime.serde.tsv.TsvSerializer;
-import org.apache.storm.utils.Utils;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-public final class SerdeUtils {
-    public static Scheme getScheme(String inputFormatClass, Properties properties, List<String> fieldNames) {
-        Scheme scheme;
-        if (isNotEmpty(inputFormatClass)) {
-            if (JsonScheme.class.getName().equals(inputFormatClass)) {
-                scheme = new JsonScheme(fieldNames);
-            } else if (TsvScheme.class.getName().equals(inputFormatClass)) {
-                String delimiter = properties.getProperty("input.tsv.delimiter", "\t");
-                scheme = new TsvScheme(fieldNames, delimiter.charAt(0));
-            } else if (CsvScheme.class.getName().equals(inputFormatClass)) {
-                scheme = new CsvScheme(fieldNames);
-            } else if (AvroScheme.class.getName().equals(inputFormatClass)) {
-                String schemaString = properties.getProperty("input.avro.schema");
-                Preconditions.checkArgument(isNotEmpty(schemaString), "input.avro.schema can not be empty");
-                scheme = new AvroScheme(schemaString, fieldNames);
-            } else {
-                scheme = Utils.newInstance(inputFormatClass);
-            }
-        } else {
-            //use JsonScheme as the default scheme
-            scheme = new JsonScheme(fieldNames);
-        }
-        return scheme;
-    }
-
-    public static IOutputSerializer getSerializer(String outputFormatClass, Properties properties, List<String> fieldNames) {
-        IOutputSerializer serializer;
-        if (isNotEmpty(outputFormatClass)) {
-            if (JsonSerializer.class.getName().equals(outputFormatClass)) {
-                serializer = new JsonSerializer(fieldNames);
-            } else if (TsvSerializer.class.getName().equals(outputFormatClass)) {
-                String delimiter = properties.getProperty("output.tsv.delimiter", "\t");
-                serializer = new TsvSerializer(fieldNames, delimiter.charAt(0));
-            } else if (CsvSerializer.class.getName().equals(outputFormatClass)) {
-                serializer = new CsvSerializer(fieldNames);
-            } else if (AvroSerializer.class.getName().equals(outputFormatClass)) {
-                String schemaString = properties.getProperty("output.avro.schema");
-                Preconditions.checkArgument(isNotEmpty(schemaString), "output.avro.schema can not be empty");
-                serializer = new AvroSerializer(schemaString, fieldNames);
-            } else {
-                serializer = Utils.newInstance(outputFormatClass);
-            }
-        } else {
-            //use JsonSerializer as the default serializer
-            serializer = new JsonSerializer(fieldNames);
-        }
-        return serializer;
-    }
-
-    public static Object convertAvroUtf8(Object value){
-        Object ret;
-        if (value instanceof Utf8) {
-            ret = value.toString();
-        } else if (value instanceof Map<?, ?>) {
-            ret = convertAvroUtf8Map((Map<Object,Object>)value);
-        } else if (value instanceof GenericData.Array) {
-            ret = convertAvroUtf8Array((GenericData.Array)value);
-        } else {
-            ret = value;
-        }
-        return ret;
-    }
-
-    public static Object convertAvroUtf8Map(Map<Object,Object> value) {
-        Map<Object, Object> map = new HashMap<>(value.size());
-        for (Map.Entry<Object, Object> entry : value.entrySet()) {
-            Object k = convertAvroUtf8(entry.getKey());
-            Object v = convertAvroUtf8(entry.getValue());
-            map.put(k, v);
-        }
-        return map;
-    }
-
-    public static Object convertAvroUtf8Array(GenericData.Array value){
-        List<Object> ls = new ArrayList<>(value.size());
-        for(Object o : value){
-            ls.add(convertAvroUtf8(o));
-        }
-        return ls;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/Utils.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/Utils.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/Utils.java
deleted file mode 100644
index a0f3af3..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/Utils.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.runtime.utils;
-
-import java.util.LinkedList;
-import java.util.List;
-
-public final class Utils {
-
-    /**
-     * This method for splitting string into parts by a delimiter
-     * It has higher performance than String.split(String regex)
-     *
-     * @param data
-     * @param delimiter
-     * @return
-     */
-    public static List<String> split(String data, char delimiter){
-        List<String> list = new LinkedList<>();
-        //do not use .toCharArray avoid system copy
-        StringBuilder sb = new StringBuilder(512);
-        int len = data.length();
-        for (int i=0; i < len; i++) {
-            char ch = data.charAt(i);
-            if(ch == delimiter){
-                list.add(sb.toString());
-                sb.setLength(0);
-                if(i == len - 1){
-                    list.add("");
-                }
-            }else{
-                sb.append(ch);
-            }
-        }
-        if (sb.length() > 0) {
-            list.add(sb.toString());
-        }
-        return list;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider b/external/sql/storm-sql-runtime/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
deleted file mode 100644
index 9a945f7..0000000
--- a/external/sql/storm-sql-runtime/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
+++ /dev/null
@@ -1,16 +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.
-
-org.apache.storm.sql.runtime.datasource.socket.SocketDataSourcesProvider
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestAvroSerializer.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestAvroSerializer.java b/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestAvroSerializer.java
deleted file mode 100644
index 174bfde..0000000
--- a/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestAvroSerializer.java
+++ /dev/null
@@ -1,72 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import org.apache.storm.sql.runtime.serde.avro.AvroScheme;
-import org.apache.storm.sql.runtime.serde.avro.AvroSerializer;
-import org.junit.Test;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-import java.util.Map;
-
-import static org.junit.Assert.assertArrayEquals;
-
-public class TestAvroSerializer {
-  private static final String schemaString = "{\"type\":\"record\"," +
-          "\"name\":\"avrotest\"," +
-          "\"fields\":[{\"name\":\"ID\",\"type\":\"int\"}," +
-          "{ \"name\":\"val\", \"type\":\"string\" }]}";
-
-  private static final String schemaString1 = "{\"type\":\"record\"," +
-          "\"name\":\"avrotest\"," +
-          "\"fields\":[{\"name\":\"ID\",\"type\":\"int\"}," +
-          "{ \"type\":{\"type\":\"map\",\"values\": \"long\"}, \"name\":\"val1\" }," +
-          "{ \"type\":{\"type\":\"array\",\"items\": \"string\"}, \"name\":\"val2\" }]}";
-
-  @Test
-  public void testAvroSchemeAndSerializer() {
-    List<String> fields = Lists.newArrayList("ID", "val");
-    List<Object> o = Lists.newArrayList(1, "2");
-
-    AvroSerializer serializer = new AvroSerializer(schemaString, fields);
-    ByteBuffer byteBuffer = serializer.write(o, null);
-
-    AvroScheme scheme = new AvroScheme(schemaString, fields);
-    assertArrayEquals(o.toArray(), scheme.deserialize(byteBuffer).toArray());
-  }
-
-  @Test
-  public void testAvroComplexSchemeAndSerializer() {
-    List<String> fields = Lists.newArrayList("ID", "val1", "val2");
-
-    Map<String,Long> mp = Maps.newHashMap();;
-    mp.put("l1",1234L);
-    mp.put("l2",56789L);
-    List<String> ls = Lists.newArrayList("s1", "s2");
-    List<Object> o = Lists.newArrayList(1, mp, ls);
-
-    AvroSerializer serializer = new AvroSerializer(schemaString1, fields);
-    ByteBuffer byteBuffer = serializer.write(o, null);
-
-    AvroScheme scheme = new AvroScheme(schemaString1, fields);
-    assertArrayEquals(o.toArray(), scheme.deserialize(byteBuffer).toArray());
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestCsvSerializer.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestCsvSerializer.java b/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestCsvSerializer.java
deleted file mode 100644
index 0108949..0000000
--- a/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestCsvSerializer.java
+++ /dev/null
@@ -1,54 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql;
-
-import com.google.common.collect.Lists;
-import org.apache.storm.sql.runtime.serde.csv.CsvScheme;
-import org.apache.storm.sql.runtime.serde.csv.CsvSerializer;
-import org.junit.Test;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import static org.junit.Assert.assertArrayEquals;
-
-public class TestCsvSerializer {
-
-  @Test
-  public void testCsvSchemeAndSerializer() {
-    List<String> fields = Lists.newArrayList("ID", "val");
-    List<Object> o = Lists.newArrayList("1", "2");
-
-    CsvSerializer serializer = new CsvSerializer(fields);
-    ByteBuffer byteBuffer = serializer.write(o, null);
-
-    CsvScheme scheme = new CsvScheme(fields);
-    assertArrayEquals(o.toArray(), scheme.deserialize(byteBuffer).toArray());
-
-    // Fields with embedded commas or double-quote characters
-    fields = Lists.newArrayList("ID", "val", "v");
-    o = Lists.newArrayList("1,9", "2,\"3\",5", "\"8\"");
-
-    serializer = new CsvSerializer(fields);
-    byteBuffer = serializer.write(o, null);
-
-    scheme = new CsvScheme(fields);
-    assertArrayEquals(o.toArray(), scheme.deserialize(byteBuffer).toArray());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestJsonRepresentation.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestJsonRepresentation.java b/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestJsonRepresentation.java
deleted file mode 100644
index 6ca1877..0000000
--- a/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestJsonRepresentation.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql;
-
-import org.apache.storm.sql.runtime.serde.json.JsonScheme;
-import org.apache.storm.sql.runtime.serde.json.JsonSerializer;
-import org.apache.storm.utils.Utils;
-import com.google.common.collect.Lists;
-import org.junit.Test;
-
-import java.nio.ByteBuffer;
-import java.nio.charset.Charset;
-import java.util.List;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-
-public class TestJsonRepresentation {
-  @Test
-  public void testJsonScheme() {
-    final List<String> fields = Lists.newArrayList("ID", "val");
-    final String s = "{\"ID\": 1, \"val\": \"2\"}";
-    JsonScheme scheme = new JsonScheme(fields);
-    List<Object> o = scheme.deserialize(ByteBuffer.wrap(s.getBytes(Charset.defaultCharset())));
-    assertArrayEquals(new Object[] {1, "2"}, o.toArray());
-  }
-
-  @Test
-  public void testJsonSerializer() {
-    final List<String> fields = Lists.newArrayList("ID", "val");
-    List<Object> o = Lists.<Object> newArrayList(1, "2");
-    JsonSerializer s = new JsonSerializer(fields);
-    ByteBuffer buf = s.write(o, null);
-    byte[] b = Utils.toByteArray(buf);
-    assertEquals("{\"ID\":1,\"val\":\"2\"}", new String(b));
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestTsvSerializer.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestTsvSerializer.java b/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestTsvSerializer.java
deleted file mode 100644
index 1798828..0000000
--- a/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestTsvSerializer.java
+++ /dev/null
@@ -1,46 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql;
-
-import com.google.common.collect.Lists;
-import org.apache.storm.sql.runtime.serde.tsv.TsvScheme;
-import org.apache.storm.sql.runtime.serde.tsv.TsvSerializer;
-import org.junit.Test;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import static org.junit.Assert.assertArrayEquals;
-
-public class TestTsvSerializer {
-
-  @Test
-  public void testTsvSchemeAndSerializer() {
-    final char delimiter = '\t';
-
-    List<String> fields = Lists.newArrayList("ID", "val");
-    List<Object> o = Lists.newArrayList("1", "2");
-
-    TsvSerializer serializer = new TsvSerializer(fields, delimiter);
-    ByteBuffer byteBuffer = serializer.write(o, null);
-
-    TsvScheme scheme = new TsvScheme(fields, delimiter);
-    assertArrayEquals(o.toArray(), scheme.deserialize(byteBuffer).toArray());
-  }
-
-}


[04/23] storm git commit: STORM-2453 Move non-connectors into the top directory

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/test/org/apache/storm/sql/TestStormSql.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/test/org/apache/storm/sql/TestStormSql.java b/sql/storm-sql-core/src/test/org/apache/storm/sql/TestStormSql.java
new file mode 100644
index 0000000..82dc184
--- /dev/null
+++ b/sql/storm-sql-core/src/test/org/apache/storm/sql/TestStormSql.java
@@ -0,0 +1,492 @@
+/**
+ * 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.sql;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.calcite.tools.ValidationException;
+import org.apache.storm.sql.javac.CompilingClassLoader;
+import org.apache.storm.sql.runtime.ChannelHandler;
+import org.apache.storm.sql.runtime.DataSource;
+import org.apache.storm.sql.runtime.DataSourcesProvider;
+import org.apache.storm.sql.runtime.DataSourcesRegistry;
+import org.apache.storm.sql.runtime.FieldInfo;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.tuple.Values;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+public class TestStormSql {
+  private static class MockDataSourceProvider implements DataSourcesProvider {
+    @Override
+    public String scheme() {
+      return "mock";
+    }
+
+    @Override
+    public DataSource construct(
+        URI uri, String inputFormatClass, String outputFormatClass,
+        List<FieldInfo> fields) {
+      return new TestUtils.MockDataSource();
+    }
+
+    @Override
+    public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
+                                                  Properties properties, List<FieldInfo> fields) {
+      return new TestUtils.MockSqlTridentDataSource();
+    }
+  }
+
+  private static class MockNestedDataSourceProvider implements DataSourcesProvider {
+    @Override
+    public String scheme() {
+      return "mocknested";
+    }
+
+    @Override
+    public DataSource construct(
+            URI uri, String inputFormatClass, String outputFormatClass,
+            List<FieldInfo> fields) {
+      return new TestUtils.MockNestedDataSource();
+    }
+
+    @Override
+    public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
+                                                  Properties properties, List<FieldInfo> fields) {
+      return new TestUtils.MockSqlTridentDataSource();
+    }
+  }
+
+  private static class MockGroupDataSourceProvider implements DataSourcesProvider {
+    @Override
+    public String scheme() {
+      return "mockgroup";
+    }
+
+    @Override
+    public DataSource construct(
+            URI uri, String inputFormatClass, String outputFormatClass,
+            List<FieldInfo> fields) {
+      return new TestUtils.MockGroupDataSource();
+    }
+
+    @Override
+    public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
+                                                  Properties properties, List<FieldInfo> fields) {
+      return new TestUtils.MockSqlTridentGroupedDataSource();
+    }
+  }
+
+  private static class MockEmpDataSourceProvider implements DataSourcesProvider {
+    @Override
+    public String scheme() {
+      return "mockemp";
+    }
+
+    @Override
+    public DataSource construct(
+            URI uri, String inputFormatClass, String outputFormatClass,
+            List<FieldInfo> fields) {
+      return new TestUtils.MockEmpDataSource();
+    }
+
+    @Override
+    public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
+                                                  Properties properties, List<FieldInfo> fields) {
+      return new TestUtils.MockSqlTridentJoinDataSourceEmp();
+    }
+  }
+
+  private static class MockDeptDataSourceProvider implements DataSourcesProvider {
+    @Override
+    public String scheme() {
+      return "mockdept";
+    }
+
+    @Override
+    public DataSource construct(
+            URI uri, String inputFormatClass, String outputFormatClass,
+            List<FieldInfo> fields) {
+      return new TestUtils.MockDeptDataSource();
+    }
+
+    @Override
+    public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
+                                                  Properties properties, List<FieldInfo> fields) {
+      return new TestUtils.MockSqlTridentJoinDataSourceDept();
+    }
+  }
+
+
+  @BeforeClass
+  public static void setUp() {
+    DataSourcesRegistry.providerMap().put("mock", new MockDataSourceProvider());
+    DataSourcesRegistry.providerMap().put("mocknested", new MockNestedDataSourceProvider());
+    DataSourcesRegistry.providerMap().put("mockgroup", new MockGroupDataSourceProvider());
+    DataSourcesRegistry.providerMap().put("mockemp", new MockEmpDataSourceProvider());
+    DataSourcesRegistry.providerMap().put("mockdept", new MockDeptDataSourceProvider());
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    DataSourcesRegistry.providerMap().remove("mock");
+    DataSourcesRegistry.providerMap().remove("mocknested");
+  }
+
+  @Test
+  public void testExternalDataSource() throws Exception {
+    List<String> stmt = new ArrayList<>();
+    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT) LOCATION 'mock:///foo'");
+    stmt.add("SELECT STREAM ID + 1 FROM FOO WHERE ID > 2");
+    StormSql sql = StormSql.construct();
+    List<Values> values = new ArrayList<>();
+    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+    sql.execute(stmt, h);
+    Assert.assertEquals(2, values.size());
+    Assert.assertEquals(4, values.get(0).get(0));
+    Assert.assertEquals(5, values.get(1).get(0));
+  }
+
+  @Test
+  public void testExternalDataSourceNested() throws Exception {
+    List<String> stmt = new ArrayList<>();
+    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, MAPFIELD ANY, NESTEDMAPFIELD ANY, ARRAYFIELD ANY) LOCATION 'mocknested:///foo'");
+    stmt.add("SELECT STREAM ID, MAPFIELD['c'], NESTEDMAPFIELD, ARRAYFIELD " +
+                     "FROM FOO " +
+                     "WHERE CAST(MAPFIELD['b'] AS INTEGER) = 2 AND CAST(ARRAYFIELD[2] AS INTEGER) = 200");
+    StormSql sql = StormSql.construct();
+    List<Values> values = new ArrayList<>();
+    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+    sql.execute(stmt, h);
+    System.out.println(values);
+    Map<String, Integer> map = ImmutableMap.of("b", 2, "c", 4);
+    Map<String, Map<String, Integer>> nestedMap = ImmutableMap.of("a", map);
+    Assert.assertEquals(new Values(2, 4, nestedMap, Arrays.asList(100, 200, 300)), values.get(0));
+  }
+
+  @Test
+  public void testExternalNestedNonExistKeyAccess() throws Exception {
+    List<String> stmt = new ArrayList<>();
+    // this triggers java.lang.RuntimeException: Cannot convert null to int
+    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, MAPFIELD ANY, NESTEDMAPFIELD ANY, ARRAYFIELD ANY) LOCATION 'mocknested:///foo'");
+    stmt.add("SELECT STREAM ID, MAPFIELD, NESTEDMAPFIELD, ARRAYFIELD " +
+             "FROM FOO " +
+             "WHERE CAST(MAPFIELD['a'] AS INTEGER) = 2");
+    StormSql sql = StormSql.construct();
+    List<Values> values = new ArrayList<>();
+    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+    sql.execute(stmt, h);
+    Assert.assertEquals(0, values.size());
+  }
+
+  @Test
+  public void testExternalNestedNonExistKeyAccess2() throws Exception {
+    List<String> stmt = new ArrayList<>();
+    // this triggers java.lang.RuntimeException: Cannot convert null to int
+    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, MAPFIELD ANY, NESTEDMAPFIELD ANY, ARRAYFIELD ANY) LOCATION 'mocknested:///foo'");
+    stmt.add("SELECT STREAM ID, MAPFIELD, NESTEDMAPFIELD, ARRAYFIELD " +
+             "FROM FOO " +
+             "WHERE CAST(NESTEDMAPFIELD['b']['c'] AS INTEGER) = 4");
+    StormSql sql = StormSql.construct();
+    List<Values> values = new ArrayList<>();
+    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+    sql.execute(stmt, h);
+    Assert.assertEquals(0, values.size());
+  }
+
+  @Test
+  public void testExternalNestedInvalidAccessStringIndexOnArray() throws Exception {
+    List<String> stmt = new ArrayList<>();
+    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, MAPFIELD ANY, NESTEDMAPFIELD ANY, ARRAYFIELD ANY) LOCATION 'mocknested:///foo'");
+    stmt.add("SELECT STREAM ID, MAPFIELD, NESTEDMAPFIELD, ARRAYFIELD " +
+             "FROM FOO " +
+             "WHERE CAST(ARRAYFIELD['a'] AS INTEGER) = 200");
+    StormSql sql = StormSql.construct();
+    List<Values> values = new ArrayList<>();
+    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+    sql.execute(stmt, h);
+    Assert.assertEquals(0, values.size());
+  }
+
+  @Test
+  public void testExternalNestedArrayOutOfBoundAccess() throws Exception {
+    List<String> stmt = new ArrayList<>();
+    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, MAPFIELD ANY, NESTEDMAPFIELD ANY, ARRAYFIELD ANY) LOCATION 'mocknested:///foo'");
+    stmt.add("SELECT STREAM ID, MAPFIELD, NESTEDMAPFIELD, ARRAYFIELD " +
+             "FROM FOO " +
+             "WHERE CAST(ARRAYFIELD[10] AS INTEGER) = 200");
+    StormSql sql = StormSql.construct();
+    List<Values> values = new ArrayList<>();
+    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+    sql.execute(stmt, h);
+    Assert.assertEquals(0, values.size());
+  }
+
+  @Test(expected = ValidationException.class)
+  public void testExternalUdfType() throws Exception {
+    List<String> stmt = new ArrayList<>();
+    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, NAME VARCHAR) LOCATION 'mock:///foo'");
+    stmt.add("CREATE FUNCTION MYPLUS AS 'org.apache.storm.sql.TestUtils$MyPlus'");
+    stmt.add("SELECT STREAM MYPLUS(NAME, 1) FROM FOO WHERE ID = 0");
+    StormSql sql = StormSql.construct();
+    List<Values> values = new ArrayList<>();
+    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+    sql.execute(stmt, h);
+    System.out.println(values);
+
+  }
+
+  @Test(expected = CompilingClassLoader.CompilerException.class)
+  public void testExternalUdfType2() throws Exception {
+    List<String> stmt = new ArrayList<>();
+    // generated code will be not compilable since return type of MYPLUS and type of 'x' are different
+    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, NAME VARCHAR) LOCATION 'mock:///foo'");
+    stmt.add("CREATE FUNCTION MYPLUS AS 'org.apache.storm.sql.TestUtils$MyPlus'");
+    stmt.add("SELECT STREAM ID FROM FOO WHERE MYPLUS(ID, 1) = 'x'");
+    StormSql sql = StormSql.construct();
+    List<Values> values = new ArrayList<>();
+    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+    sql.execute(stmt, h);
+    Assert.assertEquals(0, values.size());
+  }
+
+  @Test
+  public void testExternalUdf() throws Exception {
+    List<String> stmt = new ArrayList<>();
+    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT) LOCATION 'mock:///foo'");
+    stmt.add("CREATE FUNCTION MYPLUS AS 'org.apache.storm.sql.TestUtils$MyPlus'");
+    stmt.add("SELECT STREAM MYPLUS(ID, 1) FROM FOO WHERE ID > 2");
+    StormSql sql = StormSql.construct();
+    List<Values> values = new ArrayList<>();
+    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+    sql.execute(stmt, h);
+    Assert.assertEquals(2, values.size());
+    Assert.assertEquals(4, values.get(0).get(0));
+    Assert.assertEquals(5, values.get(1).get(0));
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void testExternalUdfUsingJar() throws Exception {
+    List<String> stmt = new ArrayList<>();
+    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT) LOCATION 'mock:///foo'");
+    stmt.add("CREATE FUNCTION MYPLUS AS 'org.apache.storm.sql.TestUtils$MyPlus' USING JAR 'foo'");
+    stmt.add("SELECT STREAM MYPLUS(ID, 1) FROM FOO WHERE ID > 2");
+    StormSql sql = StormSql.construct();
+    List<Values> values = new ArrayList<>();
+    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+    sql.execute(stmt, h);
+  }
+
+  @Test
+  public void testGroupbyBuiltin() throws Exception {
+    List<String> stmt = new ArrayList<>();
+    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
+    stmt.add("SELECT STREAM ID, COUNT(*), SUM(SALARY), AVG(SALARY) FROM FOO GROUP BY (ID)");
+    StormSql sql = StormSql.construct();
+    List<Values> values = new ArrayList<>();
+    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+    sql.execute(stmt, h);
+    Assert.assertEquals(4, values.size());
+    Assert.assertEquals(3, values.get(0).get(2));
+    Assert.assertEquals(12, values.get(1).get(2));
+    Assert.assertEquals(21, values.get(2).get(2));
+    Assert.assertEquals(9, values.get(3).get(2));
+  }
+
+  @Test
+  public void testGroupbyBuiltinWithFilter() throws Exception {
+    List<String> stmt = new ArrayList<>();
+    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
+    stmt.add("SELECT STREAM ID, COUNT(*), SUM(SALARY), AVG(PCT) FROM FOO WHERE ID = 1 GROUP BY (ID)");
+    StormSql sql = StormSql.construct();
+    List<Values> values = new ArrayList<>();
+    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+    sql.execute(stmt, h);
+    Assert.assertEquals(1, values.size());
+    Assert.assertEquals(1, values.get(0).get(0));
+    Assert.assertEquals(3L, values.get(0).get(1));
+    Assert.assertEquals(12, values.get(0).get(2));
+    Assert.assertEquals(2.5, values.get(0).get(3));
+  }
+
+  @Test
+  public void testGroupbyBuiltinAndUDF() throws Exception {
+    List<String> stmt = new ArrayList<>();
+    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
+    stmt.add("CREATE FUNCTION MYCONCAT AS 'org.apache.storm.sql.TestUtils$MyConcat'");
+    stmt.add("CREATE FUNCTION TOPN AS 'org.apache.storm.sql.TestUtils$TopN'");
+    stmt.add("SELECT STREAM ID, SUM(SALARY), MYCONCAT(NAME), TOPN(2, SALARY) FROM FOO GROUP BY (ID)");
+    StormSql sql = StormSql.construct();
+    List<Values> values = new ArrayList<>();
+    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+    sql.execute(stmt, h);
+    Assert.assertEquals(4, values.size());
+    Assert.assertEquals(3, values.get(0).get(1));
+    Assert.assertEquals("xxx", values.get(0).get(2));
+    Assert.assertEquals(Arrays.asList(2, 1), values.get(0).get(3));
+    Assert.assertEquals(12, values.get(1).get(1));
+    Assert.assertEquals("xxx", values.get(1).get(2));
+    Assert.assertEquals(Arrays.asList(5, 4), values.get(1).get(3));
+    Assert.assertEquals(21, values.get(2).get(1));
+    Assert.assertEquals("xxx", values.get(2).get(2));
+    Assert.assertEquals(Arrays.asList(8, 7), values.get(2).get(3));
+    Assert.assertEquals(9, values.get(3).get(1));
+    Assert.assertEquals("x", values.get(3).get(2));
+    Assert.assertEquals(Arrays.asList(9), values.get(3).get(3));
+  }
+
+  @Test
+  public void testAggFnNonSqlReturnType() throws Exception {
+    List<String> stmt = new ArrayList<>();
+    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
+    stmt.add("CREATE FUNCTION TOPN AS 'org.apache.storm.sql.TestUtils$TopN'");
+    stmt.add("SELECT STREAM ID, SUM(SALARY), TOPN(1, SALARY) FROM FOO WHERE ID >= 0 GROUP BY (ID) HAVING MAX(SALARY) > 0");
+    StormSql sql = StormSql.construct();
+    List<Values> values = new ArrayList<>();
+    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+    sql.execute(stmt, h);
+    Assert.assertEquals(4, values.size());
+    Assert.assertEquals(Collections.singletonList(2), values.get(0).get(2));
+    Assert.assertEquals(Collections.singletonList(5), values.get(1).get(2));
+    Assert.assertEquals(Collections.singletonList(8), values.get(2).get(2));
+    Assert.assertEquals(Collections.singletonList(9), values.get(3).get(2));
+  }
+
+  @Test
+  public void testGroupbySameAggregateOnDifferentColumns() throws Exception {
+    List<String> stmt = new ArrayList<>();
+    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
+    stmt.add("SELECT STREAM ID, COUNT(*), AVG(SALARY), AVG(PCT) FROM FOO WHERE ID = 1 GROUP BY (ID)");
+    StormSql sql = StormSql.construct();
+    List<Values> values = new ArrayList<>();
+    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+    sql.execute(stmt, h);
+    Assert.assertEquals(1, values.size());
+    Assert.assertEquals(1, values.get(0).get(0));
+    Assert.assertEquals(3L, values.get(0).get(1));
+    Assert.assertEquals(4, values.get(0).get(2));
+    Assert.assertEquals(2.5, values.get(0).get(3));
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void testGroupbyBuiltinNotimplemented() throws Exception {
+    List<String> stmt = new ArrayList<>();
+    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
+    stmt.add("SELECT STREAM ID, COUNT(*), STDDEV_POP(SALARY) FROM FOO GROUP BY (ID)");
+    StormSql sql = StormSql.construct();
+    List<Values> values = new ArrayList<>();
+    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+    sql.execute(stmt, h);
+  }
+
+  @Test
+  public void testMinMax() throws Exception {
+    List<String> stmt = new ArrayList<>();
+    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
+    stmt.add("SELECT STREAM ID, COUNT(*), MIN(SALARY), MAX(PCT) FROM FOO GROUP BY (ID)");
+    StormSql sql = StormSql.construct();
+    List<Values> values = new ArrayList<>();
+    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+    sql.execute(stmt, h);
+    Assert.assertEquals(4, values.size());
+    Assert.assertEquals(0, values.get(0).get(2));
+    Assert.assertEquals(3, values.get(1).get(2));
+    Assert.assertEquals(6, values.get(2).get(2));
+    Assert.assertEquals(9, values.get(3).get(2));
+
+    Assert.assertEquals(1.5, values.get(0).get(3));
+    Assert.assertEquals(3.0, values.get(1).get(3));
+    Assert.assertEquals(4.5, values.get(2).get(3));
+    Assert.assertEquals(5.0, values.get(3).get(3));
+  }
+  @Test
+  public void testFilterGroupbyHaving() throws Exception {
+    List<String> stmt = new ArrayList<>();
+    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
+    stmt.add("SELECT STREAM ID, MIN(SALARY) FROM FOO where ID > 0 GROUP BY (ID) HAVING ID > 2 AND MAX(SALARY) > 5");
+    StormSql sql = StormSql.construct();
+    List<Values> values = new ArrayList<>();
+    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+    sql.execute(stmt, h);
+    Assert.assertEquals(1, values.size());
+    Assert.assertEquals(3, values.get(0).get(0));
+    Assert.assertEquals(9, values.get(0).get(1));
+  }
+
+  @Test
+  public void testGroupByMultipleFields() throws Exception {
+    List<String> stmt = new ArrayList<>();
+    stmt.add("CREATE EXTERNAL TABLE FOO (DEPTID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR, EMPID INT) LOCATION 'mockgroup:///foo'");
+    stmt.add("SELECT STREAM DEPTID, EMPID, COUNT(*), MIN(SALARY), MAX(PCT) FROM FOO GROUP BY DEPTID, EMPID");
+    StormSql sql = StormSql.construct();
+    List<Values> values = new ArrayList<>();
+    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+    sql.execute(stmt, h);
+    Assert.assertEquals(7, values.size());
+    Assert.assertEquals(0, values.get(0).get(0));
+    Assert.assertEquals(0, values.get(0).get(1));
+    Assert.assertEquals(2L, values.get(0).get(2));
+  }
+
+  @Test
+  public void testjoin() throws Exception {
+    List<String> stmt = new ArrayList<>();
+    stmt.add("CREATE EXTERNAL TABLE EMP (EMPID INT PRIMARY KEY, EMPNAME VARCHAR, DEPTID INT) LOCATION 'mockemp:///foo'");
+    stmt.add("CREATE EXTERNAL TABLE DEPT (DEPTID INT PRIMARY KEY, DEPTNAME VARCHAR) LOCATION 'mockdept:///foo'");
+    stmt.add("SELECT STREAM EMPID, EMPNAME, DEPTNAME FROM EMP AS e JOIN DEPT AS d ON e.DEPTID = d.DEPTID WHERE e.empid > 0");
+    StormSql sql = StormSql.construct();
+    List<Values> values = new ArrayList<>();
+    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+    sql.execute(stmt, h);
+    System.out.println(values);
+    Assert.assertEquals(3, values.size());
+    Assert.assertEquals("emp1", values.get(0).get(1));
+    Assert.assertEquals("dept1", values.get(0).get(2));
+    Assert.assertEquals("emp2", values.get(1).get(1));
+    Assert.assertEquals("dept1", values.get(1).get(2));
+    Assert.assertEquals("emp3", values.get(2).get(1));
+    Assert.assertEquals("dept2", values.get(2).get(2));
+  }
+
+  @Test
+  public void testjoinAndGroupby() throws Exception {
+    List<String> stmt = new ArrayList<>();
+    stmt.add("CREATE EXTERNAL TABLE EMP (EMPID INT PRIMARY KEY, EMPNAME VARCHAR, DEPTID INT) LOCATION 'mockemp:///foo'");
+    stmt.add("CREATE EXTERNAL TABLE DEPT (DEPTID INT PRIMARY KEY, DEPTNAME VARCHAR) LOCATION 'mockdept:///foo'");
+    stmt.add("SELECT STREAM d.DEPTID, count(EMPID) FROM EMP AS e JOIN DEPT AS d ON e.DEPTID = d.DEPTID WHERE e.empid > 0" +
+                     "GROUP BY d.DEPTID");
+    StormSql sql = StormSql.construct();
+    List<Values> values = new ArrayList<>();
+    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+    sql.execute(stmt, h);
+    Assert.assertEquals(2, values.size());
+    Assert.assertEquals(1, values.get(0).get(0));
+    Assert.assertEquals(2L, values.get(0).get(1));
+    Assert.assertEquals(2, values.get(1).get(0));
+    Assert.assertEquals(1L, values.get(1).get(1));
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprSemantic.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprSemantic.java b/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprSemantic.java
new file mode 100644
index 0000000..634e454
--- /dev/null
+++ b/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprSemantic.java
@@ -0,0 +1,410 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.compiler;
+
+import com.google.common.base.Function;
+import org.apache.storm.sql.compiler.backends.standalone.TestCompilerUtils;
+import org.apache.storm.tuple.Values;
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.storm.sql.TestUtils;
+import org.apache.storm.sql.compiler.backends.standalone.PlanCompiler;
+import org.apache.storm.sql.runtime.ChannelHandler;
+import org.apache.storm.sql.runtime.DataSource;
+import org.apache.storm.sql.runtime.AbstractValuesProcessor;
+import org.junit.Test;
+
+import javax.annotation.Nullable;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class TestExprSemantic {
+  private final JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(
+      RelDataTypeSystem.DEFAULT);
+
+  @Test
+  public void testLogicalExpr() throws Exception {
+    Values v = testExpr(
+        Lists.newArrayList("ID > 0 OR ID < 1", "ID > 0 AND ID < 1",
+                           "NOT (ID > 0 AND ID < 1)"));
+    assertEquals(new Values(true, false, true), v);
+  }
+
+  @Test
+  public void testExpectOperator() throws Exception {
+    Values v = testExpr(
+        Lists.newArrayList("TRUE IS TRUE", "TRUE IS NOT TRUE",
+                           "UNKNOWN IS TRUE", "UNKNOWN IS NOT TRUE",
+                           "TRUE IS FALSE", "UNKNOWN IS NULL",
+                           "UNKNOWN IS NOT NULL"));
+    assertEquals(new Values(true, false, false, true, false, true, false), v);
+  }
+
+  @Test
+  public void testDistinctBetweenLikeSimilarIn() throws Exception {
+    Values v = testExpr(
+            Lists.newArrayList("TRUE IS DISTINCT FROM TRUE",
+                    "TRUE IS NOT DISTINCT FROM FALSE", "3 BETWEEN 1 AND 5",
+                    "10 NOT BETWEEN 1 AND 5", "'hello' LIKE '_e%'",
+                    "'world' NOT LIKE 'wor%'", "'abc' SIMILAR TO '[a-zA-Z]+[cd]{1}'",
+                    "'abe' NOT SIMILAR TO '[a-zA-Z]+[cd]{1}'", "'3' IN ('1', '2', '3', '4')",
+                    "2 NOT IN (1, 3, 5)"));
+    assertEquals(new Values(false, false, true, true, true,
+          false, true, true, true, true), v);
+  }
+
+  @Test
+  public void testCaseStatement() throws Exception {
+    Values v = testExpr(
+            Lists.newArrayList(
+                    "CASE WHEN 'abcd' IN ('a', 'abc', 'abcde') THEN UPPER('a') " +
+                    "WHEN UPPER('abcd') = 'AB' THEN 'b' ELSE {fn CONCAT('abcd', '#')} END",
+                    "CASE WHEN 'ab' IN ('a', 'abc', 'abcde') THEN UPPER('a') " +
+                    "WHEN UPPER('ab') = 'AB' THEN 'b' ELSE {fn CONCAT('ab', '#')} END",
+                    "CASE WHEN 'abc' IN ('a', 'abc', 'abcde') THEN UPPER('a') " +
+                    "WHEN UPPER('abc') = 'AB' THEN 'b' ELSE {fn CONCAT('abc', '#')} END"
+                    )
+    );
+
+    // TODO: The data type of literal Calcite assigns seems to be out of expectation. Please see below logical plan.
+    // LogicalProject(EXPR$0=[CASE(OR(=('abcd', 'a'), =('abcd', 'abc'), =('abcd', 'abcde')), CAST(UPPER('a')):VARCHAR(5) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL, =(UPPER('abcd'), CAST('AB'):CHAR(4) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL), 'b', CAST(||('abcd', '#')):VARCHAR(5) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL)], EXPR$1=[CASE(OR(=('ab', 'a'), =('ab', 'abc'), =('ab', 'abcde')), CAST(UPPER('a')):CHAR(3) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL, =(UPPER('ab'), 'AB'), CAST('b'):CHAR(3) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL, ||('ab', '#'))], EXPR$2=[CASE(OR(=('abc', 'a'), =('abc', 'abc'), =('abc', 'abcde')), CAST(UPPER('a')):CHAR(4) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL, =(UPPER('abc'), CAST('AB'):CHAR(3) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL), CAST('b'):CHAR(4) C
 HARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL, ||('abc', '#'))]): rowcount = 1.0, cumulative cost = {2.0 rows, 5.0 cpu, 0.0 io}, id = 5
+    //   LogicalFilter(condition=[AND(>($0, 0), <($0, 2))]): rowcount = 1.0, cumulative cost = {1.0 rows, 2.0 cpu, 0.0 io}, id = 4
+    //     EnumerableTableScan(table=[[FOO]]): rowcount = 1.0, cumulative cost = {0.0 rows, 1.0 cpu, 0.0 io}, id = 3
+    // in result, both 'b' and UPPER('a') hence 'A' are having some spaces which is not expected.
+    // When we use CASE with actual column (Java String type hence VARCHAR), it seems to work as expected.
+    // Please refer trident/TestPlanCompiler#testCaseStatement(), and see below logical plan.
+    // LogicalProject(EXPR$0=[CASE(OR(=($1, 'a'), =($1, 'abc'), =($1, 'abcde')), CAST(UPPER('a')):VARCHAR(1) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary", =(CAST(UPPER($1)):VARCHAR(2) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary", 'AB'), 'b', CAST(||($1, '#')):VARCHAR(1) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary")]): rowcount = 1.0, cumulative cost = {1.0 rows, 2.0 cpu, 0.0 io}, id = 3
+    List<Object> v2 = Lists.transform(v, new Function<Object, Object>() {
+      @Nullable
+      @Override
+      public String apply(@Nullable Object o) {
+        return ((String) o).trim();
+      }
+    });
+    assertArrayEquals(new Values("abcd#", "b", "A").toArray(), v2.toArray());
+  }
+
+  @Test
+  public void testNullIfAndCoalesce() throws Exception {
+    Values v = testExpr(
+            Lists.newArrayList(
+                    "NULLIF(5, 5)", "NULLIF(5, 0)", "COALESCE(NULL, NULL, 5, 4, NULL)", "COALESCE(1, 5)"
+            ));
+    assertEquals(new Values(null, 5, 5, 1), v);
+  }
+
+  @Test
+  public void testCollectionFunctions() throws Exception {
+    Values v = testExpr(
+            Lists.newArrayList(
+                    "ELEMENT(ARRAY[3])", "CARDINALITY(ARRAY[1, 2, 3, 4, 5])"
+            ));
+    assertEquals(new Values(3, 5), v);
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testElementFunctionMoreThanOneValue() throws Exception {
+    testExpr(
+            Lists.newArrayList(
+                    "ELEMENT(ARRAY[1, 2, 3])"
+            ));
+    fail("ELEMENT with array which has multiple elements should throw exception in runtime.");
+  }
+
+  @Test
+  public void testArithmeticWithNull() throws Exception {
+    Values v = testExpr(
+      Lists.newArrayList(
+          "1 + CAST(NULL AS INT)", "CAST(NULL AS INT) + 1", "CAST(NULL AS INT) + CAST(NULL AS INT)", "1 + 2"
+      ));
+    assertEquals(new Values(null, null, null, 3), v);
+  }
+
+  @Test
+  public void testNotWithNull() throws Exception {
+    Values v = testExpr(
+        Lists.newArrayList(
+            "NOT TRUE", "NOT FALSE", "NOT UNKNOWN"
+        ));
+    assertEquals(new Values(false, true, null), v);
+  }
+
+  @Test
+  public void testAndWithNull() throws Exception {
+    Values v = testExpr(
+        Lists.newArrayList(
+            "UNKNOWN AND TRUE", "UNKNOWN AND FALSE", "UNKNOWN AND UNKNOWN",
+            "TRUE AND TRUE", "TRUE AND FALSE", "TRUE AND UNKNOWN",
+            "FALSE AND TRUE", "FALSE AND FALSE", "FALSE AND UNKNOWN"
+        ));
+    assertEquals(new Values(null, false, null, true, false, null, false,
+                            false, false), v);
+  }
+
+  @Test
+  public void testAndWithNullable() throws Exception {
+    Values v = testExpr(
+            Lists.newArrayList(
+                    "ADDR = 'a' AND NAME = 'a'", "NAME = 'a' AND ADDR = 'a'", "NAME = 'x' AND ADDR = 'a'", "ADDR = 'a' AND NAME = 'x'"
+            ));
+    assertEquals(new Values(false, false, null, null), v);
+  }
+
+  @Test
+  public void testOrWithNullable() throws Exception {
+    Values v = testExpr(
+            Lists.newArrayList(
+                    "ADDR = 'a'  OR NAME = 'a'", "NAME = 'a' OR ADDR = 'a' ", "NAME = 'x' OR ADDR = 'a' ", "ADDR = 'a'  OR NAME = 'x'"
+            ));
+    assertEquals(new Values(null, null, true, true), v);
+  }
+
+  @Test
+  public void testOrWithNull() throws Exception {
+    Values v = testExpr(
+        Lists.newArrayList(
+            "UNKNOWN OR TRUE", "UNKNOWN OR FALSE", "UNKNOWN OR UNKNOWN",
+            "TRUE OR TRUE", "TRUE OR FALSE", "TRUE OR UNKNOWN",
+            "FALSE OR TRUE", "FALSE OR FALSE", "FALSE OR UNKNOWN"
+            ));
+    assertEquals(new Values(true, null, null, true, true, true, true,
+                            false, null), v);
+  }
+
+  @Test
+  public void testEquals() throws Exception {
+    Values v = testExpr(
+        Lists.newArrayList(
+            "1 = 2", "UNKNOWN = UNKNOWN", "'a' = 'a'", "'a' = UNKNOWN", "UNKNOWN = 'a'", "'a' = 'b'",
+            "1 <> 2", "UNKNOWN <> UNKNOWN", "'a' <> 'a'", "'a' <> UNKNOWN", "UNKNOWN <> 'a'", "'a' <> 'b'"
+        ));
+    assertEquals(new Values(false, null, true, null, null, false,
+        true, null, false, null, null, true), v);
+  }
+
+  @Test
+  public void testArithmeticFunctions() throws Exception {
+    Values v = testExpr(
+            Lists.newArrayList(
+                    "POWER(3, 2)", "ABS(-10)", "MOD(10, 3)", "MOD(-10, 3)",
+                    "CEIL(123.45)", "FLOOR(123.45)"
+            ));
+
+    assertEquals(new Values(9.0d, 10, 1, -1, new BigDecimal(124), new BigDecimal(123)), v);
+
+    // Belows are floating numbers so comparing this with literal is tend to be failing...
+    // Picking int value and compare
+    Values v2 = testExpr(
+            Lists.newArrayList(
+                    "SQRT(255)", "LN(16)", "LOG10(10000)", "EXP(10)"
+            ));
+    List<Object> v2m = Lists.transform(v2, new Function<Object, Object>() {
+      @Nullable
+      @Override
+      public Object apply(@Nullable Object o) {
+        // only takes int value
+        return ((Number) o).intValue();
+      }
+    });
+
+    // 15.9687, 2.7725, 4.0, 22026.465794
+    assertEquals(new Values(15, 2, 4, 22026), v2m);
+  }
+
+  @Test
+  public void testStringFunctions() throws Exception {
+    Values v = testExpr(
+        Lists.newArrayList(
+                "'ab' || 'cd'", "CHAR_LENGTH('foo')", "CHARACTER_LENGTH('foo')",
+                "UPPER('a')", "LOWER('A')", "POSITION('bc' IN 'abcd')",
+                "TRIM(BOTH ' ' FROM '  abcdeabcdeabc  ')",
+                "TRIM(LEADING ' ' FROM '  abcdeabcdeabc  ')",
+                "TRIM(TRAILING ' ' FROM '  abcdeabcdeabc  ')",
+                "OVERLAY('abcde' PLACING 'bc' FROM 3)",
+                "SUBSTRING('abcde' FROM 3)", "SUBSTRING('abcdeabcde' FROM 3 FOR 4)",
+                "INITCAP('foo')"
+        ));
+    assertEquals(new Values("abcd", 3, 3, "A", "a", 2, "abcdeabcdeabc", "abcdeabcdeabc  ", "  abcdeabcdeabc", "abbce", "cde", "cdea", "Foo"), v);
+  }
+
+  @Test
+  public void testBinaryStringFunctions() throws Exception {
+    Values v = testExpr(
+            Lists.newArrayList(
+                    "x'45F0AB' || x'45F0AB'",
+                    "POSITION(x'F0' IN x'453423F0ABBC')",
+                    "OVERLAY(x'453423F0ABBC45' PLACING x'4534' FROM 3)"
+                    // "SUBSTRING(x'453423F0ABBC' FROM 3)",
+                    // "SUBSTRING(x'453423F0ABBC453423F0ABBC' FROM 3 FOR 4)"
+            ));
+
+    // TODO: Calcite 1.9.0 has bugs on binary SUBSTRING functions
+    // as there's no SqlFunctions.substring(org.apache.calcite.avatica.util.ByteString, ...)
+    // commented out testing substring function
+
+    assertEquals("45f0ab45f0ab", v.get(0).toString());
+    assertEquals(4, v.get(1));
+    assertEquals("45344534abbc45", v.get(2).toString());
+    // assertEquals("23f0abbc", v.get(3).toString());
+    // assertEquals("23f0ab", v.get(4).toString());
+  }
+
+  @Test
+  public void testDateAndTimestampLiteral() throws Exception {
+    Values v = testExpr(
+            Lists.newArrayList(
+                    "DATE '1970-05-15' AS datefield",
+                    "TIME '00:00:00' AS timefield",
+                    "TIMESTAMP '2016-01-01 00:00:00' as timestampfield"
+            )
+    );
+
+    assertEquals(3, v.size());
+    assertEquals(134, v.get(0));
+    assertEquals(0, v.get(1));
+    assertEquals(1451606400000L, v.get(2));
+  }
+
+  @Test
+  public void testInterval() throws Exception {
+    Values v = testExpr(
+            Lists.newArrayList(
+                    "INTERVAL '1-5' YEAR TO MONTH AS intervalfield",
+                    "(DATE '1970-01-01', DATE '1970-01-15') AS anchoredinterval_field"
+            )
+    );
+
+    assertEquals(3, v.size());
+    assertEquals(17, v.get(0));
+    assertEquals(0, v.get(1));
+    assertEquals(14, v.get(2));
+  }
+
+  @Test
+  public void testDateFunctions() throws Exception {
+    Values v = testExpr(
+            Lists.newArrayList(
+                    "LOCALTIME = CURRENT_TIME, LOCALTIMESTAMP = CURRENT_TIMESTAMP, CURRENT_DATE",
+                    "EXTRACT(MONTH FROM TIMESTAMP '2010-01-23 12:34:56')",
+                    "FLOOR(DATE '2016-01-23' TO MONTH)",
+                    "CEIL(TIME '12:34:56' TO MINUTE)"
+            )
+    );
+
+    assertEquals(6, v.size());
+    assertTrue((boolean) v.get(0));
+    assertTrue((boolean) v.get(1));
+    // skip checking CURRENT_DATE since we don't inject dataContext so don't know about current timestamp
+    // we can do it from trident test
+    assertEquals(1L, v.get(3));
+    assertEquals(0L, v.get(4));
+    assertEquals(45300000, v.get(5));
+  }
+
+  @Test
+  public void testJDBCNumericFunctions() throws Exception {
+    Values v = testExpr(
+            Lists.newArrayList(
+                    "{fn POWER(3, 2)}", "{fn ABS(-10)}", "{fn MOD(10, 3)}", "{fn MOD(-10, 3)}"
+            ));
+
+    assertEquals(new Values(9.0d, 10, 1, -1), v);
+
+    // Belows are floating numbers so comparing this with literal is tend to be failing...
+    // Picking int value and compare
+    Values v2 = testExpr(
+            Lists.newArrayList(
+                    "{fn LOG(16)}", "{fn LOG10(10000)}", "{fn EXP(10)}"
+            ));
+    List<Object> v2m = Lists.transform(v2, new Function<Object, Object>() {
+      @Nullable
+      @Override
+      public Object apply(@Nullable Object o) {
+        // only takes int value
+        return ((Number) o).intValue();
+      }
+    });
+
+    // 2.7725, 4.0, 22026.465794
+    assertEquals(new Values(2, 4, 22026), v2m);
+  }
+
+  @Test
+  public void testJDBCStringFunctions() throws Exception {
+    Values v = testExpr(
+            Lists.newArrayList(
+                    "{fn CONCAT('ab', 'cd')}",
+                    "{fn LOCATE('bc', 'abcdeabcde')}",
+                    //"{fn LOCATE('bc', 'abcdeabcde', 4)}",
+                    "{fn INSERT('abcd', 2, 3, 'de')}",
+                    "{fn LCASE('AbCdE')}",
+                    "{fn LENGTH('AbCdE')}",
+                    //"{fn LTRIM('  abcde  ')}",
+                    //"{fn RTRIM('  abcde  ')}",
+                    "{fn SUBSTRING('abcdeabcde', 3, 4)}",
+                    "{fn UCASE('AbCdE')}"
+            )
+    );
+
+    // TODO: Calcite 1.9.0 doesn't support {fn LOCATE(string1, string2 [, integer])}
+    // while it's on support list on SQL reference
+    // and bugs on LTRIM and RTRIM : throwing AssertionError: Internal error: pre-condition failed: pos != null
+    // commented out problematic function tests
+
+    assertEquals(new Values("abcd", 2, "ade", "abcde", 5, "cdea", "ABCDE"), v);
+  }
+
+  @Test
+  public void testJDBCDateTimeFunctions() throws Exception {
+    Values v = testExpr(
+            Lists.newArrayList(
+                    "{fn CURDATE()} = CURRENT_DATE", "{fn CURTIME()} = LOCALTIME", "{fn NOW()} = LOCALTIMESTAMP",
+                    "{fn QUARTER(DATE '2016-10-07')}", "{fn TIMESTAMPADD(MINUTE, 15, TIMESTAMP '2016-10-07 00:00:00')}",
+                    "{fn TIMESTAMPDIFF(SECOND, TIMESTAMP '2016-10-06 00:00:00', TIMESTAMP '2016-10-07 00:00:00')}"
+            )
+    );
+
+    assertEquals(new Values(true, true, true, 4L, 1475799300000L, 86400), v);
+  }
+
+  private Values testExpr(List<String> exprs) throws Exception {
+    String sql = "SELECT " + Joiner.on(',').join(exprs) + " FROM FOO" +
+        " WHERE ID > 0 AND ID < 2";
+    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
+    PlanCompiler compiler = new PlanCompiler(typeFactory);
+    AbstractValuesProcessor proc = compiler.compile(state.tree());
+    Map<String, DataSource> data = new HashMap<>();
+    data.put("FOO", new TestUtils.MockDataSource());
+    List<Values> values = new ArrayList<>();
+    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+    proc.initialize(data, h);
+    return values.get(0);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestCompilerUtils.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestCompilerUtils.java b/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestCompilerUtils.java
new file mode 100644
index 0000000..8e64e9c
--- /dev/null
+++ b/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestCompilerUtils.java
@@ -0,0 +1,183 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.sql.compiler.backends.standalone;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.prepare.CalciteCatalogReader;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.StreamableTable;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.impl.ScalarFunctionImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
+import org.apache.calcite.tools.FrameworkConfig;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.tools.Planner;
+import org.apache.calcite.tools.RelConversionException;
+import org.apache.calcite.tools.ValidationException;
+import org.apache.storm.sql.compiler.CompilerUtil;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class TestCompilerUtils {
+
+    public static class MyPlus {
+        public static Integer eval(Integer x, Integer y) {
+            return x + y;
+        }
+    }
+
+    public static class MyStaticSumFunction {
+        public static long init() {
+            return 0L;
+        }
+        public static long add(long accumulator, int v) {
+            return accumulator + v;
+        }
+    }
+
+    public static class MySumFunction {
+        public MySumFunction() {
+        }
+        public long init() {
+            return 0L;
+        }
+        public long add(long accumulator, int v) {
+            return accumulator + v;
+        }
+        public long result(long accumulator) {
+            return accumulator;
+        }
+    }
+
+    public static CalciteState sqlOverDummyTable(String sql)
+            throws RelConversionException, ValidationException, SqlParseException {
+        SchemaPlus schema = Frameworks.createRootSchema(true);
+        JavaTypeFactory typeFactory = new JavaTypeFactoryImpl
+                (RelDataTypeSystem.DEFAULT);
+        StreamableTable streamableTable = new CompilerUtil.TableBuilderInfo(typeFactory)
+                .field("ID", SqlTypeName.INTEGER)
+                .field("NAME", typeFactory.createType(String.class))
+                .field("ADDR", typeFactory.createType(String.class))
+                .build();
+        Table table = streamableTable.stream();
+        schema.add("FOO", table);
+        schema.add("BAR", table);
+        schema.add("MYPLUS", ScalarFunctionImpl.create(MyPlus.class, "eval"));
+
+        List<SqlOperatorTable> sqlOperatorTables = new ArrayList<>();
+        sqlOperatorTables.add(SqlStdOperatorTable.instance());
+        sqlOperatorTables.add(new CalciteCatalogReader(CalciteSchema.from(schema),
+                false,
+                Collections.<String>emptyList(), typeFactory));
+        SqlOperatorTable chainedSqlOperatorTable = new ChainedSqlOperatorTable(sqlOperatorTables);
+        FrameworkConfig config = Frameworks.newConfigBuilder().defaultSchema(
+                schema).operatorTable(chainedSqlOperatorTable).build();
+        Planner planner = Frameworks.getPlanner(config);
+        SqlNode parse = planner.parse(sql);
+        SqlNode validate = planner.validate(parse);
+        RelNode tree = planner.convert(validate);
+        System.out.println(RelOptUtil.toString(tree, SqlExplainLevel.ALL_ATTRIBUTES));
+        return new CalciteState(schema, tree);
+    }
+
+    public static CalciteState sqlOverNestedTable(String sql)
+            throws RelConversionException, ValidationException, SqlParseException {
+        SchemaPlus schema = Frameworks.createRootSchema(true);
+        JavaTypeFactory typeFactory = new JavaTypeFactoryImpl
+                (RelDataTypeSystem.DEFAULT);
+
+        StreamableTable streamableTable = new CompilerUtil.TableBuilderInfo(typeFactory)
+                .field("ID", SqlTypeName.INTEGER)
+                .field("MAPFIELD",
+                        typeFactory.createTypeWithNullability(
+                                typeFactory.createMapType(
+                                        typeFactory.createTypeWithNullability(
+                                                typeFactory.createSqlType(SqlTypeName.VARCHAR), true),
+                                        typeFactory.createTypeWithNullability(
+                                                typeFactory.createSqlType(SqlTypeName.INTEGER), true))
+                                , true))
+                .field("NESTEDMAPFIELD",
+                        typeFactory.createTypeWithNullability(
+                            typeFactory.createMapType(
+                                    typeFactory.createTypeWithNullability(
+                                            typeFactory.createSqlType(SqlTypeName.VARCHAR), true),
+                                    typeFactory.createTypeWithNullability(
+                                            typeFactory.createMapType(
+                                                    typeFactory.createTypeWithNullability(
+                                                            typeFactory.createSqlType(SqlTypeName.VARCHAR), true),
+                                                    typeFactory.createTypeWithNullability(
+                                                            typeFactory.createSqlType(SqlTypeName.INTEGER), true))
+                                            , true))
+                                        , true))
+                .field("ARRAYFIELD", typeFactory.createTypeWithNullability(
+                        typeFactory.createArrayType(
+                            typeFactory.createTypeWithNullability(
+                                typeFactory.createSqlType(SqlTypeName.INTEGER), true), -1L)
+                        , true))
+                .build();
+        Table table = streamableTable.stream();
+        schema.add("FOO", table);
+        schema.add("BAR", table);
+        schema.add("MYPLUS", ScalarFunctionImpl.create(MyPlus.class, "eval"));
+        List<SqlOperatorTable> sqlOperatorTables = new ArrayList<>();
+        sqlOperatorTables.add(SqlStdOperatorTable.instance());
+        sqlOperatorTables.add(new CalciteCatalogReader(CalciteSchema.from(schema),
+                                                       false,
+                                                       Collections.<String>emptyList(), typeFactory));
+        SqlOperatorTable chainedSqlOperatorTable = new ChainedSqlOperatorTable(sqlOperatorTables);
+        FrameworkConfig config = Frameworks.newConfigBuilder().defaultSchema(
+                schema).operatorTable(chainedSqlOperatorTable).build();
+        Planner planner = Frameworks.getPlanner(config);
+        SqlNode parse = planner.parse(sql);
+        SqlNode validate = planner.validate(parse);
+        RelNode tree = planner.convert(validate);
+        System.out.println(RelOptUtil.toString(tree, SqlExplainLevel.ALL_ATTRIBUTES));
+        return new CalciteState(schema, tree);
+    }
+
+    public static class CalciteState {
+        final SchemaPlus schema;
+        final RelNode tree;
+
+        private CalciteState(SchemaPlus schema, RelNode tree) {
+            this.schema = schema;
+            this.tree = tree;
+        }
+
+        public SchemaPlus schema() {
+            return schema;
+        }
+
+        public RelNode tree() {
+            return tree;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestPlanCompiler.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestPlanCompiler.java b/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestPlanCompiler.java
new file mode 100644
index 0000000..3226810
--- /dev/null
+++ b/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestPlanCompiler.java
@@ -0,0 +1,104 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.compiler.backends.standalone;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.storm.tuple.Values;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.storm.sql.TestUtils;
+import org.apache.storm.sql.runtime.AbstractValuesProcessor;
+import org.apache.storm.sql.runtime.ChannelHandler;
+import org.apache.storm.sql.runtime.DataSource;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class TestPlanCompiler {
+  private final JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(
+      RelDataTypeSystem.DEFAULT);
+
+  @Test
+  public void testCompile() throws Exception {
+    String sql = "SELECT ID + 1 FROM FOO WHERE ID > 2";
+    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
+    PlanCompiler compiler = new PlanCompiler(typeFactory);
+    AbstractValuesProcessor proc = compiler.compile(state.tree());
+    Map<String, DataSource> data = new HashMap<>();
+    data.put("FOO", new TestUtils.MockDataSource());
+    List<Values> values = new ArrayList<>();
+    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+    proc.initialize(data, h);
+    Assert.assertArrayEquals(new Values[] { new Values(4), new Values(5)},
+                             values.toArray());
+  }
+
+  @Test
+  public void testLogicalExpr() throws Exception {
+    String sql = "SELECT ID > 0 OR ID < 1, ID > 0 AND ID < 1, NOT (ID > 0 AND ID < 1) FROM FOO WHERE ID > 0 AND ID < 2";
+    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
+    PlanCompiler compiler = new PlanCompiler(typeFactory);
+    AbstractValuesProcessor proc = compiler.compile(state.tree());
+    Map<String, DataSource> data = new HashMap<>();
+    data.put("FOO", new TestUtils.MockDataSource());
+    List<Values> values = new ArrayList<>();
+    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+    proc.initialize(data, h);
+    Assert.assertEquals(new Values(true, false, true), values.get(0));
+  }
+
+  @Test
+  public void testNested() throws Exception {
+    String sql = "SELECT ID, MAPFIELD['c'], NESTEDMAPFIELD, ARRAYFIELD " +
+            "FROM FOO " +
+            "WHERE NESTEDMAPFIELD['a']['b'] = 2 AND ARRAYFIELD[2] = 200";
+    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverNestedTable(sql);
+    PlanCompiler compiler = new PlanCompiler(typeFactory);
+    AbstractValuesProcessor proc = compiler.compile(state.tree());
+    Map<String, DataSource> data = new HashMap<>();
+    data.put("FOO", new TestUtils.MockNestedDataSource());
+    List<Values> values = new ArrayList<>();
+    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+    proc.initialize(data, h);
+    Map<String, Integer> map = ImmutableMap.of("b", 2, "c", 4);
+    Map<String, Map<String, Integer>> nestedMap = ImmutableMap.of("a", map);
+    Assert.assertEquals(new Values(2, 4, nestedMap, Arrays.asList(100, 200, 300)), values.get(0));
+  }
+
+  @Test
+  public void testUdf() throws Exception {
+    String sql = "SELECT MYPLUS(ID, 3)" +
+            "FROM FOO " +
+            "WHERE ID = 2";
+    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverNestedTable(sql);
+    PlanCompiler compiler = new PlanCompiler(typeFactory);
+    AbstractValuesProcessor proc = compiler.compile(state.tree());
+    Map<String, DataSource> data = new HashMap<>();
+    data.put("FOO", new TestUtils.MockDataSource());
+    List<Values> values = new ArrayList<>();
+    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
+    proc.initialize(data, h);
+    Assert.assertEquals(new Values(5), values.get(0));
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestRelNodeCompiler.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestRelNodeCompiler.java b/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestRelNodeCompiler.java
new file mode 100644
index 0000000..4bee9aa
--- /dev/null
+++ b/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestRelNodeCompiler.java
@@ -0,0 +1,64 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.compiler.backends.standalone;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.Collections;
+
+import static org.hamcrest.CoreMatchers.containsString;
+
+public class TestRelNodeCompiler {
+  @Test
+  public void testFilter() throws Exception {
+    String sql = "SELECT ID + 1 FROM FOO WHERE ID > 3";
+    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
+    JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(
+        RelDataTypeSystem.DEFAULT);
+    LogicalProject project = (LogicalProject) state.tree();
+    LogicalFilter filter = (LogicalFilter) project.getInput();
+
+    try (StringWriter sw = new StringWriter();
+         PrintWriter pw = new PrintWriter(sw)
+    ) {
+      RelNodeCompiler compiler = new RelNodeCompiler(pw, typeFactory);
+      // standalone mode doesn't use inputstreams argument
+      compiler.visitFilter(filter, Collections.EMPTY_LIST);
+      pw.flush();
+      Assert.assertThat(sw.toString(), containsString("> 3"));
+    }
+
+    try (StringWriter sw = new StringWriter();
+         PrintWriter pw = new PrintWriter(sw)
+    ) {
+      RelNodeCompiler compiler = new RelNodeCompiler(pw, typeFactory);
+      // standalone mode doesn't use inputstreams argument
+      compiler.visitProject(project, Collections.EMPTY_LIST);
+      pw.flush();
+      Assert.assertThat(sw.toString(), containsString(" + 1"));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestCompilerUtils.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestCompilerUtils.java b/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestCompilerUtils.java
new file mode 100644
index 0000000..f6ef1ca
--- /dev/null
+++ b/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestCompilerUtils.java
@@ -0,0 +1,208 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.sql.compiler.backends.trident;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.StreamableTable;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.impl.AggregateFunctionImpl;
+import org.apache.calcite.schema.impl.ScalarFunctionImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.tools.RelConversionException;
+import org.apache.calcite.tools.ValidationException;
+import org.apache.storm.sql.compiler.CompilerUtil;
+import org.apache.storm.sql.planner.trident.QueryPlanner;
+import org.apache.storm.sql.planner.trident.rel.TridentRel;
+import org.apache.storm.sql.planner.StormRelUtils;
+
+public class TestCompilerUtils {
+
+    public static class MyPlus {
+        public static Integer eval(Integer x, Integer y) {
+            return x + y;
+        }
+    }
+
+    public static class MyStaticSumFunction {
+        public static long init() {
+            return 0L;
+        }
+        public static long add(long accumulator, int v) {
+            return accumulator + v;
+        }
+    }
+
+    public static class MySumFunction {
+        public MySumFunction() {
+        }
+        public long init() {
+            return 0L;
+        }
+        public long add(long accumulator, int v) {
+            return accumulator + v;
+        }
+        public long result(long accumulator) {
+            return accumulator;
+        }
+    }
+
+    public static CalciteState sqlOverDummyTable(String sql)
+            throws RelConversionException, ValidationException, SqlParseException {
+        SchemaPlus schema = Frameworks.createRootSchema(true);
+        JavaTypeFactory typeFactory = new JavaTypeFactoryImpl
+                (RelDataTypeSystem.DEFAULT);
+        StreamableTable streamableTable = new CompilerUtil.TableBuilderInfo(typeFactory)
+                .field("ID", SqlTypeName.INTEGER)
+                .field("NAME", typeFactory.createType(String.class))
+                .field("ADDR", typeFactory.createType(String.class))
+                .build();
+        Table table = streamableTable.stream();
+        schema.add("FOO", table);
+        schema.add("BAR", table);
+        schema.add("MYPLUS", ScalarFunctionImpl.create(MyPlus.class, "eval"));
+
+        QueryPlanner queryPlanner = new QueryPlanner(schema);
+        TridentRel tree = queryPlanner.getPlan(sql);
+        System.out.println(StormRelUtils.explain(tree, SqlExplainLevel.ALL_ATTRIBUTES));
+        return new CalciteState(schema, tree);
+    }
+
+    public static CalciteState sqlOverDummyGroupByTable(String sql)
+            throws RelConversionException, ValidationException, SqlParseException {
+        SchemaPlus schema = Frameworks.createRootSchema(true);
+        JavaTypeFactory typeFactory = new JavaTypeFactoryImpl
+                (RelDataTypeSystem.DEFAULT);
+        StreamableTable streamableTable = new CompilerUtil.TableBuilderInfo(typeFactory)
+                .field("ID", SqlTypeName.INTEGER)
+                .field("GRPID", SqlTypeName.INTEGER)
+                .field("NAME", typeFactory.createType(String.class))
+                .field("ADDR", typeFactory.createType(String.class))
+                .field("AGE", SqlTypeName.INTEGER)
+                .field("SCORE", SqlTypeName.INTEGER)
+                .build();
+        Table table = streamableTable.stream();
+        schema.add("FOO", table);
+        schema.add("BAR", table);
+        schema.add("MYSTATICSUM", AggregateFunctionImpl.create(MyStaticSumFunction.class));
+        schema.add("MYSUM", AggregateFunctionImpl.create(MySumFunction.class));
+
+        QueryPlanner queryPlanner = new QueryPlanner(schema);
+        TridentRel tree = queryPlanner.getPlan(sql);
+        System.out.println(StormRelUtils.explain(tree, SqlExplainLevel.ALL_ATTRIBUTES));
+        return new CalciteState(schema, tree);
+    }
+
+    public static CalciteState sqlOverNestedTable(String sql)
+            throws RelConversionException, ValidationException, SqlParseException {
+        SchemaPlus schema = Frameworks.createRootSchema(true);
+        JavaTypeFactory typeFactory = new JavaTypeFactoryImpl
+                (RelDataTypeSystem.DEFAULT);
+
+        StreamableTable streamableTable = new CompilerUtil.TableBuilderInfo(typeFactory)
+                .field("ID", SqlTypeName.INTEGER)
+                .field("MAPFIELD",
+                        typeFactory.createTypeWithNullability(
+                                typeFactory.createMapType(
+                                        typeFactory.createTypeWithNullability(
+                                                typeFactory.createSqlType(SqlTypeName.VARCHAR), true),
+                                        typeFactory.createTypeWithNullability(
+                                                typeFactory.createSqlType(SqlTypeName.INTEGER), true))
+                                , true))
+                .field("NESTEDMAPFIELD",
+                        typeFactory.createTypeWithNullability(
+                            typeFactory.createMapType(
+                                    typeFactory.createTypeWithNullability(
+                                            typeFactory.createSqlType(SqlTypeName.VARCHAR), true),
+                                    typeFactory.createTypeWithNullability(
+                                            typeFactory.createMapType(
+                                                    typeFactory.createTypeWithNullability(
+                                                            typeFactory.createSqlType(SqlTypeName.VARCHAR), true),
+                                                    typeFactory.createTypeWithNullability(
+                                                            typeFactory.createSqlType(SqlTypeName.INTEGER), true))
+                                            , true))
+                                        , true))
+                .field("ARRAYFIELD", typeFactory.createTypeWithNullability(
+                        typeFactory.createArrayType(
+                            typeFactory.createTypeWithNullability(
+                                typeFactory.createSqlType(SqlTypeName.INTEGER), true), -1L)
+                        , true))
+                .build();
+        Table table = streamableTable.stream();
+        schema.add("FOO", table);
+        schema.add("BAR", table);
+        schema.add("MYPLUS", ScalarFunctionImpl.create(MyPlus.class, "eval"));
+
+        QueryPlanner queryPlanner = new QueryPlanner(schema);
+        TridentRel tree = queryPlanner.getPlan(sql);
+        System.out.println(StormRelUtils.explain(tree, SqlExplainLevel.ALL_ATTRIBUTES));
+        return new CalciteState(schema, tree);
+    }
+
+    public static CalciteState sqlOverSimpleEquiJoinTables(String sql)
+            throws RelConversionException, ValidationException, SqlParseException {
+        SchemaPlus schema = Frameworks.createRootSchema(true);
+        JavaTypeFactory typeFactory = new JavaTypeFactoryImpl
+                (RelDataTypeSystem.DEFAULT);
+
+        StreamableTable streamableTable = new CompilerUtil.TableBuilderInfo(typeFactory)
+                .field("EMPID", SqlTypeName.INTEGER)
+                .field("EMPNAME", SqlTypeName.VARCHAR)
+                .field("DEPTID", SqlTypeName.INTEGER)
+                .build();
+        Table table = streamableTable.stream();
+
+        StreamableTable streamableTable2 = new CompilerUtil.TableBuilderInfo(typeFactory)
+                .field("DEPTID", SqlTypeName.INTEGER)
+                .field("DEPTNAME", SqlTypeName.VARCHAR)
+                .build();
+        Table table2 = streamableTable2.stream();
+
+        schema.add("EMP", table);
+        schema.add("DEPT", table2);
+
+        QueryPlanner queryPlanner = new QueryPlanner(schema);
+        TridentRel tree = queryPlanner.getPlan(sql);
+        System.out.println(StormRelUtils.explain(tree, SqlExplainLevel.ALL_ATTRIBUTES));
+        return new CalciteState(schema, tree);
+    }
+
+    public static class CalciteState {
+        final SchemaPlus schema;
+        final RelNode tree;
+
+        private CalciteState(SchemaPlus schema, RelNode tree) {
+            this.schema = schema;
+            this.tree = tree;
+        }
+
+        public SchemaPlus schema() {
+            return schema;
+        }
+
+        public RelNode tree() {
+            return tree;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestPlanCompiler.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestPlanCompiler.java b/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestPlanCompiler.java
new file mode 100644
index 0000000..d3f30b9
--- /dev/null
+++ b/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestPlanCompiler.java
@@ -0,0 +1,232 @@
+/*
+ * *
+ *  * 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
+ *  * <p>
+ *  * http://www.apache.org/licenses/LICENSE-2.0
+ *  * <p>
+ *  * 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.sql.compiler.backends.trident;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.avatica.util.DateTimeUtils;
+import org.apache.storm.Config;
+import org.apache.storm.LocalCluster;
+import org.apache.storm.LocalCluster.LocalTopology;
+import org.apache.storm.sql.TestUtils;
+import org.apache.storm.sql.javac.CompilingClassLoader;
+import org.apache.storm.sql.planner.trident.QueryPlanner;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.sql.AbstractTridentProcessor;
+import org.apache.storm.trident.TridentTopology;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.Utils;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.time.ZoneOffset;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Callable;
+
+import static org.apache.storm.sql.TestUtils.MockState.getCollectedValues;
+
+public class TestPlanCompiler {
+  private static LocalCluster cluster;
+
+  @BeforeClass
+  public static void staticSetup() throws Exception {
+    cluster = new LocalCluster();
+  }
+
+  @AfterClass
+  public static void staticCleanup() {
+    if (cluster!= null) {
+      cluster.shutdown();
+      cluster = null;
+    }
+  }
+
+  @Before
+  public void setUp() {
+    getCollectedValues().clear();
+  }
+
+  @Test
+  public void testCompile() throws Exception {
+    final int EXPECTED_VALUE_SIZE = 2;
+    String sql = "SELECT ID FROM FOO WHERE ID > 2";
+    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
+    final Map<String, ISqlTridentDataSource> data = new HashMap<>();
+    data.put("FOO", new TestUtils.MockSqlTridentDataSource());
+    QueryPlanner planner = new QueryPlanner(state.schema());
+    AbstractTridentProcessor proc = planner.compile(data, sql);
+    final TridentTopology topo = proc.build();
+    Fields f = proc.outputStream().getOutputFields();
+    proc.outputStream().partitionPersist(new TestUtils.MockStateFactory(),
+            f, new TestUtils.MockStateUpdater(), new Fields());
+    runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo);
+    Assert.assertArrayEquals(new Values[] { new Values(3), new Values(4)}, getCollectedValues().toArray());
+  }
+
+  @Test
+  public void testInsert() throws Exception {
+    final int EXPECTED_VALUE_SIZE = 1;
+    String sql = "INSERT INTO BAR SELECT ID, NAME, ADDR FROM FOO WHERE ID > 3";
+    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
+    final Map<String, ISqlTridentDataSource> data = new HashMap<>();
+    data.put("FOO", new TestUtils.MockSqlTridentDataSource());
+    data.put("BAR", new TestUtils.MockSqlTridentDataSource());
+
+    QueryPlanner planner = new QueryPlanner(state.schema());
+    AbstractTridentProcessor proc = planner.compile(data, sql);
+    final TridentTopology topo = proc.build();
+    runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo);
+    Assert.assertArrayEquals(new Values[] { new Values(4, "abcde", "y")}, getCollectedValues().toArray());
+  }
+
+  @Test
+  public void testUdf() throws Exception {
+    int EXPECTED_VALUE_SIZE = 1;
+    String sql = "SELECT MYPLUS(ID, 3)" +
+            "FROM FOO " +
+            "WHERE ID = 2";
+    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
+    Map<String, ISqlTridentDataSource> data = new HashMap<>();
+    data.put("FOO", new TestUtils.MockSqlTridentDataSource());
+
+    QueryPlanner planner = new QueryPlanner(state.schema());
+    AbstractTridentProcessor proc = planner.compile(data, sql);
+    final TridentTopology topo = proc.build();
+    Fields f = proc.outputStream().getOutputFields();
+    proc.outputStream().partitionPersist(new TestUtils.MockStateFactory(),
+            f, new TestUtils.MockStateUpdater(), new Fields());
+    runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo);
+    Assert.assertArrayEquals(new Values[] { new Values(5) }, getCollectedValues().toArray());
+  }
+
+  @Test
+  public void testCaseStatement() throws Exception {
+    int EXPECTED_VALUE_SIZE = 5;
+    String sql = "SELECT CASE WHEN NAME IN ('a', 'abc', 'abcde') THEN UPPER('a') " +
+            "WHEN UPPER(NAME) = 'AB' THEN 'b' ELSE {fn CONCAT(NAME, '#')} END FROM FOO";
+    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
+
+    final Map<String, ISqlTridentDataSource> data = new HashMap<>();
+    data.put("FOO", new TestUtils.MockSqlTridentDataSource());
+
+    QueryPlanner planner = new QueryPlanner(state.schema());
+    AbstractTridentProcessor proc = planner.compile(data, sql);
+    final TridentTopology topo = proc.build();
+    Fields f = proc.outputStream().getOutputFields();
+    proc.outputStream().partitionPersist(new TestUtils.MockStateFactory(), f, new TestUtils.MockStateUpdater(), new Fields());
+    runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo);
+
+    Assert.assertArrayEquals(new Values[]{new Values("A"), new Values("b"), new Values("A"), new Values("abcd#"), new Values("A")}, getCollectedValues().toArray());
+  }
+
+  @Test
+  public void testNested() throws Exception {
+    int EXPECTED_VALUE_SIZE = 1;
+    String sql = "SELECT ID, MAPFIELD['c'], NESTEDMAPFIELD, ARRAYFIELD " +
+            "FROM FOO " +
+            "WHERE NESTEDMAPFIELD['a']['b'] = 2 AND ARRAYFIELD[2] = 200";
+    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverNestedTable(sql);
+
+    final Map<String, ISqlTridentDataSource> data = new HashMap<>();
+    data.put("FOO", new TestUtils.MockSqlTridentNestedDataSource());
+
+    QueryPlanner planner = new QueryPlanner(state.schema());
+    AbstractTridentProcessor proc = planner.compile(data, sql);
+    final TridentTopology topo = proc.build();
+    Fields f = proc.outputStream().getOutputFields();
+    proc.outputStream().partitionPersist(new TestUtils.MockStateFactory(), f, new TestUtils.MockStateUpdater(), new Fields());
+    runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo);
+
+    Map<String, Integer> map = ImmutableMap.of("b", 2, "c", 4);
+    Map<String, Map<String, Integer>> nestedMap = ImmutableMap.of("a", map);
+    Assert.assertArrayEquals(new Values[]{new Values(2, 4, nestedMap, Arrays.asList(100, 200, 300))}, getCollectedValues().toArray());
+  }
+
+  @Test
+  public void testDateKeywords() throws Exception {
+    int EXPECTED_VALUE_SIZE = 1;
+    String sql = "SELECT " +
+            "LOCALTIME, CURRENT_TIME, LOCALTIMESTAMP, CURRENT_TIMESTAMP, CURRENT_DATE " +
+            "FROM FOO " +
+            "WHERE ID > 0 AND ID < 2";
+    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
+
+    final Map<String, ISqlTridentDataSource> data = new HashMap<>();
+    data.put("FOO", new TestUtils.MockSqlTridentDataSource());
+    QueryPlanner planner = new QueryPlanner(state.schema());
+    AbstractTridentProcessor proc = planner.compile(data, sql);
+    final DataContext dataContext = proc.getDataContext();
+    final TridentTopology topo = proc.build();
+    Fields f = proc.outputStream().getOutputFields();
+    proc.outputStream().partitionPersist(new TestUtils.MockStateFactory(), f, new TestUtils.MockStateUpdater(), new Fields());
+    runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo);
+
+    long utcTimestamp = (long) dataContext.get(DataContext.Variable.UTC_TIMESTAMP.camelName);
+    long currentTimestamp = (long) dataContext.get(DataContext.Variable.CURRENT_TIMESTAMP.camelName);
+    long localTimestamp = (long) dataContext.get(DataContext.Variable.LOCAL_TIMESTAMP.camelName);
+
+    System.out.println(getCollectedValues());
+
+    java.sql.Timestamp timestamp = new java.sql.Timestamp(utcTimestamp);
+    int dateInt = (int) timestamp.toLocalDateTime().atOffset(ZoneOffset.UTC).toLocalDate().toEpochDay();
+    int localTimeInt = (int) (localTimestamp % DateTimeUtils.MILLIS_PER_DAY);
+    int currentTimeInt = (int) (currentTimestamp % DateTimeUtils.MILLIS_PER_DAY);
+
+    Assert.assertArrayEquals(new Values[]{new Values(localTimeInt, currentTimeInt, localTimestamp, currentTimestamp, dateInt)}, getCollectedValues().toArray());
+  }
+
+  private void runTridentTopology(final int expectedValueSize, AbstractTridentProcessor proc,
+                                  TridentTopology topo) throws Exception {
+    final Config conf = new Config();
+    conf.setMaxSpoutPending(20);
+
+    if (proc.getClassLoaders() != null && proc.getClassLoaders().size() > 0) {
+      CompilingClassLoader lastClassloader = proc.getClassLoaders().get(proc.getClassLoaders().size() - 1);
+      Utils.setClassLoaderForJavaDeSerialize(lastClassloader);
+    }
+
+    try (LocalTopology stormTopo = cluster.submitTopology("storm-sql", conf, topo.build())) {
+      waitForCompletion(1000 * 1000, new Callable<Boolean>() {
+        @Override
+        public Boolean call() throws Exception {
+          return getCollectedValues().size() < expectedValueSize;
+        }
+      });
+    } finally {
+      while(cluster.getClusterInfo().get_topologies_size() > 0) {
+        Thread.sleep(10);
+      }
+      Utils.resetClassLoaderForJavaDeSerialize();
+    }
+  }
+
+  private void waitForCompletion(long timeout, Callable<Boolean> cond) throws Exception {
+    long start = TestUtils.monotonicNow();
+    while (TestUtils.monotonicNow() - start < timeout && cond.call()) {
+      Thread.sleep(100);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/test/org/apache/storm/sql/parser/TestSqlParser.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/test/org/apache/storm/sql/parser/TestSqlParser.java b/sql/storm-sql-core/src/test/org/apache/storm/sql/parser/TestSqlParser.java
new file mode 100644
index 0000000..68054d8
--- /dev/null
+++ b/sql/storm-sql-core/src/test/org/apache/storm/sql/parser/TestSqlParser.java
@@ -0,0 +1,54 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.parser;
+
+import org.apache.calcite.sql.SqlNode;
+import org.apache.storm.sql.parser.impl.ParseException;
+import org.junit.Test;
+
+public class TestSqlParser {
+  @Test
+  public void testCreateTable() throws Exception {
+    String sql = "CREATE EXTERNAL TABLE foo (bar INT) LOCATION 'kafka:///foo'";
+    parse(sql);
+  }
+
+  @Test
+  public void testCreateTableWithPrimaryKey() throws Exception {
+    String sql = "CREATE EXTERNAL TABLE foo (bar INT PRIMARY KEY ASC) LOCATION 'kafka:///foo'";
+    parse(sql);
+  }
+
+  @Test(expected = ParseException.class)
+  public void testCreateTableWithoutLocation() throws Exception {
+    String sql = "CREATE EXTERNAL TABLE foo (bar INT)";
+    parse(sql);
+  }
+
+  @Test
+  public void testCreateFunction() throws Exception {
+    String sql = "CREATE FUNCTION foo AS 'org.apache.storm.sql.MyUDF'";
+    parse(sql);
+  }
+
+  private static SqlNode parse(String sql) throws Exception {
+    StormParser parser = new StormParser(sql);
+    return parser.impl().parseSqlStmtEof();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-external/storm-sql-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/sql/storm-sql-external/storm-sql-hdfs/pom.xml b/sql/storm-sql-external/storm-sql-hdfs/pom.xml
new file mode 100644
index 0000000..9275fbb
--- /dev/null
+++ b/sql/storm-sql-external/storm-sql-hdfs/pom.xml
@@ -0,0 +1,104 @@
+<?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>
+        <relativePath>../../../pom.xml</relativePath>
+    </parent>
+
+    <artifactId>storm-sql-hdfs</artifactId>
+
+    <developers>
+        <developer>
+            <id>vesense</id>
+            <name>Xin Wang</name>
+            <email>data.xinwang@gmail.com</email>
+        </developer>
+    </developers>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-core</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <!--log4j-over-slf4j must be excluded for hadoop-minicluster
+                    see: http://stackoverflow.com/q/20469026/3542091 -->
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>log4j-over-slf4j</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-sql-runtime</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-sql-runtime</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+            <type>test-jar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-hdfs</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-minicluster</artifactId>
+            <version>${hadoop.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+            </exclusions>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-all</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+    <build>
+        <sourceDirectory>src/jvm</sourceDirectory>
+        <testSourceDirectory>src/test</testSourceDirectory>
+        <resources>
+            <resource>
+                <directory>${basedir}/src/resources</directory>
+            </resource>
+        </resources>
+    </build>
+</project>


[13/23] storm git commit: STORM-2453 Move non-connectors into the top directory

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/KafkaDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/KafkaDataSourcesProvider.java b/external/sql/storm-sql-external/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/KafkaDataSourcesProvider.java
deleted file mode 100644
index 2aa98ba..0000000
--- a/external/sql/storm-sql-external/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/KafkaDataSourcesProvider.java
+++ /dev/null
@@ -1,170 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.kafka;
-
-import com.google.common.base.Preconditions;
-import org.apache.storm.kafka.ZkHosts;
-import org.apache.storm.kafka.trident.OpaqueTridentKafkaSpout;
-import org.apache.storm.kafka.trident.TridentKafkaConfig;
-import org.apache.storm.kafka.trident.TridentKafkaStateFactory;
-import org.apache.storm.kafka.trident.TridentKafkaUpdater;
-import org.apache.storm.kafka.trident.mapper.TridentTupleToKafkaMapper;
-import org.apache.storm.kafka.trident.selector.DefaultTopicSelector;
-import org.apache.storm.spout.Scheme;
-import org.apache.storm.spout.SchemeAsMultiScheme;
-import org.apache.storm.sql.runtime.DataSource;
-import org.apache.storm.sql.runtime.DataSourcesProvider;
-import org.apache.storm.sql.runtime.FieldInfo;
-import org.apache.storm.sql.runtime.IOutputSerializer;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.sql.runtime.SimpleSqlTridentConsumer;
-import org.apache.storm.sql.runtime.utils.SerdeUtils;
-import org.apache.storm.trident.spout.ITridentDataSource;
-import org.apache.storm.trident.tuple.TridentTuple;
-
-import java.net.URI;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-/**
- * Create a Kafka spout/sink based on the URI and properties. The URI has the format of
- * kafka://zkhost:port/broker_path?topic=topic. The properties are in JSON format which specifies the producer config
- * of the Kafka broker.
- */
-public class KafkaDataSourcesProvider implements DataSourcesProvider {
-  private static final int DEFAULT_ZK_PORT = 2181;
-
-  private static class SqlKafkaMapper implements TridentTupleToKafkaMapper<Object, ByteBuffer> {
-    private final int primaryKeyIndex;
-    private final IOutputSerializer serializer;
-
-    private SqlKafkaMapper(int primaryKeyIndex, IOutputSerializer serializer) {
-      this.primaryKeyIndex = primaryKeyIndex;
-      this.serializer = serializer;
-    }
-
-    @Override
-    public Object getKeyFromTuple(TridentTuple tuple) {
-      return tuple.get(primaryKeyIndex);
-    }
-
-    @Override
-    public ByteBuffer getMessageFromTuple(TridentTuple tuple) {
-      return serializer.write(tuple.getValues(), null);
-    }
-  }
-
-  private static class KafkaTridentDataSource implements ISqlTridentDataSource {
-    private final TridentKafkaConfig conf;
-    private final String topic;
-    private final int primaryKeyIndex;
-    private final Properties props;
-    private final IOutputSerializer serializer;
-    private KafkaTridentDataSource(TridentKafkaConfig conf, String topic, int primaryKeyIndex,
-                                   Properties props, IOutputSerializer serializer) {
-      this.conf = conf;
-      this.topic = topic;
-      this.primaryKeyIndex = primaryKeyIndex;
-      this.props = props;
-      this.serializer = serializer;
-    }
-
-    @Override
-    public ITridentDataSource getProducer() {
-      return new OpaqueTridentKafkaSpout(conf);
-    }
-
-    @Override
-    public SqlTridentConsumer getConsumer() {
-      Preconditions.checkArgument(!props.isEmpty(),
-              "Writable Kafka Table " + topic + " must contain producer config");
-      HashMap<String, Object> producerConfig = (HashMap<String, Object>) props.get("producer");
-      props.putAll(producerConfig);
-      Preconditions.checkState(props.containsKey("bootstrap.servers"),
-              "Writable Kafka Table " + topic + " must contain \"bootstrap.servers\" config");
-
-      SqlKafkaMapper mapper = new SqlKafkaMapper(primaryKeyIndex, serializer);
-
-      TridentKafkaStateFactory stateFactory = new TridentKafkaStateFactory()
-              .withKafkaTopicSelector(new DefaultTopicSelector(topic))
-              .withProducerProperties(props)
-              .withTridentTupleToKafkaMapper(mapper);
-
-      TridentKafkaUpdater stateUpdater = new TridentKafkaUpdater();
-
-      return new SimpleSqlTridentConsumer(stateFactory, stateUpdater);
-    }
-  }
-
-  @Override
-  public String scheme() {
-    return "kafka";
-  }
-
-  @Override
-  public DataSource construct(URI uri, String inputFormatClass, String outputFormatClass,
-                              List<FieldInfo> fields) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
-                                                Properties properties, List<FieldInfo> fields) {
-    int port = uri.getPort() != -1 ? uri.getPort() : DEFAULT_ZK_PORT;
-    ZkHosts zk = new ZkHosts(uri.getHost() + ":" + port, uri.getPath());
-    Map<String, String> values = parseURIParams(uri.getQuery());
-    String topic = values.get("topic");
-    Preconditions.checkNotNull(topic, "No topic of the spout is specified");
-    TridentKafkaConfig conf = new TridentKafkaConfig(zk, topic);
-    List<String> fieldNames = new ArrayList<>();
-    int primaryIndex = -1;
-    for (int i = 0; i < fields.size(); ++i) {
-      FieldInfo f = fields.get(i);
-      fieldNames.add(f.name());
-      if (f.isPrimary()) {
-        primaryIndex = i;
-      }
-    }
-    Preconditions.checkState(primaryIndex != -1, "Kafka stream table must have a primary key");
-    Scheme scheme = SerdeUtils.getScheme(inputFormatClass, properties, fieldNames);
-    conf.scheme = new SchemeAsMultiScheme(scheme);
-    IOutputSerializer serializer = SerdeUtils.getSerializer(outputFormatClass, properties, fieldNames);
-
-    return new KafkaTridentDataSource(conf, topic, primaryIndex, properties, serializer);
-  }
-
-  private static Map<String, String> parseURIParams(String query) {
-    HashMap<String, String> res = new HashMap<>();
-    if (query == null) {
-      return res;
-    }
-
-    String[] params = query.split("&");
-    for (String p : params) {
-      String[] v = p.split("=", 2);
-      if (v.length > 1) {
-        res.put(v[0], v[1]);
-      }
-    }
-    return res;
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-kafka/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-kafka/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider b/external/sql/storm-sql-external/storm-sql-kafka/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
deleted file mode 100644
index 7f687cc..0000000
--- a/external/sql/storm-sql-external/storm-sql-kafka/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
+++ /dev/null
@@ -1,16 +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.
-
-org.apache.storm.sql.kafka.KafkaDataSourcesProvider
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java b/external/sql/storm-sql-external/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java
deleted file mode 100644
index 0cde492..0000000
--- a/external/sql/storm-sql-external/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.kafka;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.storm.kafka.trident.TridentKafkaState;
-import org.apache.storm.kafka.trident.TridentKafkaStateFactory;
-import org.apache.storm.kafka.trident.TridentKafkaUpdater;
-import org.apache.storm.sql.runtime.DataSourcesRegistry;
-import org.apache.storm.sql.runtime.FieldInfo;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.sql.runtime.serde.json.JsonSerializer;
-import org.apache.storm.trident.tuple.TridentTuple;
-import org.junit.Assert;
-import org.junit.Test;
-import org.mockito.ArgumentMatcher;
-import org.mockito.internal.util.reflection.Whitebox;
-
-import java.net.URI;
-import java.nio.ByteBuffer;
-import java.util.*;
-import java.util.concurrent.Future;
-
-import static org.mockito.Mockito.any;
-import static org.mockito.Mockito.argThat;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.verifyNoMoreInteractions;
-
-public class TestKafkaDataSourcesProvider {
-  private static final List<FieldInfo> FIELDS = ImmutableList.of(
-          new FieldInfo("ID", int.class, true),
-          new FieldInfo("val", String.class, false));
-  private static final List<String> FIELD_NAMES = ImmutableList.of("ID", "val");
-  private static final JsonSerializer SERIALIZER = new JsonSerializer(FIELD_NAMES);
-  private static final Properties TBL_PROPERTIES = new Properties();
-
-  static {
-    Map<String,Object> map = new HashMap<>();
-    map.put("bootstrap.servers", "localhost:9092");
-    map.put("acks", "1");
-    map.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
-    map.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
-    TBL_PROPERTIES.put("producer", map);
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testKafkaSink() {
-    ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(
-            URI.create("kafka://mock?topic=foo"), null, null, TBL_PROPERTIES, FIELDS);
-    Assert.assertNotNull(ds);
-
-    ISqlTridentDataSource.SqlTridentConsumer consumer = ds.getConsumer();
-
-    Assert.assertEquals(TridentKafkaStateFactory.class, consumer.getStateFactory().getClass());
-    Assert.assertEquals(TridentKafkaUpdater.class, consumer.getStateUpdater().getClass());
-
-    TridentKafkaState state = (TridentKafkaState) consumer.getStateFactory().makeState(Collections.emptyMap(), null, 0, 1);
-    KafkaProducer producer = mock(KafkaProducer.class);
-    doReturn(mock(Future.class)).when(producer).send(any(ProducerRecord.class));
-    Whitebox.setInternalState(state, "producer", producer);
-
-    List<TridentTuple> tupleList = mockTupleList();
-    for (TridentTuple t : tupleList) {
-      state.updateState(Collections.singletonList(t), null);
-      verify(producer).send(argThat(new KafkaMessageMatcher(t)));
-    }
-    verifyNoMoreInteractions(producer);
-  }
-
-  private static List<TridentTuple> mockTupleList() {
-    List<TridentTuple> tupleList = new ArrayList<>();
-    TridentTuple t0 = mock(TridentTuple.class);
-    TridentTuple t1 = mock(TridentTuple.class);
-    doReturn(1).when(t0).get(0);
-    doReturn(2).when(t1).get(0);
-    doReturn(Lists.<Object>newArrayList(1, "2")).when(t0).getValues();
-    doReturn(Lists.<Object>newArrayList(2, "3")).when(t1).getValues();
-    tupleList.add(t0);
-    tupleList.add(t1);
-    return tupleList;
-  }
-
-  private static class KafkaMessageMatcher extends ArgumentMatcher<ProducerRecord> {
-    private static final int PRIMARY_INDEX = 0;
-    private final TridentTuple tuple;
-
-    private KafkaMessageMatcher(TridentTuple tuple) {
-      this.tuple = tuple;
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public boolean matches(Object o) {
-      ProducerRecord<Object, ByteBuffer> m = (ProducerRecord<Object,ByteBuffer>)o;
-      if (m.key() != tuple.get(PRIMARY_INDEX)) {
-        return false;
-      }
-      ByteBuffer buf = m.value();
-      ByteBuffer b = SERIALIZER.write(tuple.getValues(), null);
-      return b.equals(buf);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-mongodb/pom.xml
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-mongodb/pom.xml b/external/sql/storm-sql-external/storm-sql-mongodb/pom.xml
deleted file mode 100644
index 40fde8a..0000000
--- a/external/sql/storm-sql-external/storm-sql-mongodb/pom.xml
+++ /dev/null
@@ -1,84 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements.  See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License.  You may obtain a copy of the License at
-
-     http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/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>
-        <relativePath>../../../../pom.xml</relativePath>
-    </parent>
-
-    <artifactId>storm-sql-mongodb</artifactId>
-
-    <developers>
-        <developer>
-            <id>vesense</id>
-            <name>Xin Wang</name>
-            <email>data.xinwang@gmail.com</email>
-        </developer>
-    </developers>
-
-    <dependencies>
-        <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-sql-runtime</artifactId>
-            <version>${project.version}</version>
-            <scope>provided</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>storm-sql-runtime</artifactId>
-            <version>${project.version}</version>
-            <scope>test</scope>
-            <type>test-jar</type>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>storm-mongodb</artifactId>
-            <version>${project.version}</version>
-            <scope>provided</scope>
-        </dependency>
-        <dependency>
-            <groupId>junit</groupId>
-            <artifactId>junit</artifactId>
-            <scope>test</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.mockito</groupId>
-            <artifactId>mockito-all</artifactId>
-            <scope>test</scope>
-        </dependency>
-    </dependencies>
-    <build>
-        <sourceDirectory>src/jvm</sourceDirectory>
-        <testSourceDirectory>src/test</testSourceDirectory>
-        <resources>
-            <resource>
-                <directory>${basedir}/src/resources</directory>
-            </resource>
-        </resources>
-    </build>
-</project>

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-mongodb/src/jvm/org/apache/storm/sql/mongodb/MongoDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-mongodb/src/jvm/org/apache/storm/sql/mongodb/MongoDataSourcesProvider.java b/external/sql/storm-sql-external/storm-sql-mongodb/src/jvm/org/apache/storm/sql/mongodb/MongoDataSourcesProvider.java
deleted file mode 100644
index 60d52d1..0000000
--- a/external/sql/storm-sql-external/storm-sql-mongodb/src/jvm/org/apache/storm/sql/mongodb/MongoDataSourcesProvider.java
+++ /dev/null
@@ -1,126 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.mongodb;
-
-import com.google.common.base.Preconditions;
-import org.apache.storm.mongodb.common.mapper.MongoMapper;
-import org.apache.storm.mongodb.trident.state.MongoState;
-import org.apache.storm.mongodb.trident.state.MongoStateFactory;
-import org.apache.storm.mongodb.trident.state.MongoStateUpdater;
-import org.apache.storm.sql.runtime.DataSource;
-import org.apache.storm.sql.runtime.DataSourcesProvider;
-import org.apache.storm.sql.runtime.FieldInfo;
-import org.apache.storm.sql.runtime.IOutputSerializer;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.sql.runtime.SimpleSqlTridentConsumer;
-import org.apache.storm.sql.runtime.utils.FieldInfoUtils;
-import org.apache.storm.sql.runtime.utils.SerdeUtils;
-import org.apache.storm.trident.spout.ITridentDataSource;
-import org.apache.storm.trident.state.StateFactory;
-import org.apache.storm.trident.state.StateUpdater;
-import org.apache.storm.tuple.ITuple;
-import org.bson.Document;
-
-import java.net.URI;
-import java.util.List;
-import java.util.Properties;
-
-/**
- * Create a MongoDB sink based on the URI and properties. The URI has the format of
- * mongodb://[username:password@]host1[:port1][,host2[:port2],...[,hostN[:portN]]][/[database][?options]].
- * The properties are in JSON format which specifies the name of the MongoDB collection and etc.
- */
-public class MongoDataSourcesProvider implements DataSourcesProvider {
-
-  private static class MongoTridentDataSource implements ISqlTridentDataSource {
-    private final String url;
-    private final Properties props;
-    private final IOutputSerializer serializer;
-
-    private MongoTridentDataSource(String url, Properties props, IOutputSerializer serializer) {
-      this.url = url;
-      this.props = props;
-      this.serializer = serializer;
-    }
-
-    @Override
-    public ITridentDataSource getProducer() {
-      throw new UnsupportedOperationException(this.getClass().getName() + " doesn't provide Producer");
-    }
-
-    @Override
-    public SqlTridentConsumer getConsumer() {
-      Preconditions.checkArgument(!props.isEmpty(), "Writable MongoDB must contain collection config");
-      String serField = props.getProperty("trident.ser.field", "tridentSerField");
-      MongoMapper mapper = new TridentMongoMapper(serField, serializer);
-
-      MongoState.Options options = new MongoState.Options()
-          .withUrl(url)
-          .withCollectionName(props.getProperty("collection.name"))
-          .withMapper(mapper);
-
-      StateFactory stateFactory = new MongoStateFactory(options);
-      StateUpdater stateUpdater = new MongoStateUpdater();
-
-      return new SimpleSqlTridentConsumer(stateFactory, stateUpdater);
-    }
-  }
-
-  private static class TridentMongoMapper implements MongoMapper {
-    private final String serField;
-    private final IOutputSerializer serializer;
-
-    private TridentMongoMapper(String serField, IOutputSerializer serializer) {
-      this.serField = serField;
-      this.serializer = serializer;
-    }
-
-    @Override
-    public Document toDocument(ITuple tuple) {
-      Document document = new Document();
-      byte[] array = serializer.write(tuple.getValues(), null).array();
-      document.append(serField, array);
-      return document;
-    }
-
-    @Override
-    public Document toDocumentByKeys(List<Object> keys) {
-      return null;
-    }
-  }
-
-  @Override
-  public String scheme() {
-    return "mongodb";
-  }
-
-  @Override
-  public DataSource construct(URI uri, String inputFormatClass, String outputFormatClass,
-                              List<FieldInfo> fields) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
-                                                Properties properties, List<FieldInfo> fields) {
-    List<String> fieldNames = FieldInfoUtils.getFieldNames(fields);
-    IOutputSerializer serializer = SerdeUtils.getSerializer(outputFormatClass, properties, fieldNames);
-    return new MongoTridentDataSource(uri.toString(), properties, serializer);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-mongodb/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-mongodb/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider b/external/sql/storm-sql-external/storm-sql-mongodb/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
deleted file mode 100644
index e46d794..0000000
--- a/external/sql/storm-sql-external/storm-sql-mongodb/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
+++ /dev/null
@@ -1,16 +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.
-
-org.apache.storm.sql.mongodb.MongoDataSourcesProvider
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-mongodb/src/test/org/apache/storm/sql/mongodb/TestMongoDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-mongodb/src/test/org/apache/storm/sql/mongodb/TestMongoDataSourcesProvider.java b/external/sql/storm-sql-external/storm-sql-mongodb/src/test/org/apache/storm/sql/mongodb/TestMongoDataSourcesProvider.java
deleted file mode 100644
index 3b15345..0000000
--- a/external/sql/storm-sql-external/storm-sql-mongodb/src/test/org/apache/storm/sql/mongodb/TestMongoDataSourcesProvider.java
+++ /dev/null
@@ -1,122 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.mongodb;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import org.apache.storm.mongodb.common.MongoDBClient;
-import org.apache.storm.mongodb.trident.state.MongoState;
-import org.apache.storm.mongodb.trident.state.MongoStateFactory;
-import org.apache.storm.mongodb.trident.state.MongoStateUpdater;
-import org.apache.storm.sql.runtime.DataSourcesRegistry;
-import org.apache.storm.sql.runtime.FieldInfo;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.sql.runtime.serde.json.JsonSerializer;
-import org.apache.storm.trident.state.StateUpdater;
-import org.apache.storm.trident.tuple.TridentTuple;
-import org.bson.Document;
-import org.junit.Assert;
-import org.junit.Test;
-import org.mockito.ArgumentMatcher;
-import org.mockito.internal.util.reflection.Whitebox;
-
-import java.net.URI;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Properties;
-
-import static org.mockito.Matchers.argThat;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.verifyNoMoreInteractions;
-
-public class TestMongoDataSourcesProvider {
-  private static final List<FieldInfo> FIELDS = ImmutableList.of(
-      new FieldInfo("ID", int.class, true),
-      new FieldInfo("val", String.class, false));
-  private static final List<String> FIELD_NAMES = ImmutableList.of("ID", "val");
-  private static final JsonSerializer SERIALIZER = new JsonSerializer(FIELD_NAMES);
-  private static final Properties TBL_PROPERTIES = new Properties();
-
-  static {
-    TBL_PROPERTIES.put("collection.name", "collection1");
-    TBL_PROPERTIES.put("trident.ser.field", "tridentSerField");
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testMongoSink() {
-    ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(
-            URI.create("mongodb://127.0.0.1:27017/test"), null, null, TBL_PROPERTIES, FIELDS);
-    Assert.assertNotNull(ds);
-
-    ISqlTridentDataSource.SqlTridentConsumer consumer = ds.getConsumer();
-
-    Assert.assertEquals(MongoStateFactory.class, consumer.getStateFactory().getClass());
-    Assert.assertEquals(MongoStateUpdater.class, consumer.getStateUpdater().getClass());
-
-    MongoState state = (MongoState) consumer.getStateFactory().makeState(Collections.emptyMap(), null, 0, 1);
-    StateUpdater stateUpdater = consumer.getStateUpdater();
-
-    MongoDBClient mongoClient = mock(MongoDBClient.class);
-    Whitebox.setInternalState(state, "mongoClient", mongoClient);
-
-    List<TridentTuple> tupleList = mockTupleList();
-
-    for (TridentTuple t : tupleList) {
-      stateUpdater.updateState(state, Collections.singletonList(t), null);
-      verify(mongoClient).insert(argThat(new MongoArgMatcher(t)) , eq(true));
-    }
-
-    verifyNoMoreInteractions(mongoClient);
-  }
-
-  private static List<TridentTuple> mockTupleList() {
-    List<TridentTuple> tupleList = new ArrayList<>();
-    TridentTuple t0 = mock(TridentTuple.class);
-    TridentTuple t1 = mock(TridentTuple.class);
-    doReturn(1).when(t0).get(0);
-    doReturn(2).when(t1).get(0);
-    doReturn(Lists.<Object>newArrayList(1, "2")).when(t0).getValues();
-    doReturn(Lists.<Object>newArrayList(2, "3")).when(t1).getValues();
-    tupleList.add(t0);
-    tupleList.add(t1);
-    return tupleList;
-  }
-
-  private static class MongoArgMatcher extends ArgumentMatcher<List<Document>> {
-    private final TridentTuple tuple;
-
-    private MongoArgMatcher(TridentTuple tuple) {
-      this.tuple = tuple;
-    }
-
-    @SuppressWarnings("unchecked")
-    @Override
-    public boolean matches(Object o) {
-      Document doc = ((List<Document>)o).get(0);
-      ByteBuffer buf = ByteBuffer.wrap((byte[])doc.get(TBL_PROPERTIES.getProperty("trident.ser.field")));
-      ByteBuffer b = SERIALIZER.write(tuple.getValues(), null);
-      return b.equals(buf);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-redis/pom.xml
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-redis/pom.xml b/external/sql/storm-sql-external/storm-sql-redis/pom.xml
deleted file mode 100644
index 19cc699..0000000
--- a/external/sql/storm-sql-external/storm-sql-redis/pom.xml
+++ /dev/null
@@ -1,76 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements.  See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License.  You may obtain a copy of the License at
-
-     http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/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>
-        <relativePath>../../../../pom.xml</relativePath>
-    </parent>
-
-    <artifactId>storm-sql-redis</artifactId>
-
-    <dependencies>
-        <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-sql-runtime</artifactId>
-            <version>${project.version}</version>
-            <scope>provided</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>storm-sql-runtime</artifactId>
-            <version>${project.version}</version>
-            <scope>test</scope>
-            <type>test-jar</type>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>storm-redis</artifactId>
-            <version>${project.version}</version>
-            <scope>provided</scope>
-        </dependency>
-        <dependency>
-            <groupId>junit</groupId>
-            <artifactId>junit</artifactId>
-            <scope>test</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.mockito</groupId>
-            <artifactId>mockito-all</artifactId>
-            <scope>test</scope>
-        </dependency>
-    </dependencies>
-    <build>
-        <sourceDirectory>src/jvm</sourceDirectory>
-        <testSourceDirectory>src/test</testSourceDirectory>
-        <resources>
-            <resource>
-                <directory>${basedir}/src/resources</directory>
-            </resource>
-        </resources>
-    </build>
-</project>

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-redis/src/jvm/org/apache/storm/sql/redis/RedisDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-redis/src/jvm/org/apache/storm/sql/redis/RedisDataSourcesProvider.java b/external/sql/storm-sql-external/storm-sql-redis/src/jvm/org/apache/storm/sql/redis/RedisDataSourcesProvider.java
deleted file mode 100644
index 68933b2..0000000
--- a/external/sql/storm-sql-external/storm-sql-redis/src/jvm/org/apache/storm/sql/redis/RedisDataSourcesProvider.java
+++ /dev/null
@@ -1,228 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.redis;
-
-import com.google.common.base.Preconditions;
-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.apache.storm.redis.trident.state.RedisClusterState;
-import org.apache.storm.redis.trident.state.RedisClusterStateUpdater;
-import org.apache.storm.redis.trident.state.RedisState;
-import org.apache.storm.redis.trident.state.RedisStateUpdater;
-import org.apache.storm.sql.runtime.DataSource;
-import org.apache.storm.sql.runtime.DataSourcesProvider;
-import org.apache.storm.sql.runtime.FieldInfo;
-import org.apache.storm.sql.runtime.IOutputSerializer;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.sql.runtime.SimpleSqlTridentConsumer;
-import org.apache.storm.sql.runtime.utils.SerdeUtils;
-import org.apache.storm.sql.runtime.utils.FieldInfoUtils;
-import org.apache.storm.trident.spout.ITridentDataSource;
-import org.apache.storm.trident.state.StateFactory;
-import org.apache.storm.trident.state.StateUpdater;
-import org.apache.storm.tuple.ITuple;
-import redis.clients.util.JedisURIHelper;
-
-import java.io.Serializable;
-import java.net.InetSocketAddress;
-import java.net.URI;
-import java.util.Collections;
-import java.util.List;
-import java.util.Properties;
-
-/**
- * Create a Redis sink based on the URI and properties. The URI has the format of
- * redis://:[password]@[host]:[port]/[dbIdx]. Only host is mandatory and others can be set to default.
- *
- * The properties are in JSON format which specifies the config of the Redis data type and etc.
- * Please note that when "use.redis.cluster" is "true", cluster discovery is only done from given URI.
- */
-public class RedisDataSourcesProvider implements DataSourcesProvider {
-  private static final int DEFAULT_REDIS_PORT = 6379;
-  private static final int DEFAULT_TIMEOUT = 2000;
-
-  private abstract static class AbstractRedisTridentDataSource implements ISqlTridentDataSource, Serializable {
-    protected abstract StateFactory newStateFactory();
-    protected abstract StateUpdater newStateUpdater(RedisStoreMapper storeMapper);
-
-    private final Properties props;
-    private final List<FieldInfo> fields;
-    private final IOutputSerializer serializer;
-
-    AbstractRedisTridentDataSource(Properties props, List<FieldInfo> fields, IOutputSerializer serializer) {
-      this.props = props;
-      this.fields = fields;
-      this.serializer = serializer;
-    }
-
-    @Override
-    public ITridentDataSource getProducer() {
-      throw new UnsupportedOperationException(this.getClass().getName() + " doesn't provide Producer");
-    }
-
-    @Override
-    public SqlTridentConsumer getConsumer() {
-      RedisDataTypeDescription dataTypeDescription = getDataTypeDesc(props);
-
-      RedisStoreMapper storeMapper = new TridentRedisStoreMapper(dataTypeDescription, fields, serializer);
-
-      StateFactory stateFactory = newStateFactory();
-      StateUpdater stateUpdater = newStateUpdater(storeMapper);
-
-      return new SimpleSqlTridentConsumer(stateFactory, stateUpdater);
-    }
-
-    private RedisDataTypeDescription getDataTypeDesc(Properties props) {
-      Preconditions.checkArgument(props.containsKey("data.type"),
-              "Redis data source must contain \"data.type\" config");
-
-      RedisDataTypeDescription.RedisDataType dataType = RedisDataTypeDescription.RedisDataType.valueOf(props.getProperty("data.type").toUpperCase());
-      String additionalKey = props.getProperty("data.additional.key");
-
-      return new RedisDataTypeDescription(dataType, additionalKey);
-    }
-  }
-
-  private static class RedisClusterTridentDataSource extends AbstractRedisTridentDataSource {
-    private final JedisClusterConfig config;
-
-    RedisClusterTridentDataSource(JedisClusterConfig config, Properties props, List<FieldInfo> fields, IOutputSerializer serializer) {
-      super(props, fields, serializer);
-      this.config = config;
-    }
-
-    @Override
-    protected StateFactory newStateFactory() {
-      return new RedisClusterState.Factory(config);
-    }
-
-    @Override
-    protected StateUpdater newStateUpdater(RedisStoreMapper storeMapper) {
-      return new RedisClusterStateUpdater(storeMapper);
-    }
-  }
-
-  private static class RedisTridentDataSource extends AbstractRedisTridentDataSource {
-    private final JedisPoolConfig config;
-
-    RedisTridentDataSource(JedisPoolConfig config, Properties props, List<FieldInfo> fields, IOutputSerializer serializer) {
-      super(props, fields, serializer);
-      this.config = config;
-    }
-
-    @Override
-    protected StateFactory newStateFactory() {
-      return new RedisState.Factory(config);
-    }
-
-    @Override
-    protected StateUpdater newStateUpdater(RedisStoreMapper storeMapper) {
-      return new RedisStateUpdater(storeMapper);
-    }
-  }
-
-  @Override
-  public String scheme() {
-    return "redis";
-  }
-
-  @Override
-  public DataSource construct(URI uri, String inputFormatClass, String outputFormatClass, List<FieldInfo> fields) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass, Properties props, List<FieldInfo> fields) {
-    Preconditions.checkArgument(JedisURIHelper.isValid(uri), "URI is not valid for Redis: " + uri);
-
-    String host = uri.getHost();
-    int port = uri.getPort() != -1 ? uri.getPort() : DEFAULT_REDIS_PORT;
-    int dbIdx = JedisURIHelper.getDBIndex(uri);
-    String password = JedisURIHelper.getPassword(uri);
-
-    int timeout = Integer.parseInt(props.getProperty("redis.timeout", String.valueOf(DEFAULT_TIMEOUT)));
-
-    boolean clusterMode = Boolean.valueOf(props.getProperty("use.redis.cluster", "false"));
-
-    List<String> fieldNames = FieldInfoUtils.getFieldNames(fields);
-    IOutputSerializer serializer = SerdeUtils.getSerializer(outputFormatClass, props, fieldNames);
-    if (clusterMode) {
-      JedisClusterConfig config = new JedisClusterConfig.Builder()
-              .setNodes(Collections.singleton(new InetSocketAddress(host, port)))
-              .setTimeout(timeout)
-              .build();
-      return new RedisClusterTridentDataSource(config, props, fields, serializer);
-    } else {
-      JedisPoolConfig config = new JedisPoolConfig(host, port, timeout, password, dbIdx);
-      return new RedisTridentDataSource(config, props, fields, serializer);
-    }
-  }
-
-  private static class TridentRedisStoreMapper implements RedisStoreMapper {
-    private final RedisDataTypeDescription dataTypeDescription;
-    private final FieldInfo primaryKeyField;
-    private final IOutputSerializer outputSerializer;
-
-    private TridentRedisStoreMapper(RedisDataTypeDescription dataTypeDescription, List<FieldInfo> fields, IOutputSerializer outputSerializer) {
-      this.dataTypeDescription = dataTypeDescription;
-      this.outputSerializer = outputSerializer;
-
-      // find primary key from constructor
-      FieldInfo pkField = findPrimaryKeyField(fields);
-      Preconditions.checkArgument(pkField != null, "Primary key must be presented to field list");
-
-      this.primaryKeyField = pkField;
-    }
-
-    private FieldInfo findPrimaryKeyField(List<FieldInfo> fields) {
-      FieldInfo pkField = null;
-      for (FieldInfo field : fields) {
-        if (field.isPrimary()) {
-          // TODO: this assumes key is only from the one field
-          // if not we need to have order of fields in PK
-          pkField = field;
-          break;
-        }
-      }
-      return pkField;
-    }
-
-    @Override
-    public RedisDataTypeDescription getDataTypeDescription() {
-      return dataTypeDescription;
-    }
-
-    @Override
-    public String getKeyFromTuple(ITuple tuple) {
-      String keyFieldName = primaryKeyField.name();
-      Object key = tuple.getValueByField(keyFieldName);
-      if (key == null) {
-        throw new NullPointerException("key field " + keyFieldName + " is null");
-      }
-      return String.valueOf(key);
-    }
-
-    @Override
-    public String getValueFromTuple(ITuple tuple) {
-      byte[] array = outputSerializer.write(tuple.getValues(), null).array();
-      return new String(array);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-redis/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-redis/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider b/external/sql/storm-sql-external/storm-sql-redis/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
deleted file mode 100644
index 23b0444..0000000
--- a/external/sql/storm-sql-external/storm-sql-redis/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
+++ /dev/null
@@ -1,16 +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.
-
-org.apache.storm.sql.redis.RedisDataSourcesProvider
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-redis/src/test/org/apache/storm/sql/redis/TestRedisDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-redis/src/test/org/apache/storm/sql/redis/TestRedisDataSourcesProvider.java b/external/sql/storm-sql-external/storm-sql-redis/src/test/org/apache/storm/sql/redis/TestRedisDataSourcesProvider.java
deleted file mode 100644
index 94d4949..0000000
--- a/external/sql/storm-sql-external/storm-sql-redis/src/test/org/apache/storm/sql/redis/TestRedisDataSourcesProvider.java
+++ /dev/null
@@ -1,158 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.redis;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import org.apache.storm.redis.trident.state.RedisClusterState;
-import org.apache.storm.redis.trident.state.RedisClusterStateUpdater;
-import org.apache.storm.redis.trident.state.RedisState;
-import org.apache.storm.redis.trident.state.RedisStateUpdater;
-import org.apache.storm.sql.runtime.DataSourcesRegistry;
-import org.apache.storm.sql.runtime.FieldInfo;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.sql.runtime.serde.json.JsonSerializer;
-import org.apache.storm.trident.state.StateUpdater;
-import org.apache.storm.trident.tuple.TridentTuple;
-import org.junit.Assert;
-import org.junit.Test;
-import org.mockito.internal.util.reflection.Whitebox;
-import redis.clients.jedis.Jedis;
-import redis.clients.jedis.JedisCluster;
-import redis.clients.jedis.JedisPool;
-import redis.clients.jedis.Pipeline;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Properties;
-
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-public class TestRedisDataSourcesProvider {
-  private static final List<FieldInfo> FIELDS = ImmutableList.of(
-      new FieldInfo("ID", int.class, true),
-      new FieldInfo("val", String.class, false));
-  private static final List<String> FIELD_NAMES = ImmutableList.of("ID", "val");
-  private static final String ADDITIONAL_KEY = "hello";
-  private static final JsonSerializer SERIALIZER = new JsonSerializer(FIELD_NAMES);
-  private static final Properties TBL_PROPERTIES = new Properties();
-  private static final Properties CLUSTER_TBL_PROPERTIES = new Properties();
-
-  static {
-    TBL_PROPERTIES.put("data.type", "HASH");
-    TBL_PROPERTIES.put("data.additional.key", ADDITIONAL_KEY);
-    CLUSTER_TBL_PROPERTIES.put("data.type", "HASH");
-    CLUSTER_TBL_PROPERTIES.put("data.additional.key", ADDITIONAL_KEY);
-    CLUSTER_TBL_PROPERTIES.put("use.redis.cluster", "true");
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testRedisSink() {
-    ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(
-        URI.create("redis://:foobared@localhost:6380/2"), null, null, TBL_PROPERTIES, FIELDS);
-    Assert.assertNotNull(ds);
-
-    ISqlTridentDataSource.SqlTridentConsumer consumer = ds.getConsumer();
-
-    Assert.assertEquals(RedisState.Factory.class, consumer.getStateFactory().getClass());
-    Assert.assertEquals(RedisStateUpdater.class, consumer.getStateUpdater().getClass());
-
-    RedisState state = (RedisState) consumer.getStateFactory().makeState(Collections.emptyMap(), null, 0, 1);
-    StateUpdater stateUpdater = consumer.getStateUpdater();
-
-    JedisPool mockJedisPool = mock(JedisPool.class);
-    Jedis mockJedis = mock(Jedis.class);
-    Pipeline mockPipeline = mock(Pipeline.class);
-
-    Whitebox.setInternalState(state, "jedisPool", mockJedisPool);
-    when(mockJedisPool.getResource()).thenReturn(mockJedis);
-    when(mockJedis.pipelined()).thenReturn(mockPipeline);
-
-    List<TridentTuple> tupleList = mockTupleList();
-
-    stateUpdater.updateState(state, tupleList, null);
-    for (TridentTuple t : tupleList) {
-      // PK goes to the key
-      String id = String.valueOf(t.getValueByField("ID"));
-      String serializedValue = new String(SERIALIZER.write(t.getValues(), null).array());
-      verify(mockPipeline).hset(eq(ADDITIONAL_KEY), eq(id), eq(serializedValue));
-    }
-
-    verify(mockPipeline).sync();
-    verify(mockJedis).close();
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testRedisClusterSink() throws IOException {
-    ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(
-        URI.create("redis://localhost:6380"), null, null, CLUSTER_TBL_PROPERTIES, FIELDS);
-    Assert.assertNotNull(ds);
-
-    ISqlTridentDataSource.SqlTridentConsumer consumer = ds.getConsumer();
-
-    Assert.assertEquals(RedisClusterState.Factory.class, consumer.getStateFactory().getClass());
-    Assert.assertEquals(RedisClusterStateUpdater.class, consumer.getStateUpdater().getClass());
-
-    RedisClusterState state = (RedisClusterState) consumer.getStateFactory().makeState(Collections.emptyMap(), null, 0, 1);
-    StateUpdater stateUpdater = consumer.getStateUpdater();
-
-    JedisCluster mockJedisCluster = mock(JedisCluster.class);
-
-    Whitebox.setInternalState(state, "jedisCluster", mockJedisCluster);
-
-    List<TridentTuple> tupleList = mockTupleList();
-
-    stateUpdater.updateState(state, tupleList, null);
-    for (TridentTuple t : tupleList) {
-      // PK goes to the key
-      String id = String.valueOf(t.getValueByField("ID"));
-      String serializedValue = new String(SERIALIZER.write(t.getValues(), null).array());
-      verify(mockJedisCluster).hset(eq(ADDITIONAL_KEY), eq(id), eq(serializedValue));
-    }
-
-    verify(mockJedisCluster, never()).close();
-  }
-
-  private static List<TridentTuple> mockTupleList() {
-    List<TridentTuple> tupleList = new ArrayList<>();
-    TridentTuple t0 = mock(TridentTuple.class);
-    TridentTuple t1 = mock(TridentTuple.class);
-    when(t0.getValueByField("ID")).thenReturn(1);
-    when(t0.getValueByField("val")).thenReturn("2");
-    doReturn(Lists.<Object>newArrayList(1, "2")).when(t0).getValues();
-
-    when(t1.getValueByField("ID")).thenReturn(2);
-    when(t1.getValueByField("val")).thenReturn("3");
-    doReturn(Lists.<Object>newArrayList(2, "3")).when(t1).getValues();
-
-    tupleList.add(t0);
-    tupleList.add(t1);
-    return tupleList;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/pom.xml
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/pom.xml b/external/sql/storm-sql-runtime/pom.xml
deleted file mode 100644
index 4666df4..0000000
--- a/external/sql/storm-sql-runtime/pom.xml
+++ /dev/null
@@ -1,136 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements.  See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License.  You may obtain a copy of the License at
-
-     http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/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>
-        <relativePath>../../../pom.xml</relativePath>
-    </parent>
-
-    <artifactId>storm-sql-runtime</artifactId>
-
-    <developers>
-        <developer>
-            <id>haohui</id>
-            <name>Haohui Mai</name>
-            <email>ricetons@gmail.com</email>
-        </developer>
-    </developers>
-
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
-            <version>${project.version}</version>
-            <scope>provided</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.calcite</groupId>
-            <artifactId>calcite-core</artifactId>
-            <version>${calcite.version}</version>
-            <exclusions>
-                <exclusion>
-                    <groupId>commons-dbcp</groupId>
-                    <artifactId>commons-dbcp</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>com.google.code.findbugs</groupId>
-                    <artifactId>jsr305</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.pentaho</groupId>
-                    <artifactId>pentaho-aggdesigner-algorithm</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>com.fasterxml.jackson.core</groupId>
-                    <artifactId>jackson-annotations</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-        <dependency>
-            <groupId>com.fasterxml.jackson.core</groupId>
-            <artifactId>jackson-databind</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.avro</groupId>
-            <artifactId>avro</artifactId>
-            <version>1.7.7</version>
-        </dependency>
-        <dependency>
-            <groupId>commons-lang</groupId>
-            <artifactId>commons-lang</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.commons</groupId>
-            <artifactId>commons-csv</artifactId>
-            <version>1.4</version>
-        </dependency>
-        <dependency>
-            <groupId>org.mockito</groupId>
-            <artifactId>mockito-all</artifactId>
-            <scope>test</scope>
-        </dependency>
-        <dependency>
-            <groupId>junit</groupId>
-            <artifactId>junit</artifactId>
-            <scope>test</scope>
-        </dependency>
-    </dependencies>
-    <build>
-        <sourceDirectory>src/jvm</sourceDirectory>
-        <testSourceDirectory>src/test</testSourceDirectory>
-        <resources>
-            <resource>
-                <directory>${basedir}/src/resources</directory>
-            </resource>
-        </resources>
-        <plugins>
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-jar-plugin</artifactId>
-                <executions>
-                    <execution>
-                        <goals>
-                            <goal>test-jar</goal>
-                        </goals>
-                    </execution>
-                </executions>
-            </plugin>
-            <plugin>
-                <groupId>org.codehaus.mojo</groupId>
-                <artifactId>appassembler-maven-plugin</artifactId>
-                <version>1.9</version>
-                <executions>
-                    <execution>
-                        <id>create-repo</id>
-                        <goals>
-                            <goal>create-repository</goal>
-                        </goals>
-                        <configuration>
-                            <assembleDirectory>${project.build.directory}/app-assembler</assembleDirectory>
-                            <repositoryLayout>flat</repositoryLayout>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-        </plugins>
-    </build>
-</project>

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/calcite/interpreter/StormContext.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/calcite/interpreter/StormContext.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/calcite/interpreter/StormContext.java
deleted file mode 100644
index aa7e435..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/calcite/interpreter/StormContext.java
+++ /dev/null
@@ -1,31 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.calcite.interpreter;
-
-import org.apache.calcite.DataContext;
-
-import java.io.Serializable;
-
-/**
- * This is a hack to use Calcite Context.
- */
-public class StormContext extends Context implements Serializable {
-    public StormContext(DataContext root) {
-        super(root);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractChannelHandler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractChannelHandler.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractChannelHandler.java
deleted file mode 100644
index 64be39d..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractChannelHandler.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.runtime;
-
-import org.apache.storm.tuple.Values;
-
-public abstract class AbstractChannelHandler implements ChannelHandler {
-  @Override
-  public abstract void dataReceived(ChannelContext ctx, Values data);
-
-  @Override
-  public void channelInactive(ChannelContext ctx) {
-
-  }
-
-  @Override
-  public void exceptionCaught(Throwable cause) {
-
-  }
-
-  @Override
-  public void flush(ChannelContext ctx) {
-    ctx.flush();
-  }
-
-  @Override
-  public void setSource(ChannelContext ctx, Object source) {
-
-  }
-
-  public static final AbstractChannelHandler PASS_THROUGH = new AbstractChannelHandler() {
-    @Override
-    public void dataReceived(ChannelContext ctx, Values data) {
-      ctx.emit(data);
-    }
-  };
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractValuesProcessor.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractValuesProcessor.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractValuesProcessor.java
deleted file mode 100644
index 5b18b5b..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractValuesProcessor.java
+++ /dev/null
@@ -1,46 +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.sql.runtime;
-
-import org.apache.storm.tuple.Values;
-import org.apache.storm.sql.runtime.ChannelHandler;
-import org.apache.storm.sql.runtime.DataSource;
-
-import java.util.Map;
-
-/**
- * Subclass of AbstractTupleProcessor provides a series of tuple. It
- * takes a series of iterators of {@link Values} and produces a stream of
- * tuple.
- *
- * The subclass implements the {@see next()} method to provide
- * the output of the stream. It can choose to return null in {@see next()} to
- * indicate that this particular iteration is a no-op. SQL processors depend
- * on this semantic to implement filtering and nullable records.
- */
-public abstract class AbstractValuesProcessor {
-
-  /**
-   * Initialize the data sources.
-   *
-   * @param data a map from the table name to the iterators of the values.
-   *
-   */
-  public abstract void initialize(Map<String, DataSource> data, ChannelHandler
-      result);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelContext.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelContext.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelContext.java
deleted file mode 100644
index 65ad01c..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelContext.java
+++ /dev/null
@@ -1,30 +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.sql.runtime;
-
-import org.apache.storm.tuple.Values;
-
-public interface ChannelContext {
-  /**
-   * Emit data to the next stage of the data pipeline.
-   */
-  void emit(Values data);
-  void fireChannelInactive();
-  void flush();
-  void setSource(Object source);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelHandler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelHandler.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelHandler.java
deleted file mode 100644
index af02b7e..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelHandler.java
+++ /dev/null
@@ -1,41 +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.sql.runtime;
-
-import org.apache.storm.tuple.Values;
-
-/**
- * DataListener provides an event-driven interface for the user to process
- * series of events.
- */
-public interface ChannelHandler {
-  void dataReceived(ChannelContext ctx, Values data);
-
-  /**
-   * The producer of the data has indicated that the channel is no longer
-   * active.
-   * @param ctx
-   */
-  void channelInactive(ChannelContext ctx);
-
-  void exceptionCaught(Throwable cause);
-
-  void flush(ChannelContext ctx);
-
-  void setSource(ChannelContext ctx, Object source);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/Channels.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/Channels.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/Channels.java
deleted file mode 100644
index 3b5eedd..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/Channels.java
+++ /dev/null
@@ -1,109 +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.sql.runtime;
-
-import org.apache.storm.tuple.Values;
-
-public class Channels {
-  private static final ChannelContext VOID_CTX = new ChannelContext() {
-    @Override
-    public void emit(Values data) {}
-
-    @Override
-    public void fireChannelInactive() {}
-
-    @Override
-    public void flush() {
-
-    }
-
-    @Override
-    public void setSource(java.lang.Object source) {
-
-    }
-  };
-
-  private static class ChannelContextAdapter implements ChannelContext {
-    private final ChannelHandler handler;
-    private final ChannelContext next;
-
-    public ChannelContextAdapter(
-        ChannelContext next, ChannelHandler handler) {
-      this.handler = handler;
-      this.next = next;
-    }
-
-    @Override
-    public void emit(Values data) {
-      handler.dataReceived(next, data);
-    }
-
-    @Override
-    public void fireChannelInactive() {
-      handler.channelInactive(next);
-    }
-
-    @Override
-    public void flush() {
-      handler.flush(next);
-    }
-
-    @Override
-    public void setSource(java.lang.Object source) {
-      handler.setSource(next, source);
-      next.setSource(source); // propagate through the chain
-    }
-  }
-
-  private static class ForwardingChannelContext implements ChannelContext {
-    private final ChannelContext next;
-
-    public ForwardingChannelContext(ChannelContext next) {
-      this.next = next;
-    }
-
-    @Override
-    public void emit(Values data) {
-      next.emit(data);
-    }
-
-    @Override
-    public void fireChannelInactive() {
-      next.fireChannelInactive();
-    }
-
-    @Override
-    public void flush() {
-      next.flush();
-    }
-
-    @Override
-    public void setSource(Object source) {
-      next.setSource(source);
-    }
-  }
-
-  public static ChannelContext chain(
-      ChannelContext next, ChannelHandler handler) {
-    return new ChannelContextAdapter(next, handler);
-  }
-
-  public static ChannelContext voidContext() {
-    return VOID_CTX;
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSource.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSource.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSource.java
deleted file mode 100644
index 352af73..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSource.java
+++ /dev/null
@@ -1,27 +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.sql.runtime;
-
-/**
- * A DataSource ingests data in StormSQL. It provides a series of tuple to
- * the downstream {@link ChannelHandler}.
- *
- */
-public interface DataSource {
-  void open(ChannelContext ctx);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesProvider.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesProvider.java
deleted file mode 100644
index dbece9c..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesProvider.java
+++ /dev/null
@@ -1,47 +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.sql.runtime;
-
-import java.net.URI;
-import java.util.List;
-import java.util.Properties;
-
-public interface DataSourcesProvider {
-  /**
-   * @return the scheme of the data source
-   */
-  String scheme();
-
-  /**
-   * Construct a new data source.
-   * @param uri The URI that specifies the data source. The format of the URI
-   *            is fully customizable.
-   * @param inputFormatClass the name of the class that deserializes data.
-   *                         It is null when unspecified.
-   * @param outputFormatClass the name of the class that serializes data. It
-   *                          is null when unspecified.
-   * @param fields The name of the fields and the schema of the table.
-   */
-  DataSource construct(
-      URI uri, String inputFormatClass, String outputFormatClass,
-      List<FieldInfo> fields);
-
-  ISqlTridentDataSource constructTrident(
-          URI uri, String inputFormatClass, String outputFormatClass,
-          Properties properties, List<FieldInfo> fields);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesRegistry.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesRegistry.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesRegistry.java
deleted file mode 100644
index dfefb01..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesRegistry.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.sql.runtime;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.URI;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.ServiceLoader;
-
-public class DataSourcesRegistry {
-  private static final Logger LOG = LoggerFactory.getLogger(
-      DataSourcesRegistry.class);
-  private static final Map<String, DataSourcesProvider> providers;
-
-  static {
-    providers = new HashMap<>();
-    ServiceLoader<DataSourcesProvider> loader = ServiceLoader.load(
-        DataSourcesProvider.class);
-    for (DataSourcesProvider p : loader) {
-      LOG.info("Registering scheme {} with {}", p.scheme(), p);
-      providers.put(p.scheme(), p);
-    }
-  }
-
-  private DataSourcesRegistry() {
-  }
-
-  public static DataSource construct(
-      URI uri, String inputFormatClass, String outputFormatClass,
-      List<FieldInfo> fields) {
-    DataSourcesProvider provider = providers.get(uri.getScheme());
-    if (provider == null) {
-      return null;
-    }
-
-    return provider.construct(uri, inputFormatClass, outputFormatClass, fields);
-  }
-
-  public static ISqlTridentDataSource constructTridentDataSource(
-          URI uri, String inputFormatClass, String outputFormatClass,
-          Properties properties, List<FieldInfo> fields) {
-    DataSourcesProvider provider = providers.get(uri.getScheme());
-    if (provider == null) {
-      return null;
-    }
-
-    return provider.constructTrident(uri, inputFormatClass, outputFormatClass, properties, fields);
-  }
-
-  /**
-   * Allow unit tests to inject data sources.
-   */
-  public static Map<String, DataSourcesProvider> providerMap() {
-    return providers;
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/FieldInfo.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/FieldInfo.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/FieldInfo.java
deleted file mode 100644
index 03b030b..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/FieldInfo.java
+++ /dev/null
@@ -1,47 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.runtime;
-
-import java.io.Serializable;
-
-/**
- * Describe each column of the field
- */
-public class FieldInfo implements Serializable {
-  private final String name;
-  private final Class<?> type;
-  private final boolean isPrimary;
-
-  public FieldInfo(String name, Class<?> type, boolean isPrimary) {
-    this.name = name;
-    this.type = type;
-    this.isPrimary = isPrimary;
-  }
-
-  public String name() {
-    return name;
-  }
-
-  public Class<?> type() {
-    return type;
-  }
-
-  public boolean isPrimary() {
-    return isPrimary;
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/IOutputSerializer.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/IOutputSerializer.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/IOutputSerializer.java
deleted file mode 100644
index b6670d9..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/IOutputSerializer.java
+++ /dev/null
@@ -1,31 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.runtime;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-public interface IOutputSerializer {
-  /**
-   * Serialize the data to a ByteBuffer. The caller can pass in a ByteBuffer so that the serializer can reuse the
-   * memory.
-   *
-   * @return A ByteBuffer contains the serialized result.
-   */
-  ByteBuffer write(List<Object> data, ByteBuffer buffer);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ISqlTridentDataSource.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ISqlTridentDataSource.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ISqlTridentDataSource.java
deleted file mode 100644
index 9eae5ae..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ISqlTridentDataSource.java
+++ /dev/null
@@ -1,65 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.runtime;
-
-import org.apache.storm.trident.spout.ITridentDataSource;
-import org.apache.storm.trident.state.StateFactory;
-import org.apache.storm.trident.state.StateUpdater;
-
-/**
- * A ISqlTridentDataSource specifies how an external data source produces and consumes data.
- */
-public interface ISqlTridentDataSource {
-  /**
-   * SqlTridentConsumer is a data structure containing StateFactory and StateUpdater for consuming tuples with State.
-   *
-   * Please note that StateFactory and StateUpdater should use same class which implements State.
-   *
-   * @see org.apache.storm.trident.state.StateFactory
-   * @see org.apache.storm.trident.state.StateUpdater
-   */
-  interface SqlTridentConsumer {
-    StateFactory getStateFactory();
-    StateUpdater getStateUpdater();
-  }
-
-  /**
-   * Provides instance of ITridentDataSource which can be used as producer in Trident.
-   *
-   * Since ITridentDataSource is a marker interface for Trident Spout interfaces, this method should effectively
-   * return an instance of one of these interfaces (can be changed if Trident API evolves) or descendants:
-   * - IBatchSpout
-   * - ITridentSpout
-   * - IPartitionedTridentSpout
-   * - IOpaquePartitionedTridentSpout
-   *
-   * @see org.apache.storm.trident.spout.ITridentDataSource
-   * @see org.apache.storm.trident.spout.IBatchSpout
-   * @see org.apache.storm.trident.spout.ITridentSpout
-   * @see org.apache.storm.trident.spout.IPartitionedTridentSpout
-   * @see org.apache.storm.trident.spout.IOpaquePartitionedTridentSpout
-   */
-  ITridentDataSource getProducer();
-
-  /**
-   * Provides instance of SqlTridentConsumer which can be used as consumer (State) in Trident.
-   *
-   * @see SqlTridentConsumer
-   */
-  SqlTridentConsumer getConsumer();
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/SimpleSqlTridentConsumer.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/SimpleSqlTridentConsumer.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/SimpleSqlTridentConsumer.java
deleted file mode 100644
index c9abd16..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/SimpleSqlTridentConsumer.java
+++ /dev/null
@@ -1,41 +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.sql.runtime;
-
-import org.apache.storm.trident.state.StateFactory;
-import org.apache.storm.trident.state.StateUpdater;
-
-public class SimpleSqlTridentConsumer implements ISqlTridentDataSource.SqlTridentConsumer {
-    private final StateFactory stateFactory;
-    private final StateUpdater stateUpdater;
-
-    public SimpleSqlTridentConsumer(StateFactory stateFactory, StateUpdater stateUpdater) {
-        this.stateFactory = stateFactory;
-        this.stateUpdater = stateUpdater;
-    }
-
-    @Override
-    public StateFactory getStateFactory() {
-        return stateFactory;
-    }
-
-    @Override
-    public StateUpdater getStateUpdater() {
-        return stateUpdater;
-    }
-}


[22/23] storm git commit: Merge branch 'STORM-2453' of https://github.com/HeartSaVioR/storm into STORM-2453-merge

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


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

Branch: refs/heads/master
Commit: 1dd4bbc4874f38eee0e96eb19a342cc1217baab1
Parents: e84b39d e9d7833
Author: Jungtaek Lim <ka...@gmail.com>
Authored: Thu Apr 6 08:11:02 2017 +0900
Committer: Jungtaek Lim <ka...@gmail.com>
Committed: Thu Apr 6 08:11:02 2017 +0900

----------------------------------------------------------------------
 bin/storm-kafka-monitor                         |   2 +-
 bin/storm.py                                    |   9 +-
 examples/flux-examples/README.md                |  93 --
 examples/flux-examples/pom.xml                  | 152 ----
 .../flux/examples/StatefulWordCounter.java      |  64 --
 .../storm/flux/examples/TestPrintBolt.java      |  39 -
 .../storm/flux/examples/TestWindowBolt.java     |  47 -
 .../storm/flux/examples/WordCountClient.java    |  74 --
 .../apache/storm/flux/examples/WordCounter.java |  71 --
 .../src/main/resources/hbase_bolt.properties    |  18 -
 .../src/main/resources/hdfs_bolt.properties     |  26 -
 .../src/main/resources/kafka_spout.yaml         | 136 ---
 .../src/main/resources/multilang.yaml           |  89 --
 .../src/main/resources/simple_hbase.yaml        |  92 --
 .../src/main/resources/simple_hdfs.yaml         | 105 ---
 .../resources/simple_stateful_wordcount.yaml    |  60 --
 .../src/main/resources/simple_windowing.yaml    |  69 --
 .../src/main/resources/simple_wordcount.yaml    |  68 --
 external/flux/README.md                         | 872 -------------------
 external/flux/flux-core/pom.xml                 | 112 ---
 .../main/java/org/apache/storm/flux/Flux.java   | 261 ------
 .../java/org/apache/storm/flux/FluxBuilder.java | 630 --------------
 .../apache/storm/flux/api/TopologySource.java   |  39 -
 .../org/apache/storm/flux/model/BeanDef.java    |  39 -
 .../storm/flux/model/BeanListReference.java     |  37 -
 .../apache/storm/flux/model/BeanReference.java  |  39 -
 .../org/apache/storm/flux/model/BoltDef.java    |  24 -
 .../storm/flux/model/ConfigMethodDef.java       |  67 --
 .../storm/flux/model/ExecutionContext.java      |  77 --
 .../apache/storm/flux/model/GroupingDef.java    |  77 --
 .../org/apache/storm/flux/model/IncludeDef.java |  54 --
 .../org/apache/storm/flux/model/ObjectDef.java  |  95 --
 .../apache/storm/flux/model/PropertyDef.java    |  58 --
 .../org/apache/storm/flux/model/SpoutDef.java   |  24 -
 .../org/apache/storm/flux/model/StreamDef.java  |  64 --
 .../apache/storm/flux/model/TopologyDef.java    | 216 -----
 .../storm/flux/model/TopologySourceDef.java     |  36 -
 .../org/apache/storm/flux/model/VertexDef.java  |  36 -
 .../apache/storm/flux/parser/FluxParser.java    | 208 -----
 .../flux-core/src/main/resources/splash.txt     |   9 -
 .../org/apache/storm/flux/FluxBuilderTest.java  |  31 -
 .../org/apache/storm/flux/IntegrationTest.java  |  39 -
 .../java/org/apache/storm/flux/TCKTest.java     | 256 ------
 .../multilang/MultilangEnvirontmentTest.java    |  89 --
 .../apache/storm/flux/test/SimpleTopology.java  |  59 --
 .../storm/flux/test/SimpleTopologySource.java   |  52 --
 .../test/SimpleTopologyWithConfigParam.java     |  55 --
 .../org/apache/storm/flux/test/TestBolt.java    | 131 ---
 .../storm/flux/test/TridentTopologySource.java  |  71 --
 .../src/test/resources/configs/bad_hbase.yaml   |  98 ---
 .../test/resources/configs/bad_shell_test.yaml  | 118 ---
 .../resources/configs/config-methods-test.yaml  |  92 --
 .../resources/configs/diamond-topology.yaml     |  87 --
 .../existing-topology-method-override.yaml      |  25 -
 .../existing-topology-reflection-config.yaml    |  24 -
 .../configs/existing-topology-reflection.yaml   |  24 -
 .../configs/existing-topology-trident.yaml      |  24 -
 .../resources/configs/existing-topology.yaml    |  23 -
 .../src/test/resources/configs/hdfs_test.yaml   |  97 ---
 .../test/resources/configs/include_test.yaml    |  25 -
 .../configs/invalid-existing-topology.yaml      |  33 -
 .../src/test/resources/configs/kafka_test.yaml  | 126 ---
 .../src/test/resources/configs/shell_test.yaml  | 118 ---
 .../test/resources/configs/simple_hbase.yaml    | 120 ---
 .../resources/configs/substitution-test.yaml    | 106 ---
 .../src/test/resources/configs/tck.yaml         |  95 --
 .../src/test/resources/configs/test.properties  |  18 -
 .../flux-core/src/test/resources/log4j2.xml     |  34 -
 external/flux/flux-ui/README.md                 |   3 -
 external/flux/flux-wrappers/pom.xml             |  51 --
 .../flux/wrappers/bolts/FluxShellBolt.java      | 175 ----
 .../storm/flux/wrappers/bolts/LogInfoBolt.java  |  44 -
 .../flux/wrappers/spouts/FluxShellSpout.java    | 175 ----
 .../main/resources/resources/randomsentence.js  |  93 --
 .../main/resources/resources/splitsentence.py   |  24 -
 external/flux/pom.xml                           |  64 --
 external/sql/README.md                          | 207 -----
 external/sql/pom.xml                            |  47 -
 external/sql/storm-sql-core/pom.xml             | 279 ------
 .../sql/storm-sql-core/src/codegen/config.fmpp  |  23 -
 .../storm-sql-core/src/codegen/data/Parser.tdd  |  80 --
 .../src/codegen/includes/license.ftl            |  17 -
 .../src/codegen/includes/parserImpls.ftl        | 113 ---
 .../storm/sql/AbstractTridentProcessor.java     |  58 --
 .../src/jvm/org/apache/storm/sql/StormSql.java  |  59 --
 .../jvm/org/apache/storm/sql/StormSqlImpl.java  | 290 ------
 .../org/apache/storm/sql/StormSqlRunner.java    |  85 --
 .../sql/calcite/ParallelStreamableTable.java    |  35 -
 .../apache/storm/sql/compiler/CompilerUtil.java | 183 ----
 .../sql/compiler/RexNodeToJavaCodeCompiler.java | 231 -----
 .../sql/compiler/StormSqlTypeFactoryImpl.java   |  51 --
 .../standalone/BuiltinAggregateFunctions.java   | 238 -----
 .../backends/standalone/PlanCompiler.java       | 139 ---
 .../standalone/PostOrderRelNodeVisitor.java     | 132 ---
 .../backends/standalone/RelNodeCompiler.java    | 484 ----------
 .../storm/sql/javac/CompilingClassLoader.java   | 225 -----
 .../storm/sql/parser/ColumnConstraint.java      |  42 -
 .../storm/sql/parser/ColumnDefinition.java      |  44 -
 .../storm/sql/parser/SqlCreateFunction.java     |  94 --
 .../apache/storm/sql/parser/SqlCreateTable.java | 166 ----
 .../apache/storm/sql/parser/SqlDDLKeywords.java |  27 -
 .../apache/storm/sql/parser/StormParser.java    |  47 -
 .../apache/storm/sql/parser/UnparseUtil.java    |  60 --
 .../sql/planner/StormRelDataTypeSystem.java     |  37 -
 .../apache/storm/sql/planner/StormRelUtils.java |  68 --
 .../planner/UnsupportedOperatorsVisitor.java    |  24 -
 .../storm/sql/planner/rel/StormCalcRelBase.java |  32 -
 .../sql/planner/rel/StormFilterRelBase.java     |  32 -
 .../storm/sql/planner/rel/StormJoinRelBase.java |  36 -
 .../sql/planner/rel/StormProjectRelBase.java    |  35 -
 .../storm/sql/planner/rel/StormRelNode.java     |  24 -
 .../planner/rel/StormStreamInsertRelBase.java   |  36 -
 .../sql/planner/rel/StormStreamScanRelBase.java |  34 -
 .../storm/sql/planner/trident/QueryPlanner.java | 156 ----
 .../sql/planner/trident/TridentPlanCreator.java | 125 ---
 .../planner/trident/TridentStormRuleSets.java   | 110 ---
 .../sql/planner/trident/rel/TridentCalcRel.java |  97 ---
 .../planner/trident/rel/TridentFilterRel.java   |  65 --
 .../trident/rel/TridentLogicalConvention.java   |  67 --
 .../planner/trident/rel/TridentProjectRel.java  |  69 --
 .../sql/planner/trident/rel/TridentRel.java     |  26 -
 .../trident/rel/TridentStreamInsertRel.java     |  76 --
 .../trident/rel/TridentStreamScanRel.java       |  55 --
 .../trident/rules/TridentAggregateRule.java     |  39 -
 .../planner/trident/rules/TridentCalcRule.java  |  45 -
 .../trident/rules/TridentFilterRule.java        |  45 -
 .../planner/trident/rules/TridentJoinRule.java  |  37 -
 .../trident/rules/TridentModifyRule.java        |  71 --
 .../trident/rules/TridentProjectRule.java       |  45 -
 .../planner/trident/rules/TridentScanRule.java  |  58 --
 .../test/org/apache/storm/sql/TestStormSql.java | 492 -----------
 .../storm/sql/compiler/TestExprSemantic.java    | 410 ---------
 .../backends/standalone/TestCompilerUtils.java  | 183 ----
 .../backends/standalone/TestPlanCompiler.java   | 104 ---
 .../standalone/TestRelNodeCompiler.java         |  64 --
 .../backends/trident/TestCompilerUtils.java     | 208 -----
 .../backends/trident/TestPlanCompiler.java      | 235 -----
 .../apache/storm/sql/parser/TestSqlParser.java  |  54 --
 .../storm-sql-external/storm-sql-hdfs/pom.xml   | 104 ---
 .../storm/sql/hdfs/HdfsDataSourcesProvider.java | 135 ---
 ...apache.storm.sql.runtime.DataSourcesProvider |  16 -
 .../sql/hdfs/TestHdfsDataSourcesProvider.java   | 129 ---
 .../storm-sql-external/storm-sql-kafka/pom.xml  |  93 --
 .../sql/kafka/KafkaDataSourcesProvider.java     | 170 ----
 ...apache.storm.sql.runtime.DataSourcesProvider |  16 -
 .../sql/kafka/TestKafkaDataSourcesProvider.java | 125 ---
 .../storm-sql-mongodb/pom.xml                   |  84 --
 .../sql/mongodb/MongoDataSourcesProvider.java   | 126 ---
 ...apache.storm.sql.runtime.DataSourcesProvider |  16 -
 .../mongodb/TestMongoDataSourcesProvider.java   | 122 ---
 .../storm-sql-external/storm-sql-redis/pom.xml  |  76 --
 .../sql/redis/RedisDataSourcesProvider.java     | 228 -----
 ...apache.storm.sql.runtime.DataSourcesProvider |  16 -
 .../sql/redis/TestRedisDataSourcesProvider.java | 158 ----
 external/sql/storm-sql-runtime/pom.xml          | 136 ---
 .../calcite/interpreter/StormContext.java       |  31 -
 .../sql/runtime/AbstractChannelHandler.java     |  52 --
 .../sql/runtime/AbstractValuesProcessor.java    |  46 -
 .../storm/sql/runtime/ChannelContext.java       |  30 -
 .../storm/sql/runtime/ChannelHandler.java       |  41 -
 .../org/apache/storm/sql/runtime/Channels.java  | 109 ---
 .../apache/storm/sql/runtime/DataSource.java    |  27 -
 .../storm/sql/runtime/DataSourcesProvider.java  |  47 -
 .../storm/sql/runtime/DataSourcesRegistry.java  |  76 --
 .../org/apache/storm/sql/runtime/FieldInfo.java |  47 -
 .../storm/sql/runtime/IOutputSerializer.java    |  31 -
 .../sql/runtime/ISqlTridentDataSource.java      |  65 --
 .../sql/runtime/SimpleSqlTridentConsumer.java   |  41 -
 .../storm/sql/runtime/StormSqlFunctions.java    |  34 -
 .../calcite/DebuggableExecutableExpression.java |  45 -
 .../runtime/calcite/ExecutableExpression.java   |  31 -
 .../sql/runtime/calcite/StormDataContext.java   |  79 --
 .../socket/SocketDataSourcesProvider.java       | 101 ---
 .../datasource/socket/trident/SocketState.java  |  89 --
 .../socket/trident/SocketStateUpdater.java      |  59 --
 .../socket/trident/TridentSocketSpout.java      | 177 ----
 .../sql/runtime/serde/avro/AvroScheme.java      |  74 --
 .../sql/runtime/serde/avro/AvroSerializer.java  |  72 --
 .../sql/runtime/serde/avro/CachedSchemas.java   |  41 -
 .../storm/sql/runtime/serde/csv/CsvScheme.java  |  70 --
 .../sql/runtime/serde/csv/CsvSerializer.java    |  59 --
 .../sql/runtime/serde/json/JsonScheme.java      |  58 --
 .../sql/runtime/serde/json/JsonSerializer.java  |  57 --
 .../storm/sql/runtime/serde/tsv/TsvScheme.java  |  58 --
 .../sql/runtime/serde/tsv/TsvSerializer.java    |  54 --
 .../trident/functions/EvaluationCalc.java       |  86 --
 .../trident/functions/EvaluationFilter.java     |  61 --
 .../trident/functions/EvaluationFunction.java   |  67 --
 .../trident/functions/ForwardFunction.java      |  30 -
 .../storm/sql/runtime/utils/FieldInfoUtils.java |  39 -
 .../storm/sql/runtime/utils/SerdeUtils.java     | 123 ---
 .../apache/storm/sql/runtime/utils/Utils.java   |  55 --
 ...apache.storm.sql.runtime.DataSourcesProvider |  16 -
 .../apache/storm/sql/TestAvroSerializer.java    |  72 --
 .../org/apache/storm/sql/TestCsvSerializer.java |  54 --
 .../storm/sql/TestJsonRepresentation.java       |  52 --
 .../org/apache/storm/sql/TestTsvSerializer.java |  46 -
 .../test/org/apache/storm/sql/TestUtils.java    | 584 -------------
 .../socket/TestSocketDataSourceProvider.java    |  94 --
 external/storm-submit-tools/pom.xml             | 216 -----
 .../submit/command/DependencyResolverMain.java  | 158 ----
 .../storm/submit/dependency/AetherUtils.java    |  91 --
 .../apache/storm/submit/dependency/Booter.java  |  51 --
 .../submit/dependency/DependencyResolver.java   |  98 ---
 .../dependency/RepositorySystemFactory.java     |  67 --
 .../submit/dependency/AetherUtilsTest.java      | 102 ---
 .../dependency/DependencyResolverTest.java      |  83 --
 flux/README.md                                  | 872 +++++++++++++++++++
 flux/flux-core/pom.xml                          | 112 +++
 .../main/java/org/apache/storm/flux/Flux.java   | 259 ++++++
 .../java/org/apache/storm/flux/FluxBuilder.java | 630 ++++++++++++++
 .../apache/storm/flux/api/TopologySource.java   |  39 +
 .../org/apache/storm/flux/model/BeanDef.java    |  34 +
 .../storm/flux/model/BeanListReference.java     |  37 +
 .../apache/storm/flux/model/BeanReference.java  |  39 +
 .../org/apache/storm/flux/model/BoltDef.java    |  24 +
 .../storm/flux/model/ConfigMethodDef.java       |  67 ++
 .../storm/flux/model/ExecutionContext.java      |  77 ++
 .../apache/storm/flux/model/GroupingDef.java    |  77 ++
 .../org/apache/storm/flux/model/IncludeDef.java |  54 ++
 .../org/apache/storm/flux/model/ObjectDef.java  |  95 ++
 .../apache/storm/flux/model/PropertyDef.java    |  58 ++
 .../org/apache/storm/flux/model/SpoutDef.java   |  24 +
 .../org/apache/storm/flux/model/StreamDef.java  |  64 ++
 .../apache/storm/flux/model/TopologyDef.java    | 216 +++++
 .../storm/flux/model/TopologySourceDef.java     |  36 +
 .../org/apache/storm/flux/model/VertexDef.java  |  36 +
 .../apache/storm/flux/parser/FluxParser.java    | 206 +++++
 flux/flux-core/src/main/resources/splash.txt    |   9 +
 .../org/apache/storm/flux/FluxBuilderTest.java  |  31 +
 .../org/apache/storm/flux/IntegrationTest.java  |  39 +
 .../java/org/apache/storm/flux/TCKTest.java     | 254 ++++++
 .../multilang/MultilangEnvirontmentTest.java    |  89 ++
 .../apache/storm/flux/test/SimpleTopology.java  |  58 ++
 .../storm/flux/test/SimpleTopologySource.java   |  52 ++
 .../test/SimpleTopologyWithConfigParam.java     |  53 ++
 .../org/apache/storm/flux/test/TestBolt.java    | 131 +++
 .../storm/flux/test/TridentTopologySource.java  |  71 ++
 .../src/test/resources/configs/bad_hbase.yaml   |  98 +++
 .../test/resources/configs/bad_shell_test.yaml  | 118 +++
 .../resources/configs/config-methods-test.yaml  |  92 ++
 .../resources/configs/diamond-topology.yaml     |  87 ++
 .../existing-topology-method-override.yaml      |  25 +
 .../existing-topology-reflection-config.yaml    |  24 +
 .../configs/existing-topology-reflection.yaml   |  24 +
 .../configs/existing-topology-trident.yaml      |  24 +
 .../resources/configs/existing-topology.yaml    |  23 +
 .../src/test/resources/configs/hdfs_test.yaml   |  97 +++
 .../test/resources/configs/include_test.yaml    |  25 +
 .../configs/invalid-existing-topology.yaml      |  33 +
 .../src/test/resources/configs/kafka_test.yaml  | 126 +++
 .../src/test/resources/configs/shell_test.yaml  | 118 +++
 .../test/resources/configs/simple_hbase.yaml    | 120 +++
 .../resources/configs/substitution-test.yaml    | 106 +++
 .../src/test/resources/configs/tck.yaml         |  95 ++
 .../src/test/resources/configs/test.properties  |  18 +
 flux/flux-core/src/test/resources/log4j2.xml    |  34 +
 flux/flux-examples/README.md                    |  93 ++
 flux/flux-examples/pom.xml                      | 146 ++++
 .../flux/examples/StatefulWordCounter.java      |  64 ++
 .../storm/flux/examples/TestPrintBolt.java      |  39 +
 .../storm/flux/examples/TestWindowBolt.java     |  47 +
 .../storm/flux/examples/WordCountClient.java    |  74 ++
 .../apache/storm/flux/examples/WordCounter.java |  71 ++
 .../src/main/resources/hbase_bolt.properties    |  18 +
 .../src/main/resources/hdfs_bolt.properties     |  26 +
 .../src/main/resources/kafka_spout.yaml         | 136 +++
 .../src/main/resources/multilang.yaml           |  89 ++
 .../src/main/resources/simple_hbase.yaml        |  92 ++
 .../src/main/resources/simple_hdfs.yaml         | 105 +++
 .../resources/simple_stateful_wordcount.yaml    |  60 ++
 .../src/main/resources/simple_windowing.yaml    |  69 ++
 .../src/main/resources/simple_wordcount.yaml    |  68 ++
 flux/flux-ui/README.md                          |   3 +
 flux/flux-wrappers/pom.xml                      |  51 ++
 .../flux/wrappers/bolts/FluxShellBolt.java      | 175 ++++
 .../storm/flux/wrappers/bolts/LogInfoBolt.java  |  44 +
 .../flux/wrappers/spouts/FluxShellSpout.java    | 175 ++++
 .../main/resources/resources/randomsentence.js  |  93 ++
 .../main/resources/resources/splitsentence.py   |  24 +
 flux/pom.xml                                    |  64 ++
 pom.xml                                         |   6 +-
 sql/README.md                                   | 207 +++++
 sql/pom.xml                                     |  47 +
 sql/storm-sql-core/pom.xml                      | 279 ++++++
 sql/storm-sql-core/src/codegen/config.fmpp      |  23 +
 sql/storm-sql-core/src/codegen/data/Parser.tdd  |  80 ++
 .../src/codegen/includes/license.ftl            |  17 +
 .../src/codegen/includes/parserImpls.ftl        | 113 +++
 .../storm/sql/AbstractTridentProcessor.java     |  56 ++
 .../src/jvm/org/apache/storm/sql/StormSql.java  |  59 ++
 .../jvm/org/apache/storm/sql/StormSqlImpl.java  | 290 ++++++
 .../org/apache/storm/sql/StormSqlRunner.java    |  85 ++
 .../sql/calcite/ParallelStreamableTable.java    |  35 +
 .../apache/storm/sql/compiler/CompilerUtil.java | 183 ++++
 .../sql/compiler/RexNodeToJavaCodeCompiler.java | 231 +++++
 .../sql/compiler/StormSqlTypeFactoryImpl.java   |  51 ++
 .../standalone/BuiltinAggregateFunctions.java   | 238 +++++
 .../backends/standalone/PlanCompiler.java       | 139 +++
 .../standalone/PostOrderRelNodeVisitor.java     | 132 +++
 .../backends/standalone/RelNodeCompiler.java    | 484 ++++++++++
 .../storm/sql/javac/CompilingClassLoader.java   | 225 +++++
 .../storm/sql/parser/ColumnConstraint.java      |  42 +
 .../storm/sql/parser/ColumnDefinition.java      |  44 +
 .../storm/sql/parser/SqlCreateFunction.java     |  94 ++
 .../apache/storm/sql/parser/SqlCreateTable.java | 166 ++++
 .../apache/storm/sql/parser/SqlDDLKeywords.java |  27 +
 .../apache/storm/sql/parser/StormParser.java    |  47 +
 .../apache/storm/sql/parser/UnparseUtil.java    |  59 ++
 .../sql/planner/StormRelDataTypeSystem.java     |  37 +
 .../apache/storm/sql/planner/StormRelUtils.java |  68 ++
 .../planner/UnsupportedOperatorsVisitor.java    |  24 +
 .../storm/sql/planner/rel/StormCalcRelBase.java |  31 +
 .../sql/planner/rel/StormFilterRelBase.java     |  31 +
 .../storm/sql/planner/rel/StormJoinRelBase.java |  35 +
 .../sql/planner/rel/StormProjectRelBase.java    |  34 +
 .../storm/sql/planner/rel/StormRelNode.java     |  24 +
 .../planner/rel/StormStreamInsertRelBase.java   |  35 +
 .../sql/planner/rel/StormStreamScanRelBase.java |  33 +
 .../storm/sql/planner/trident/QueryPlanner.java | 156 ++++
 .../sql/planner/trident/TridentPlanCreator.java | 123 +++
 .../planner/trident/TridentStormRuleSets.java   | 110 +++
 .../sql/planner/trident/rel/TridentCalcRel.java |  97 +++
 .../planner/trident/rel/TridentFilterRel.java   |  65 ++
 .../trident/rel/TridentLogicalConvention.java   |  67 ++
 .../planner/trident/rel/TridentProjectRel.java  |  69 ++
 .../sql/planner/trident/rel/TridentRel.java     |  25 +
 .../trident/rel/TridentStreamInsertRel.java     |  76 ++
 .../trident/rel/TridentStreamScanRel.java       |  55 ++
 .../trident/rules/TridentAggregateRule.java     |  39 +
 .../planner/trident/rules/TridentCalcRule.java  |  45 +
 .../trident/rules/TridentFilterRule.java        |  45 +
 .../planner/trident/rules/TridentJoinRule.java  |  37 +
 .../trident/rules/TridentModifyRule.java        |  71 ++
 .../trident/rules/TridentProjectRule.java       |  45 +
 .../planner/trident/rules/TridentScanRule.java  |  58 ++
 .../test/org/apache/storm/sql/TestStormSql.java | 492 +++++++++++
 .../storm/sql/compiler/TestExprSemantic.java    | 410 +++++++++
 .../backends/standalone/TestCompilerUtils.java  | 183 ++++
 .../backends/standalone/TestPlanCompiler.java   | 104 +++
 .../standalone/TestRelNodeCompiler.java         |  64 ++
 .../backends/trident/TestCompilerUtils.java     | 208 +++++
 .../backends/trident/TestPlanCompiler.java      | 232 +++++
 .../apache/storm/sql/parser/TestSqlParser.java  |  54 ++
 sql/storm-sql-external/storm-sql-hdfs/pom.xml   | 104 +++
 .../storm/sql/hdfs/HdfsDataSourcesProvider.java | 135 +++
 ...apache.storm.sql.runtime.DataSourcesProvider |  16 +
 .../sql/hdfs/TestHdfsDataSourcesProvider.java   | 129 +++
 sql/storm-sql-external/storm-sql-kafka/pom.xml  |  93 ++
 .../sql/kafka/KafkaDataSourcesProvider.java     | 170 ++++
 ...apache.storm.sql.runtime.DataSourcesProvider |  16 +
 .../sql/kafka/TestKafkaDataSourcesProvider.java | 125 +++
 .../storm-sql-mongodb/pom.xml                   |  84 ++
 .../sql/mongodb/MongoDataSourcesProvider.java   | 126 +++
 ...apache.storm.sql.runtime.DataSourcesProvider |  16 +
 .../mongodb/TestMongoDataSourcesProvider.java   | 122 +++
 sql/storm-sql-external/storm-sql-redis/pom.xml  |  76 ++
 .../sql/redis/RedisDataSourcesProvider.java     | 228 +++++
 ...apache.storm.sql.runtime.DataSourcesProvider |  16 +
 .../sql/redis/TestRedisDataSourcesProvider.java | 158 ++++
 sql/storm-sql-runtime/pom.xml                   | 136 +++
 .../calcite/interpreter/StormContext.java       |  31 +
 .../sql/runtime/AbstractChannelHandler.java     |  52 ++
 .../sql/runtime/AbstractValuesProcessor.java    |  44 +
 .../storm/sql/runtime/ChannelContext.java       |  30 +
 .../storm/sql/runtime/ChannelHandler.java       |  41 +
 .../org/apache/storm/sql/runtime/Channels.java  | 109 +++
 .../apache/storm/sql/runtime/DataSource.java    |  27 +
 .../storm/sql/runtime/DataSourcesProvider.java  |  47 +
 .../storm/sql/runtime/DataSourcesRegistry.java  |  76 ++
 .../org/apache/storm/sql/runtime/FieldInfo.java |  47 +
 .../storm/sql/runtime/IOutputSerializer.java    |  31 +
 .../sql/runtime/ISqlTridentDataSource.java      |  65 ++
 .../sql/runtime/SimpleSqlTridentConsumer.java   |  41 +
 .../storm/sql/runtime/StormSqlFunctions.java    |  34 +
 .../calcite/DebuggableExecutableExpression.java |  45 +
 .../runtime/calcite/ExecutableExpression.java   |  31 +
 .../sql/runtime/calcite/StormDataContext.java   |  79 ++
 .../socket/SocketDataSourcesProvider.java       | 100 +++
 .../datasource/socket/trident/SocketState.java  |  89 ++
 .../socket/trident/SocketStateUpdater.java      |  59 ++
 .../socket/trident/TridentSocketSpout.java      | 177 ++++
 .../sql/runtime/serde/avro/AvroScheme.java      |  74 ++
 .../sql/runtime/serde/avro/AvroSerializer.java  |  72 ++
 .../sql/runtime/serde/avro/CachedSchemas.java   |  41 +
 .../storm/sql/runtime/serde/csv/CsvScheme.java  |  70 ++
 .../sql/runtime/serde/csv/CsvSerializer.java    |  59 ++
 .../sql/runtime/serde/json/JsonScheme.java      |  58 ++
 .../sql/runtime/serde/json/JsonSerializer.java  |  57 ++
 .../storm/sql/runtime/serde/tsv/TsvScheme.java  |  58 ++
 .../sql/runtime/serde/tsv/TsvSerializer.java    |  54 ++
 .../trident/functions/EvaluationCalc.java       |  86 ++
 .../trident/functions/EvaluationFilter.java     |  61 ++
 .../trident/functions/EvaluationFunction.java   |  67 ++
 .../trident/functions/ForwardFunction.java      |  30 +
 .../storm/sql/runtime/utils/FieldInfoUtils.java |  39 +
 .../storm/sql/runtime/utils/SerdeUtils.java     | 123 +++
 .../apache/storm/sql/runtime/utils/Utils.java   |  55 ++
 ...apache.storm.sql.runtime.DataSourcesProvider |  16 +
 .../apache/storm/sql/TestAvroSerializer.java    |  72 ++
 .../org/apache/storm/sql/TestCsvSerializer.java |  54 ++
 .../storm/sql/TestJsonRepresentation.java       |  52 ++
 .../org/apache/storm/sql/TestTsvSerializer.java |  46 +
 .../test/org/apache/storm/sql/TestUtils.java    | 584 +++++++++++++
 .../socket/TestSocketDataSourceProvider.java    |  94 ++
 .../final-package/src/main/assembly/binary.xml  |  87 +-
 storm-submit-tools/pom.xml                      | 216 +++++
 .../submit/command/DependencyResolverMain.java  | 158 ++++
 .../storm/submit/dependency/AetherUtils.java    |  91 ++
 .../apache/storm/submit/dependency/Booter.java  |  51 ++
 .../submit/dependency/DependencyResolver.java   |  98 +++
 .../dependency/RepositorySystemFactory.java     |  67 ++
 .../submit/dependency/AetherUtilsTest.java      | 102 +++
 .../dependency/DependencyResolverTest.java      |  83 ++
 414 files changed, 19519 insertions(+), 19591 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/1dd4bbc4/flux/pom.xml
----------------------------------------------------------------------
diff --cc flux/pom.xml
index 0000000,287b04a..ce8f6a0
mode 000000,100644..100644
--- a/flux/pom.xml
+++ b/flux/pom.xml
@@@ -1,0 -1,85 +1,64 @@@
+ <?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>
+ 
+     <artifactId>flux</artifactId>
+     <packaging>pom</packaging>
+     <name>flux</name>
+ 
+ 
+     <parent>
+         <artifactId>storm</artifactId>
+         <groupId>org.apache.storm</groupId>
+         <version>2.0.0-SNAPSHOT</version>
+         <relativePath>../pom.xml</relativePath>
+     </parent>
+ 
+     <developers>
+         <developer>
+             <id>ptgoetz</id>
+             <name>P. Taylor Goetz</name>
+             <email>ptgoetz@apache.org</email>
+         </developer>
+     </developers>
+ 
 -    <properties>
 -        <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
 -        <!-- see comment below... This fixes an annoyance with intellij -->
 -        <provided.scope>provided</provided.scope>
 -    </properties>
 -
 -    <profiles>
 -        <!--
 -            Hack to make intellij behave.
 -            If you use intellij, enable this profile in your IDE.
 -            It should make life easier.
 -        -->
 -        <profile>
 -            <id>intellij</id>
 -            <properties>
 -                <provided.scope>compile</provided.scope>
 -            </properties>
 -        </profile>
 -    </profiles>
 -
+     <modules>
+         <module>flux-wrappers</module>
+         <module>flux-core</module>
 -        <module>flux-examples</module>
+     </modules>
+ 
+     <dependencies>
+         <dependency>
+             <groupId>org.apache.storm</groupId>
+             <artifactId>storm-core</artifactId>
+             <version>${project.version}</version>
+             <scope>${provided.scope}</scope>
+         </dependency>
+         <dependency>
+             <groupId>commons-cli</groupId>
+             <artifactId>commons-cli</artifactId>
+             <version>1.2</version>
+         </dependency>
+         <dependency>
+             <groupId>org.apache.kafka</groupId>
+             <artifactId>${storm.kafka.artifact.id}</artifactId>
+             <scope>test</scope>
+         </dependency>
+     </dependencies>
+ </project>

http://git-wip-us.apache.org/repos/asf/storm/blob/1dd4bbc4/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/1dd4bbc4/sql/storm-sql-core/pom.xml
----------------------------------------------------------------------
diff --cc sql/storm-sql-core/pom.xml
index 0000000,ca84970..e409658
mode 000000,100644..100644
--- a/sql/storm-sql-core/pom.xml
+++ b/sql/storm-sql-core/pom.xml
@@@ -1,0 -1,279 +1,279 @@@
+ <?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>
+         <relativePath>../../pom.xml</relativePath>
+     </parent>
+ 
+     <artifactId>storm-sql-core</artifactId>
+ 
+     <developers>
+         <developer>
+             <id>haohui</id>
+             <name>Haohui Mai</name>
+             <email>ricetons@gmail.com</email>
+         </developer>
+     </developers>
+ 
+     <dependencies>
+         <dependency>
+             <groupId>org.apache.storm</groupId>
+             <artifactId>storm-core</artifactId>
+             <version>${project.version}</version>
 -            <scope>provided</scope>
++            <scope>${provided.scope}</scope>
+         </dependency>
+         <dependency>
+             <groupId>org.apache.storm</groupId>
+             <artifactId>storm-sql-runtime</artifactId>
+             <version>${project.version}</version>
 -            <scope>provided</scope>
++            <scope>${provided.scope}</scope>
+         </dependency>
+         <dependency>
+             <groupId>org.apache.storm</groupId>
+             <artifactId>storm-sql-runtime</artifactId>
+             <version>${project.version}</version>
+             <scope>test</scope>
+             <type>test-jar</type>
+         </dependency>
+         <dependency>
+             <groupId>org.apache.calcite</groupId>
+             <artifactId>calcite-core</artifactId>
+             <version>${calcite.version}</version>
+             <exclusions>
+                 <exclusion>
+                     <groupId>com.fasterxml.jackson.core</groupId>
+                     <artifactId>jackson-annotations</artifactId>
+                 </exclusion>
+             </exclusions>
+         </dependency>
+         <dependency>
+             <groupId>com.fasterxml.jackson.core</groupId>
+             <artifactId>jackson-annotations</artifactId>
+             <version>${jackson.version}</version>
+         </dependency>
+         <dependency>
+             <groupId>commons-cli</groupId>
+             <artifactId>commons-cli</artifactId>
+         </dependency>
+         <dependency>
+             <groupId>commons-lang</groupId>
+             <artifactId>commons-lang</artifactId>
+             <scope>runtime</scope>
+         </dependency>
+         <dependency>
+             <groupId>junit</groupId>
+             <artifactId>junit</artifactId>
+             <scope>test</scope>
+         </dependency>
+         <dependency>
+             <groupId>org.mockito</groupId>
+             <artifactId>mockito-all</artifactId>
+             <scope>test</scope>
+         </dependency>
+         <dependency>
+             <groupId>org.apache.commons</groupId>
+             <artifactId>commons-collections4</artifactId>
+             <version>4.1</version>
+             <scope>test</scope>
+         </dependency>
+     </dependencies>
+     <build>
+         <sourceDirectory>src/jvm</sourceDirectory>
+         <testSourceDirectory>src/test</testSourceDirectory>
+         <plugins>
+             <plugin>
+                 <artifactId>maven-resources-plugin</artifactId>
+                 <executions>
+                     <execution>
+                         <id>copy-fmpp-resources</id>
+                         <phase>initialize</phase>
+                         <goals>
+                             <goal>copy-resources</goal>
+                         </goals>
+                         <configuration>
+                             <outputDirectory>${project.build.directory}/codegen</outputDirectory>
+                             <resources>
+                                 <resource>
+                                     <directory>src/codegen</directory>
+                                     <filtering>false</filtering>
+                                 </resource>
+                             </resources>
+                         </configuration>
+                     </execution>
+                     <execution>
+                         <id>copy-java-sources</id>
+                         <phase>process-sources</phase>
+                         <goals>
+                             <goal>copy-resources</goal>
+                         </goals>
+                         <configuration>
+                             <outputDirectory>${basedir}/target/classes/</outputDirectory>
+                             <resources>
+                                 <resource>
+                                     <directory>src/jvm</directory>
+                                     <filtering>true</filtering>
+                                 </resource>
+                                 <resource>
+                                     <directory>src/test</directory>
+                                     <filtering>true</filtering>
+                                 </resource>
+                                 <resource>
+                                     <directory>target/generated-sources</directory>
+                                     <!-- <include>*/org</include> -->
+                                     <filtering>true</filtering>
+                                 </resource>
+                             </resources>
+                         </configuration>
+                     </execution>
+                 </executions>
+             </plugin>
+             <plugin>
+                 <groupId>org.apache.maven.plugins</groupId>
+                 <artifactId>maven-jar-plugin</artifactId>
+                 <version>2.2</version>
+                 <executions>
+                     <execution>
+                         <goals>
+                             <goal>test-jar</goal>
+                         </goals>
+                     </execution>
+                 </executions>
+             </plugin>
+             <plugin>
+                 <!-- Extract parser grammar template from calcite-core.jar and put
+                      it under ${project.build.directory} where all freemarker templates are. -->
+                 <groupId>org.apache.maven.plugins</groupId>
+                 <artifactId>maven-dependency-plugin</artifactId>
+                 <version>2.8</version>
+                 <executions>
+                     <execution>
+                         <id>unpack-parser-template</id>
+                         <phase>initialize</phase>
+                         <goals>
+                             <goal>unpack</goal>
+                         </goals>
+                         <configuration>
+                             <artifactItems>
+                                 <artifactItem>
+                                     <groupId>org.apache.calcite</groupId>
+                                     <artifactId>calcite-core</artifactId>
+                                     <type>jar</type>
+                                     <overWrite>true</overWrite>
+                                     <outputDirectory>${project.build.directory}/</outputDirectory>
+                                     <includes>**/Parser.jj</includes>
+                                 </artifactItem>
+                             </artifactItems>
+                         </configuration>
+                     </execution>
+                 </executions>
+             </plugin>
+             <!-- using appassembler-maven-plugin instead of maven-dependency-plugin to copy dependencies
+             as copy and unpack goal are not working together -->
+             <plugin>
+                 <groupId>org.codehaus.mojo</groupId>
+                 <artifactId>appassembler-maven-plugin</artifactId>
+                 <version>1.9</version>
+                 <executions>
+                     <execution>
+                         <id>create-repo</id>
+                         <goals>
+                             <goal>create-repository</goal>
+                         </goals>
+                         <configuration>
+                             <assembleDirectory>${project.build.directory}/app-assembler</assembleDirectory>
+                             <repositoryLayout>flat</repositoryLayout>
+                         </configuration>
+                     </execution>
+                 </executions>
+             </plugin>
+             <plugin>
+                 <groupId>com.googlecode.fmpp-maven-plugin</groupId>
+                 <artifactId>fmpp-maven-plugin</artifactId>
+                 <version>1.0</version>
+                 <dependencies>
+                     <dependency>
+                         <groupId>org.freemarker</groupId>
+                         <artifactId>freemarker</artifactId>
+                         <version>2.3.25-incubating</version>
+                     </dependency>
+                 </dependencies>
+                 <executions>
+                     <execution>
+                         <id>generate-fmpp-sources</id>
+                         <phase>generate-sources</phase>
+                         <goals>
+                             <goal>generate</goal>
+                         </goals>
+                         <configuration>
+                             <cfgFile>${project.build.directory}/codegen/config.fmpp</cfgFile>
+                             <outputDirectory>target/generated-sources</outputDirectory>
+                             <templateDirectory>${project.build.directory}/codegen/templates</templateDirectory>
+                         </configuration>
+                     </execution>
+                 </executions>
+             </plugin>
+             <plugin>
+                 <groupId>org.codehaus.mojo</groupId>
+                 <artifactId>build-helper-maven-plugin</artifactId>
+                 <version>1.5</version>
+                 <executions>
+                     <execution>
+                         <id>add-generated-sources</id>
+                         <phase>process-sources</phase>
+                         <goals>
+                             <goal>add-source</goal>
+                         </goals>
+                         <configuration>
+                             <sources>
+                                 <source>${project.build.directory}/generated-sources</source>
+                             </sources>
+                         </configuration>
+                     </execution>
+                 </executions>
+             </plugin>
+             <plugin>
+                 <groupId>org.codehaus.mojo</groupId>
+                 <artifactId>javacc-maven-plugin</artifactId>
+                 <version>2.4</version>
+                 <executions>
+                     <execution>
+                         <phase>generate-sources</phase>
+                         <id>javacc</id>
+                         <goals>
+                             <goal>javacc</goal>
+                         </goals>
+                         <configuration>
+                             <sourceDirectory>${project.build.directory}/generated-sources/</sourceDirectory>
+                             <includes>
+                                 <include>**/Parser.jj</include>
+                             </includes>
+                             <lookAhead>2</lookAhead>
+                             <isStatic>false</isStatic>
+                             <outputDirectory>${project.build.directory}/generated-sources/</outputDirectory>
+                         </configuration>
+                     </execution>
+                 </executions>
+             </plugin>
+         </plugins>
+     </build>
+ </project>

http://git-wip-us.apache.org/repos/asf/storm/blob/1dd4bbc4/sql/storm-sql-external/storm-sql-hdfs/pom.xml
----------------------------------------------------------------------
diff --cc sql/storm-sql-external/storm-sql-hdfs/pom.xml
index 0000000,9275fbb..d1f37b7
mode 000000,100644..100644
--- a/sql/storm-sql-external/storm-sql-hdfs/pom.xml
+++ b/sql/storm-sql-external/storm-sql-hdfs/pom.xml
@@@ -1,0 -1,104 +1,104 @@@
+ <?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>
+         <relativePath>../../../pom.xml</relativePath>
+     </parent>
+ 
+     <artifactId>storm-sql-hdfs</artifactId>
+ 
+     <developers>
+         <developer>
+             <id>vesense</id>
+             <name>Xin Wang</name>
+             <email>data.xinwang@gmail.com</email>
+         </developer>
+     </developers>
+ 
+     <dependencies>
+         <dependency>
+             <groupId>org.apache.storm</groupId>
+             <artifactId>storm-core</artifactId>
+             <version>${project.version}</version>
 -            <scope>provided</scope>
++            <scope>${provided.scope}</scope>
+             <exclusions>
+                 <!--log4j-over-slf4j must be excluded for hadoop-minicluster
+                     see: http://stackoverflow.com/q/20469026/3542091 -->
+                 <exclusion>
+                     <groupId>org.slf4j</groupId>
+                     <artifactId>log4j-over-slf4j</artifactId>
+                 </exclusion>
+             </exclusions>
+         </dependency>
+         <dependency>
+             <groupId>org.apache.storm</groupId>
+             <artifactId>storm-sql-runtime</artifactId>
+             <version>${project.version}</version>
 -            <scope>provided</scope>
++            <scope>${provided.scope}</scope>
+         </dependency>
+         <dependency>
+             <groupId>org.apache.storm</groupId>
+             <artifactId>storm-sql-runtime</artifactId>
+             <version>${project.version}</version>
+             <scope>test</scope>
+             <type>test-jar</type>
+         </dependency>
+         <dependency>
+             <groupId>org.apache.storm</groupId>
+             <artifactId>storm-hdfs</artifactId>
+             <version>${project.version}</version>
 -            <scope>provided</scope>
++            <scope>${provided.scope}</scope>
+         </dependency>
+         <dependency>
+             <groupId>org.apache.hadoop</groupId>
+             <artifactId>hadoop-minicluster</artifactId>
+             <version>${hadoop.version}</version>
+             <exclusions>
+                 <exclusion>
+                     <groupId>org.slf4j</groupId>
+                     <artifactId>slf4j-log4j12</artifactId>
+                 </exclusion>
+             </exclusions>
+             <scope>test</scope>
+         </dependency>
+         <dependency>
+             <groupId>junit</groupId>
+             <artifactId>junit</artifactId>
+             <scope>test</scope>
+         </dependency>
+         <dependency>
+             <groupId>org.mockito</groupId>
+             <artifactId>mockito-all</artifactId>
+             <scope>test</scope>
+         </dependency>
+     </dependencies>
+     <build>
+         <sourceDirectory>src/jvm</sourceDirectory>
+         <testSourceDirectory>src/test</testSourceDirectory>
+         <resources>
+             <resource>
+                 <directory>${basedir}/src/resources</directory>
+             </resource>
+         </resources>
+     </build>
+ </project>

http://git-wip-us.apache.org/repos/asf/storm/blob/1dd4bbc4/sql/storm-sql-external/storm-sql-kafka/pom.xml
----------------------------------------------------------------------
diff --cc sql/storm-sql-external/storm-sql-kafka/pom.xml
index 0000000,881343d..c7562bb
mode 000000,100644..100644
--- a/sql/storm-sql-external/storm-sql-kafka/pom.xml
+++ b/sql/storm-sql-external/storm-sql-kafka/pom.xml
@@@ -1,0 -1,93 +1,93 @@@
+ <?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>
+         <relativePath>../../../pom.xml</relativePath>
+     </parent>
+ 
+     <artifactId>storm-sql-kafka</artifactId>
+ 
+     <developers>
+         <developer>
+             <id>haohui</id>
+             <name>Haohui Mai</name>
+             <email>ricetons@gmail.com</email>
+         </developer>
+     </developers>
+ 
+     <dependencies>
+         <dependency>
+             <groupId>org.apache.storm</groupId>
+             <artifactId>storm-core</artifactId>
+             <version>${project.version}</version>
 -            <scope>provided</scope>
++            <scope>${provided.scope}</scope>
+         </dependency>
+         <dependency>
+             <groupId>org.apache.storm</groupId>
+             <artifactId>storm-sql-runtime</artifactId>
+             <version>${project.version}</version>
 -            <scope>provided</scope>
++            <scope>${provided.scope}</scope>
+         </dependency>
+         <dependency>
+             <groupId>org.apache.storm</groupId>
+             <artifactId>storm-sql-runtime</artifactId>
+             <version>${project.version}</version>
+             <scope>test</scope>
+             <type>test-jar</type>
+         </dependency>
+         <dependency>
+             <groupId>org.apache.storm</groupId>
+             <artifactId>storm-kafka</artifactId>
+             <version>${project.version}</version>
 -            <scope>provided</scope>
++            <scope>${provided.scope}</scope>
+         </dependency>
+         <dependency>
+             <groupId>org.apache.kafka</groupId>
+             <artifactId>${storm.kafka.artifact.id}</artifactId>
+         </dependency>
+         <dependency>
+             <groupId>org.apache.kafka</groupId>
+             <artifactId>kafka-clients</artifactId>
+             <version>${storm.kafka.version}</version>
+         </dependency>
+         <dependency>
+             <groupId>junit</groupId>
+             <artifactId>junit</artifactId>
+             <scope>test</scope>
+         </dependency>
+         <dependency>
+             <groupId>org.mockito</groupId>
+             <artifactId>mockito-all</artifactId>
+             <scope>test</scope>
+         </dependency>
+     </dependencies>
+     <build>
+         <sourceDirectory>src/jvm</sourceDirectory>
+         <testSourceDirectory>src/test</testSourceDirectory>
+         <resources>
+             <resource>
+                 <directory>${basedir}/src/resources</directory>
+             </resource>
+         </resources>
+     </build>
+ </project>

http://git-wip-us.apache.org/repos/asf/storm/blob/1dd4bbc4/sql/storm-sql-external/storm-sql-mongodb/pom.xml
----------------------------------------------------------------------
diff --cc sql/storm-sql-external/storm-sql-mongodb/pom.xml
index 0000000,218d89b..cb0baba
mode 000000,100644..100644
--- a/sql/storm-sql-external/storm-sql-mongodb/pom.xml
+++ b/sql/storm-sql-external/storm-sql-mongodb/pom.xml
@@@ -1,0 -1,84 +1,84 @@@
+ <?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>
+         <relativePath>../../../pom.xml</relativePath>
+     </parent>
+ 
+     <artifactId>storm-sql-mongodb</artifactId>
+ 
+     <developers>
+         <developer>
+             <id>vesense</id>
+             <name>Xin Wang</name>
+             <email>data.xinwang@gmail.com</email>
+         </developer>
+     </developers>
+ 
+     <dependencies>
+         <dependency>
+             <groupId>org.apache.storm</groupId>
+             <artifactId>storm-core</artifactId>
+             <version>${project.version}</version>
 -            <scope>provided</scope>
++            <scope>${provided.scope}</scope>
+         </dependency>
+         <dependency>
+             <groupId>org.apache.storm</groupId>
+             <artifactId>storm-sql-runtime</artifactId>
+             <version>${project.version}</version>
 -            <scope>provided</scope>
++            <scope>${provided.scope}</scope>
+         </dependency>
+         <dependency>
+             <groupId>org.apache.storm</groupId>
+             <artifactId>storm-sql-runtime</artifactId>
+             <version>${project.version}</version>
+             <scope>test</scope>
+             <type>test-jar</type>
+         </dependency>
+         <dependency>
+             <groupId>org.apache.storm</groupId>
+             <artifactId>storm-mongodb</artifactId>
+             <version>${project.version}</version>
 -            <scope>provided</scope>
++            <scope>${provided.scope}</scope>
+         </dependency>
+         <dependency>
+             <groupId>junit</groupId>
+             <artifactId>junit</artifactId>
+             <scope>test</scope>
+         </dependency>
+         <dependency>
+             <groupId>org.mockito</groupId>
+             <artifactId>mockito-all</artifactId>
+             <scope>test</scope>
+         </dependency>
+     </dependencies>
+     <build>
+         <sourceDirectory>src/jvm</sourceDirectory>
+         <testSourceDirectory>src/test</testSourceDirectory>
+         <resources>
+             <resource>
+                 <directory>${basedir}/src/resources</directory>
+             </resource>
+         </resources>
+     </build>
+ </project>

http://git-wip-us.apache.org/repos/asf/storm/blob/1dd4bbc4/sql/storm-sql-external/storm-sql-redis/pom.xml
----------------------------------------------------------------------
diff --cc sql/storm-sql-external/storm-sql-redis/pom.xml
index 0000000,32ce432..38648a4
mode 000000,100644..100644
--- a/sql/storm-sql-external/storm-sql-redis/pom.xml
+++ b/sql/storm-sql-external/storm-sql-redis/pom.xml
@@@ -1,0 -1,76 +1,76 @@@
+ <?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>
+         <relativePath>../../../pom.xml</relativePath>
+     </parent>
+ 
+     <artifactId>storm-sql-redis</artifactId>
+ 
+     <dependencies>
+         <dependency>
+             <groupId>org.apache.storm</groupId>
+             <artifactId>storm-core</artifactId>
+             <version>${project.version}</version>
 -            <scope>provided</scope>
++            <scope>${provided.scope}</scope>
+         </dependency>
+         <dependency>
+             <groupId>org.apache.storm</groupId>
+             <artifactId>storm-sql-runtime</artifactId>
+             <version>${project.version}</version>
 -            <scope>provided</scope>
++            <scope>${provided.scope}</scope>
+         </dependency>
+         <dependency>
+             <groupId>org.apache.storm</groupId>
+             <artifactId>storm-sql-runtime</artifactId>
+             <version>${project.version}</version>
+             <scope>test</scope>
+             <type>test-jar</type>
+         </dependency>
+         <dependency>
+             <groupId>org.apache.storm</groupId>
+             <artifactId>storm-redis</artifactId>
+             <version>${project.version}</version>
 -            <scope>provided</scope>
++            <scope>${provided.scope}</scope>
+         </dependency>
+         <dependency>
+             <groupId>junit</groupId>
+             <artifactId>junit</artifactId>
+             <scope>test</scope>
+         </dependency>
+         <dependency>
+             <groupId>org.mockito</groupId>
+             <artifactId>mockito-all</artifactId>
+             <scope>test</scope>
+         </dependency>
+     </dependencies>
+     <build>
+         <sourceDirectory>src/jvm</sourceDirectory>
+         <testSourceDirectory>src/test</testSourceDirectory>
+         <resources>
+             <resource>
+                 <directory>${basedir}/src/resources</directory>
+             </resource>
+         </resources>
+     </build>
+ </project>

http://git-wip-us.apache.org/repos/asf/storm/blob/1dd4bbc4/sql/storm-sql-runtime/pom.xml
----------------------------------------------------------------------
diff --cc sql/storm-sql-runtime/pom.xml
index 0000000,ce57cb2..060a1ab
mode 000000,100644..100644
--- a/sql/storm-sql-runtime/pom.xml
+++ b/sql/storm-sql-runtime/pom.xml
@@@ -1,0 -1,136 +1,136 @@@
+ <?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>
+         <relativePath>../../pom.xml</relativePath>
+     </parent>
+ 
+     <artifactId>storm-sql-runtime</artifactId>
+ 
+     <developers>
+         <developer>
+             <id>haohui</id>
+             <name>Haohui Mai</name>
+             <email>ricetons@gmail.com</email>
+         </developer>
+     </developers>
+ 
+     <dependencies>
+         <dependency>
+             <groupId>org.apache.storm</groupId>
+             <artifactId>storm-core</artifactId>
+             <version>${project.version}</version>
 -            <scope>provided</scope>
++            <scope>${provided.scope}</scope>
+         </dependency>
+         <dependency>
+             <groupId>org.apache.calcite</groupId>
+             <artifactId>calcite-core</artifactId>
+             <version>${calcite.version}</version>
+             <exclusions>
+                 <exclusion>
+                     <groupId>commons-dbcp</groupId>
+                     <artifactId>commons-dbcp</artifactId>
+                 </exclusion>
+                 <exclusion>
+                     <groupId>com.google.code.findbugs</groupId>
+                     <artifactId>jsr305</artifactId>
+                 </exclusion>
+                 <exclusion>
+                     <groupId>org.pentaho</groupId>
+                     <artifactId>pentaho-aggdesigner-algorithm</artifactId>
+                 </exclusion>
+                 <exclusion>
+                     <groupId>com.fasterxml.jackson.core</groupId>
+                     <artifactId>jackson-annotations</artifactId>
+                 </exclusion>
+             </exclusions>
+         </dependency>
+         <dependency>
+             <groupId>com.fasterxml.jackson.core</groupId>
+             <artifactId>jackson-databind</artifactId>
+         </dependency>
+         <dependency>
+             <groupId>org.apache.avro</groupId>
+             <artifactId>avro</artifactId>
+             <version>1.7.7</version>
+         </dependency>
+         <dependency>
+             <groupId>commons-lang</groupId>
+             <artifactId>commons-lang</artifactId>
+         </dependency>
+         <dependency>
+             <groupId>org.apache.commons</groupId>
+             <artifactId>commons-csv</artifactId>
+             <version>1.4</version>
+         </dependency>
+         <dependency>
+             <groupId>org.mockito</groupId>
+             <artifactId>mockito-all</artifactId>
+             <scope>test</scope>
+         </dependency>
+         <dependency>
+             <groupId>junit</groupId>
+             <artifactId>junit</artifactId>
+             <scope>test</scope>
+         </dependency>
+     </dependencies>
+     <build>
+         <sourceDirectory>src/jvm</sourceDirectory>
+         <testSourceDirectory>src/test</testSourceDirectory>
+         <resources>
+             <resource>
+                 <directory>${basedir}/src/resources</directory>
+             </resource>
+         </resources>
+         <plugins>
+             <plugin>
+                 <groupId>org.apache.maven.plugins</groupId>
+                 <artifactId>maven-jar-plugin</artifactId>
+                 <executions>
+                     <execution>
+                         <goals>
+                             <goal>test-jar</goal>
+                         </goals>
+                     </execution>
+                 </executions>
+             </plugin>
+             <plugin>
+                 <groupId>org.codehaus.mojo</groupId>
+                 <artifactId>appassembler-maven-plugin</artifactId>
+                 <version>1.9</version>
+                 <executions>
+                     <execution>
+                         <id>create-repo</id>
+                         <goals>
+                             <goal>create-repository</goal>
+                         </goals>
+                         <configuration>
+                             <assembleDirectory>${project.build.directory}/app-assembler</assembleDirectory>
+                             <repositoryLayout>flat</repositoryLayout>
+                         </configuration>
+                     </execution>
+                 </executions>
+             </plugin>
+         </plugins>
+     </build>
+ </project>

http://git-wip-us.apache.org/repos/asf/storm/blob/1dd4bbc4/storm-dist/binary/final-package/src/main/assembly/binary.xml
----------------------------------------------------------------------
diff --cc storm-dist/binary/final-package/src/main/assembly/binary.xml
index 9321ea2,7ee1445..02b405f
--- a/storm-dist/binary/final-package/src/main/assembly/binary.xml
+++ b/storm-dist/binary/final-package/src/main/assembly/binary.xml
@@@ -180,59 -290,14 +180,7 @@@
                  <include>README.*</include>
              </includes>
          </fileSet>
-         <!-- Storm SQL -->
-         <fileSet>
-             <directory>${project.basedir}/../../../external/sql/storm-sql-core/target/app-assembler/repo</directory>
-             <outputDirectory>external/sql/storm-sql-core</outputDirectory>
-             <includes>
-                 <include>*jar</include>
-             </includes>
-         </fileSet>
-         <fileSet>
-             <directory>${project.basedir}/../../../external/sql/storm-sql-external/storm-sql-kafka/target</directory>
-             <outputDirectory>external/sql/storm-sql-external/storm-sql-kafka</outputDirectory>
-             <includes>
-                 <include>storm*jar</include>
-             </includes>
-         </fileSet>
-         <fileSet>
-             <directory>${project.basedir}/../../../external/sql/storm-sql-external/storm-sql-redis/target</directory>
-             <outputDirectory>external/sql/storm-sql-external/storm-sql-redis</outputDirectory>
-             <includes>
-                 <include>storm*jar</include>
-             </includes>
-         </fileSet>
-         <fileSet>
-             <directory>${project.basedir}/../../../external/sql/storm-sql-external/storm-sql-mongodb/target</directory>
-             <outputDirectory>external/sql/storm-sql-external/storm-sql-mongodb</outputDirectory>
-             <includes>
-                 <include>storm*jar</include>
-             </includes>
-         </fileSet>
-         <fileSet>
-             <directory>${project.basedir}/../../../external/sql/storm-sql-external/storm-sql-hdfs/target</directory>
-             <outputDirectory>external/sql/storm-sql-external/storm-sql-hdfs</outputDirectory>
-             <includes>
-                 <include>storm*jar</include>
-             </includes>
-         </fileSet>
-         <fileSet>
-             <directory>${project.basedir}/../../../external/sql/storm-sql-runtime/target/app-assembler/repo</directory>
-             <outputDirectory>external/sql/storm-sql-runtime</outputDirectory>
-             <includes>
-                 <include>*jar</include>
-             </includes>
-         </fileSet>
- 
-         <fileSet>
-             <directory>${project.basedir}/../../../external/sql</directory>
-             <outputDirectory>external/sql</outputDirectory>
-             <includes>
-                 <include>README.*</include>
-             </includes>
-         </fileSet>
-         <!-- END Storm SQL -->
          <fileSet>
 -            <directory>${project.basedir}/../../../external/storm-cassandra/target</directory>
 -            <outputDirectory>external/storm-cassandra</outputDirectory>
 -            <includes>
 -                <include>storm*jar</include>
 -            </includes>
 -        </fileSet>
 -        <fileSet>
              <directory>${project.basedir}/../../../external/storm-mqtt</directory>
              <outputDirectory>external/storm-mqtt</outputDirectory>
              <includes>
@@@ -322,6 -451,62 +263,32 @@@
                  <include>storm*jar</include>
              </includes>
          </fileSet>
+ 
 -        <fileSet>
 -            <directory>${project.basedir}/../../storm-perf/target</directory>
 -            <outputDirectory>perf</outputDirectory>
 -            <includes>
 -                <include>storm*jar</include>
 -            </includes>
 -        </fileSet>
 -        <fileSet>
 -            <directory>${project.basedir}/../../storm-perf/src/main/conf</directory>
 -            <outputDirectory>perf/conf</outputDirectory>
 -            <includes>
 -                <include>*yaml</include>
 -            </includes>
 -        </fileSet>
 -        <fileSet>
 -            <directory>${project.basedir}/../../storm-perf/src/main/sampledata</directory>
 -            <outputDirectory>perf/resources</outputDirectory>
 -            <includes>
 -                <include>*</include>
 -            </includes>
 -        </fileSet>
 -        <fileSet>
 -            <directory>${project.basedir}/../../storm-perf/</directory>
 -            <outputDirectory>perf</outputDirectory>
 -            <includes>
 -                <include>README.*</include>
 -            </includes>
 -        </fileSet>
 -
+         <!-- storm-sql -->
+         <fileSet>
+             <directory>${project.basedir}/../../../sql/storm-sql-core/target/app-assembler/repo</directory>
+             <outputDirectory>lib-tools/sql/core</outputDirectory>
+             <includes>
+                 <include>*jar</include>
+             </includes>
+         </fileSet>
+         <fileSet>
+             <directory>${project.basedir}/../../../sql/storm-sql-runtime/target/app-assembler/repo</directory>
+             <outputDirectory>lib-tools/sql/runtime</outputDirectory>
+             <includes>
+                 <include>*jar</include>
+             </includes>
+         </fileSet>
+ 
+         <!-- storm-submit-tools -->
+         <fileSet>
+             <directory>${project.basedir}/../../../storm-submit-tools/target</directory>
+             <outputDirectory>lib-tools/submit-tools</outputDirectory>
+             <includes>
+                 <include>storm*jar</include>
+             </includes>
+         </fileSet>
+ 
 -
      </fileSets>
  
      <files>


[07/23] storm git commit: STORM-2453 Move non-connectors into the top directory

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/FluxShellBolt.java
----------------------------------------------------------------------
diff --git a/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/FluxShellBolt.java b/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/FluxShellBolt.java
new file mode 100644
index 0000000..05b8e7a
--- /dev/null
+++ b/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/FluxShellBolt.java
@@ -0,0 +1,175 @@
+/*
+ * 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.flux.wrappers.bolts;
+
+import org.apache.storm.task.ShellBolt;
+import org.apache.storm.topology.IRichBolt;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.tuple.Fields;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A generic `ShellBolt` implementation that allows you specify output fields
+ * and even streams without having to subclass `ShellBolt` to do so.
+ *
+ */
+public class FluxShellBolt extends ShellBolt implements IRichBolt{
+    private Map<String, String[]> outputFields;
+    private Map<String, Object> componentConfig;
+    
+    /**
+     * Create a ShellBolt with command line arguments
+     * @param command Command line arguments for the bolt
+     */
+    public FluxShellBolt(String[] command){
+        super(command);
+        this.outputFields = new HashMap<String, String[]>();
+    }
+
+    /**
+     * Create a ShellBolt with command line arguments and output fields
+     * 
+     * Keep this constructor for backward compatibility.
+     * 
+     * @param command Command line arguments for the bolt
+     * @param outputFields Names of fields the bolt will emit (if any).
+     */
+    public FluxShellBolt(String[] command, String[] outputFields){
+        this(command);
+        this.setDefaultStream(outputFields);
+    }
+
+    /**
+     * Add configuration for this bolt. This method is called from YAML file:
+     *
+     * ```
+     * className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
+     * constructorArgs:
+     * # command line
+     * - ["python", "splitsentence.py"]
+     * # output fields
+     * - ["word"]
+     * configMethods:
+     * - name: "addComponentConfig"
+     *   args: ["publisher.data_paths", "actions"]
+     * ```
+     *
+     * @param key
+     * @param value
+     */
+    public void addComponentConfig(String key, Object value) {
+        if (this.componentConfig == null) {
+            this.componentConfig = new HashMap<String, Object>();
+        }
+        this.componentConfig.put(key, value);
+    }
+
+    /**
+     * Add configuration for this bolt. This method is called from YAML file:
+     *
+     * ```
+     * className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
+     * constructorArgs:
+     * # command line
+     * - ["python", "splitsentence.py"]
+     * # output fields
+     * - ["word"]
+     * configMethods:
+     * - name: "addComponentConfig"
+     *   args:
+     *   - "publisher.data_paths"
+     *   - ["actions"]
+     * ```
+     *
+     * @param key
+     * @param values
+     */
+    public void addComponentConfig(String key, List<Object> values) {
+        if (this.componentConfig == null) {
+            this.componentConfig = new HashMap<String, Object>();
+        }
+        this.componentConfig.put(key, values);
+    }
+
+    /**
+     * Set default stream outputFields, this method is called from YAML file:
+     * 
+     * ```
+     * bolts:
+     * - className: org.apache.storm.flux.wrappers.bolts.FluxShellBolt
+     *   id: my_bolt
+     *   constructorArgs:
+     *   - [python, my_bolt.py]
+     *   configMethods:
+     *   - name: setDefaultStream
+     *     args:
+     *     - [word, count]
+     * ```
+     * 
+     * @param outputFields Names of fields the bolt will emit (if any) in default stream.
+     */
+    public void setDefaultStream(String[] outputFields) {
+        this.setNamedStream("default", outputFields);
+    }
+
+    /**
+     * Set custom *named* stream outputFields, this method is called from YAML file:
+     * 
+     * ```
+     * bolts:
+     * - className: org.apache.storm.flux.wrappers.bolts.FluxShellBolt
+     *   id: my_bolt
+     *   constructorArgs:
+     *   - [python, my_bolt.py]
+     *   configMethods:
+     *   - name: setNamedStream
+     *     args:
+     *     - first
+     *     - [word, count]
+     * ```
+     * @param name Name of stream the bolt will emit into.
+     * @param outputFields Names of fields the bolt will emit in custom *named* stream.
+     */
+    public void setNamedStream(String name, String[] outputFields) {
+        this.outputFields.put(name, outputFields);
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        Iterator it = this.outputFields.entrySet().iterator();
+        while (it.hasNext()) {
+            Map.Entry entryTuple = (Map.Entry)it.next();
+            String key = (String)entryTuple.getKey();
+            String[] value = (String[])entryTuple.getValue();
+            if(key.equals("default")) {
+                declarer.declare(new Fields(value));
+            } else {
+                declarer.declareStream(key, new Fields(value));
+            }
+        }
+    }
+
+    @Override
+    public Map<String, Object> getComponentConfiguration() {
+        return this.componentConfig;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/LogInfoBolt.java
----------------------------------------------------------------------
diff --git a/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/LogInfoBolt.java b/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/LogInfoBolt.java
new file mode 100644
index 0000000..5f0e84b
--- /dev/null
+++ b/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/bolts/LogInfoBolt.java
@@ -0,0 +1,44 @@
+/*
+ * 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.flux.wrappers.bolts;
+
+import org.apache.storm.topology.BasicOutputCollector;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseBasicBolt;
+import org.apache.storm.tuple.Tuple;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Simple bolt that does nothing other than LOG.info() every tuple recieveed.
+ *
+ */
+public class LogInfoBolt extends BaseBasicBolt {
+    private static final Logger LOG = LoggerFactory.getLogger(LogInfoBolt.class);
+
+    @Override
+    public void execute(Tuple tuple, BasicOutputCollector basicOutputCollector) {
+       LOG.info("{}", tuple);
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer outputFieldsDeclarer) {
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/spouts/FluxShellSpout.java
----------------------------------------------------------------------
diff --git a/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/spouts/FluxShellSpout.java b/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/spouts/FluxShellSpout.java
new file mode 100644
index 0000000..5fd378d
--- /dev/null
+++ b/flux/flux-wrappers/src/main/java/org/apache/storm/flux/wrappers/spouts/FluxShellSpout.java
@@ -0,0 +1,175 @@
+/*
+ * 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.flux.wrappers.spouts;
+
+import org.apache.storm.spout.ShellSpout;
+import org.apache.storm.topology.IRichSpout;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.tuple.Fields;
+
+import java.util.List;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.Iterator;
+
+/**
+ * A generic `ShellSpout` implementation that allows you specify output fields
+ * and even streams without having to subclass `ShellSpout` to do so.
+ *
+ */
+public class FluxShellSpout extends ShellSpout implements IRichSpout {
+    private Map<String, String[]> outputFields;
+    private Map<String, Object> componentConfig;
+    
+    /**
+     * Create a ShellSpout with command line arguments
+     * @param command Command line arguments for the bolt
+     */
+    public FluxShellSpout(String[] command){
+        super(command);
+        this.outputFields = new HashMap<String, String[]>();
+    }
+
+    /**
+     * Create a ShellSpout with command line arguments and output fields
+     * 
+     * Keep this constructor for backward compatibility.
+     * 
+     * @param args Command line arguments for the spout
+     * @param outputFields Names of fields the spout will emit.
+     */
+    public FluxShellSpout(String[] args, String[] outputFields){
+        this(args);
+        this.setDefaultStream(outputFields);
+    }
+
+    /**
+     * Add configuration for this spout. This method is called from YAML file:
+     *
+     * ```
+     * className: "org.apache.storm.flux.wrappers.bolts.FluxShellSpout"
+     * constructorArgs:
+     * # command line
+     * - ["python", "splitsentence.py"]
+     * # output fields
+     * - ["word"]
+     * configMethods:
+     * - name: "addComponentConfig"
+     *   args: ["publisher.data_paths", "actions"]
+     * ```
+     *
+     * @param key
+     * @param value
+     */
+    public void addComponentConfig(String key, Object value) {
+        if (this.componentConfig == null) {
+            this.componentConfig = new HashMap<String, Object>();
+        }
+        this.componentConfig.put(key, value);
+    }
+
+    /**
+     * Add configuration for this spout. This method is called from YAML file:
+     *
+     * ```
+     * className: "org.apache.storm.flux.wrappers.bolts.FluxShellSpout"
+     * constructorArgs:
+     * # command line
+     * - ["python", "splitsentence.py"]
+     * # output fields
+     * - ["word"]
+     * configMethods:
+     * - name: "addComponentConfig"
+     *   args:
+     *   - "publisher.data_paths"
+     *   - ["actions"]
+     * ```
+     *
+     * @param key
+     * @param values
+     */
+    public void addComponentConfig(String key, List<Object> values) {
+        if (this.componentConfig == null) {
+            this.componentConfig = new HashMap<String, Object>();
+        }
+        this.componentConfig.put(key, values);
+    }
+
+    /**
+     * Set default stream outputFields, this method is called from YAML file:
+     * 
+     * ```
+     * spouts:
+     * - className: org.apache.storm.flux.wrappers.bolts.FluxShellSpout
+     *   id: my_spout
+     *   constructorArgs:
+     *   - [python, my_spout.py]
+     *   configMethods:
+     *   - name: setDefaultStream
+     *     args:
+     *     - [word, count]
+     * ```
+     * 
+     * @param outputFields Names of fields the spout will emit (if any) in default stream.
+     */
+    public void setDefaultStream(String[] outputFields) {
+        this.setNamedStream("default", outputFields);
+    }
+
+    /**
+     * Set custom *named* stream outputFields, this method is called from YAML file:
+     * 
+     * ```
+     * spouts:
+     * - className: org.apache.storm.flux.wrappers.bolts.FluxShellSpout
+     *   id: my_spout
+     *   constructorArgs:
+     *   - [python, my_spout.py]
+     *   configMethods:
+     *   - name: setNamedStream
+     *     args:
+     *     - first
+     *     - [word, count]
+     * ```
+     * @param name Name of stream the spout will emit into.
+     * @param outputFields Names of fields the spout will emit in custom *named* stream.
+     */
+    public void setNamedStream(String name, String[] outputFields) {
+        this.outputFields.put(name, outputFields);
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer declarer) {
+        Iterator it = this.outputFields.entrySet().iterator();
+        while (it.hasNext()) {
+            Map.Entry entryTuple = (Map.Entry)it.next();
+            String key = (String)entryTuple.getKey();
+            String[] value = (String[])entryTuple.getValue();
+            if(key.equals("default")) {
+                declarer.declare(new Fields(value));
+            } else {
+                declarer.declareStream(key, new Fields(value));
+            }
+        }
+    }
+
+    @Override
+    public Map<String, Object> getComponentConfiguration() {
+        return this.componentConfig;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-wrappers/src/main/resources/resources/randomsentence.js
----------------------------------------------------------------------
diff --git a/flux/flux-wrappers/src/main/resources/resources/randomsentence.js b/flux/flux-wrappers/src/main/resources/resources/randomsentence.js
new file mode 100644
index 0000000..b121915
--- /dev/null
+++ b/flux/flux-wrappers/src/main/resources/resources/randomsentence.js
@@ -0,0 +1,93 @@
+/*
+ * 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.
+ */
+
+/**
+ * Example for storm spout. Emits random sentences.
+ * The original class in java - org.apache.storm.starter.spout.RandomSentenceSpout.
+ *
+ */
+
+var storm = require('./storm');
+var Spout = storm.Spout;
+
+
+var SENTENCES = [
+    "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"]
+
+function RandomSentenceSpout(sentences) {
+    Spout.call(this);
+    this.runningTupleId = 0;
+    this.sentences = sentences;
+    this.pending = {};
+};
+
+RandomSentenceSpout.prototype = Object.create(Spout.prototype);
+RandomSentenceSpout.prototype.constructor = RandomSentenceSpout;
+
+RandomSentenceSpout.prototype.getRandomSentence = function() {
+    return this.sentences[getRandomInt(0, this.sentences.length - 1)];
+}
+
+RandomSentenceSpout.prototype.nextTuple = function(done) {
+    var self = this;
+    var sentence = this.getRandomSentence();
+    var tup = [sentence];
+    var id = this.createNextTupleId();
+    this.pending[id] = tup;
+    //This timeout can be removed if TOPOLOGY_SLEEP_SPOUT_WAIT_STRATEGY_TIME_MS is configured to 100
+    setTimeout(function() {
+        self.emit({tuple: tup, id: id}, function(taskIds) {
+            self.log(tup + ' sent to task ids - ' + taskIds);
+        });
+        done();
+    },100);
+}
+
+RandomSentenceSpout.prototype.createNextTupleId = function() {
+    var id = this.runningTupleId;
+    this.runningTupleId++;
+    return id;
+}
+
+RandomSentenceSpout.prototype.ack = function(id, done) {
+    this.log('Received ack for - ' + id);
+    delete this.pending[id];
+    done();
+}
+
+RandomSentenceSpout.prototype.fail = function(id, done) {
+    var self = this;
+    this.log('Received fail for - ' + id + '. Retrying.');
+    this.emit({tuple: this.pending[id], id:id}, function(taskIds) {
+        self.log(self.pending[id] + ' sent to task ids - ' + taskIds);
+    });
+    done();
+}
+
+/**
+ * Returns a random integer between min (inclusive) and max (inclusive)
+ */
+function getRandomInt(min, max) {
+    return Math.floor(Math.random() * (max - min + 1)) + min;
+}
+
+new RandomSentenceSpout(SENTENCES).run();

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-wrappers/src/main/resources/resources/splitsentence.py
----------------------------------------------------------------------
diff --git a/flux/flux-wrappers/src/main/resources/resources/splitsentence.py b/flux/flux-wrappers/src/main/resources/resources/splitsentence.py
new file mode 100644
index 0000000..300105f
--- /dev/null
+++ b/flux/flux-wrappers/src/main/resources/resources/splitsentence.py
@@ -0,0 +1,24 @@
+# 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.
+import storm
+
+class SplitSentenceBolt(storm.BasicBolt):
+    def process(self, tup):
+        words = tup.values[0].split(" ")
+        for word in words:
+          storm.emit([word])
+
+SplitSentenceBolt().run()
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/pom.xml
----------------------------------------------------------------------
diff --git a/flux/pom.xml b/flux/pom.xml
new file mode 100644
index 0000000..287b04a
--- /dev/null
+++ b/flux/pom.xml
@@ -0,0 +1,85 @@
+<?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>
+
+    <artifactId>flux</artifactId>
+    <packaging>pom</packaging>
+    <name>flux</name>
+
+
+    <parent>
+        <artifactId>storm</artifactId>
+        <groupId>org.apache.storm</groupId>
+        <version>2.0.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+
+    <developers>
+        <developer>
+            <id>ptgoetz</id>
+            <name>P. Taylor Goetz</name>
+            <email>ptgoetz@apache.org</email>
+        </developer>
+    </developers>
+
+    <properties>
+        <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+        <!-- see comment below... This fixes an annoyance with intellij -->
+        <provided.scope>provided</provided.scope>
+    </properties>
+
+    <profiles>
+        <!--
+            Hack to make intellij behave.
+            If you use intellij, enable this profile in your IDE.
+            It should make life easier.
+        -->
+        <profile>
+            <id>intellij</id>
+            <properties>
+                <provided.scope>compile</provided.scope>
+            </properties>
+        </profile>
+    </profiles>
+
+    <modules>
+        <module>flux-wrappers</module>
+        <module>flux-core</module>
+        <module>flux-examples</module>
+    </modules>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-core</artifactId>
+            <version>${project.version}</version>
+            <scope>${provided.scope}</scope>
+        </dependency>
+        <dependency>
+            <groupId>commons-cli</groupId>
+            <artifactId>commons-cli</artifactId>
+            <version>1.2</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.kafka</groupId>
+            <artifactId>${storm.kafka.artifact.id}</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 896d735..d3f94e2 100644
--- a/pom.xml
+++ b/pom.xml
@@ -320,6 +320,9 @@
         <module>storm-drpc-server</module>
         <module>storm-rename-hack</module>
         <module>storm-clojure</module>
+        <module>storm-submit-tools</module>
+        <module>flux</module>
+        <module>sql</module>
 
         <!-- externals -->
         <module>external/storm-kafka</module>
@@ -329,8 +332,6 @@
         <module>external/storm-jdbc</module>
         <module>external/storm-redis</module>
         <module>external/storm-eventhubs</module>
-        <module>external/flux</module>
-        <module>external/sql</module>
         <module>external/storm-elasticsearch</module>
         <module>external/storm-solr</module>
         <module>external/storm-metrics</module>
@@ -342,7 +343,6 @@
         <module>external/storm-kafka-monitor</module>
         <module>external/storm-kinesis</module>
         <module>external/storm-druid</module>
-        <module>external/storm-submit-tools</module>
         <module>external/storm-jms</module>
         <module>external/storm-pmml</module>
 

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/README.md
----------------------------------------------------------------------
diff --git a/sql/README.md b/sql/README.md
new file mode 100644
index 0000000..a4b44fb
--- /dev/null
+++ b/sql/README.md
@@ -0,0 +1,207 @@
+# Storm SQL
+
+Compile SQL queries to Storm topologies.
+
+## Usage
+
+Run the ``storm sql`` command to compile SQL statements into Trident topology, and submit it to the Storm cluster
+
+```
+$ bin/storm sql <sql-file> <topo-name>
+```
+
+In which `sql-file` contains a list of SQL statements to be executed, and `topo-name` is the name of the topology.
+
+StormSQL activates `explain mode` and shows query plan instead of submitting topology when user specifies `topo-name` as `--explain`.
+Detailed explanation is available from `Showing Query Plan (explain mode)` section.
+
+## Supported Features
+
+The following features are supported in the current repository:
+
+* Streaming from and to external data sources
+* Filtering tuples
+* Projections
+* Aggregations (Grouping)
+* User defined function (scalar and aggregate)
+* Join (Inner, Left outer, Right outer, Full outer)
+
+## Specifying External Data Sources
+
+In StormSQL data is represented by external tables. Users can specify data sources using the `CREATE EXTERNAL TABLE`
+statement. For example, the following statement specifies a Kafka spouts and sink:
+
+```
+CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY) LOCATION 'kafka://localhost:2181/brokers?topic=test' TBLPROPERTIES '{"producer":{"bootstrap.servers":"localhost:9092","acks":"1","key.serializer":"org.apache.storm.kafka.IntSerializer","value.serializer":"org.apache.storm.kafka.ByteBufferSerializer"}}'
+```
+
+The syntax of `CREATE EXTERNAL TABLE` closely follows the one defined in
+[Hive Data Definition Language](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL).
+
+`PARALLELISM` is StormSQL's own keyword which describes parallelism hint for input data source. This is same as providing parallelism hint to Trident Spout.
+Downstream operators are executed with same parallelism before repartition (Aggregation triggers repartition).
+
+Default value is 1, and this option is no effect on output data source. (We might change if needed. Normally repartition is the thing to avoid.)
+
+## Plugging in External Data Sources
+
+Users plug in external data sources through implementing the `ISqlTridentDataSource` interface and registers them using
+the mechanisms of Java's service loader. The external data source will be chosen based on the scheme of the URI of the
+tables. Please refer to the implementation of `storm-sql-kafka` for more details.
+
+## Specifying User Defined Function (UDF)
+
+Users can define user defined function (scalar or aggregate) using `CREATE FUNCTION` statement.
+For example, the following statement defines `MYPLUS` function which uses `org.apache.storm.sql.TestUtils$MyPlus` class.
+
+```
+CREATE FUNCTION MYPLUS AS 'org.apache.storm.sql.TestUtils$MyPlus'
+```
+
+Storm SQL determines whether the function as scalar or aggregate by checking which methods are defined.
+If the class defines `evaluate` method, Storm SQL treats the function as `scalar`,
+and if the class defines `add` method, Storm SQL treats the function as `aggregate`.
+
+Example of class for scalar function is here:
+
+```
+  public class MyPlus {
+    public static Integer evaluate(Integer x, Integer y) {
+      return x + y;
+    }
+  }
+
+```
+
+and class for aggregate function is here:
+
+```
+  public class MyConcat {
+    public static String init() {
+      return "";
+    }
+    public static String add(String accumulator, String val) {
+      return accumulator + val;
+    }
+    public static String result(String accumulator) {
+      return accumulator;
+    }
+  }
+```
+
+If users don't define `result` method, result is the last return value of `add` method.
+Users need to define `result` method only when we need to transform accumulated value.
+
+## Example: Filtering Kafka Stream
+
+Let's say there is a Kafka stream that represents the transactions of orders. Each message in the stream contains the id
+of the order, the unit price of the product and the quantity of the orders. The goal is to filter orders where the
+transactions are significant and to insert these orders into another Kafka stream for further analysis.
+
+The user can specify the following SQL statements in the SQL file:
+
+```
+CREATE EXTERNAL TABLE ORDERS (ID INT PRIMARY KEY, UNIT_PRICE INT, QUANTITY INT) LOCATION 'kafka://localhost:2181/brokers?topic=orders' TBLPROPERTIES '{"producer":{"bootstrap.servers":"localhost:9092","acks":"1","key.serializer":"org.apache.storm.kafka.IntSerializer","value.serializer":"org.apache.storm.kafka.ByteBufferSerializer"}}'
+
+CREATE EXTERNAL TABLE LARGE_ORDERS (ID INT PRIMARY KEY, TOTAL INT) LOCATION 'kafka://localhost:2181/brokers?topic=large_orders' TBLPROPERTIES '{"producer":{"bootstrap.servers":"localhost:9092","acks":"1","key.serializer":"org.apache.storm.kafka.IntSerializer","value.serializer":"org.apache.storm.kafka.ByteBufferSerializer"}}'
+
+INSERT INTO LARGE_ORDERS SELECT ID, UNIT_PRICE * QUANTITY AS TOTAL FROM ORDERS WHERE UNIT_PRICE * QUANTITY > 50
+```
+
+The first statement defines the table `ORDER` which represents the input stream. The `LOCATION` clause specifies the
+ZkHost (`localhost:2181`), the path of the brokers in ZooKeeper (`/brokers`) and the topic (`orders`).
+The `TBLPROPERTIES` clause specifies the configuration of
+[KafkaProducer](http://kafka.apache.org/documentation.html#newproducerconfigs).
+Current implementation of `storm-sql-kafka` requires specifying both `LOCATION` and `TBLPROPERTIES` clauses even though
+the table is read-only or write-only.
+
+Similarly, the second statement specifies the table `LARGE_ORDERS` which represents the output stream. The third
+statement is a `SELECT` statement which defines the topology: it instructs StormSQL to filter all orders in the external
+table `ORDERS`, calculates the total price and inserts matching records into the Kafka stream specified by
+`LARGE_ORDER`.
+
+To run this example, users need to include the data sources (`storm-sql-kafka` in this case) and its dependency in the
+class path. Dependencies for Storm SQL are automatically handled when users run `storm sql`. Users can include data sources at the submission step like below:
+
+```
+$ bin/storm sql order_filtering.sql order_filtering --artifacts "org.apache.storm:storm-sql-kafka:2.0.0-SNAPSHOT,org.apache.storm:storm-kafka:2.0.0-SNAPSHOT,org.apache.kafka:kafka_2.10:0.8.2.2\!org.slf4j:slf4j-log4j12,org.apache.kafka:kafka-clients:0.8.2.2"
+```
+
+Above command submits the SQL statements to StormSQL. Users need to modify each artifacts' version if users are using different version of Storm or Kafka. 
+
+By now you should be able to see the `order_filtering` topology in the Storm UI.
+
+## Showing Query Plan (explain mode)
+
+Like `explain` on SQL statement, StormSQL provides `explain mode` when running Storm SQL Runner. In explain mode, StormSQL analyzes each query statement (only DML) and show plan instead of submitting topology.
+
+In order to run `explain mode`, you need to provide topology name as `--explain` and run `storm sql` as same as submitting.
+
+For example, when you run the example seen above with explain mode:
+ 
+```
+$ bin/storm sql order_filtering.sql --explain --artifacts "org.apache.storm:storm-sql-kafka:2.0.0-SNAPSHOT,org.apache.storm:storm-kafka:2.0.0-SNAPSHOT,org.apache.kafka:kafka_2.10:0.8.2.2\!org.slf4j:slf4j-log4j12,org.apache.kafka:kafka-clients:0.8.2.2"
+```
+
+StormSQL prints out like below:
+ 
+```
+
+===========================================================
+query>
+CREATE EXTERNAL TABLE ORDERS (ID INT PRIMARY KEY, UNIT_PRICE INT, QUANTITY INT) LOCATION 'kafka://localhost:2181/brokers?topic=orders' TBLPROPERTIES '{"producer":{"bootstrap.servers":"localhost:9092","acks":"1","key.serializer":"org.apache.storm.kafka.IntSerializer","value.serializer":"org.apache.storm.kafka.ByteBufferSerializer"}}'
+-----------------------------------------------------------
+16:53:43.951 [main] INFO  o.a.s.s.r.DataSourcesRegistry - Registering scheme kafka with org.apache.storm.sql.kafka.KafkaDataSourcesProvider@4d1bf319
+No plan presented on DDL
+===========================================================
+===========================================================
+query>
+CREATE EXTERNAL TABLE LARGE_ORDERS (ID INT PRIMARY KEY, TOTAL INT) LOCATION 'kafka://localhost:2181/brokers?topic=large_orders' TBLPROPERTIES '{"producer":{"bootstrap.servers":"localhost:9092","acks":"1","key.serializer":"org.apache.storm.kafka.IntSerializer","value.serializer":"org.apache.storm.kafka.ByteBufferSerializer"}}'
+-----------------------------------------------------------
+No plan presented on DDL
+===========================================================
+===========================================================
+query>
+INSERT INTO LARGE_ORDERS SELECT ID, UNIT_PRICE * QUANTITY AS TOTAL FROM ORDERS WHERE UNIT_PRICE * QUANTITY > 50
+-----------------------------------------------------------
+plan>
+LogicalTableModify(table=[[LARGE_ORDERS]], operation=[INSERT], updateColumnList=[[]], flattened=[true]), id = 8
+  LogicalProject(ID=[$0], TOTAL=[*($1, $2)]), id = 7
+    LogicalFilter(condition=[>(*($1, $2), 50)]), id = 6
+      EnumerableTableScan(table=[[ORDERS]]), id = 5
+
+===========================================================
+
+```
+
+## Current Limitations
+
+- Windowing is yet to be implemented.
+- Only equi-join (single field equality) is supported for joining table.
+- Joining table only applies within each small batch that comes off of the spout.
+  - Not across batches.
+  - Limitation came from `join` feature of Trident.
+  - Please refer this doc: `Trident API Overview` for details.
+
+## License
+
+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.
+
+## Committer Sponsors
+ * Sriharsha Chintalapani ([sriharsha@apache.org](mailto:sriharsha@apache.org))
+ * P. Taylor Goetz ([ptgoetz@apache.org](mailto:ptgoetz@apache.org))
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/pom.xml
----------------------------------------------------------------------
diff --git a/sql/pom.xml b/sql/pom.xml
new file mode 100644
index 0000000..f085dd8
--- /dev/null
+++ b/sql/pom.xml
@@ -0,0 +1,47 @@
+<?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>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+
+    <artifactId>sql</artifactId>
+    <packaging>pom</packaging>
+
+    <developers>
+        <developer>
+            <id>haohui</id>
+            <name>Haohui Mai</name>
+            <email>ricetons@gmail.com</email>
+        </developer>
+    </developers>
+
+    <modules>
+        <module>storm-sql-core</module>
+        <module>storm-sql-runtime</module>
+        <module>storm-sql-external/storm-sql-kafka</module>
+        <module>storm-sql-external/storm-sql-redis</module>
+        <module>storm-sql-external/storm-sql-mongodb</module>
+        <module>storm-sql-external/storm-sql-hdfs</module>
+    </modules>
+</project>

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/pom.xml
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/pom.xml b/sql/storm-sql-core/pom.xml
new file mode 100644
index 0000000..ca84970
--- /dev/null
+++ b/sql/storm-sql-core/pom.xml
@@ -0,0 +1,279 @@
+<?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>
+        <relativePath>../../pom.xml</relativePath>
+    </parent>
+
+    <artifactId>storm-sql-core</artifactId>
+
+    <developers>
+        <developer>
+            <id>haohui</id>
+            <name>Haohui Mai</name>
+            <email>ricetons@gmail.com</email>
+        </developer>
+    </developers>
+
+    <dependencies>
+        <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-sql-runtime</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-sql-runtime</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+            <type>test-jar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.calcite</groupId>
+            <artifactId>calcite-core</artifactId>
+            <version>${calcite.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.fasterxml.jackson.core</groupId>
+                    <artifactId>jackson-annotations</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-annotations</artifactId>
+            <version>${jackson.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>commons-cli</groupId>
+            <artifactId>commons-cli</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>commons-lang</groupId>
+            <artifactId>commons-lang</artifactId>
+            <scope>runtime</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-all</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-collections4</artifactId>
+            <version>4.1</version>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+    <build>
+        <sourceDirectory>src/jvm</sourceDirectory>
+        <testSourceDirectory>src/test</testSourceDirectory>
+        <plugins>
+            <plugin>
+                <artifactId>maven-resources-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>copy-fmpp-resources</id>
+                        <phase>initialize</phase>
+                        <goals>
+                            <goal>copy-resources</goal>
+                        </goals>
+                        <configuration>
+                            <outputDirectory>${project.build.directory}/codegen</outputDirectory>
+                            <resources>
+                                <resource>
+                                    <directory>src/codegen</directory>
+                                    <filtering>false</filtering>
+                                </resource>
+                            </resources>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>copy-java-sources</id>
+                        <phase>process-sources</phase>
+                        <goals>
+                            <goal>copy-resources</goal>
+                        </goals>
+                        <configuration>
+                            <outputDirectory>${basedir}/target/classes/</outputDirectory>
+                            <resources>
+                                <resource>
+                                    <directory>src/jvm</directory>
+                                    <filtering>true</filtering>
+                                </resource>
+                                <resource>
+                                    <directory>src/test</directory>
+                                    <filtering>true</filtering>
+                                </resource>
+                                <resource>
+                                    <directory>target/generated-sources</directory>
+                                    <!-- <include>*/org</include> -->
+                                    <filtering>true</filtering>
+                                </resource>
+                            </resources>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <version>2.2</version>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <!-- Extract parser grammar template from calcite-core.jar and put
+                     it under ${project.build.directory} where all freemarker templates are. -->
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-dependency-plugin</artifactId>
+                <version>2.8</version>
+                <executions>
+                    <execution>
+                        <id>unpack-parser-template</id>
+                        <phase>initialize</phase>
+                        <goals>
+                            <goal>unpack</goal>
+                        </goals>
+                        <configuration>
+                            <artifactItems>
+                                <artifactItem>
+                                    <groupId>org.apache.calcite</groupId>
+                                    <artifactId>calcite-core</artifactId>
+                                    <type>jar</type>
+                                    <overWrite>true</overWrite>
+                                    <outputDirectory>${project.build.directory}/</outputDirectory>
+                                    <includes>**/Parser.jj</includes>
+                                </artifactItem>
+                            </artifactItems>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <!-- using appassembler-maven-plugin instead of maven-dependency-plugin to copy dependencies
+            as copy and unpack goal are not working together -->
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>appassembler-maven-plugin</artifactId>
+                <version>1.9</version>
+                <executions>
+                    <execution>
+                        <id>create-repo</id>
+                        <goals>
+                            <goal>create-repository</goal>
+                        </goals>
+                        <configuration>
+                            <assembleDirectory>${project.build.directory}/app-assembler</assembleDirectory>
+                            <repositoryLayout>flat</repositoryLayout>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>com.googlecode.fmpp-maven-plugin</groupId>
+                <artifactId>fmpp-maven-plugin</artifactId>
+                <version>1.0</version>
+                <dependencies>
+                    <dependency>
+                        <groupId>org.freemarker</groupId>
+                        <artifactId>freemarker</artifactId>
+                        <version>2.3.25-incubating</version>
+                    </dependency>
+                </dependencies>
+                <executions>
+                    <execution>
+                        <id>generate-fmpp-sources</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>generate</goal>
+                        </goals>
+                        <configuration>
+                            <cfgFile>${project.build.directory}/codegen/config.fmpp</cfgFile>
+                            <outputDirectory>target/generated-sources</outputDirectory>
+                            <templateDirectory>${project.build.directory}/codegen/templates</templateDirectory>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+                <version>1.5</version>
+                <executions>
+                    <execution>
+                        <id>add-generated-sources</id>
+                        <phase>process-sources</phase>
+                        <goals>
+                            <goal>add-source</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>${project.build.directory}/generated-sources</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>javacc-maven-plugin</artifactId>
+                <version>2.4</version>
+                <executions>
+                    <execution>
+                        <phase>generate-sources</phase>
+                        <id>javacc</id>
+                        <goals>
+                            <goal>javacc</goal>
+                        </goals>
+                        <configuration>
+                            <sourceDirectory>${project.build.directory}/generated-sources/</sourceDirectory>
+                            <includes>
+                                <include>**/Parser.jj</include>
+                            </includes>
+                            <lookAhead>2</lookAhead>
+                            <isStatic>false</isStatic>
+                            <outputDirectory>${project.build.directory}/generated-sources/</outputDirectory>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+</project>

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/codegen/config.fmpp
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/codegen/config.fmpp b/sql/storm-sql-core/src/codegen/config.fmpp
new file mode 100644
index 0000000..be5a792
--- /dev/null
+++ b/sql/storm-sql-core/src/codegen/config.fmpp
@@ -0,0 +1,23 @@
+# 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.
+
+data: {
+  parser:                   tdd(../data/Parser.tdd)
+}
+
+freemarkerLinks: {
+  includes: includes/
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/codegen/data/Parser.tdd
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/codegen/data/Parser.tdd b/sql/storm-sql-core/src/codegen/data/Parser.tdd
new file mode 100644
index 0000000..b0dccb6
--- /dev/null
+++ b/sql/storm-sql-core/src/codegen/data/Parser.tdd
@@ -0,0 +1,80 @@
+# 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.
+
+{
+  # Generated parser implementation class package and name
+  package: "org.apache.storm.sql.parser.impl",
+  class: "StormParserImpl",
+
+  # List of import statements.
+  imports: [
+    "org.apache.calcite.sql.validate.*",
+    "org.apache.calcite.util.*",
+    "org.apache.storm.sql.parser.*",
+    "java.util.*"
+  ]
+
+  # List of keywords.
+  keywords: [
+    "LOCATION",
+    "INPUTFORMAT",
+    "OUTPUTFORMAT",
+    "PARALLELISM",
+    "STORED",
+    "TBLPROPERTIES",
+    "JAR"
+  ]
+
+  # List of methods for parsing custom SQL statements.
+  statementParserMethods: [
+    "SqlCreateTable()",
+    "SqlCreateFunction()"
+  ]
+
+  # List of methods for parsing custom literals.
+  # Example: ParseJsonLiteral().
+  literalParserMethods: [
+  ]
+
+  # List of methods for parsing custom data types.
+  dataTypeParserMethods: [
+  ]
+
+  nonReservedKeywords: [
+  ]
+
+  createStatementParserMethods: [
+  ]
+
+  alterStatementParserMethods: [
+  ]
+
+  dropStatementParserMethods: [
+  ]
+
+  # List of files in @includes directory that have parser method
+  # implementations for custom SQL statements, literals or types
+  # given as part of "statementParserMethods", "literalParserMethods" or
+  # "dataTypeParserMethods".
+  implementationFiles: [
+    "parserImpls.ftl"
+  ]
+
+  includeCompoundIdentifier: true,
+  includeBraces: true,
+  includeAdditionalDeclarations: false,
+  allowBangEqual: false
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/codegen/includes/license.ftl
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/codegen/includes/license.ftl b/sql/storm-sql-core/src/codegen/includes/license.ftl
new file mode 100644
index 0000000..7e66353
--- /dev/null
+++ b/sql/storm-sql-core/src/codegen/includes/license.ftl
@@ -0,0 +1,17 @@
+/**
+ * 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.
+ */

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/codegen/includes/parserImpls.ftl
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/codegen/includes/parserImpls.ftl b/sql/storm-sql-core/src/codegen/includes/parserImpls.ftl
new file mode 100644
index 0000000..4143840
--- /dev/null
+++ b/sql/storm-sql-core/src/codegen/includes/parserImpls.ftl
@@ -0,0 +1,113 @@
+<#-- 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. -->
+
+
+private void ColumnDef(List<ColumnDefinition> list) :
+{
+    SqlParserPos pos;
+    SqlIdentifier name;
+    SqlDataTypeSpec type;
+    ColumnConstraint constraint = null;
+    SqlMonotonicity monotonicity = SqlMonotonicity.NOT_MONOTONIC;
+}
+{
+    name = SimpleIdentifier() { pos = getPos(); }
+    type = DataType()
+    [
+      <PRIMARY> <KEY>
+      [ <ASC>   { monotonicity = SqlMonotonicity.INCREASING; }
+      | <DESC>  { monotonicity = SqlMonotonicity.DECREASING; }
+      ]
+      { constraint = new ColumnConstraint.PrimaryKey(monotonicity, getPos()); }
+    ]
+    {
+        list.add(new ColumnDefinition(name, type, constraint, pos));
+    }
+}
+
+SqlNodeList ColumnDefinitionList() :
+{
+    SqlParserPos pos;
+    List<ColumnDefinition> list = Lists.newArrayList();
+}
+{
+    <LPAREN> { pos = getPos(); }
+    ColumnDef(list)
+    ( <COMMA> ColumnDef(list) )*
+    <RPAREN> {
+        return new SqlNodeList(list, pos.plus(getPos()));
+    }
+}
+
+/**
+ * CREATE EXTERNAL TABLE ( IF NOT EXISTS )?
+ *   ( database_name '.' )? table_name ( '(' column_def ( ',' column_def )* ')'
+ *   ( STORED AS INPUTFORMAT input_format_classname OUTPUTFORMAT output_format_classname )?
+ *   LOCATION location_uri
+ *   ( TBLPROPERTIES tbl_properties )?
+ *   ( AS select_stmt )
+ */
+SqlNode SqlCreateTable() :
+{
+    SqlParserPos pos;
+    SqlIdentifier tblName;
+    SqlNodeList fieldList;
+    SqlNode location;
+    SqlNode parallelism = null;
+    SqlNode input_format_class_name = null, output_format_class_name = null;
+    SqlNode tbl_properties = null;
+    SqlNode select = null;
+}
+{
+    <CREATE> { pos = getPos(); }
+    <EXTERNAL> <TABLE>
+    tblName = CompoundIdentifier()
+    fieldList = ColumnDefinitionList()
+    [
+      <STORED> <AS>
+      <INPUTFORMAT> input_format_class_name = StringLiteral()
+      <OUTPUTFORMAT> output_format_class_name = StringLiteral()
+    ]
+    <LOCATION>
+    location = StringLiteral()
+    [ <PARALLELISM> parallelism = UnsignedNumericLiteral() ]
+    [ <TBLPROPERTIES> tbl_properties = StringLiteral() ]
+    [ <AS> select = OrderedQueryOrExpr(ExprContext.ACCEPT_QUERY) ] {
+        return new SqlCreateTable(pos, tblName, fieldList,
+        input_format_class_name, output_format_class_name, location,
+        parallelism, tbl_properties, select);
+    }
+}
+
+/**
+ * CREATE FUNCTION functionname AS 'classname'
+ */
+SqlNode SqlCreateFunction() :
+{
+    SqlParserPos pos;
+    SqlIdentifier functionName;
+    SqlNode className;
+    SqlNode jarName = null;
+}
+{
+    <CREATE> { pos = getPos(); }
+    <FUNCTION>
+        functionName = CompoundIdentifier()
+    <AS>
+        className = StringLiteral()
+    [
+      <USING> <JAR>
+      jarName = StringLiteral()
+    ]
+    {
+      return new SqlCreateFunction(pos, functionName, className, jarName);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/AbstractTridentProcessor.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/AbstractTridentProcessor.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/AbstractTridentProcessor.java
new file mode 100644
index 0000000..843339f
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/AbstractTridentProcessor.java
@@ -0,0 +1,56 @@
+/**
+ * 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.sql;
+
+import org.apache.calcite.DataContext;
+import org.apache.storm.sql.javac.CompilingClassLoader;
+import org.apache.storm.trident.Stream;
+import org.apache.storm.trident.TridentTopology;
+
+import java.util.List;
+
+public abstract class AbstractTridentProcessor {
+  protected Stream outputStream;
+  protected DataContext dataContext;
+  protected List<CompilingClassLoader> classLoaders;
+  /**
+   * @return the output stream of the SQL
+   */
+  public Stream outputStream() {
+    return outputStream;
+  }
+
+  /**
+   * Construct the trident topology based on the SQL.
+   */
+  public abstract TridentTopology build();
+
+  /**
+   * @return DataContext instance which is used with execution of query
+   */
+  public DataContext getDataContext() {
+    return dataContext;
+  }
+
+  /**
+   * @return Classloaders to compile. They're all chaining so the last classloader can access all classes.
+   */
+  public List<CompilingClassLoader> getClassLoaders() {
+    return classLoaders;
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSql.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSql.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSql.java
new file mode 100644
index 0000000..5dec4af
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSql.java
@@ -0,0 +1,59 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql;
+
+import org.apache.storm.StormSubmitter;
+import org.apache.storm.generated.SubmitOptions;
+import org.apache.storm.sql.runtime.ChannelHandler;
+
+import java.util.Map;
+
+/**
+ * The StormSql class provides standalone, interactive interfaces to execute
+ * SQL statements over streaming data.
+ * <p>
+ * The StormSql class is stateless. The user needs to submit the data
+ * definition language (DDL) statements and the query statements in the same
+ * batch.
+ */
+public abstract class StormSql {
+  /**
+   * Execute the SQL statements in stand-alone mode. The user can retrieve the result by passing in an instance
+   * of {@see ChannelHandler}.
+   */
+  public abstract void execute(Iterable<String> statements,
+                               ChannelHandler handler) throws Exception;
+
+  /**
+   * Submit the SQL statements to Nimbus and run it as a topology.
+   */
+  public abstract void submit(
+      String name, Iterable<String> statements, Map<String, ?> stormConf, SubmitOptions opts,
+      StormSubmitter.ProgressListener progressListener, String asUser)
+      throws Exception;
+
+  /**
+   * Print out query plan for each query.
+   */
+  public abstract void explain(Iterable<String> statements) throws Exception;
+
+  public static StormSql construct() {
+    return new StormSqlImpl();
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlImpl.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlImpl.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlImpl.java
new file mode 100644
index 0000000..007daa7
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlImpl.java
@@ -0,0 +1,290 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.prepare.CalciteCatalogReader;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.schema.Function;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.impl.AggregateFunctionImpl;
+import org.apache.calcite.schema.impl.ScalarFunctionImpl;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
+import org.apache.calcite.tools.FrameworkConfig;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.tools.Planner;
+import org.apache.storm.StormSubmitter;
+import org.apache.storm.generated.SubmitOptions;
+import org.apache.storm.sql.compiler.StormSqlTypeFactoryImpl;
+import org.apache.storm.sql.compiler.backends.standalone.PlanCompiler;
+import org.apache.storm.sql.javac.CompilingClassLoader;
+import org.apache.storm.sql.parser.ColumnConstraint;
+import org.apache.storm.sql.parser.ColumnDefinition;
+import org.apache.storm.sql.parser.SqlCreateFunction;
+import org.apache.storm.sql.parser.SqlCreateTable;
+import org.apache.storm.sql.parser.StormParser;
+import org.apache.storm.sql.planner.StormRelUtils;
+import org.apache.storm.sql.planner.trident.QueryPlanner;
+import org.apache.storm.sql.runtime.AbstractValuesProcessor;
+import org.apache.storm.sql.runtime.ChannelHandler;
+import org.apache.storm.sql.runtime.DataSource;
+import org.apache.storm.sql.runtime.DataSourcesRegistry;
+import org.apache.storm.sql.runtime.FieldInfo;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.trident.TridentTopology;
+
+import java.io.BufferedOutputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.jar.Attributes;
+import java.util.jar.JarOutputStream;
+import java.util.jar.Manifest;
+import java.util.zip.ZipEntry;
+
+import static org.apache.storm.sql.compiler.CompilerUtil.TableBuilderInfo;
+
+class StormSqlImpl extends StormSql {
+  private final JavaTypeFactory typeFactory = new StormSqlTypeFactoryImpl(
+      RelDataTypeSystem.DEFAULT);
+  private final SchemaPlus schema = Frameworks.createRootSchema(true);
+  private boolean hasUdf = false;
+
+  @Override
+  public void execute(
+      Iterable<String> statements, ChannelHandler result)
+      throws Exception {
+    Map<String, DataSource> dataSources = new HashMap<>();
+    for (String sql : statements) {
+      StormParser parser = new StormParser(sql);
+      SqlNode node = parser.impl().parseSqlStmtEof();
+      if (node instanceof SqlCreateTable) {
+        handleCreateTable((SqlCreateTable) node, dataSources);
+      } else if (node instanceof SqlCreateFunction) {
+        handleCreateFunction((SqlCreateFunction) node);
+      } else {
+        FrameworkConfig config = buildFrameWorkConfig();
+        Planner planner = Frameworks.getPlanner(config);
+        SqlNode parse = planner.parse(sql);
+        SqlNode validate = planner.validate(parse);
+        RelNode tree = planner.convert(validate);
+        PlanCompiler compiler = new PlanCompiler(typeFactory);
+        AbstractValuesProcessor proc = compiler.compile(tree);
+        proc.initialize(dataSources, result);
+      }
+    }
+  }
+
+  @Override
+  public void submit(
+      String name, Iterable<String> statements, Map<String, ?> stormConf, SubmitOptions opts,
+      StormSubmitter.ProgressListener progressListener, String asUser)
+      throws Exception {
+    Map<String, ISqlTridentDataSource> dataSources = new HashMap<>();
+    for (String sql : statements) {
+      StormParser parser = new StormParser(sql);
+      SqlNode node = parser.impl().parseSqlStmtEof();
+      if (node instanceof SqlCreateTable) {
+        handleCreateTableForTrident((SqlCreateTable) node, dataSources);
+      } else if (node instanceof SqlCreateFunction) {
+        handleCreateFunction((SqlCreateFunction) node);
+      }  else {
+        QueryPlanner planner = new QueryPlanner(schema);
+        AbstractTridentProcessor processor = planner.compile(dataSources, sql);
+        TridentTopology topo = processor.build();
+
+        Path jarPath = null;
+        try {
+          // QueryPlanner on Trident mode configures the topology with compiled classes,
+          // so we need to add new classes into topology jar
+          // Topology will be serialized and sent to Nimbus, and deserialized and executed in workers.
+
+          jarPath = Files.createTempFile("storm-sql", ".jar");
+          System.setProperty("storm.jar", jarPath.toString());
+          packageTopology(jarPath, processor);
+          StormSubmitter.submitTopologyAs(name, stormConf, topo.build(), opts, progressListener, asUser);
+        } finally {
+          if (jarPath != null) {
+            Files.delete(jarPath);
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public void explain(Iterable<String> statements) throws Exception {
+    Map<String, ISqlTridentDataSource> dataSources = new HashMap<>();
+    for (String sql : statements) {
+      StormParser parser = new StormParser(sql);
+      SqlNode node = parser.impl().parseSqlStmtEof();
+
+      System.out.println("===========================================================");
+      System.out.println("query>");
+      System.out.println(sql);
+      System.out.println("-----------------------------------------------------------");
+
+      if (node instanceof SqlCreateTable) {
+        handleCreateTableForTrident((SqlCreateTable) node, dataSources);
+        System.out.println("No plan presented on DDL");
+      } else if (node instanceof SqlCreateFunction) {
+        handleCreateFunction((SqlCreateFunction) node);
+        System.out.println("No plan presented on DDL");
+      } else {
+        FrameworkConfig config = buildFrameWorkConfig();
+        Planner planner = Frameworks.getPlanner(config);
+        SqlNode parse = planner.parse(sql);
+        SqlNode validate = planner.validate(parse);
+        RelNode tree = planner.convert(validate);
+
+        String plan = StormRelUtils.explain(tree, SqlExplainLevel.ALL_ATTRIBUTES);
+        System.out.println("plan>");
+        System.out.println(plan);
+      }
+
+      System.out.println("===========================================================");
+    }
+  }
+
+  private void packageTopology(Path jar, AbstractTridentProcessor processor) throws IOException {
+    Manifest manifest = new Manifest();
+    Attributes attr = manifest.getMainAttributes();
+    attr.put(Attributes.Name.MANIFEST_VERSION, "1.0");
+    attr.put(Attributes.Name.MAIN_CLASS, processor.getClass().getCanonicalName());
+    try (JarOutputStream out = new JarOutputStream(
+            new BufferedOutputStream(new FileOutputStream(jar.toFile())), manifest)) {
+      List<CompilingClassLoader> classLoaders = processor.getClassLoaders();
+      if (classLoaders != null && !classLoaders.isEmpty()) {
+        for (CompilingClassLoader classLoader : classLoaders) {
+          for (Map.Entry<String, ByteArrayOutputStream> e : classLoader.getClasses().entrySet()) {
+            out.putNextEntry(new ZipEntry(e.getKey().replace(".", "/") + ".class"));
+            out.write(e.getValue().toByteArray());
+            out.closeEntry();
+          }
+        }
+      }
+    }
+  }
+
+  private void handleCreateTable(
+      SqlCreateTable n, Map<String, DataSource> dataSources) {
+    List<FieldInfo> fields = updateSchema(n);
+    DataSource ds = DataSourcesRegistry.construct(n.location(), n
+        .inputFormatClass(), n.outputFormatClass(), fields);
+    if (ds == null) {
+      throw new RuntimeException("Cannot construct data source for " + n
+          .tableName());
+    } else if (dataSources.containsKey(n.tableName())) {
+      throw new RuntimeException("Duplicated definition for table " + n
+          .tableName());
+    }
+    dataSources.put(n.tableName(), ds);
+  }
+
+  private void handleCreateFunction(SqlCreateFunction sqlCreateFunction) throws ClassNotFoundException {
+    if(sqlCreateFunction.jarName() != null) {
+      throw new UnsupportedOperationException("UDF 'USING JAR' not implemented");
+    }
+    Method method;
+    Function function;
+    if ((method=findMethod(sqlCreateFunction.className(), "evaluate")) != null) {
+      function = ScalarFunctionImpl.create(method);
+    } else if (findMethod(sqlCreateFunction.className(), "add") != null) {
+      function = AggregateFunctionImpl.create(Class.forName(sqlCreateFunction.className()));
+    } else {
+      throw new RuntimeException("Invalid scalar or aggregate function");
+    }
+    schema.add(sqlCreateFunction.functionName().toUpperCase(), function);
+    hasUdf = true;
+  }
+
+  private Method findMethod(String clazzName, String methodName) throws ClassNotFoundException {
+    Class<?> clazz = Class.forName(clazzName);
+    for (Method method : clazz.getMethods()) {
+      if (method.getName().equals(methodName)) {
+        return method;
+      }
+    }
+    return null;
+  }
+
+  private void handleCreateTableForTrident(
+      SqlCreateTable n, Map<String, ISqlTridentDataSource> dataSources) {
+    List<FieldInfo> fields = updateSchema(n);
+    ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(n.location(), n
+        .inputFormatClass(), n.outputFormatClass(), n.properties(), fields);
+    if (ds == null) {
+      throw new RuntimeException("Failed to find data source for " + n
+          .tableName() + " URI: " + n.location());
+    } else if (dataSources.containsKey(n.tableName())) {
+      throw new RuntimeException("Duplicated definition for table " + n
+          .tableName());
+    }
+    dataSources.put(n.tableName(), ds);
+  }
+
+  private List<FieldInfo> updateSchema(SqlCreateTable n) {
+    TableBuilderInfo builder = new TableBuilderInfo(typeFactory);
+    List<FieldInfo> fields = new ArrayList<>();
+    for (ColumnDefinition col : n.fieldList()) {
+      builder.field(col.name(), col.type(), col.constraint());
+      RelDataType dataType = col.type().deriveType(typeFactory);
+      Class<?> javaType = (Class<?>)typeFactory.getJavaClass(dataType);
+      ColumnConstraint constraint = col.constraint();
+      boolean isPrimary = constraint != null && constraint instanceof ColumnConstraint.PrimaryKey;
+      fields.add(new FieldInfo(col.name(), javaType, isPrimary));
+    }
+
+    if (n.parallelism() != null) {
+      builder.parallelismHint(n.parallelism());
+    }
+    Table table = builder.build();
+    schema.add(n.tableName(), table);
+    return fields;
+  }
+
+  private FrameworkConfig buildFrameWorkConfig() {
+    if (hasUdf) {
+      List<SqlOperatorTable> sqlOperatorTables = new ArrayList<>();
+      sqlOperatorTables.add(SqlStdOperatorTable.instance());
+      sqlOperatorTables.add(new CalciteCatalogReader(CalciteSchema.from(schema),
+                                                     false,
+                                                     Collections.<String>emptyList(), typeFactory));
+      return Frameworks.newConfigBuilder().defaultSchema(schema)
+              .operatorTable(new ChainedSqlOperatorTable(sqlOperatorTables)).build();
+    } else {
+      return Frameworks.newConfigBuilder().defaultSchema(schema).build();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlRunner.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlRunner.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlRunner.java
new file mode 100644
index 0000000..5618647
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlRunner.java
@@ -0,0 +1,85 @@
+/**
+ * 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.sql;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.DefaultParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Options;
+import org.apache.storm.generated.SubmitOptions;
+import org.apache.storm.generated.TopologyInitialStatus;
+import org.apache.storm.utils.Utils;
+
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.List;
+import java.util.Map;
+
+public class StormSqlRunner {
+    private static final String OPTION_SQL_FILE_SHORT = "f";
+    private static final String OPTION_SQL_FILE_LONG = "file";
+    private static final String OPTION_SQL_TOPOLOGY_NAME_SHORT = "t";
+    private static final String OPTION_SQL_TOPOLOGY_NAME_LONG = "topology";
+    private static final String OPTION_SQL_EXPLAIN_SHORT = "e";
+    private static final String OPTION_SQL_EXPLAIN_LONG = "explain";
+
+    public static void main(String[] args) throws Exception {
+        Options options = buildOptions();
+        CommandLineParser parser = new DefaultParser();
+        CommandLine commandLine = parser.parse(options, args);
+
+        if (!commandLine.hasOption(OPTION_SQL_FILE_LONG)) {
+            printUsageAndExit(options, OPTION_SQL_FILE_LONG + " is required");
+        }
+
+        String filePath = commandLine.getOptionValue(OPTION_SQL_FILE_LONG);
+        List<String> stmts = Files.readAllLines(Paths.get(filePath), StandardCharsets.UTF_8);
+        StormSql sql = StormSql.construct();
+        @SuppressWarnings("unchecked")
+        Map<String, ?> conf = Utils.readStormConfig();
+
+        if (commandLine.hasOption(OPTION_SQL_EXPLAIN_LONG)) {
+            sql.explain(stmts);
+        } else if (commandLine.hasOption(OPTION_SQL_TOPOLOGY_NAME_LONG)) {
+            String topoName = commandLine.getOptionValue(OPTION_SQL_TOPOLOGY_NAME_LONG);
+            SubmitOptions submitOptions = new SubmitOptions(TopologyInitialStatus.ACTIVE);
+            sql.submit(topoName, stmts, conf, submitOptions, null, null);
+        } else {
+            printUsageAndExit(options, "Either " + OPTION_SQL_TOPOLOGY_NAME_LONG + " or " + OPTION_SQL_EXPLAIN_LONG +
+                    " must be presented");
+        }
+    }
+
+    private static void printUsageAndExit(Options options, String message) {
+        System.out.println(message);
+        HelpFormatter formatter = new HelpFormatter();
+        formatter.printHelp("storm-sql-runner ", options);
+        System.exit(1);
+    }
+
+    private static Options buildOptions() {
+        Options options = new Options();
+        options.addOption(OPTION_SQL_FILE_SHORT, OPTION_SQL_FILE_LONG, true, "REQUIRED SQL file which has sql statements");
+        options.addOption(OPTION_SQL_TOPOLOGY_NAME_SHORT, OPTION_SQL_TOPOLOGY_NAME_LONG, true, "Topology name to submit");
+        options.addOption(OPTION_SQL_EXPLAIN_SHORT, OPTION_SQL_EXPLAIN_LONG, false, "Activate explain mode (topology name will be ignored)");
+        return options;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/calcite/ParallelStreamableTable.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/calcite/ParallelStreamableTable.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/calcite/ParallelStreamableTable.java
new file mode 100644
index 0000000..c6b584d
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/calcite/ParallelStreamableTable.java
@@ -0,0 +1,35 @@
+/**
+ * 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.sql.calcite;
+
+import org.apache.calcite.rel.stream.Delta;
+import org.apache.calcite.schema.StreamableTable;
+
+/**
+ * Table that can be converted to a stream. This table also has its parallelism information.
+ *
+ * @see Delta
+ */
+public interface ParallelStreamableTable extends StreamableTable {
+
+    /**
+     * Returns parallelism hint of this table. Returns null if don't know.
+     */
+    Integer parallelismHint();
+}


[09/23] storm git commit: STORM-2453 Move non-connectors into the top directory

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/ExecutionContext.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/ExecutionContext.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/ExecutionContext.java
new file mode 100644
index 0000000..1520006
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/ExecutionContext.java
@@ -0,0 +1,77 @@
+/*
+ * 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.flux.model;
+
+import org.apache.storm.Config;
+import org.apache.storm.task.IBolt;
+import org.apache.storm.topology.IRichSpout;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Container for all the objects required to instantiate a topology.
+ */
+public class ExecutionContext {
+    // parsed Topology definition
+    TopologyDef topologyDef;
+
+    // Storm config
+    private Config config;
+
+    // components
+    private List<Object> compontents;
+    // indexed by id
+    private Map<String, Object> componentMap = new HashMap<String, Object>();
+
+    private Map<String, IRichSpout> spoutMap = new HashMap<String, IRichSpout>();
+
+    private List<IBolt> bolts;
+    private Map<String, Object> boltMap = new HashMap<String, Object>();
+
+    public ExecutionContext(TopologyDef topologyDef, Config config){
+        this.topologyDef = topologyDef;
+        this.config = config;
+    }
+
+    public TopologyDef getTopologyDef(){
+        return this.topologyDef;
+    }
+
+    public void addSpout(String id, IRichSpout spout){
+        this.spoutMap.put(id, spout);
+    }
+
+    public void addBolt(String id, Object bolt){
+        this.boltMap.put(id, bolt);
+    }
+
+    public Object getBolt(String id){
+        return this.boltMap.get(id);
+    }
+
+    public void addComponent(String id, Object value){
+        this.componentMap.put(id, value);
+    }
+
+    public Object getComponent(String id){
+        return this.componentMap.get(id);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/GroupingDef.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/GroupingDef.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/GroupingDef.java
new file mode 100644
index 0000000..e4fac8e
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/GroupingDef.java
@@ -0,0 +1,77 @@
+/*
+ * 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.flux.model;
+
+import java.util.List;
+
+/**
+ * Bean representation of a Storm stream grouping.
+ */
+public class GroupingDef {
+
+    /**
+     * Types of stream groupings Storm allows
+     */
+    public static enum Type {
+        ALL,
+        CUSTOM,
+        DIRECT,
+        SHUFFLE,
+        LOCAL_OR_SHUFFLE,
+        FIELDS,
+        GLOBAL,
+        NONE
+    }
+
+    private Type type;
+    private String streamId;
+    private List<String> args;
+    private ObjectDef customClass;
+
+    public List<String> getArgs() {
+        return args;
+    }
+
+    public void setArgs(List<String> args) {
+        this.args = args;
+    }
+
+    public Type getType() {
+        return type;
+    }
+
+    public void setType(Type type) {
+        this.type = type;
+    }
+
+    public String getStreamId() {
+        return streamId;
+    }
+
+    public void setStreamId(String streamId) {
+        this.streamId = streamId;
+    }
+
+    public ObjectDef getCustomClass() {
+        return customClass;
+    }
+
+    public void setCustomClass(ObjectDef customClass) {
+        this.customClass = customClass;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/IncludeDef.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/IncludeDef.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/IncludeDef.java
new file mode 100644
index 0000000..23fd9d2
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/IncludeDef.java
@@ -0,0 +1,54 @@
+/*
+ * 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.flux.model;
+
+/**
+ * Represents an include. Includes can be either a file or a classpath resource.
+ *
+ * If an include is marked as `override=true` then existing properties will be replaced.
+ *
+ */
+public class IncludeDef {
+    private boolean resource = false;
+    boolean override = false;
+    private String file;
+
+    public boolean isResource() {
+        return resource;
+    }
+
+    public void setResource(boolean resource) {
+        this.resource = resource;
+    }
+
+    public String getFile() {
+        return file;
+    }
+
+    public void setFile(String file) {
+        this.file = file;
+    }
+
+    public boolean isOverride() {
+        return override;
+    }
+
+    public void setOverride(boolean override) {
+        this.override = override;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/ObjectDef.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/ObjectDef.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/ObjectDef.java
new file mode 100644
index 0000000..04a7e8a
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/ObjectDef.java
@@ -0,0 +1,95 @@
+/*
+ * 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.flux.model;
+
+import org.apache.storm.Config;
+
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A representation of a Java object that given a className, constructor arguments,
+ * and properties, can be instantiated.
+ */
+public class ObjectDef {
+    private String className;
+    private List<Object> constructorArgs;
+    private boolean hasReferences;
+    private List<PropertyDef> properties;
+    private List<ConfigMethodDef> configMethods;
+
+    public String getClassName() {
+        return className;
+    }
+
+    public void setClassName(String className) {
+        this.className = className;
+    }
+
+    public List<Object> getConstructorArgs() {
+        return constructorArgs;
+    }
+
+    public void setConstructorArgs(List<Object> constructorArgs) {
+
+        List<Object> newVal = new ArrayList<Object>();
+        for(Object obj : constructorArgs){
+            if(obj instanceof LinkedHashMap){
+                Map map = (Map)obj;
+                if(map.containsKey("ref") && map.size() == 1) {
+                    newVal.add(new BeanReference((String) map.get("ref")));
+                    this.hasReferences = true;
+                } else if (map.containsKey("reflist") && map.size() == 1) {
+                    newVal.add(new BeanListReference((List<String>) map.get("reflist")));
+                    this.hasReferences = true;
+                } else {
+                    newVal.add(obj);
+                }
+            } else {
+                newVal.add(obj);
+            }
+        }
+        this.constructorArgs = newVal;
+    }
+
+    public boolean hasConstructorArgs(){
+        return this.constructorArgs != null && this.constructorArgs.size() > 0;
+    }
+
+    public boolean hasReferences(){
+        return this.hasReferences;
+    }
+
+    public List<PropertyDef> getProperties() {
+        return properties;
+    }
+
+    public void setProperties(List<PropertyDef> properties) {
+        this.properties = properties;
+    }
+
+    public List<ConfigMethodDef> getConfigMethods() {
+        return configMethods;
+    }
+
+    public void setConfigMethods(List<ConfigMethodDef> configMethods) {
+        this.configMethods = configMethods;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/PropertyDef.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/PropertyDef.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/PropertyDef.java
new file mode 100644
index 0000000..f3d7704
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/PropertyDef.java
@@ -0,0 +1,58 @@
+/*
+ * 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.flux.model;
+
+public class PropertyDef {
+    private String name;
+    private Object value;
+    private String ref;
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public Object getValue() {
+        return value;
+    }
+
+    public void setValue(Object value) {
+        if(this.ref != null){
+            throw new IllegalStateException("A property can only have a value OR a reference, not both.");
+        }
+        this.value = value;
+    }
+
+    public String getRef() {
+        return ref;
+    }
+
+    public void setRef(String ref) {
+        if(this.value != null){
+            throw new IllegalStateException("A property can only have a value OR a reference, not both.");
+        }
+        this.ref = ref;
+    }
+
+    public boolean isReference(){
+        return this.ref != null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/SpoutDef.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/SpoutDef.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/SpoutDef.java
new file mode 100644
index 0000000..277c601
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/SpoutDef.java
@@ -0,0 +1,24 @@
+/*
+ * 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.flux.model;
+
+/**
+ * Bean representation of a Storm spout.
+ */
+public class SpoutDef extends VertexDef {
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/StreamDef.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/StreamDef.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/StreamDef.java
new file mode 100644
index 0000000..da80f1c
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/StreamDef.java
@@ -0,0 +1,64 @@
+/*
+ * 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.flux.model;
+
+/**
+ * Represents a stream of tuples from one Storm component (Spout or Bolt) to another (an edge in the topology DAG).
+ *
+ * Required fields are `from` and `to`, which define the source and destination, and the stream `grouping`.
+ *
+ */
+public class StreamDef {
+
+    private String name; // not used, placeholder for GUI, etc.
+    private String from;
+    private String to;
+    private GroupingDef grouping;
+
+    public String getTo() {
+        return to;
+    }
+
+    public void setTo(String to) {
+        this.to = to;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public String getFrom() {
+        return from;
+    }
+
+    public void setFrom(String from) {
+        this.from = from;
+    }
+
+    public GroupingDef getGrouping() {
+        return grouping;
+    }
+
+    public void setGrouping(GroupingDef grouping) {
+        this.grouping = grouping;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologyDef.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologyDef.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologyDef.java
new file mode 100644
index 0000000..86614f1
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologyDef.java
@@ -0,0 +1,216 @@
+/*
+ * 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.flux.model;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
+
+/**
+ * Bean represenation of a topology.
+ *
+ * It consists of the following:
+ *   1. The topology name
+ *   2. A `java.util.Map` representing the `org.apache.storm.config` for the topology
+ *   3. A list of spout definitions
+ *   4. A list of bolt definitions
+ *   5. A list of stream definitions that define the flow between spouts and bolts.
+ *
+ */
+public class TopologyDef {
+    private static Logger LOG = LoggerFactory.getLogger(TopologyDef.class);
+
+    private String name;
+    private Map<String, BeanDef> componentMap = new LinkedHashMap<String, BeanDef>(); // not required
+    private List<IncludeDef> includes; // not required
+    private Map<String, Object> config = new HashMap<String, Object>();
+
+    // a "topology source" is a class that can produce a `StormTopology` thrift object.
+    private TopologySourceDef topologySource;
+
+    // the following are required if we're defining a core storm topology DAG in YAML, etc.
+    private Map<String, BoltDef> boltMap = new LinkedHashMap<String, BoltDef>();
+    private Map<String, SpoutDef> spoutMap = new LinkedHashMap<String, SpoutDef>();
+    private List<StreamDef> streams = new ArrayList<StreamDef>();
+
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public void setName(String name, boolean override){
+        if(this.name == null || override){
+            this.name = name;
+        } else {
+            LOG.warn("Ignoring attempt to set property 'name' with override == false.");
+        }
+    }
+
+    public List<SpoutDef> getSpouts() {
+        ArrayList<SpoutDef> retval = new ArrayList<SpoutDef>();
+        retval.addAll(this.spoutMap.values());
+        return retval;
+    }
+
+    public void setSpouts(List<SpoutDef> spouts) {
+        this.spoutMap = new LinkedHashMap<String, SpoutDef>();
+        for(SpoutDef spout : spouts){
+            this.spoutMap.put(spout.getId(), spout);
+        }
+    }
+
+    public List<BoltDef> getBolts() {
+        ArrayList<BoltDef> retval = new ArrayList<BoltDef>();
+        retval.addAll(this.boltMap.values());
+        return retval;
+    }
+
+    public void setBolts(List<BoltDef> bolts) {
+        this.boltMap = new LinkedHashMap<String, BoltDef>();
+        for(BoltDef bolt : bolts){
+            this.boltMap.put(bolt.getId(), bolt);
+        }
+    }
+
+    public List<StreamDef> getStreams() {
+        return streams;
+    }
+
+    public void setStreams(List<StreamDef> streams) {
+        this.streams = streams;
+    }
+
+    public Map<String, Object> getConfig() {
+        return config;
+    }
+
+    public void setConfig(Map<String, Object> config) {
+        this.config = config;
+    }
+
+    public List<BeanDef> getComponents() {
+        ArrayList<BeanDef> retval = new ArrayList<BeanDef>();
+        retval.addAll(this.componentMap.values());
+        return retval;
+    }
+
+    public void setComponents(List<BeanDef> components) {
+        this.componentMap = new LinkedHashMap<String, BeanDef>();
+        for(BeanDef component : components){
+            this.componentMap.put(component.getId(), component);
+        }
+    }
+
+    public List<IncludeDef> getIncludes() {
+        return includes;
+    }
+
+    public void setIncludes(List<IncludeDef> includes) {
+        this.includes = includes;
+    }
+
+    // utility methods
+    public int parallelismForBolt(String boltId){
+        return this.boltMap.get(boltId).getParallelism();
+    }
+
+    public BoltDef getBoltDef(String id){
+        return this.boltMap.get(id);
+    }
+
+    public SpoutDef getSpoutDef(String id){
+        return this.spoutMap.get(id);
+    }
+
+    public BeanDef getComponent(String id){
+        return this.componentMap.get(id);
+    }
+
+    // used by includes implementation
+    public void addAllBolts(List<BoltDef> bolts, boolean override){
+        for(BoltDef bolt : bolts){
+            String id = bolt.getId();
+            if(this.boltMap.get(id) == null || override) {
+                this.boltMap.put(bolt.getId(), bolt);
+            } else {
+                LOG.warn("Ignoring attempt to create bolt '{}' with override == false.", id);
+            }
+        }
+    }
+
+    public void addAllSpouts(List<SpoutDef> spouts, boolean override){
+        for(SpoutDef spout : spouts){
+            String id = spout.getId();
+            if(this.spoutMap.get(id) == null || override) {
+                this.spoutMap.put(spout.getId(), spout);
+            } else {
+                LOG.warn("Ignoring attempt to create spout '{}' with override == false.", id);
+            }
+        }
+    }
+
+    public void addAllComponents(List<BeanDef> components, boolean override) {
+        for(BeanDef bean : components){
+            String id = bean.getId();
+            if(this.componentMap.get(id) == null || override) {
+                this.componentMap.put(bean.getId(), bean);
+            } else {
+                LOG.warn("Ignoring attempt to create component '{}' with override == false.", id);
+            }
+        }
+    }
+
+    public void addAllStreams(List<StreamDef> streams, boolean override) {
+        //TODO figure out how we want to deal with overrides. Users may want to add streams even when overriding other
+        // properties. For now we just add them blindly which could lead to a potentially invalid topology.
+        this.streams.addAll(streams);
+    }
+
+    public TopologySourceDef getTopologySource() {
+        return topologySource;
+    }
+
+    public void setTopologySource(TopologySourceDef topologySource) {
+        this.topologySource = topologySource;
+    }
+
+    public boolean isDslTopology(){
+        return this.topologySource == null;
+    }
+
+
+    public boolean validate(){
+        boolean hasSpouts = this.spoutMap != null && this.spoutMap.size() > 0;
+        boolean hasBolts = this.boltMap != null && this.boltMap.size() > 0;
+        boolean hasStreams = this.streams != null && this.streams.size() > 0;
+        boolean hasSpoutsBoltsStreams = hasStreams && hasBolts && hasSpouts;
+        // you cant define a topologySource and a DSL topology at the same time...
+        if (!isDslTopology() && ((hasSpouts || hasBolts || hasStreams))) {
+            return false;
+        }
+        if(isDslTopology() && (hasSpouts && hasBolts && hasStreams)) {
+            return true;
+        }
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologySourceDef.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologySourceDef.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologySourceDef.java
new file mode 100644
index 0000000..d6a2f57
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/TopologySourceDef.java
@@ -0,0 +1,36 @@
+/*
+ * 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.flux.model;
+
+public class TopologySourceDef extends ObjectDef {
+    public static final String DEFAULT_METHOD_NAME = "getTopology";
+
+    private String methodName;
+
+    public TopologySourceDef(){
+        this.methodName = DEFAULT_METHOD_NAME;
+    }
+
+    public String getMethodName() {
+        return methodName;
+    }
+
+    public void setMethodName(String methodName) {
+        this.methodName = methodName;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/model/VertexDef.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/model/VertexDef.java b/flux/flux-core/src/main/java/org/apache/storm/flux/model/VertexDef.java
new file mode 100644
index 0000000..e71bcc2
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/model/VertexDef.java
@@ -0,0 +1,36 @@
+/*
+ * 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.flux.model;
+
+/**
+ * Abstract parent class of component definitions
+ * (spouts/bolts)
+ */
+public abstract class VertexDef extends BeanDef {
+
+    // default parallelism to 1 so if it's ommitted, the topology will still function.
+    private int parallelism = 1;
+
+    public int getParallelism() {
+        return parallelism;
+    }
+
+    public void setParallelism(int parallelism) {
+        this.parallelism = parallelism;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/java/org/apache/storm/flux/parser/FluxParser.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/java/org/apache/storm/flux/parser/FluxParser.java b/flux/flux-core/src/main/java/org/apache/storm/flux/parser/FluxParser.java
new file mode 100644
index 0000000..35904a2
--- /dev/null
+++ b/flux/flux-core/src/main/java/org/apache/storm/flux/parser/FluxParser.java
@@ -0,0 +1,206 @@
+/*
+ * 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.flux.parser;
+
+import org.apache.storm.flux.model.BoltDef;
+import org.apache.storm.flux.model.IncludeDef;
+import org.apache.storm.flux.model.SpoutDef;
+import org.apache.storm.flux.model.TopologyDef;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.yaml.snakeyaml.TypeDescription;
+import org.yaml.snakeyaml.Yaml;
+import org.yaml.snakeyaml.constructor.Constructor;
+
+import java.io.ByteArrayOutputStream;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+import java.util.Properties;
+
+public class FluxParser {
+    private static final Logger LOG = LoggerFactory.getLogger(FluxParser.class);
+
+    private FluxParser(){}
+
+    // TODO refactor input stream processing (see parseResource() method).
+    public static TopologyDef parseFile(String inputFile, boolean dumpYaml, boolean processIncludes,
+    	String propertiesFile, boolean envSub) throws IOException {
+   
+        FileInputStream in = new FileInputStream(inputFile);
+        TopologyDef topology = parseInputStream(in, dumpYaml, processIncludes, propertiesFile, envSub);
+        in.close();
+        
+        return topology;
+    }
+
+    public static TopologyDef parseResource(String resource, boolean dumpYaml, boolean processIncludes,
+    	String propertiesFile, boolean envSub) throws IOException {
+        
+        InputStream in = FluxParser.class.getResourceAsStream(resource);
+        TopologyDef topology = parseInputStream(in, dumpYaml, processIncludes, propertiesFile, envSub);
+        in.close();
+        
+        return topology;
+    }
+    
+    public static TopologyDef parseInputStream(InputStream inputStream, boolean dumpYaml, boolean processIncludes,
+    	String propertiesFile, boolean envSub) throws IOException {
+		
+	Yaml yaml = yaml();
+    	
+	if (inputStream == null) {
+		LOG.error("Unable to load input stream");
+		System.exit(1);
+	}
+		
+	TopologyDef topology = loadYaml(yaml, inputStream, propertiesFile, envSub);
+		
+	if (dumpYaml) {
+		dumpYaml(topology, yaml);
+	}
+	
+	if (processIncludes) {
+		return processIncludes(yaml, topology, propertiesFile, envSub);
+	} else {
+		return topology;
+	}
+    }
+
+    private static TopologyDef loadYaml(Yaml yaml, InputStream in, String propsFile, boolean envSubstitution) throws IOException {
+        ByteArrayOutputStream bos = new ByteArrayOutputStream();
+        LOG.info("loading YAML from input stream...");
+        int b = -1;
+        while((b = in.read()) != -1){
+            bos.write(b);
+        }
+
+        // TODO substitution implementation is not exactly efficient or kind to memory...
+        String str = bos.toString();
+        // properties file substitution
+        if(propsFile != null){
+            LOG.info("Performing property substitution.");
+            InputStream propsIn = new FileInputStream(propsFile);
+            Properties props = new Properties();
+            props.load(propsIn);
+            for(Object key : props.keySet()){
+                str = str.replace("${" + key + "}", props.getProperty((String)key));
+            }
+        } else {
+            LOG.info("Not performing property substitution.");
+        }
+
+        // environment variable substitution
+        if(envSubstitution){
+            LOG.info("Performing environment variable substitution...");
+            Map<String, String> envs = System.getenv();
+            for(String key : envs.keySet()){
+                str = str.replace("${ENV-" + key + "}", envs.get(key));
+            }
+        } else {
+            LOG.info("Not performing environment variable substitution.");
+        }
+        return (TopologyDef)yaml.load(str);
+    }
+
+    private static void dumpYaml(TopologyDef topology, Yaml yaml){
+        System.out.println("Configuration (interpreted): \n" + yaml.dump(topology));
+    }
+
+    private static Yaml yaml(){
+        Constructor constructor = new Constructor(TopologyDef.class);
+
+        TypeDescription topologyDescription = new TypeDescription(TopologyDef.class);
+        topologyDescription.putListPropertyType("spouts", SpoutDef.class);
+        topologyDescription.putListPropertyType("bolts", BoltDef.class);
+        topologyDescription.putListPropertyType("includes", IncludeDef.class);
+        constructor.addTypeDescription(topologyDescription);
+
+        Yaml  yaml = new Yaml(constructor);
+        return yaml;
+    }
+
+    /**
+     *
+     * @param yaml the yaml parser for parsing the include file(s)
+     * @param topologyDef the topology definition containing (possibly zero) includes
+     * @return The TopologyDef with includes resolved.
+     */
+    private static TopologyDef processIncludes(Yaml yaml, TopologyDef topologyDef, String propsFile, boolean envSub)
+            throws IOException {
+        //TODO support multiple levels of includes
+        if(topologyDef.getIncludes() != null) {
+            for (IncludeDef include : topologyDef.getIncludes()){
+                TopologyDef includeTopologyDef = null;
+                if (include.isResource()) {
+                    LOG.info("Loading includes from resource: {}", include.getFile());
+                    includeTopologyDef = parseResource(include.getFile(), true, false, propsFile, envSub);
+                } else {
+                    LOG.info("Loading includes from file: {}", include.getFile());
+                    includeTopologyDef = parseFile(include.getFile(), true, false, propsFile, envSub);
+                }
+
+                // if overrides are disabled, we won't replace anything that already exists
+                boolean override = include.isOverride();
+                // name
+                if(includeTopologyDef.getName() != null){
+                    topologyDef.setName(includeTopologyDef.getName(), override);
+                }
+
+                // config
+                if(includeTopologyDef.getConfig() != null) {
+                    //TODO move this logic to the model class
+                    Map<String, Object> config = topologyDef.getConfig();
+                    Map<String, Object> includeConfig = includeTopologyDef.getConfig();
+                    if(override) {
+                        config.putAll(includeTopologyDef.getConfig());
+                    } else {
+                        for(String key : includeConfig.keySet()){
+                            if(config.containsKey(key)){
+                                LOG.warn("Ignoring attempt to set topology config property '{}' with override == false", key);
+                            }
+                            else {
+                                config.put(key, includeConfig.get(key));
+                            }
+                        }
+                    }
+                }
+
+                //component overrides
+                if(includeTopologyDef.getComponents() != null){
+                    topologyDef.addAllComponents(includeTopologyDef.getComponents(), override);
+                }
+                //bolt overrides
+                if(includeTopologyDef.getBolts() != null){
+                    topologyDef.addAllBolts(includeTopologyDef.getBolts(), override);
+                }
+                //spout overrides
+                if(includeTopologyDef.getSpouts() != null) {
+                    topologyDef.addAllSpouts(includeTopologyDef.getSpouts(), override);
+                }
+                //stream overrides
+                //TODO streams should be uniquely identifiable
+                if(includeTopologyDef.getStreams() != null) {
+                    topologyDef.addAllStreams(includeTopologyDef.getStreams(), override);
+                }
+            } // end include processing
+        }
+        return topologyDef;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/main/resources/splash.txt
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/main/resources/splash.txt b/flux/flux-core/src/main/resources/splash.txt
new file mode 100644
index 0000000..337931a
--- /dev/null
+++ b/flux/flux-core/src/main/resources/splash.txt
@@ -0,0 +1,9 @@
+\u2588\u2588\u2588\u2588\u2588\u2588\u2588\u2557\u2588\u2588\u2557     \u2588\u2588\u2557   \u2588\u2588\u2557\u2588\u2588\u2557  \u2588\u2588\u2557
+\u2588\u2588\u2554\u2550\u2550\u2550\u2550\u255d\u2588\u2588\u2551     \u2588\u2588\u2551   \u2588\u2588\u2551\u255a\u2588\u2588\u2557\u2588\u2588\u2554\u255d
+\u2588\u2588\u2588\u2588\u2588\u2557  \u2588\u2588\u2551     \u2588\u2588\u2551   \u2588\u2588\u2551 \u255a\u2588\u2588\u2588\u2554\u255d
+\u2588\u2588\u2554\u2550\u2550\u255d  \u2588\u2588\u2551     \u2588\u2588\u2551   \u2588\u2588\u2551 \u2588\u2588\u2554\u2588\u2588\u2557
+\u2588\u2588\u2551     \u2588\u2588\u2588\u2588\u2588\u2588\u2588\u2557\u255a\u2588\u2588\u2588\u2588\u2588\u2588\u2554\u255d\u2588\u2588\u2554\u255d \u2588\u2588\u2557
+\u255a\u2550\u255d     \u255a\u2550\u2550\u2550\u2550\u2550\u2550\u255d \u255a\u2550\u2550\u2550\u2550\u2550\u255d \u255a\u2550\u255d  \u255a\u2550\u255d
++-         Apache Storm        -+
++-  data FLow User eXperience  -+
+Version: ${project.version}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/java/org/apache/storm/flux/FluxBuilderTest.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/java/org/apache/storm/flux/FluxBuilderTest.java b/flux/flux-core/src/test/java/org/apache/storm/flux/FluxBuilderTest.java
new file mode 100644
index 0000000..ff67867
--- /dev/null
+++ b/flux/flux-core/src/test/java/org/apache/storm/flux/FluxBuilderTest.java
@@ -0,0 +1,31 @@
+/*
+ * 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.flux;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+public class FluxBuilderTest {
+
+    @Test
+    public void testIsPrimitiveNumber() throws Exception {
+        assertTrue(FluxBuilder.isPrimitiveNumber(int.class));
+        assertFalse(FluxBuilder.isPrimitiveNumber(boolean.class));
+        assertFalse(FluxBuilder.isPrimitiveNumber(String.class));
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/java/org/apache/storm/flux/IntegrationTest.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/java/org/apache/storm/flux/IntegrationTest.java b/flux/flux-core/src/test/java/org/apache/storm/flux/IntegrationTest.java
new file mode 100644
index 0000000..c5807f8
--- /dev/null
+++ b/flux/flux-core/src/test/java/org/apache/storm/flux/IntegrationTest.java
@@ -0,0 +1,39 @@
+/*
+ * 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.flux;
+
+import org.junit.Test;
+
+public class IntegrationTest {
+
+    private static boolean skipTest = true;
+
+    static {
+        String skipStr = System.getProperty("skipIntegration");
+        if(skipStr != null && skipStr.equalsIgnoreCase("false")){
+            skipTest = false;
+        }
+    }
+
+    @Test
+    public void testRunTopologySource() throws Exception {
+        if(!skipTest) {
+            Flux.main(new String[]{"-s", "30000", "src/test/resources/configs/existing-topology.yaml"});
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/java/org/apache/storm/flux/TCKTest.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/java/org/apache/storm/flux/TCKTest.java b/flux/flux-core/src/test/java/org/apache/storm/flux/TCKTest.java
new file mode 100644
index 0000000..c9227f6
--- /dev/null
+++ b/flux/flux-core/src/test/java/org/apache/storm/flux/TCKTest.java
@@ -0,0 +1,254 @@
+/*
+ * 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.flux;
+
+import org.apache.storm.Config;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.flux.model.ExecutionContext;
+import org.apache.storm.flux.model.TopologyDef;
+import org.apache.storm.flux.parser.FluxParser;
+import org.apache.storm.flux.test.TestBolt;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class TCKTest {
+    @Test
+    public void testTCK() throws Exception {
+        TopologyDef topologyDef = FluxParser.parseResource("/configs/tck.yaml", false, true, null, false);
+        Config conf = FluxBuilder.buildConfig(topologyDef);
+        ExecutionContext context = new ExecutionContext(topologyDef, conf);
+        StormTopology topology = FluxBuilder.buildTopology(context);
+        assertNotNull(topology);
+        topology.validate();
+    }
+
+    @Test
+    public void testShellComponents() throws Exception {
+        TopologyDef topologyDef = FluxParser.parseResource("/configs/shell_test.yaml", false, true, null, false);
+        Config conf = FluxBuilder.buildConfig(topologyDef);
+        ExecutionContext context = new ExecutionContext(topologyDef, conf);
+        StormTopology topology = FluxBuilder.buildTopology(context);
+        assertNotNull(topology);
+        topology.validate();
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testBadShellComponents() throws Exception {
+        TopologyDef topologyDef = FluxParser.parseResource("/configs/bad_shell_test.yaml", false, true, null, false);
+        Config conf = FluxBuilder.buildConfig(topologyDef);
+        ExecutionContext context = new ExecutionContext(topologyDef, conf);
+        StormTopology topology = FluxBuilder.buildTopology(context);
+        assertNotNull(topology);
+        topology.validate();
+    }
+
+    @Test
+    public void testKafkaSpoutConfig() throws Exception {
+        TopologyDef topologyDef = FluxParser.parseResource("/configs/kafka_test.yaml", false, true, null, false);
+        Config conf = FluxBuilder.buildConfig(topologyDef);
+        ExecutionContext context = new ExecutionContext(topologyDef, conf);
+        StormTopology topology = FluxBuilder.buildTopology(context);
+        assertNotNull(topology);
+        topology.validate();
+    }
+
+    @Test
+    public void testLoadFromResource() throws Exception {
+        TopologyDef topologyDef = FluxParser.parseResource("/configs/kafka_test.yaml", false, true, null, false);
+        Config conf = FluxBuilder.buildConfig(topologyDef);
+        ExecutionContext context = new ExecutionContext(topologyDef, conf);
+        StormTopology topology = FluxBuilder.buildTopology(context);
+        assertNotNull(topology);
+        topology.validate();
+    }
+
+
+    @Test
+    public void testHdfs() throws Exception {
+        TopologyDef topologyDef = FluxParser.parseResource("/configs/hdfs_test.yaml", false, true, null, false);
+        Config conf = FluxBuilder.buildConfig(topologyDef);
+        ExecutionContext context = new ExecutionContext(topologyDef, conf);
+        StormTopology topology = FluxBuilder.buildTopology(context);
+        assertNotNull(topology);
+        topology.validate();
+    }
+
+    @Test
+    public void testDiamondTopology() throws Exception {
+        TopologyDef topologyDef = FluxParser.parseResource("/configs/diamond-topology.yaml", false, true, null, false);
+        Config conf = FluxBuilder.buildConfig(topologyDef);
+        ExecutionContext context = new ExecutionContext(topologyDef, conf);
+        StormTopology topology = FluxBuilder.buildTopology(context);
+        assertNotNull(topology);
+        topology.validate();
+    }
+
+
+    @Test
+    public void testHbase() throws Exception {
+        TopologyDef topologyDef = FluxParser.parseResource("/configs/simple_hbase.yaml", false, true, null, false);
+        Config conf = FluxBuilder.buildConfig(topologyDef);
+        ExecutionContext context = new ExecutionContext(topologyDef, conf);
+        StormTopology topology = FluxBuilder.buildTopology(context);
+        assertNotNull(topology);
+        topology.validate();
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testBadHbase() throws Exception {
+        TopologyDef topologyDef = FluxParser.parseResource("/configs/bad_hbase.yaml", false, true, null, false);
+        Config conf = FluxBuilder.buildConfig(topologyDef);
+        ExecutionContext context = new ExecutionContext(topologyDef, conf);
+        StormTopology topology = FluxBuilder.buildTopology(context);
+        assertNotNull(topology);
+        topology.validate();
+    }
+
+    @Test
+    public void testIncludes() throws Exception {
+        TopologyDef topologyDef = FluxParser.parseResource("/configs/include_test.yaml", false, true, null, false);
+        Config conf = FluxBuilder.buildConfig(topologyDef);
+        ExecutionContext context = new ExecutionContext(topologyDef, conf);
+        StormTopology topology = FluxBuilder.buildTopology(context);
+        assertNotNull(topology);
+        assertTrue(topologyDef.getName().equals("include-topology"));
+        assertTrue(topologyDef.getBolts().size() > 0);
+        assertTrue(topologyDef.getSpouts().size() > 0);
+        topology.validate();
+    }
+
+    @Test
+    public void testTopologySource() throws Exception {
+        TopologyDef topologyDef = FluxParser.parseResource("/configs/existing-topology.yaml", false, true, null, false);
+        assertTrue(topologyDef.validate());
+        Config conf = FluxBuilder.buildConfig(topologyDef);
+        ExecutionContext context = new ExecutionContext(topologyDef, conf);
+        StormTopology topology = FluxBuilder.buildTopology(context);
+        assertNotNull(topology);
+        topology.validate();
+    }
+
+    @Test
+    public void testTopologySourceWithReflection() throws Exception {
+        TopologyDef topologyDef = FluxParser.parseResource("/configs/existing-topology-reflection.yaml", false, true, null, false);
+        assertTrue(topologyDef.validate());
+        Config conf = FluxBuilder.buildConfig(topologyDef);
+        ExecutionContext context = new ExecutionContext(topologyDef, conf);
+        StormTopology topology = FluxBuilder.buildTopology(context);
+        assertNotNull(topology);
+        topology.validate();
+    }
+
+    @Test
+    public void testTopologySourceWithConfigParam() throws Exception {
+        TopologyDef topologyDef = FluxParser.parseResource("/configs/existing-topology-reflection-config.yaml", false, true, null, false);
+        assertTrue(topologyDef.validate());
+        Config conf = FluxBuilder.buildConfig(topologyDef);
+        ExecutionContext context = new ExecutionContext(topologyDef, conf);
+        StormTopology topology = FluxBuilder.buildTopology(context);
+        assertNotNull(topology);
+        topology.validate();
+    }
+
+    @Test
+    public void testTopologySourceWithMethodName() throws Exception {
+        TopologyDef topologyDef = FluxParser.parseResource("/configs/existing-topology-method-override.yaml", false, true, null, false);
+        assertTrue(topologyDef.validate());
+        Config conf = FluxBuilder.buildConfig(topologyDef);
+        ExecutionContext context = new ExecutionContext(topologyDef, conf);
+        StormTopology topology = FluxBuilder.buildTopology(context);
+        assertNotNull(topology);
+        topology.validate();
+    }
+
+
+    @Test
+    public void testTridentTopologySource() throws Exception {
+        TopologyDef topologyDef = FluxParser.parseResource("/configs/existing-topology-trident.yaml", false, true, null, false);
+        assertTrue(topologyDef.validate());
+        Config conf = FluxBuilder.buildConfig(topologyDef);
+        ExecutionContext context = new ExecutionContext(topologyDef, conf);
+        StormTopology topology = FluxBuilder.buildTopology(context);
+        assertNotNull(topology);
+        topology.validate();
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testInvalidTopologySource() throws Exception {
+        TopologyDef topologyDef = FluxParser.parseResource("/configs/invalid-existing-topology.yaml", false, true, null, false);
+        assertFalse("Topology config is invalid.", topologyDef.validate());
+        Config conf = FluxBuilder.buildConfig(topologyDef);
+        ExecutionContext context = new ExecutionContext(topologyDef, conf);
+        StormTopology topology = FluxBuilder.buildTopology(context);
+    }
+
+
+    @Test
+    public void testTopologySourceWithGetMethodName() throws Exception {
+        TopologyDef topologyDef = FluxParser.parseResource("/configs/existing-topology-reflection.yaml", false, true, null, false);
+        assertTrue(topologyDef.validate());
+        Config conf = FluxBuilder.buildConfig(topologyDef);
+        ExecutionContext context = new ExecutionContext(topologyDef, conf);
+        StormTopology topology = FluxBuilder.buildTopology(context);
+        assertNotNull(topology);
+        topology.validate();
+    }
+
+    @Test
+    public void testTopologySourceWithConfigMethods() throws Exception {
+        TopologyDef topologyDef = FluxParser.parseResource("/configs/config-methods-test.yaml", false, true, null, false);
+        assertTrue(topologyDef.validate());
+        Config conf = FluxBuilder.buildConfig(topologyDef);
+        ExecutionContext context = new ExecutionContext(topologyDef, conf);
+        StormTopology topology = FluxBuilder.buildTopology(context);
+        assertNotNull(topology);
+        topology.validate();
+
+        // make sure the property was actually set
+        TestBolt bolt = (TestBolt)context.getBolt("bolt-1");
+        assertTrue(bolt.getFoo().equals("foo"));
+        assertTrue(bolt.getBar().equals("bar"));
+        assertTrue(bolt.getFooBar().equals("foobar"));
+        assertArrayEquals(new TestBolt.TestClass[] {new TestBolt.TestClass("foo"), new TestBolt.TestClass("bar"), new TestBolt.TestClass("baz")}, bolt.getClasses());
+    }
+
+    @Test
+    public void testVariableSubstitution() throws Exception {
+        TopologyDef topologyDef = FluxParser.parseResource("/configs/substitution-test.yaml", false, true, "src/test/resources/configs/test.properties", true);
+        assertTrue(topologyDef.validate());
+        Config conf = FluxBuilder.buildConfig(topologyDef);
+        ExecutionContext context = new ExecutionContext(topologyDef, conf);
+        StormTopology topology = FluxBuilder.buildTopology(context);
+        assertNotNull(topology);
+        topology.validate();
+
+        // test basic substitution
+        assertEquals("Property not replaced.",
+                "substitution-topology",
+                context.getTopologyDef().getName());
+
+        // test environment variable substitution
+        // $PATH should be defined on most systems
+        String envPath = System.getenv().get("PATH");
+        assertEquals("ENV variable not replaced.",
+                envPath,
+                context.getTopologyDef().getConfig().get("test.env.value"));
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/java/org/apache/storm/flux/multilang/MultilangEnvirontmentTest.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/java/org/apache/storm/flux/multilang/MultilangEnvirontmentTest.java b/flux/flux-core/src/test/java/org/apache/storm/flux/multilang/MultilangEnvirontmentTest.java
new file mode 100644
index 0000000..dcded17
--- /dev/null
+++ b/flux/flux-core/src/test/java/org/apache/storm/flux/multilang/MultilangEnvirontmentTest.java
@@ -0,0 +1,89 @@
+/*
+ * 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.flux.multilang;
+
+
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Sanity checks to make sure we can at least invoke the shells used.
+ */
+public class MultilangEnvirontmentTest {
+    private static final Logger LOG = LoggerFactory.getLogger(MultilangEnvirontmentTest.class);
+
+    @Test
+    public void testInvokePython() throws Exception {
+        String[] command = new String[]{"python", "--version"};
+        int exitVal = invokeCommand(command);
+        assertEquals("Exit value for python is 0.", 0, exitVal);
+    }
+
+    @Test
+    public void testInvokeNode() throws Exception {
+        String[] command = new String[]{"node", "--version"};
+        int exitVal = invokeCommand(command);
+        assertEquals("Exit value for node is 0.", 0, exitVal);
+    }
+
+    private static class StreamRedirect implements Runnable {
+        private InputStream in;
+        private OutputStream out;
+
+        public StreamRedirect(InputStream in, OutputStream out) {
+            this.in = in;
+            this.out = out;
+        }
+
+        @Override
+        public void run() {
+            try {
+                int i = -1;
+                while ((i = this.in.read()) != -1) {
+                    out.write(i);
+                }
+                this.in.close();
+                this.out.close();
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        }
+    }
+
+    private int invokeCommand(String[] args) throws Exception {
+        LOG.debug("Invoking command: {}", args);
+
+        ProcessBuilder pb = new ProcessBuilder(args);
+        pb.redirectErrorStream(true);
+        final Process proc = pb.start();
+
+        ByteArrayOutputStream out = new ByteArrayOutputStream();
+        Thread t = new Thread(new StreamRedirect(proc.getInputStream(), out));
+        t.start();
+        int exitVal = proc.waitFor();
+        LOG.debug("Command result: {}", out.toString());
+        return exitVal;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopology.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopology.java b/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopology.java
new file mode 100644
index 0000000..ff65a8a
--- /dev/null
+++ b/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopology.java
@@ -0,0 +1,58 @@
+/**
+ * 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.flux.test;
+
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.topology.TopologyBuilder;
+import org.apache.storm.flux.wrappers.bolts.LogInfoBolt;
+import org.apache.storm.flux.wrappers.spouts.FluxShellSpout;
+
+import java.util.Map;
+
+/**
+ * Test topology source that does not implement TopologySource, but has the same
+ * `getTopology()` method.
+ */
+public class SimpleTopology{
+
+
+    public SimpleTopology(){}
+
+    public SimpleTopology(String foo, String bar){}
+
+    public StormTopology getTopologyWithDifferentMethodName(Map<String, Object> config){
+        return getTopology(config);
+    }
+
+
+    public StormTopology getTopology(Map<String, Object> config) {
+        TopologyBuilder builder = new TopologyBuilder();
+
+        // spouts
+        FluxShellSpout spout = new FluxShellSpout(
+                new String[]{"node", "randomsentence.js"},
+                new String[]{"word"});
+        builder.setSpout("sentence-spout", spout, 1);
+
+        // bolts
+        builder.setBolt("log-bolt", new LogInfoBolt(), 1)
+                .shuffleGrouping("sentence-spout");
+
+        return builder.createTopology();
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologySource.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologySource.java b/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologySource.java
new file mode 100644
index 0000000..2fadacf
--- /dev/null
+++ b/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologySource.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.flux.test;
+
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.topology.TopologyBuilder;
+import org.apache.storm.flux.api.TopologySource;
+import org.apache.storm.flux.wrappers.bolts.LogInfoBolt;
+import org.apache.storm.flux.wrappers.spouts.FluxShellSpout;
+
+import java.util.Map;
+
+public class SimpleTopologySource implements TopologySource {
+
+
+    public SimpleTopologySource(){}
+
+    public SimpleTopologySource(String foo, String bar){}
+
+
+    @Override
+    public StormTopology getTopology(Map<String, Object> config) {
+        TopologyBuilder builder = new TopologyBuilder();
+
+        // spouts
+        FluxShellSpout spout = new FluxShellSpout(
+                new String[]{"node", "randomsentence.js"},
+                new String[]{"word"});
+        builder.setSpout("sentence-spout", spout, 1);
+
+        // bolts
+        builder.setBolt("log-bolt", new LogInfoBolt(), 1)
+                .shuffleGrouping("sentence-spout");
+
+        return builder.createTopology();
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologyWithConfigParam.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologyWithConfigParam.java b/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologyWithConfigParam.java
new file mode 100644
index 0000000..78195b5
--- /dev/null
+++ b/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologyWithConfigParam.java
@@ -0,0 +1,53 @@
+/**
+ * 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.flux.test;
+
+import org.apache.storm.Config;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.topology.TopologyBuilder;
+import org.apache.storm.flux.wrappers.bolts.LogInfoBolt;
+import org.apache.storm.flux.wrappers.spouts.FluxShellSpout;
+
+/**
+ * Test topology source that does not implement TopologySource, but has the same
+ * `getTopology()` method.
+ */
+public class SimpleTopologyWithConfigParam {
+
+
+    public SimpleTopologyWithConfigParam(){}
+
+    public SimpleTopologyWithConfigParam(String foo, String bar){}
+
+
+    public StormTopology getTopology(Config config) {
+        TopologyBuilder builder = new TopologyBuilder();
+
+        // spouts
+        FluxShellSpout spout = new FluxShellSpout(
+                new String[]{"node", "randomsentence.js"},
+                new String[]{"word"});
+        builder.setSpout("sentence-spout", spout, 1);
+
+        // bolts
+        builder.setBolt("log-bolt", new LogInfoBolt(), 1)
+                .shuffleGrouping("sentence-spout");
+
+        return builder.createTopology();
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/java/org/apache/storm/flux/test/TestBolt.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/java/org/apache/storm/flux/test/TestBolt.java b/flux/flux-core/src/test/java/org/apache/storm/flux/test/TestBolt.java
new file mode 100644
index 0000000..28d11b6
--- /dev/null
+++ b/flux/flux-core/src/test/java/org/apache/storm/flux/test/TestBolt.java
@@ -0,0 +1,131 @@
+/**
+ * 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.flux.test;
+
+import org.apache.storm.topology.BasicOutputCollector;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.base.BaseBasicBolt;
+import org.apache.storm.tuple.Tuple;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+
+
+public class TestBolt extends BaseBasicBolt {
+    private static final Logger LOG = LoggerFactory.getLogger(TestBolt.class);
+
+    private String foo;
+    private String bar;
+    private String fooBar;
+    private String none;
+    private TestClass[] classes;
+
+    public static class TestClass implements Serializable {
+        private String field;
+
+        public TestClass(String field) {
+            this.field = field;
+        }
+
+        public String getField() {
+            return field;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) return true;
+            if (!(o instanceof TestClass)) return false;
+
+            TestClass testClass = (TestClass) o;
+
+            return getField() != null ? getField().equals(testClass.getField()) : testClass.getField() == null;
+        }
+
+        @Override
+        public int hashCode() {
+            return getField() != null ? getField().hashCode() : 0;
+        }
+    }
+
+
+    public static enum TestEnum {
+        FOO,
+        BAR
+    }
+
+    public TestBolt(TestEnum te){
+
+    }
+
+    public TestBolt(TestEnum te, float f){
+
+    }
+
+    public TestBolt(TestEnum te, float f, boolean b){
+
+    }
+
+    public TestBolt(TestEnum te, float f, boolean b, TestClass... str) {
+
+    }
+
+    @Override
+    public void execute(Tuple tuple, BasicOutputCollector basicOutputCollector) {
+        LOG.info("{}", tuple);
+    }
+
+    @Override
+    public void declareOutputFields(OutputFieldsDeclarer outputFieldsDeclarer) {
+
+    }
+
+    // config methods
+    public void withFoo(String foo){
+        this.foo = foo;
+    }
+    public void withNone(){
+        this.none = "hit";
+    }
+    public void withBar(String bar){
+        this.bar = bar;
+    }
+
+    public void withFooBar(String foo, String bar){
+        this.fooBar = foo + bar;
+    }
+
+    public void withClasses(TestClass...classes) {
+        this.classes = classes;
+    }
+
+    public String getFoo(){
+        return this.foo;
+    }
+    public String getBar(){
+        return this.bar;
+    }
+
+    public String getFooBar(){
+        return this.fooBar;
+    }
+
+    public TestClass[] getClasses() {
+        return classes;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/java/org/apache/storm/flux/test/TridentTopologySource.java
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/java/org/apache/storm/flux/test/TridentTopologySource.java b/flux/flux-core/src/test/java/org/apache/storm/flux/test/TridentTopologySource.java
new file mode 100644
index 0000000..36b272b
--- /dev/null
+++ b/flux/flux-core/src/test/java/org/apache/storm/flux/test/TridentTopologySource.java
@@ -0,0 +1,71 @@
+/**
+ * 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.flux.test;
+
+import org.apache.storm.Config;
+import org.apache.storm.generated.StormTopology;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.kafka.StringScheme;
+import org.apache.storm.trident.TridentTopology;
+import org.apache.storm.trident.operation.BaseFunction;
+import org.apache.storm.trident.operation.TridentCollector;
+import org.apache.storm.trident.operation.builtin.Count;
+import org.apache.storm.trident.testing.FixedBatchSpout;
+import org.apache.storm.trident.testing.MemoryMapState;
+import org.apache.storm.trident.tuple.TridentTuple;
+
+/**
+ * Basic Trident example that will return a `StormTopology` from a `getTopology()` method.
+ */
+public class TridentTopologySource {
+
+    private FixedBatchSpout spout;
+
+    public StormTopology getTopology(Config config) {
+
+        this.spout = new FixedBatchSpout(new Fields("sentence"), 20,
+                new Values("one two"),
+                new Values("two three"),
+                new Values("three four"),
+                new Values("four five"),
+                new Values("five six")
+        );
+
+
+        TridentTopology trident = new TridentTopology();
+
+        trident.newStream("wordcount", spout).name("sentence").parallelismHint(1).shuffle()
+                .each(new Fields("sentence"), new Split(), new Fields("word"))
+                .parallelismHint(1)
+                .groupBy(new Fields("word"))
+                .persistentAggregate(new MemoryMapState.Factory(), new Count(), new Fields("count"))
+                .parallelismHint(1);
+        return trident.build();
+    }
+
+    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));
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/bad_hbase.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/bad_hbase.yaml b/flux/flux-core/src/test/resources/configs/bad_hbase.yaml
new file mode 100644
index 0000000..a29e314
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/bad_hbase.yaml
@@ -0,0 +1,98 @@
+# 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.
+
+# Test ability to wire together shell spouts/bolts
+---
+
+# topology definition
+# name to be used when submitting
+name: "hbase-wordcount"
+
+# Components
+# Components are analagous to Spring beans. They are meant to be used as constructor,
+# property(setter), and builder arguments.
+#
+# for the time being, components must be declared in the order they are referenced
+
+components:
+  - id: "columnFields"
+    className: "org.apache.storm.tuple.Fields"
+    constructorArgs:
+      - ["word"]
+
+  - id: "counterFields"
+    className: "org.apache.storm.tuple.Fields"
+    constructorArgs:
+      # !!! the following won't work, and should thow an IllegalArgumentException...
+      - "count"
+
+  - id: "mapper"
+    className: "org.apache.storm.hbase.bolt.mapper.SimpleHBaseMapper"
+    configMethods:
+      - name: "withRowKeyField"
+        args: ["word"]
+      - name: "withColumnFields"
+        args: [ref: "columnFields"]
+      - name: "withCounterFields"
+        args: [ref: "counterFields"]
+      - name: "withColumnFamily"
+        args: ["cf"]
+
+# topology configuration
+# this will be passed to the submitter as a map of config options
+#
+config:
+  topology.workers: 1
+  hbase.conf:
+    hbase.rootdir: "hdfs://hadoop:54310/hbase"
+    hbase.zookeeper.quorum: "hadoop"
+
+# spout definitions
+spouts:
+  - id: "word-spout"
+    className: "org.apache.storm.testing.TestWordSpout"
+    parallelism: 1
+
+# bolt definitions
+
+bolts:
+  - id: "count-bolt"
+    className: "org.apache.storm.testing.TestWordCounter"
+
+  - id: "hbase-bolt"
+    className: "org.apache.storm.hbase.bolt.HBaseBolt"
+    constructorArgs:
+      - "WordCount" # HBase table name
+      - ref: "mapper"
+    configMethods:
+      - name: "withConfigKey"
+        args: ["hbase.conf"]
+    parallelism: 1
+
+
+streams:
+  - name: "" # name isn't used (placeholder for logging, UI, etc.)
+    from: "word-spout"
+    to: "count-bolt"
+    grouping:
+      type: SHUFFLE
+
+  - name: "" # name isn't used (placeholder for logging, UI, etc.)
+    from: "count-bolt"
+    to: "hbase-bolt"
+    grouping:
+      type: FIELDS
+      args: ["word"]

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/bad_shell_test.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/bad_shell_test.yaml b/flux/flux-core/src/test/resources/configs/bad_shell_test.yaml
new file mode 100644
index 0000000..0892ce7
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/bad_shell_test.yaml
@@ -0,0 +1,118 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# Test ability to wire together shell spouts/bolts
+---
+
+# topology definition
+# name to be used when submitting
+name: "shell-topology"
+
+# Components
+# Components are analagous to Spring beans. They are meant to be used as constructor,
+# property(setter), and builder arguments.
+#components:
+#  - id: "myComponent"
+#    className: "com.foo.bar.MyComponent"
+#    constructorArgs:
+#      - ...
+#    properties:
+#      foo: "bar"
+#      bar: "foo"
+
+# NOTE: We may want to consider some level of spring integration. For example, allowing component references
+# to a spring `ApplicationContext`.
+
+# topology configuration
+# this will be passed to the submitter as a map of config options
+#
+config:
+  topology.workers: 1
+  # ...
+
+# spout definitions
+spouts:
+  - id: "sentence-spout"
+    className: "org.apache.storm.flux.wrappers.spouts.FluxShellSpout"
+    # shell spout constructor takes 2 arguments: String[], String[]
+    constructorArgs:
+      # command line
+      - ["node", "randomsentence.js"]
+      # output fields
+      - ["word"]
+    configMethods:
+      - name: "addComponentConfig"
+        args: ["rabbitmq.configfile", "etc/rabbit.yml", "hello"]
+      - name: "addComponentConfig"
+        args:
+        - "publisher.data_paths"
+        - ["actions", "hello"]
+    parallelism: 1
+    # ...
+
+# bolt definitions
+bolts:
+  - id: "splitsentence"
+    className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
+    constructorArgs:
+      # command line
+      - ["python", "splitsentence.py"]
+      # output fields
+      - ["word"]
+    configMethods:
+      - name: "addComponentConfig"
+        args: ["rabbitmq.configfile", "etc/rabbit.yml", "hello"]
+      - name: "addComponentConfig"
+        args:
+        - "publisher.data_paths"
+        - ["actions", "hello"]
+    parallelism: 1
+    # ...
+
+  - id: "log"
+    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+    parallelism: 1
+    # ...
+
+  - id: "count"
+    className: "org.apache.storm.testing.TestWordCounter"
+    parallelism: 1
+    # ...
+
+#stream definitions
+# stream definitions define connections between spouts and bolts.
+# note that such connections can be cyclical
+# custom stream groupings are also supported
+
+streams:
+  - name: "spout --> split" # name isn't used (placeholder for logging, UI, etc.)
+    from: "sentence-spout"
+    to: "splitsentence"
+    grouping:
+      type: SHUFFLE
+
+  - name: "split --> count"
+    from: "splitsentence"
+    to: "count"
+    grouping:
+      type: FIELDS
+      args: ["word"]
+
+  - name: "count --> log"
+    from: "count"
+    to: "log"
+    grouping:
+      type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/config-methods-test.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/config-methods-test.yaml b/flux/flux-core/src/test/resources/configs/config-methods-test.yaml
new file mode 100644
index 0000000..7c4ffb3
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/config-methods-test.yaml
@@ -0,0 +1,92 @@
+# 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.
+---
+name: "yaml-topology"
+
+#
+config:
+  topology.workers: 1
+  # ...
+
+components:
+  - id: "foo"
+    className: "org.apache.storm.flux.test.TestBolt$TestClass"
+    constructorArgs:
+      - "foo"
+  - id: "bar"
+    className: "org.apache.storm.flux.test.TestBolt$TestClass"
+    constructorArgs:
+      - "bar"
+  - id: "baz"
+    className: "org.apache.storm.flux.test.TestBolt$TestClass"
+    constructorArgs:
+      - "baz"
+
+# spout definitions
+spouts:
+  - id: "spout-1"
+    className: "org.apache.storm.testing.TestWordSpout"
+    parallelism: 1
+    # ...
+
+# bolt definitions
+bolts:
+  - id: "bolt-1"
+    className: "org.apache.storm.flux.test.TestBolt"
+    parallelism: 1
+    constructorArgs:
+      - FOO # enum class
+      - 1.0
+      - true
+      - reflist: ["foo", "bar"]
+    configMethods:
+      - name: "withFoo"
+        args:
+          - "foo"
+      - name: "withNone"
+      - name: "withBar"
+        args:
+          - "bar"
+      - name: "withFooBar"
+        args:
+          - "foo"
+          - "bar"
+      - name: "withClasses"
+        args:
+          - reflist:
+            - "foo"
+            - "bar"
+            - "baz"
+
+
+#stream definitions
+# stream definitions define connections between spouts and bolts.
+# note that such connections can be cyclical
+streams:
+  - name: "spout-1 --> bolt-1" # name isn't used (placeholder for logging, UI, etc.)
+#    id: "connection-1"
+    from: "spout-1"
+    to: "bolt-1"
+    grouping:
+      type: SHUFFLE
+
+
+
+
+
+
+
+

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/diamond-topology.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/diamond-topology.yaml b/flux/flux-core/src/test/resources/configs/diamond-topology.yaml
new file mode 100644
index 0000000..957c258
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/diamond-topology.yaml
@@ -0,0 +1,87 @@
+# 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.
+
+---
+
+# topology definition
+# name to be used when submitting
+name: "diamond-topology"
+
+# topology configuration
+# this will be passed to the submitter as a map of config options
+#
+config:
+  topology.workers: 1
+
+# spout definitions
+spouts:
+  - id: "spout-1"
+    className: "org.apache.storm.testing.TestWordSpout"
+    parallelism: 1
+
+# bolt definitions
+bolts:
+  - id: "A"
+    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+    parallelism: 1
+
+  - id: "B"
+    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+    parallelism: 1
+
+  - id: "C"
+    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+    parallelism: 1
+
+  - id: "D"
+    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
+    parallelism: 1
+
+#stream definitions
+# stream definitions define connections between spouts and bolts.
+# note that such connections can be cyclical
+streams:
+  - name: "spout-1 --> bolt-1" # name isn't used (placeholder for logging, UI, etc.)
+#    id: "connection-1"
+    from: "spout-1"
+    to: "A"
+    grouping:
+      type: FIELDS
+      args: ["word"]
+
+  - from: "A"
+    to: "B"
+    grouping:
+      type: SHUFFLE
+
+  - from: "A"
+    to: "C"
+    grouping:
+      type: SHUFFLE
+
+  - from: "C"
+    to: "D"
+    grouping:
+      type: SHUFFLE
+
+  - from: "B"
+    to: "D"
+    grouping:
+      type: SHUFFLE
+
+
+
+

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/existing-topology-method-override.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/existing-topology-method-override.yaml b/flux/flux-core/src/test/resources/configs/existing-topology-method-override.yaml
new file mode 100644
index 0000000..fceeeed
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/existing-topology-method-override.yaml
@@ -0,0 +1,25 @@
+# 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.
+---
+
+# configuration that uses an existing topology that does not implement TopologySource
+name: "existing-topology"
+topologySource:
+  className: "org.apache.storm.flux.test.SimpleTopology"
+  methodName: "getTopologyWithDifferentMethodName"
+  constructorArgs:
+    - "foo"
+    - "bar"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/flux/flux-core/src/test/resources/configs/existing-topology-reflection-config.yaml
----------------------------------------------------------------------
diff --git a/flux/flux-core/src/test/resources/configs/existing-topology-reflection-config.yaml b/flux/flux-core/src/test/resources/configs/existing-topology-reflection-config.yaml
new file mode 100644
index 0000000..440fe4d
--- /dev/null
+++ b/flux/flux-core/src/test/resources/configs/existing-topology-reflection-config.yaml
@@ -0,0 +1,24 @@
+# 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.
+---
+
+# configuration that uses an existing topology that does not implement TopologySource
+name: "existing-topology"
+topologySource:
+  className: "org.apache.storm.flux.test.SimpleTopologyWithConfigParam"
+  constructorArgs:
+    - "foo"
+    - "bar"
\ No newline at end of file


[15/23] storm git commit: STORM-2453 Move non-connectors into the top directory

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentStormRuleSets.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentStormRuleSets.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentStormRuleSets.java
deleted file mode 100644
index e146069..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentStormRuleSets.java
+++ /dev/null
@@ -1,110 +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.sql.planner.trident;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.rel.rules.CalcMergeRule;
-import org.apache.calcite.rel.rules.FilterCalcMergeRule;
-import org.apache.calcite.rel.rules.FilterProjectTransposeRule;
-import org.apache.calcite.rel.rules.FilterToCalcRule;
-import org.apache.calcite.rel.rules.ProjectCalcMergeRule;
-import org.apache.calcite.rel.rules.ProjectFilterTransposeRule;
-import org.apache.calcite.rel.rules.ProjectRemoveRule;
-import org.apache.calcite.rel.rules.ProjectToCalcRule;
-import org.apache.calcite.rel.rules.PruneEmptyRules;
-import org.apache.calcite.rel.rules.ReduceExpressionsRule;
-import org.apache.calcite.rel.rules.SortRemoveRule;
-import org.apache.calcite.rel.rules.UnionEliminatorRule;
-import org.apache.calcite.rel.stream.StreamRules;
-import org.apache.calcite.tools.RuleSet;
-import org.apache.storm.sql.planner.trident.rules.TridentCalcRule;
-import org.apache.storm.sql.planner.trident.rules.TridentFilterRule;
-import org.apache.storm.sql.planner.trident.rules.TridentScanRule;
-import org.apache.storm.sql.planner.trident.rules.TridentAggregateRule;
-import org.apache.storm.sql.planner.trident.rules.TridentJoinRule;
-import org.apache.storm.sql.planner.trident.rules.TridentModifyRule;
-import org.apache.storm.sql.planner.trident.rules.TridentProjectRule;
-
-import java.util.Iterator;
-
-public class TridentStormRuleSets {
-    private static final ImmutableSet<RelOptRule> calciteToStormConversionRules =
-            ImmutableSet.<RelOptRule>builder().add(
-                    SortRemoveRule.INSTANCE,
-
-                    FilterToCalcRule.INSTANCE,
-                    ProjectToCalcRule.INSTANCE,
-                    FilterCalcMergeRule.INSTANCE,
-                    ProjectCalcMergeRule.INSTANCE,
-                    CalcMergeRule.INSTANCE,
-
-                    PruneEmptyRules.FILTER_INSTANCE,
-                    PruneEmptyRules.PROJECT_INSTANCE,
-                    PruneEmptyRules.UNION_INSTANCE,
-
-                    ProjectFilterTransposeRule.INSTANCE,
-                    FilterProjectTransposeRule.INSTANCE,
-                    ProjectRemoveRule.INSTANCE,
-
-                    ReduceExpressionsRule.FILTER_INSTANCE,
-                    ReduceExpressionsRule.PROJECT_INSTANCE,
-                    ReduceExpressionsRule.CALC_INSTANCE,
-
-                    // merge and push unions rules
-                    UnionEliminatorRule.INSTANCE,
-
-                    TridentScanRule.INSTANCE,
-                    TridentFilterRule.INSTANCE,
-                    TridentProjectRule.INSTANCE,
-                    TridentAggregateRule.INSTANCE,
-                    TridentJoinRule.INSTANCE,
-                    TridentModifyRule.INSTANCE,
-                    TridentCalcRule.INSTANCE
-            ).build();
-
-    public static RuleSet[] getRuleSets() {
-        return new RuleSet[]{
-                new StormRuleSet(StreamRules.RULES),
-                new StormRuleSet(ImmutableSet.<RelOptRule>builder().addAll(StreamRules.RULES).addAll(calciteToStormConversionRules).build())
-        };
-    }
-
-    private static class StormRuleSet implements RuleSet {
-        final ImmutableSet<RelOptRule> rules;
-
-        public StormRuleSet(ImmutableSet<RelOptRule> rules) {
-            this.rules = rules;
-        }
-
-        public StormRuleSet(ImmutableList<RelOptRule> rules) {
-            this.rules = ImmutableSet.<RelOptRule>builder()
-                    .addAll(rules)
-                    .build();
-        }
-
-        @Override
-        public Iterator<RelOptRule> iterator() {
-            return rules.iterator();
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentCalcRel.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentCalcRel.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentCalcRel.java
deleted file mode 100644
index 482e841..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentCalcRel.java
+++ /dev/null
@@ -1,97 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.planner.trident.rel;
-
-import com.google.common.collect.Lists;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Calc;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexLocalRef;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexProgram;
-import org.apache.storm.sql.planner.StormRelUtils;
-import org.apache.storm.sql.planner.rel.StormCalcRelBase;
-import org.apache.storm.sql.planner.trident.TridentPlanCreator;
-import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
-import org.apache.storm.sql.runtime.trident.functions.EvaluationCalc;
-import org.apache.storm.trident.Stream;
-import org.apache.storm.tuple.Fields;
-
-import java.util.ArrayList;
-import java.util.List;
-
-public class TridentCalcRel extends StormCalcRelBase implements TridentRel {
-    public TridentCalcRel(RelOptCluster cluster, RelTraitSet traits, RelNode child, RexProgram program) {
-        super(cluster, traits, child, program);
-    }
-
-    @Override
-    public Calc copy(RelTraitSet traitSet, RelNode child, RexProgram program) {
-        return new TridentCalcRel(getCluster(), traitSet, child, program);
-    }
-
-    @Override
-    public void tridentPlan(TridentPlanCreator planCreator) throws Exception {
-        // SingleRel
-        RelNode input = getInput();
-        StormRelUtils.getStormRelInput(input).tridentPlan(planCreator);
-        Stream inputStream = planCreator.pop().toStream();
-
-        String stageName = StormRelUtils.getStageName(this);
-
-        RelDataType inputRowType = getInput(0).getRowType();
-
-        List<String> outputFieldNames = getRowType().getFieldNames();
-        int outputCount = outputFieldNames.size();
-
-        // filter
-        ExecutableExpression filterInstance = null;
-        RexLocalRef condition = program.getCondition();
-        if (condition != null) {
-            RexNode conditionNode = program.expandLocalRef(condition);
-            filterInstance = planCreator.createScalarInstance(Lists.newArrayList(conditionNode), inputRowType,
-                    StormRelUtils.getClassName(this));
-        }
-
-        // projection
-        ExecutableExpression projectionInstance = null;
-        List<RexLocalRef> projectList = program.getProjectList();
-        if (projectList != null && !projectList.isEmpty()) {
-            List<RexNode> expandedNodes = new ArrayList<>();
-            for (RexLocalRef project : projectList) {
-                expandedNodes.add(program.expandLocalRef(project));
-            }
-
-            projectionInstance = planCreator.createScalarInstance(expandedNodes, inputRowType,
-                    StormRelUtils.getClassName(this));
-        }
-
-        if (projectionInstance == null && filterInstance == null) {
-            // it shouldn't be happen
-            throw new IllegalStateException("Either projection or condition, or both should be provided.");
-        }
-
-        final Stream finalStream = inputStream
-                .flatMap(new EvaluationCalc(filterInstance, projectionInstance, outputCount, planCreator.getDataContext()), new Fields(outputFieldNames))
-                .name(stageName);
-
-        planCreator.addStream(finalStream);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentFilterRel.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentFilterRel.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentFilterRel.java
deleted file mode 100644
index 1fe0927..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentFilterRel.java
+++ /dev/null
@@ -1,65 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.planner.trident.rel;
-
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Filter;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexNode;
-import org.apache.storm.sql.planner.StormRelUtils;
-import org.apache.storm.sql.planner.rel.StormFilterRelBase;
-import org.apache.storm.sql.planner.trident.TridentPlanCreator;
-import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
-import org.apache.storm.sql.runtime.trident.functions.EvaluationFilter;
-import org.apache.storm.trident.Stream;
-import org.apache.storm.trident.fluent.IAggregatableStream;
-
-import java.util.List;
-
-public class TridentFilterRel extends StormFilterRelBase implements TridentRel {
-    public TridentFilterRel(RelOptCluster cluster, RelTraitSet traits, RelNode child, RexNode condition) {
-        super(cluster, traits, child, condition);
-    }
-
-    @Override
-    public Filter copy(RelTraitSet traitSet, RelNode input, RexNode condition) {
-        return new TridentFilterRel(getCluster(), traitSet, input, condition);
-    }
-
-    @Override
-    public void tridentPlan(TridentPlanCreator planCreator) throws Exception {
-        // SingleRel
-        RelNode input = getInput();
-        StormRelUtils.getStormRelInput(input).tridentPlan(planCreator);
-        Stream inputStream = planCreator.pop().toStream();
-
-        String stageName = StormRelUtils.getStageName(this);
-
-        List<RexNode> childExps = getChildExps();
-        RelDataType inputRowType = getInput(0).getRowType();
-
-        String filterClassName = StormRelUtils.getClassName(this);
-        ExecutableExpression filterInstance = planCreator.createScalarInstance(childExps, inputRowType, filterClassName);
-
-        IAggregatableStream finalStream = inputStream.filter(new EvaluationFilter(filterInstance, planCreator.getDataContext()))
-                .name(stageName);
-        planCreator.addStream(finalStream);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentLogicalConvention.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentLogicalConvention.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentLogicalConvention.java
deleted file mode 100644
index d221498..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentLogicalConvention.java
+++ /dev/null
@@ -1,67 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.planner.trident.rel;
-
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.plan.ConventionTraitDef;
-import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelTrait;
-import org.apache.calcite.plan.RelTraitDef;
-import org.apache.calcite.plan.RelTraitSet;
-
-public enum TridentLogicalConvention implements Convention {
-  INSTANCE;
-
-  @Override
-  public Class getInterface() {
-    return TridentRel.class;
-  }
-
-  @Override
-  public String getName() {
-    return "STORM_LOGICAL";
-  }
-
-  @Override
-  public RelTraitDef getTraitDef() {
-    return ConventionTraitDef.INSTANCE;
-  }
-
-  @Override
-  public boolean satisfies(RelTrait trait) {
-    return this == trait;
-  }
-
-  @Override
-  public void register(RelOptPlanner planner) {}
-
-  @Override
-  public String toString() {
-    return getName();
-  }
-
-  @Override
-  public boolean canConvertConvention(Convention toConvention) {
-    return false;
-  }
-
-  @Override
-  public boolean useAbstractConvertersForConversion(RelTraitSet fromTraits, RelTraitSet toTraits) {
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentProjectRel.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentProjectRel.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentProjectRel.java
deleted file mode 100644
index 06be5d7..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentProjectRel.java
+++ /dev/null
@@ -1,69 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.planner.trident.rel;
-
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexNode;
-import org.apache.storm.sql.planner.StormRelUtils;
-import org.apache.storm.sql.planner.rel.StormProjectRelBase;
-import org.apache.storm.sql.planner.trident.TridentPlanCreator;
-import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
-import org.apache.storm.sql.runtime.trident.functions.EvaluationFunction;
-import org.apache.storm.trident.Stream;
-import org.apache.storm.tuple.Fields;
-
-import java.util.List;
-
-public class TridentProjectRel extends StormProjectRelBase implements TridentRel {
-    public TridentProjectRel(RelOptCluster cluster, RelTraitSet traits, RelNode input, List<? extends RexNode> projects, RelDataType rowType) {
-        super(cluster, traits, input, projects, rowType);
-    }
-
-    @Override
-    public Project copy(RelTraitSet traitSet, RelNode input, List<RexNode> projects, RelDataType rowType) {
-        return new TridentProjectRel(getCluster(), traitSet, input, projects, rowType);
-    }
-
-    @Override
-    public void tridentPlan(TridentPlanCreator planCreator) throws Exception {
-        // SingleRel
-        RelNode input = getInput();
-        StormRelUtils.getStormRelInput(input).tridentPlan(planCreator);
-        Stream inputStream = planCreator.pop().toStream();
-
-        String stageName = StormRelUtils.getStageName(this);
-        String projectionClassName = StormRelUtils.getClassName(this);
-
-        List<String> outputFieldNames = getRowType().getFieldNames();
-        int outputCount = outputFieldNames.size();
-
-        List<RexNode> childExps = getChildExps();
-        RelDataType inputRowType = getInput(0).getRowType();
-
-        ExecutableExpression projectionInstance = planCreator.createScalarInstance(childExps, inputRowType, projectionClassName);
-        Stream finalStream = inputStream
-                .map(new EvaluationFunction(projectionInstance, outputCount, planCreator.getDataContext()), new Fields(outputFieldNames))
-                .name(stageName);
-
-        planCreator.addStream(finalStream);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentRel.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentRel.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentRel.java
deleted file mode 100644
index fa92ec9..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentRel.java
+++ /dev/null
@@ -1,26 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.planner.trident.rel;
-
-import org.apache.calcite.plan.Convention;
-import org.apache.storm.sql.planner.rel.StormRelNode;
-import org.apache.storm.sql.planner.trident.TridentPlanCreator;
-
-public interface TridentRel extends StormRelNode {
-    void tridentPlan(TridentPlanCreator planCreator) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamInsertRel.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamInsertRel.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamInsertRel.java
deleted file mode 100644
index e92c29b..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamInsertRel.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.planner.trident.rel;
-
-import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.prepare.Prepare;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rex.RexNode;
-import org.apache.storm.sql.planner.StormRelUtils;
-import org.apache.storm.sql.planner.rel.StormStreamInsertRelBase;
-import org.apache.storm.sql.planner.trident.TridentPlanCreator;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.trident.Stream;
-import org.apache.storm.trident.fluent.IAggregatableStream;
-import org.apache.storm.tuple.Fields;
-
-import java.util.List;
-
-public class TridentStreamInsertRel extends StormStreamInsertRelBase implements TridentRel {
-    public TridentStreamInsertRel(RelOptCluster cluster, RelTraitSet traits, RelOptTable table, Prepare.CatalogReader catalogReader, RelNode child, Operation operation, List<String> updateColumnList, List<RexNode> sourceExpressionList, boolean flattened) {
-        super(cluster, traits, table, catalogReader, child, operation, updateColumnList, sourceExpressionList, flattened);
-    }
-
-    @Override
-    public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
-        return new TridentStreamInsertRel(getCluster(), traitSet, getTable(), getCatalogReader(),
-                sole(inputs), getOperation(), getUpdateColumnList(), getSourceExpressionList(), isFlattened());
-    }
-
-    @Override
-    public void tridentPlan(TridentPlanCreator planCreator) throws Exception {
-        // SingleRel
-        RelNode input = getInput();
-        StormRelUtils.getStormRelInput(input).tridentPlan(planCreator);
-        Stream inputStream = planCreator.pop().toStream();
-
-        String stageName = StormRelUtils.getStageName(this);
-
-        Preconditions.checkArgument(isInsert(), "Only INSERT statement is supported.");
-
-        List<String> inputFields = this.input.getRowType().getFieldNames();
-        List<String> outputFields = getRowType().getFieldNames();
-
-        // FIXME: this should be really different...
-        String tableName = Joiner.on('.').join(getTable().getQualifiedName());
-        ISqlTridentDataSource.SqlTridentConsumer consumer = planCreator.getSources().get(tableName).getConsumer();
-
-        // In fact this is normally the end of stream, but I'm still not sure so I open new streams based on State values
-        IAggregatableStream finalStream = inputStream
-                .partitionPersist(consumer.getStateFactory(), new Fields(inputFields), consumer.getStateUpdater(),
-                        new Fields(outputFields))
-                .newValuesStream().name(stageName);
-
-        planCreator.addStream(finalStream);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamScanRel.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamScanRel.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamScanRel.java
deleted file mode 100644
index c563d73..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamScanRel.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.planner.trident.rel;
-
-import com.google.common.base.Joiner;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.storm.sql.planner.StormRelUtils;
-import org.apache.storm.sql.planner.rel.StormStreamScanRelBase;
-import org.apache.storm.sql.planner.trident.TridentPlanCreator;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.trident.fluent.IAggregatableStream;
-
-import java.util.Map;
-
-public class TridentStreamScanRel extends StormStreamScanRelBase implements TridentRel {
-    private final int parallelismHint;
-
-    public TridentStreamScanRel(RelOptCluster cluster, RelTraitSet traitSet, RelOptTable table, int parallelismHint) {
-        super(cluster, traitSet, table);
-        this.parallelismHint = parallelismHint;
-    }
-
-    @Override
-    public void tridentPlan(TridentPlanCreator planCreator) throws Exception {
-        String sourceName = Joiner.on('.').join(getTable().getQualifiedName());
-
-        // FIXME: this should be really different...
-        Map<String, ISqlTridentDataSource> sources = planCreator.getSources();
-        if (!sources.containsKey(sourceName)) {
-            throw new RuntimeException("Cannot find table " + sourceName);
-        }
-
-        String stageName = StormRelUtils.getStageName(this);
-        IAggregatableStream finalStream = planCreator.getTopology().newStream(stageName, sources.get(sourceName).getProducer())
-                .parallelismHint(parallelismHint);
-        planCreator.addStream(finalStream);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentAggregateRule.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentAggregateRule.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentAggregateRule.java
deleted file mode 100644
index ac35414..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentAggregateRule.java
+++ /dev/null
@@ -1,39 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.planner.trident.rules;
-
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.logical.LogicalAggregate;
-import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
-
-public class TridentAggregateRule extends ConverterRule {
-    public static final RelOptRule INSTANCE = new TridentAggregateRule();
-
-    private TridentAggregateRule() {
-        super(LogicalAggregate.class, Convention.NONE, TridentLogicalConvention.INSTANCE, "TridentAggregateRule");
-    }
-
-    @Override
-    public RelNode convert(RelNode rel) {
-        throw new UnsupportedOperationException("Aggregate operation is not supported.");
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentCalcRule.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentCalcRule.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentCalcRule.java
deleted file mode 100644
index 25126ec..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentCalcRule.java
+++ /dev/null
@@ -1,45 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.planner.trident.rules;
-
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Calc;
-import org.apache.calcite.rel.logical.LogicalCalc;
-import org.apache.storm.sql.planner.trident.rel.TridentCalcRel;
-import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
-
-public class TridentCalcRule extends ConverterRule {
-  public static final TridentCalcRule INSTANCE = new TridentCalcRule();
-
-  private TridentCalcRule() {
-    super(LogicalCalc.class, Convention.NONE, TridentLogicalConvention.INSTANCE, "TridentCalcRule");
-  }
-
-  @Override
-  public RelNode convert(RelNode rel) {
-    final Calc calc = (Calc) rel;
-    final RelNode input = calc.getInput();
-
-    return new TridentCalcRel(calc.getCluster(), calc.getTraitSet().replace(TridentLogicalConvention.INSTANCE),
-            convert(input, input.getTraitSet().replace(TridentLogicalConvention.INSTANCE)),
-            calc.getProgram());
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentFilterRule.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentFilterRule.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentFilterRule.java
deleted file mode 100644
index 7f9c41f..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentFilterRule.java
+++ /dev/null
@@ -1,45 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.planner.trident.rules;
-
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Filter;
-import org.apache.calcite.rel.logical.LogicalFilter;
-import org.apache.storm.sql.planner.trident.rel.TridentFilterRel;
-import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
-
-public class TridentFilterRule extends ConverterRule {
-  public static TridentFilterRule INSTANCE = new TridentFilterRule();
-
-  private TridentFilterRule() {
-    super(LogicalFilter.class, Convention.NONE, TridentLogicalConvention.INSTANCE, "TridentFilterRule");
-  }
-
-  @Override
-  public RelNode convert(RelNode rel) {
-    final Filter filter = (Filter) rel;
-    final RelNode input = filter.getInput();
-
-    return new TridentFilterRel(filter.getCluster(),
-        filter.getTraitSet().replace(TridentLogicalConvention.INSTANCE),
-        convert(input, input.getTraitSet().replace(TridentLogicalConvention.INSTANCE)),
-        filter.getCondition());
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentJoinRule.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentJoinRule.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentJoinRule.java
deleted file mode 100644
index 90f5083..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentJoinRule.java
+++ /dev/null
@@ -1,37 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.planner.trident.rules;
-
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.logical.LogicalJoin;
-import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
-
-public class TridentJoinRule extends ConverterRule {
-  public static final TridentJoinRule INSTANCE = new TridentJoinRule();
-
-  private TridentJoinRule() {
-    super(LogicalJoin.class, Convention.NONE, TridentLogicalConvention.INSTANCE, "TridentJoinRule");
-  }
-
-  @Override
-  public RelNode convert(RelNode rel) {
-    throw new UnsupportedOperationException("Join operation is not supported.");
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentModifyRule.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentModifyRule.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentModifyRule.java
deleted file mode 100644
index 2155451..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentModifyRule.java
+++ /dev/null
@@ -1,71 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.planner.trident.rules;
-
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.prepare.Prepare;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.TableModify;
-import org.apache.calcite.rel.logical.LogicalTableModify;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.schema.Table;
-import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
-import org.apache.storm.sql.planner.trident.rel.TridentStreamInsertRel;
-
-import java.util.List;
-
-public class TridentModifyRule extends ConverterRule {
-  public static final TridentModifyRule INSTANCE = new TridentModifyRule();
-
-  private TridentModifyRule() {
-    super(LogicalTableModify.class, Convention.NONE, TridentLogicalConvention.INSTANCE, "TridentModifyRule");
-  }
-
-  @Override
-  public RelNode convert(RelNode rel) {
-    final TableModify tableModify = (TableModify) rel;
-    final RelNode input = tableModify.getInput();
-
-    final RelOptCluster cluster = tableModify.getCluster();
-    final RelTraitSet traitSet = tableModify.getTraitSet().replace(TridentLogicalConvention.INSTANCE);
-    final RelOptTable relOptTable = tableModify.getTable();
-    final Prepare.CatalogReader catalogReader = tableModify.getCatalogReader();
-    final RelNode convertedInput = convert(input, input.getTraitSet().replace(TridentLogicalConvention.INSTANCE));
-    final TableModify.Operation operation = tableModify.getOperation();
-    final List<String> updateColumnList = tableModify.getUpdateColumnList();
-    final List<RexNode> sourceExpressionList = tableModify.getSourceExpressionList();
-    final boolean flattened = tableModify.isFlattened();
-
-    final Table table = tableModify.getTable().unwrap(Table.class);
-
-    switch (table.getJdbcTableType()) {
-      case STREAM:
-        if (operation != TableModify.Operation.INSERT) {
-          throw new UnsupportedOperationException(String.format("Streams doesn't support %s modify operation", operation));
-        }
-        return new TridentStreamInsertRel(cluster, traitSet, relOptTable, catalogReader, convertedInput, operation,
-            updateColumnList, sourceExpressionList, flattened);
-      default:
-        throw new IllegalArgumentException(String.format("Unsupported table type: %s", table.getJdbcTableType()));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentProjectRule.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentProjectRule.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentProjectRule.java
deleted file mode 100644
index 2922725..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentProjectRule.java
+++ /dev/null
@@ -1,45 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.planner.trident.rules;
-
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.logical.LogicalProject;
-import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
-import org.apache.storm.sql.planner.trident.rel.TridentProjectRel;
-
-public class TridentProjectRule extends ConverterRule {
-  public static final TridentProjectRule INSTANCE = new TridentProjectRule();
-
-  private TridentProjectRule() {
-    super(LogicalProject.class, Convention.NONE, TridentLogicalConvention.INSTANCE,
-        "TridentProjectRule");
-  }
-
-  @Override
-  public RelNode convert(RelNode rel) {
-    final Project project = (Project) rel;
-    final RelNode input = project.getInput();
-
-    return new TridentProjectRel(project.getCluster(),
-        project.getTraitSet().replace(TridentLogicalConvention.INSTANCE),
-        convert(input, input.getTraitSet().replace(TridentLogicalConvention.INSTANCE)), project.getProjects(), project.getRowType());
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentScanRule.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentScanRule.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentScanRule.java
deleted file mode 100644
index abbd680..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentScanRule.java
+++ /dev/null
@@ -1,58 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.planner.trident.rules;
-
-import org.apache.calcite.adapter.enumerable.EnumerableConvention;
-import org.apache.calcite.adapter.enumerable.EnumerableTableScan;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.TableScan;
-import org.apache.calcite.schema.Table;
-import org.apache.storm.sql.calcite.ParallelStreamableTable;
-import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
-import org.apache.storm.sql.planner.trident.rel.TridentStreamScanRel;
-
-public class TridentScanRule extends ConverterRule {
-  public static final TridentScanRule INSTANCE = new TridentScanRule();
-  public static final int DEFAULT_PARALLELISM_HINT = 1;
-
-  private TridentScanRule() {
-    super(EnumerableTableScan.class, EnumerableConvention.INSTANCE, TridentLogicalConvention.INSTANCE, "TridentScanRule");
-  }
-
-  @Override
-  public RelNode convert(RelNode rel) {
-    final TableScan scan = (TableScan) rel;
-    int parallelismHint = DEFAULT_PARALLELISM_HINT;
-
-    final ParallelStreamableTable parallelTable = scan.getTable().unwrap(ParallelStreamableTable.class);
-    if (parallelTable != null && parallelTable.parallelismHint() != null) {
-      parallelismHint = parallelTable.parallelismHint();
-    }
-
-    final Table table = scan.getTable().unwrap(Table.class);
-    switch (table.getJdbcTableType()) {
-      case STREAM:
-        return new TridentStreamScanRel(scan.getCluster(),
-            scan.getTraitSet().replace(TridentLogicalConvention.INSTANCE),
-            scan.getTable(), parallelismHint);
-      default:
-        throw new IllegalArgumentException(String.format("Unsupported table type: %s", table.getJdbcTableType()));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/test/org/apache/storm/sql/TestStormSql.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/TestStormSql.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/TestStormSql.java
deleted file mode 100644
index 82dc184..0000000
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/TestStormSql.java
+++ /dev/null
@@ -1,492 +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.sql;
-
-import com.google.common.collect.ImmutableMap;
-import org.apache.calcite.tools.ValidationException;
-import org.apache.storm.sql.javac.CompilingClassLoader;
-import org.apache.storm.sql.runtime.ChannelHandler;
-import org.apache.storm.sql.runtime.DataSource;
-import org.apache.storm.sql.runtime.DataSourcesProvider;
-import org.apache.storm.sql.runtime.DataSourcesRegistry;
-import org.apache.storm.sql.runtime.FieldInfo;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.tuple.Values;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-public class TestStormSql {
-  private static class MockDataSourceProvider implements DataSourcesProvider {
-    @Override
-    public String scheme() {
-      return "mock";
-    }
-
-    @Override
-    public DataSource construct(
-        URI uri, String inputFormatClass, String outputFormatClass,
-        List<FieldInfo> fields) {
-      return new TestUtils.MockDataSource();
-    }
-
-    @Override
-    public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
-                                                  Properties properties, List<FieldInfo> fields) {
-      return new TestUtils.MockSqlTridentDataSource();
-    }
-  }
-
-  private static class MockNestedDataSourceProvider implements DataSourcesProvider {
-    @Override
-    public String scheme() {
-      return "mocknested";
-    }
-
-    @Override
-    public DataSource construct(
-            URI uri, String inputFormatClass, String outputFormatClass,
-            List<FieldInfo> fields) {
-      return new TestUtils.MockNestedDataSource();
-    }
-
-    @Override
-    public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
-                                                  Properties properties, List<FieldInfo> fields) {
-      return new TestUtils.MockSqlTridentDataSource();
-    }
-  }
-
-  private static class MockGroupDataSourceProvider implements DataSourcesProvider {
-    @Override
-    public String scheme() {
-      return "mockgroup";
-    }
-
-    @Override
-    public DataSource construct(
-            URI uri, String inputFormatClass, String outputFormatClass,
-            List<FieldInfo> fields) {
-      return new TestUtils.MockGroupDataSource();
-    }
-
-    @Override
-    public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
-                                                  Properties properties, List<FieldInfo> fields) {
-      return new TestUtils.MockSqlTridentGroupedDataSource();
-    }
-  }
-
-  private static class MockEmpDataSourceProvider implements DataSourcesProvider {
-    @Override
-    public String scheme() {
-      return "mockemp";
-    }
-
-    @Override
-    public DataSource construct(
-            URI uri, String inputFormatClass, String outputFormatClass,
-            List<FieldInfo> fields) {
-      return new TestUtils.MockEmpDataSource();
-    }
-
-    @Override
-    public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
-                                                  Properties properties, List<FieldInfo> fields) {
-      return new TestUtils.MockSqlTridentJoinDataSourceEmp();
-    }
-  }
-
-  private static class MockDeptDataSourceProvider implements DataSourcesProvider {
-    @Override
-    public String scheme() {
-      return "mockdept";
-    }
-
-    @Override
-    public DataSource construct(
-            URI uri, String inputFormatClass, String outputFormatClass,
-            List<FieldInfo> fields) {
-      return new TestUtils.MockDeptDataSource();
-    }
-
-    @Override
-    public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
-                                                  Properties properties, List<FieldInfo> fields) {
-      return new TestUtils.MockSqlTridentJoinDataSourceDept();
-    }
-  }
-
-
-  @BeforeClass
-  public static void setUp() {
-    DataSourcesRegistry.providerMap().put("mock", new MockDataSourceProvider());
-    DataSourcesRegistry.providerMap().put("mocknested", new MockNestedDataSourceProvider());
-    DataSourcesRegistry.providerMap().put("mockgroup", new MockGroupDataSourceProvider());
-    DataSourcesRegistry.providerMap().put("mockemp", new MockEmpDataSourceProvider());
-    DataSourcesRegistry.providerMap().put("mockdept", new MockDeptDataSourceProvider());
-  }
-
-  @AfterClass
-  public static void tearDown() {
-    DataSourcesRegistry.providerMap().remove("mock");
-    DataSourcesRegistry.providerMap().remove("mocknested");
-  }
-
-  @Test
-  public void testExternalDataSource() throws Exception {
-    List<String> stmt = new ArrayList<>();
-    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT) LOCATION 'mock:///foo'");
-    stmt.add("SELECT STREAM ID + 1 FROM FOO WHERE ID > 2");
-    StormSql sql = StormSql.construct();
-    List<Values> values = new ArrayList<>();
-    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
-    sql.execute(stmt, h);
-    Assert.assertEquals(2, values.size());
-    Assert.assertEquals(4, values.get(0).get(0));
-    Assert.assertEquals(5, values.get(1).get(0));
-  }
-
-  @Test
-  public void testExternalDataSourceNested() throws Exception {
-    List<String> stmt = new ArrayList<>();
-    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, MAPFIELD ANY, NESTEDMAPFIELD ANY, ARRAYFIELD ANY) LOCATION 'mocknested:///foo'");
-    stmt.add("SELECT STREAM ID, MAPFIELD['c'], NESTEDMAPFIELD, ARRAYFIELD " +
-                     "FROM FOO " +
-                     "WHERE CAST(MAPFIELD['b'] AS INTEGER) = 2 AND CAST(ARRAYFIELD[2] AS INTEGER) = 200");
-    StormSql sql = StormSql.construct();
-    List<Values> values = new ArrayList<>();
-    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
-    sql.execute(stmt, h);
-    System.out.println(values);
-    Map<String, Integer> map = ImmutableMap.of("b", 2, "c", 4);
-    Map<String, Map<String, Integer>> nestedMap = ImmutableMap.of("a", map);
-    Assert.assertEquals(new Values(2, 4, nestedMap, Arrays.asList(100, 200, 300)), values.get(0));
-  }
-
-  @Test
-  public void testExternalNestedNonExistKeyAccess() throws Exception {
-    List<String> stmt = new ArrayList<>();
-    // this triggers java.lang.RuntimeException: Cannot convert null to int
-    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, MAPFIELD ANY, NESTEDMAPFIELD ANY, ARRAYFIELD ANY) LOCATION 'mocknested:///foo'");
-    stmt.add("SELECT STREAM ID, MAPFIELD, NESTEDMAPFIELD, ARRAYFIELD " +
-             "FROM FOO " +
-             "WHERE CAST(MAPFIELD['a'] AS INTEGER) = 2");
-    StormSql sql = StormSql.construct();
-    List<Values> values = new ArrayList<>();
-    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
-    sql.execute(stmt, h);
-    Assert.assertEquals(0, values.size());
-  }
-
-  @Test
-  public void testExternalNestedNonExistKeyAccess2() throws Exception {
-    List<String> stmt = new ArrayList<>();
-    // this triggers java.lang.RuntimeException: Cannot convert null to int
-    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, MAPFIELD ANY, NESTEDMAPFIELD ANY, ARRAYFIELD ANY) LOCATION 'mocknested:///foo'");
-    stmt.add("SELECT STREAM ID, MAPFIELD, NESTEDMAPFIELD, ARRAYFIELD " +
-             "FROM FOO " +
-             "WHERE CAST(NESTEDMAPFIELD['b']['c'] AS INTEGER) = 4");
-    StormSql sql = StormSql.construct();
-    List<Values> values = new ArrayList<>();
-    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
-    sql.execute(stmt, h);
-    Assert.assertEquals(0, values.size());
-  }
-
-  @Test
-  public void testExternalNestedInvalidAccessStringIndexOnArray() throws Exception {
-    List<String> stmt = new ArrayList<>();
-    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, MAPFIELD ANY, NESTEDMAPFIELD ANY, ARRAYFIELD ANY) LOCATION 'mocknested:///foo'");
-    stmt.add("SELECT STREAM ID, MAPFIELD, NESTEDMAPFIELD, ARRAYFIELD " +
-             "FROM FOO " +
-             "WHERE CAST(ARRAYFIELD['a'] AS INTEGER) = 200");
-    StormSql sql = StormSql.construct();
-    List<Values> values = new ArrayList<>();
-    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
-    sql.execute(stmt, h);
-    Assert.assertEquals(0, values.size());
-  }
-
-  @Test
-  public void testExternalNestedArrayOutOfBoundAccess() throws Exception {
-    List<String> stmt = new ArrayList<>();
-    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, MAPFIELD ANY, NESTEDMAPFIELD ANY, ARRAYFIELD ANY) LOCATION 'mocknested:///foo'");
-    stmt.add("SELECT STREAM ID, MAPFIELD, NESTEDMAPFIELD, ARRAYFIELD " +
-             "FROM FOO " +
-             "WHERE CAST(ARRAYFIELD[10] AS INTEGER) = 200");
-    StormSql sql = StormSql.construct();
-    List<Values> values = new ArrayList<>();
-    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
-    sql.execute(stmt, h);
-    Assert.assertEquals(0, values.size());
-  }
-
-  @Test(expected = ValidationException.class)
-  public void testExternalUdfType() throws Exception {
-    List<String> stmt = new ArrayList<>();
-    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, NAME VARCHAR) LOCATION 'mock:///foo'");
-    stmt.add("CREATE FUNCTION MYPLUS AS 'org.apache.storm.sql.TestUtils$MyPlus'");
-    stmt.add("SELECT STREAM MYPLUS(NAME, 1) FROM FOO WHERE ID = 0");
-    StormSql sql = StormSql.construct();
-    List<Values> values = new ArrayList<>();
-    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
-    sql.execute(stmt, h);
-    System.out.println(values);
-
-  }
-
-  @Test(expected = CompilingClassLoader.CompilerException.class)
-  public void testExternalUdfType2() throws Exception {
-    List<String> stmt = new ArrayList<>();
-    // generated code will be not compilable since return type of MYPLUS and type of 'x' are different
-    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT, NAME VARCHAR) LOCATION 'mock:///foo'");
-    stmt.add("CREATE FUNCTION MYPLUS AS 'org.apache.storm.sql.TestUtils$MyPlus'");
-    stmt.add("SELECT STREAM ID FROM FOO WHERE MYPLUS(ID, 1) = 'x'");
-    StormSql sql = StormSql.construct();
-    List<Values> values = new ArrayList<>();
-    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
-    sql.execute(stmt, h);
-    Assert.assertEquals(0, values.size());
-  }
-
-  @Test
-  public void testExternalUdf() throws Exception {
-    List<String> stmt = new ArrayList<>();
-    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT) LOCATION 'mock:///foo'");
-    stmt.add("CREATE FUNCTION MYPLUS AS 'org.apache.storm.sql.TestUtils$MyPlus'");
-    stmt.add("SELECT STREAM MYPLUS(ID, 1) FROM FOO WHERE ID > 2");
-    StormSql sql = StormSql.construct();
-    List<Values> values = new ArrayList<>();
-    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
-    sql.execute(stmt, h);
-    Assert.assertEquals(2, values.size());
-    Assert.assertEquals(4, values.get(0).get(0));
-    Assert.assertEquals(5, values.get(1).get(0));
-  }
-
-  @Test(expected = UnsupportedOperationException.class)
-  public void testExternalUdfUsingJar() throws Exception {
-    List<String> stmt = new ArrayList<>();
-    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT) LOCATION 'mock:///foo'");
-    stmt.add("CREATE FUNCTION MYPLUS AS 'org.apache.storm.sql.TestUtils$MyPlus' USING JAR 'foo'");
-    stmt.add("SELECT STREAM MYPLUS(ID, 1) FROM FOO WHERE ID > 2");
-    StormSql sql = StormSql.construct();
-    List<Values> values = new ArrayList<>();
-    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
-    sql.execute(stmt, h);
-  }
-
-  @Test
-  public void testGroupbyBuiltin() throws Exception {
-    List<String> stmt = new ArrayList<>();
-    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
-    stmt.add("SELECT STREAM ID, COUNT(*), SUM(SALARY), AVG(SALARY) FROM FOO GROUP BY (ID)");
-    StormSql sql = StormSql.construct();
-    List<Values> values = new ArrayList<>();
-    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
-    sql.execute(stmt, h);
-    Assert.assertEquals(4, values.size());
-    Assert.assertEquals(3, values.get(0).get(2));
-    Assert.assertEquals(12, values.get(1).get(2));
-    Assert.assertEquals(21, values.get(2).get(2));
-    Assert.assertEquals(9, values.get(3).get(2));
-  }
-
-  @Test
-  public void testGroupbyBuiltinWithFilter() throws Exception {
-    List<String> stmt = new ArrayList<>();
-    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
-    stmt.add("SELECT STREAM ID, COUNT(*), SUM(SALARY), AVG(PCT) FROM FOO WHERE ID = 1 GROUP BY (ID)");
-    StormSql sql = StormSql.construct();
-    List<Values> values = new ArrayList<>();
-    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
-    sql.execute(stmt, h);
-    Assert.assertEquals(1, values.size());
-    Assert.assertEquals(1, values.get(0).get(0));
-    Assert.assertEquals(3L, values.get(0).get(1));
-    Assert.assertEquals(12, values.get(0).get(2));
-    Assert.assertEquals(2.5, values.get(0).get(3));
-  }
-
-  @Test
-  public void testGroupbyBuiltinAndUDF() throws Exception {
-    List<String> stmt = new ArrayList<>();
-    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
-    stmt.add("CREATE FUNCTION MYCONCAT AS 'org.apache.storm.sql.TestUtils$MyConcat'");
-    stmt.add("CREATE FUNCTION TOPN AS 'org.apache.storm.sql.TestUtils$TopN'");
-    stmt.add("SELECT STREAM ID, SUM(SALARY), MYCONCAT(NAME), TOPN(2, SALARY) FROM FOO GROUP BY (ID)");
-    StormSql sql = StormSql.construct();
-    List<Values> values = new ArrayList<>();
-    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
-    sql.execute(stmt, h);
-    Assert.assertEquals(4, values.size());
-    Assert.assertEquals(3, values.get(0).get(1));
-    Assert.assertEquals("xxx", values.get(0).get(2));
-    Assert.assertEquals(Arrays.asList(2, 1), values.get(0).get(3));
-    Assert.assertEquals(12, values.get(1).get(1));
-    Assert.assertEquals("xxx", values.get(1).get(2));
-    Assert.assertEquals(Arrays.asList(5, 4), values.get(1).get(3));
-    Assert.assertEquals(21, values.get(2).get(1));
-    Assert.assertEquals("xxx", values.get(2).get(2));
-    Assert.assertEquals(Arrays.asList(8, 7), values.get(2).get(3));
-    Assert.assertEquals(9, values.get(3).get(1));
-    Assert.assertEquals("x", values.get(3).get(2));
-    Assert.assertEquals(Arrays.asList(9), values.get(3).get(3));
-  }
-
-  @Test
-  public void testAggFnNonSqlReturnType() throws Exception {
-    List<String> stmt = new ArrayList<>();
-    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
-    stmt.add("CREATE FUNCTION TOPN AS 'org.apache.storm.sql.TestUtils$TopN'");
-    stmt.add("SELECT STREAM ID, SUM(SALARY), TOPN(1, SALARY) FROM FOO WHERE ID >= 0 GROUP BY (ID) HAVING MAX(SALARY) > 0");
-    StormSql sql = StormSql.construct();
-    List<Values> values = new ArrayList<>();
-    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
-    sql.execute(stmt, h);
-    Assert.assertEquals(4, values.size());
-    Assert.assertEquals(Collections.singletonList(2), values.get(0).get(2));
-    Assert.assertEquals(Collections.singletonList(5), values.get(1).get(2));
-    Assert.assertEquals(Collections.singletonList(8), values.get(2).get(2));
-    Assert.assertEquals(Collections.singletonList(9), values.get(3).get(2));
-  }
-
-  @Test
-  public void testGroupbySameAggregateOnDifferentColumns() throws Exception {
-    List<String> stmt = new ArrayList<>();
-    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
-    stmt.add("SELECT STREAM ID, COUNT(*), AVG(SALARY), AVG(PCT) FROM FOO WHERE ID = 1 GROUP BY (ID)");
-    StormSql sql = StormSql.construct();
-    List<Values> values = new ArrayList<>();
-    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
-    sql.execute(stmt, h);
-    Assert.assertEquals(1, values.size());
-    Assert.assertEquals(1, values.get(0).get(0));
-    Assert.assertEquals(3L, values.get(0).get(1));
-    Assert.assertEquals(4, values.get(0).get(2));
-    Assert.assertEquals(2.5, values.get(0).get(3));
-  }
-
-  @Test(expected = UnsupportedOperationException.class)
-  public void testGroupbyBuiltinNotimplemented() throws Exception {
-    List<String> stmt = new ArrayList<>();
-    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
-    stmt.add("SELECT STREAM ID, COUNT(*), STDDEV_POP(SALARY) FROM FOO GROUP BY (ID)");
-    StormSql sql = StormSql.construct();
-    List<Values> values = new ArrayList<>();
-    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
-    sql.execute(stmt, h);
-  }
-
-  @Test
-  public void testMinMax() throws Exception {
-    List<String> stmt = new ArrayList<>();
-    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
-    stmt.add("SELECT STREAM ID, COUNT(*), MIN(SALARY), MAX(PCT) FROM FOO GROUP BY (ID)");
-    StormSql sql = StormSql.construct();
-    List<Values> values = new ArrayList<>();
-    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
-    sql.execute(stmt, h);
-    Assert.assertEquals(4, values.size());
-    Assert.assertEquals(0, values.get(0).get(2));
-    Assert.assertEquals(3, values.get(1).get(2));
-    Assert.assertEquals(6, values.get(2).get(2));
-    Assert.assertEquals(9, values.get(3).get(2));
-
-    Assert.assertEquals(1.5, values.get(0).get(3));
-    Assert.assertEquals(3.0, values.get(1).get(3));
-    Assert.assertEquals(4.5, values.get(2).get(3));
-    Assert.assertEquals(5.0, values.get(3).get(3));
-  }
-  @Test
-  public void testFilterGroupbyHaving() throws Exception {
-    List<String> stmt = new ArrayList<>();
-    stmt.add("CREATE EXTERNAL TABLE FOO (ID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR) LOCATION 'mockgroup:///foo'");
-    stmt.add("SELECT STREAM ID, MIN(SALARY) FROM FOO where ID > 0 GROUP BY (ID) HAVING ID > 2 AND MAX(SALARY) > 5");
-    StormSql sql = StormSql.construct();
-    List<Values> values = new ArrayList<>();
-    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
-    sql.execute(stmt, h);
-    Assert.assertEquals(1, values.size());
-    Assert.assertEquals(3, values.get(0).get(0));
-    Assert.assertEquals(9, values.get(0).get(1));
-  }
-
-  @Test
-  public void testGroupByMultipleFields() throws Exception {
-    List<String> stmt = new ArrayList<>();
-    stmt.add("CREATE EXTERNAL TABLE FOO (DEPTID INT PRIMARY KEY, SALARY INT, PCT DOUBLE, NAME VARCHAR, EMPID INT) LOCATION 'mockgroup:///foo'");
-    stmt.add("SELECT STREAM DEPTID, EMPID, COUNT(*), MIN(SALARY), MAX(PCT) FROM FOO GROUP BY DEPTID, EMPID");
-    StormSql sql = StormSql.construct();
-    List<Values> values = new ArrayList<>();
-    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
-    sql.execute(stmt, h);
-    Assert.assertEquals(7, values.size());
-    Assert.assertEquals(0, values.get(0).get(0));
-    Assert.assertEquals(0, values.get(0).get(1));
-    Assert.assertEquals(2L, values.get(0).get(2));
-  }
-
-  @Test
-  public void testjoin() throws Exception {
-    List<String> stmt = new ArrayList<>();
-    stmt.add("CREATE EXTERNAL TABLE EMP (EMPID INT PRIMARY KEY, EMPNAME VARCHAR, DEPTID INT) LOCATION 'mockemp:///foo'");
-    stmt.add("CREATE EXTERNAL TABLE DEPT (DEPTID INT PRIMARY KEY, DEPTNAME VARCHAR) LOCATION 'mockdept:///foo'");
-    stmt.add("SELECT STREAM EMPID, EMPNAME, DEPTNAME FROM EMP AS e JOIN DEPT AS d ON e.DEPTID = d.DEPTID WHERE e.empid > 0");
-    StormSql sql = StormSql.construct();
-    List<Values> values = new ArrayList<>();
-    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
-    sql.execute(stmt, h);
-    System.out.println(values);
-    Assert.assertEquals(3, values.size());
-    Assert.assertEquals("emp1", values.get(0).get(1));
-    Assert.assertEquals("dept1", values.get(0).get(2));
-    Assert.assertEquals("emp2", values.get(1).get(1));
-    Assert.assertEquals("dept1", values.get(1).get(2));
-    Assert.assertEquals("emp3", values.get(2).get(1));
-    Assert.assertEquals("dept2", values.get(2).get(2));
-  }
-
-  @Test
-  public void testjoinAndGroupby() throws Exception {
-    List<String> stmt = new ArrayList<>();
-    stmt.add("CREATE EXTERNAL TABLE EMP (EMPID INT PRIMARY KEY, EMPNAME VARCHAR, DEPTID INT) LOCATION 'mockemp:///foo'");
-    stmt.add("CREATE EXTERNAL TABLE DEPT (DEPTID INT PRIMARY KEY, DEPTNAME VARCHAR) LOCATION 'mockdept:///foo'");
-    stmt.add("SELECT STREAM d.DEPTID, count(EMPID) FROM EMP AS e JOIN DEPT AS d ON e.DEPTID = d.DEPTID WHERE e.empid > 0" +
-                     "GROUP BY d.DEPTID");
-    StormSql sql = StormSql.construct();
-    List<Values> values = new ArrayList<>();
-    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
-    sql.execute(stmt, h);
-    Assert.assertEquals(2, values.size());
-    Assert.assertEquals(1, values.get(0).get(0));
-    Assert.assertEquals(2L, values.get(0).get(1));
-    Assert.assertEquals(2, values.get(1).get(0));
-    Assert.assertEquals(1L, values.get(1).get(1));
-  }
-}


[14/23] storm git commit: STORM-2453 Move non-connectors into the top directory

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprSemantic.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprSemantic.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprSemantic.java
deleted file mode 100644
index 634e454..0000000
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprSemantic.java
+++ /dev/null
@@ -1,410 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.compiler;
-
-import com.google.common.base.Function;
-import org.apache.storm.sql.compiler.backends.standalone.TestCompilerUtils;
-import org.apache.storm.tuple.Values;
-import com.google.common.base.Joiner;
-import com.google.common.collect.Lists;
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.storm.sql.TestUtils;
-import org.apache.storm.sql.compiler.backends.standalone.PlanCompiler;
-import org.apache.storm.sql.runtime.ChannelHandler;
-import org.apache.storm.sql.runtime.DataSource;
-import org.apache.storm.sql.runtime.AbstractValuesProcessor;
-import org.junit.Test;
-
-import javax.annotation.Nullable;
-import java.math.BigDecimal;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-public class TestExprSemantic {
-  private final JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(
-      RelDataTypeSystem.DEFAULT);
-
-  @Test
-  public void testLogicalExpr() throws Exception {
-    Values v = testExpr(
-        Lists.newArrayList("ID > 0 OR ID < 1", "ID > 0 AND ID < 1",
-                           "NOT (ID > 0 AND ID < 1)"));
-    assertEquals(new Values(true, false, true), v);
-  }
-
-  @Test
-  public void testExpectOperator() throws Exception {
-    Values v = testExpr(
-        Lists.newArrayList("TRUE IS TRUE", "TRUE IS NOT TRUE",
-                           "UNKNOWN IS TRUE", "UNKNOWN IS NOT TRUE",
-                           "TRUE IS FALSE", "UNKNOWN IS NULL",
-                           "UNKNOWN IS NOT NULL"));
-    assertEquals(new Values(true, false, false, true, false, true, false), v);
-  }
-
-  @Test
-  public void testDistinctBetweenLikeSimilarIn() throws Exception {
-    Values v = testExpr(
-            Lists.newArrayList("TRUE IS DISTINCT FROM TRUE",
-                    "TRUE IS NOT DISTINCT FROM FALSE", "3 BETWEEN 1 AND 5",
-                    "10 NOT BETWEEN 1 AND 5", "'hello' LIKE '_e%'",
-                    "'world' NOT LIKE 'wor%'", "'abc' SIMILAR TO '[a-zA-Z]+[cd]{1}'",
-                    "'abe' NOT SIMILAR TO '[a-zA-Z]+[cd]{1}'", "'3' IN ('1', '2', '3', '4')",
-                    "2 NOT IN (1, 3, 5)"));
-    assertEquals(new Values(false, false, true, true, true,
-          false, true, true, true, true), v);
-  }
-
-  @Test
-  public void testCaseStatement() throws Exception {
-    Values v = testExpr(
-            Lists.newArrayList(
-                    "CASE WHEN 'abcd' IN ('a', 'abc', 'abcde') THEN UPPER('a') " +
-                    "WHEN UPPER('abcd') = 'AB' THEN 'b' ELSE {fn CONCAT('abcd', '#')} END",
-                    "CASE WHEN 'ab' IN ('a', 'abc', 'abcde') THEN UPPER('a') " +
-                    "WHEN UPPER('ab') = 'AB' THEN 'b' ELSE {fn CONCAT('ab', '#')} END",
-                    "CASE WHEN 'abc' IN ('a', 'abc', 'abcde') THEN UPPER('a') " +
-                    "WHEN UPPER('abc') = 'AB' THEN 'b' ELSE {fn CONCAT('abc', '#')} END"
-                    )
-    );
-
-    // TODO: The data type of literal Calcite assigns seems to be out of expectation. Please see below logical plan.
-    // LogicalProject(EXPR$0=[CASE(OR(=('abcd', 'a'), =('abcd', 'abc'), =('abcd', 'abcde')), CAST(UPPER('a')):VARCHAR(5) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL, =(UPPER('abcd'), CAST('AB'):CHAR(4) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL), 'b', CAST(||('abcd', '#')):VARCHAR(5) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL)], EXPR$1=[CASE(OR(=('ab', 'a'), =('ab', 'abc'), =('ab', 'abcde')), CAST(UPPER('a')):CHAR(3) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL, =(UPPER('ab'), 'AB'), CAST('b'):CHAR(3) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL, ||('ab', '#'))], EXPR$2=[CASE(OR(=('abc', 'a'), =('abc', 'abc'), =('abc', 'abcde')), CAST(UPPER('a')):CHAR(4) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL, =(UPPER('abc'), CAST('AB'):CHAR(3) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL), CAST('b'):CHAR(4) C
 HARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary" NOT NULL, ||('abc', '#'))]): rowcount = 1.0, cumulative cost = {2.0 rows, 5.0 cpu, 0.0 io}, id = 5
-    //   LogicalFilter(condition=[AND(>($0, 0), <($0, 2))]): rowcount = 1.0, cumulative cost = {1.0 rows, 2.0 cpu, 0.0 io}, id = 4
-    //     EnumerableTableScan(table=[[FOO]]): rowcount = 1.0, cumulative cost = {0.0 rows, 1.0 cpu, 0.0 io}, id = 3
-    // in result, both 'b' and UPPER('a') hence 'A' are having some spaces which is not expected.
-    // When we use CASE with actual column (Java String type hence VARCHAR), it seems to work as expected.
-    // Please refer trident/TestPlanCompiler#testCaseStatement(), and see below logical plan.
-    // LogicalProject(EXPR$0=[CASE(OR(=($1, 'a'), =($1, 'abc'), =($1, 'abcde')), CAST(UPPER('a')):VARCHAR(1) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary", =(CAST(UPPER($1)):VARCHAR(2) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary", 'AB'), 'b', CAST(||($1, '#')):VARCHAR(1) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary")]): rowcount = 1.0, cumulative cost = {1.0 rows, 2.0 cpu, 0.0 io}, id = 3
-    List<Object> v2 = Lists.transform(v, new Function<Object, Object>() {
-      @Nullable
-      @Override
-      public String apply(@Nullable Object o) {
-        return ((String) o).trim();
-      }
-    });
-    assertArrayEquals(new Values("abcd#", "b", "A").toArray(), v2.toArray());
-  }
-
-  @Test
-  public void testNullIfAndCoalesce() throws Exception {
-    Values v = testExpr(
-            Lists.newArrayList(
-                    "NULLIF(5, 5)", "NULLIF(5, 0)", "COALESCE(NULL, NULL, 5, 4, NULL)", "COALESCE(1, 5)"
-            ));
-    assertEquals(new Values(null, 5, 5, 1), v);
-  }
-
-  @Test
-  public void testCollectionFunctions() throws Exception {
-    Values v = testExpr(
-            Lists.newArrayList(
-                    "ELEMENT(ARRAY[3])", "CARDINALITY(ARRAY[1, 2, 3, 4, 5])"
-            ));
-    assertEquals(new Values(3, 5), v);
-  }
-
-  @Test(expected = RuntimeException.class)
-  public void testElementFunctionMoreThanOneValue() throws Exception {
-    testExpr(
-            Lists.newArrayList(
-                    "ELEMENT(ARRAY[1, 2, 3])"
-            ));
-    fail("ELEMENT with array which has multiple elements should throw exception in runtime.");
-  }
-
-  @Test
-  public void testArithmeticWithNull() throws Exception {
-    Values v = testExpr(
-      Lists.newArrayList(
-          "1 + CAST(NULL AS INT)", "CAST(NULL AS INT) + 1", "CAST(NULL AS INT) + CAST(NULL AS INT)", "1 + 2"
-      ));
-    assertEquals(new Values(null, null, null, 3), v);
-  }
-
-  @Test
-  public void testNotWithNull() throws Exception {
-    Values v = testExpr(
-        Lists.newArrayList(
-            "NOT TRUE", "NOT FALSE", "NOT UNKNOWN"
-        ));
-    assertEquals(new Values(false, true, null), v);
-  }
-
-  @Test
-  public void testAndWithNull() throws Exception {
-    Values v = testExpr(
-        Lists.newArrayList(
-            "UNKNOWN AND TRUE", "UNKNOWN AND FALSE", "UNKNOWN AND UNKNOWN",
-            "TRUE AND TRUE", "TRUE AND FALSE", "TRUE AND UNKNOWN",
-            "FALSE AND TRUE", "FALSE AND FALSE", "FALSE AND UNKNOWN"
-        ));
-    assertEquals(new Values(null, false, null, true, false, null, false,
-                            false, false), v);
-  }
-
-  @Test
-  public void testAndWithNullable() throws Exception {
-    Values v = testExpr(
-            Lists.newArrayList(
-                    "ADDR = 'a' AND NAME = 'a'", "NAME = 'a' AND ADDR = 'a'", "NAME = 'x' AND ADDR = 'a'", "ADDR = 'a' AND NAME = 'x'"
-            ));
-    assertEquals(new Values(false, false, null, null), v);
-  }
-
-  @Test
-  public void testOrWithNullable() throws Exception {
-    Values v = testExpr(
-            Lists.newArrayList(
-                    "ADDR = 'a'  OR NAME = 'a'", "NAME = 'a' OR ADDR = 'a' ", "NAME = 'x' OR ADDR = 'a' ", "ADDR = 'a'  OR NAME = 'x'"
-            ));
-    assertEquals(new Values(null, null, true, true), v);
-  }
-
-  @Test
-  public void testOrWithNull() throws Exception {
-    Values v = testExpr(
-        Lists.newArrayList(
-            "UNKNOWN OR TRUE", "UNKNOWN OR FALSE", "UNKNOWN OR UNKNOWN",
-            "TRUE OR TRUE", "TRUE OR FALSE", "TRUE OR UNKNOWN",
-            "FALSE OR TRUE", "FALSE OR FALSE", "FALSE OR UNKNOWN"
-            ));
-    assertEquals(new Values(true, null, null, true, true, true, true,
-                            false, null), v);
-  }
-
-  @Test
-  public void testEquals() throws Exception {
-    Values v = testExpr(
-        Lists.newArrayList(
-            "1 = 2", "UNKNOWN = UNKNOWN", "'a' = 'a'", "'a' = UNKNOWN", "UNKNOWN = 'a'", "'a' = 'b'",
-            "1 <> 2", "UNKNOWN <> UNKNOWN", "'a' <> 'a'", "'a' <> UNKNOWN", "UNKNOWN <> 'a'", "'a' <> 'b'"
-        ));
-    assertEquals(new Values(false, null, true, null, null, false,
-        true, null, false, null, null, true), v);
-  }
-
-  @Test
-  public void testArithmeticFunctions() throws Exception {
-    Values v = testExpr(
-            Lists.newArrayList(
-                    "POWER(3, 2)", "ABS(-10)", "MOD(10, 3)", "MOD(-10, 3)",
-                    "CEIL(123.45)", "FLOOR(123.45)"
-            ));
-
-    assertEquals(new Values(9.0d, 10, 1, -1, new BigDecimal(124), new BigDecimal(123)), v);
-
-    // Belows are floating numbers so comparing this with literal is tend to be failing...
-    // Picking int value and compare
-    Values v2 = testExpr(
-            Lists.newArrayList(
-                    "SQRT(255)", "LN(16)", "LOG10(10000)", "EXP(10)"
-            ));
-    List<Object> v2m = Lists.transform(v2, new Function<Object, Object>() {
-      @Nullable
-      @Override
-      public Object apply(@Nullable Object o) {
-        // only takes int value
-        return ((Number) o).intValue();
-      }
-    });
-
-    // 15.9687, 2.7725, 4.0, 22026.465794
-    assertEquals(new Values(15, 2, 4, 22026), v2m);
-  }
-
-  @Test
-  public void testStringFunctions() throws Exception {
-    Values v = testExpr(
-        Lists.newArrayList(
-                "'ab' || 'cd'", "CHAR_LENGTH('foo')", "CHARACTER_LENGTH('foo')",
-                "UPPER('a')", "LOWER('A')", "POSITION('bc' IN 'abcd')",
-                "TRIM(BOTH ' ' FROM '  abcdeabcdeabc  ')",
-                "TRIM(LEADING ' ' FROM '  abcdeabcdeabc  ')",
-                "TRIM(TRAILING ' ' FROM '  abcdeabcdeabc  ')",
-                "OVERLAY('abcde' PLACING 'bc' FROM 3)",
-                "SUBSTRING('abcde' FROM 3)", "SUBSTRING('abcdeabcde' FROM 3 FOR 4)",
-                "INITCAP('foo')"
-        ));
-    assertEquals(new Values("abcd", 3, 3, "A", "a", 2, "abcdeabcdeabc", "abcdeabcdeabc  ", "  abcdeabcdeabc", "abbce", "cde", "cdea", "Foo"), v);
-  }
-
-  @Test
-  public void testBinaryStringFunctions() throws Exception {
-    Values v = testExpr(
-            Lists.newArrayList(
-                    "x'45F0AB' || x'45F0AB'",
-                    "POSITION(x'F0' IN x'453423F0ABBC')",
-                    "OVERLAY(x'453423F0ABBC45' PLACING x'4534' FROM 3)"
-                    // "SUBSTRING(x'453423F0ABBC' FROM 3)",
-                    // "SUBSTRING(x'453423F0ABBC453423F0ABBC' FROM 3 FOR 4)"
-            ));
-
-    // TODO: Calcite 1.9.0 has bugs on binary SUBSTRING functions
-    // as there's no SqlFunctions.substring(org.apache.calcite.avatica.util.ByteString, ...)
-    // commented out testing substring function
-
-    assertEquals("45f0ab45f0ab", v.get(0).toString());
-    assertEquals(4, v.get(1));
-    assertEquals("45344534abbc45", v.get(2).toString());
-    // assertEquals("23f0abbc", v.get(3).toString());
-    // assertEquals("23f0ab", v.get(4).toString());
-  }
-
-  @Test
-  public void testDateAndTimestampLiteral() throws Exception {
-    Values v = testExpr(
-            Lists.newArrayList(
-                    "DATE '1970-05-15' AS datefield",
-                    "TIME '00:00:00' AS timefield",
-                    "TIMESTAMP '2016-01-01 00:00:00' as timestampfield"
-            )
-    );
-
-    assertEquals(3, v.size());
-    assertEquals(134, v.get(0));
-    assertEquals(0, v.get(1));
-    assertEquals(1451606400000L, v.get(2));
-  }
-
-  @Test
-  public void testInterval() throws Exception {
-    Values v = testExpr(
-            Lists.newArrayList(
-                    "INTERVAL '1-5' YEAR TO MONTH AS intervalfield",
-                    "(DATE '1970-01-01', DATE '1970-01-15') AS anchoredinterval_field"
-            )
-    );
-
-    assertEquals(3, v.size());
-    assertEquals(17, v.get(0));
-    assertEquals(0, v.get(1));
-    assertEquals(14, v.get(2));
-  }
-
-  @Test
-  public void testDateFunctions() throws Exception {
-    Values v = testExpr(
-            Lists.newArrayList(
-                    "LOCALTIME = CURRENT_TIME, LOCALTIMESTAMP = CURRENT_TIMESTAMP, CURRENT_DATE",
-                    "EXTRACT(MONTH FROM TIMESTAMP '2010-01-23 12:34:56')",
-                    "FLOOR(DATE '2016-01-23' TO MONTH)",
-                    "CEIL(TIME '12:34:56' TO MINUTE)"
-            )
-    );
-
-    assertEquals(6, v.size());
-    assertTrue((boolean) v.get(0));
-    assertTrue((boolean) v.get(1));
-    // skip checking CURRENT_DATE since we don't inject dataContext so don't know about current timestamp
-    // we can do it from trident test
-    assertEquals(1L, v.get(3));
-    assertEquals(0L, v.get(4));
-    assertEquals(45300000, v.get(5));
-  }
-
-  @Test
-  public void testJDBCNumericFunctions() throws Exception {
-    Values v = testExpr(
-            Lists.newArrayList(
-                    "{fn POWER(3, 2)}", "{fn ABS(-10)}", "{fn MOD(10, 3)}", "{fn MOD(-10, 3)}"
-            ));
-
-    assertEquals(new Values(9.0d, 10, 1, -1), v);
-
-    // Belows are floating numbers so comparing this with literal is tend to be failing...
-    // Picking int value and compare
-    Values v2 = testExpr(
-            Lists.newArrayList(
-                    "{fn LOG(16)}", "{fn LOG10(10000)}", "{fn EXP(10)}"
-            ));
-    List<Object> v2m = Lists.transform(v2, new Function<Object, Object>() {
-      @Nullable
-      @Override
-      public Object apply(@Nullable Object o) {
-        // only takes int value
-        return ((Number) o).intValue();
-      }
-    });
-
-    // 2.7725, 4.0, 22026.465794
-    assertEquals(new Values(2, 4, 22026), v2m);
-  }
-
-  @Test
-  public void testJDBCStringFunctions() throws Exception {
-    Values v = testExpr(
-            Lists.newArrayList(
-                    "{fn CONCAT('ab', 'cd')}",
-                    "{fn LOCATE('bc', 'abcdeabcde')}",
-                    //"{fn LOCATE('bc', 'abcdeabcde', 4)}",
-                    "{fn INSERT('abcd', 2, 3, 'de')}",
-                    "{fn LCASE('AbCdE')}",
-                    "{fn LENGTH('AbCdE')}",
-                    //"{fn LTRIM('  abcde  ')}",
-                    //"{fn RTRIM('  abcde  ')}",
-                    "{fn SUBSTRING('abcdeabcde', 3, 4)}",
-                    "{fn UCASE('AbCdE')}"
-            )
-    );
-
-    // TODO: Calcite 1.9.0 doesn't support {fn LOCATE(string1, string2 [, integer])}
-    // while it's on support list on SQL reference
-    // and bugs on LTRIM and RTRIM : throwing AssertionError: Internal error: pre-condition failed: pos != null
-    // commented out problematic function tests
-
-    assertEquals(new Values("abcd", 2, "ade", "abcde", 5, "cdea", "ABCDE"), v);
-  }
-
-  @Test
-  public void testJDBCDateTimeFunctions() throws Exception {
-    Values v = testExpr(
-            Lists.newArrayList(
-                    "{fn CURDATE()} = CURRENT_DATE", "{fn CURTIME()} = LOCALTIME", "{fn NOW()} = LOCALTIMESTAMP",
-                    "{fn QUARTER(DATE '2016-10-07')}", "{fn TIMESTAMPADD(MINUTE, 15, TIMESTAMP '2016-10-07 00:00:00')}",
-                    "{fn TIMESTAMPDIFF(SECOND, TIMESTAMP '2016-10-06 00:00:00', TIMESTAMP '2016-10-07 00:00:00')}"
-            )
-    );
-
-    assertEquals(new Values(true, true, true, 4L, 1475799300000L, 86400), v);
-  }
-
-  private Values testExpr(List<String> exprs) throws Exception {
-    String sql = "SELECT " + Joiner.on(',').join(exprs) + " FROM FOO" +
-        " WHERE ID > 0 AND ID < 2";
-    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
-    PlanCompiler compiler = new PlanCompiler(typeFactory);
-    AbstractValuesProcessor proc = compiler.compile(state.tree());
-    Map<String, DataSource> data = new HashMap<>();
-    data.put("FOO", new TestUtils.MockDataSource());
-    List<Values> values = new ArrayList<>();
-    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
-    proc.initialize(data, h);
-    return values.get(0);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestCompilerUtils.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestCompilerUtils.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestCompilerUtils.java
deleted file mode 100644
index 8e64e9c..0000000
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestCompilerUtils.java
+++ /dev/null
@@ -1,183 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * 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.sql.compiler.backends.standalone;
-
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.calcite.plan.RelOptUtil;
-import org.apache.calcite.prepare.CalciteCatalogReader;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.schema.StreamableTable;
-import org.apache.calcite.schema.Table;
-import org.apache.calcite.schema.impl.ScalarFunctionImpl;
-import org.apache.calcite.sql.SqlExplainLevel;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlOperatorTable;
-import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.sql.parser.SqlParseException;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
-import org.apache.calcite.tools.FrameworkConfig;
-import org.apache.calcite.tools.Frameworks;
-import org.apache.calcite.tools.Planner;
-import org.apache.calcite.tools.RelConversionException;
-import org.apache.calcite.tools.ValidationException;
-import org.apache.storm.sql.compiler.CompilerUtil;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-public class TestCompilerUtils {
-
-    public static class MyPlus {
-        public static Integer eval(Integer x, Integer y) {
-            return x + y;
-        }
-    }
-
-    public static class MyStaticSumFunction {
-        public static long init() {
-            return 0L;
-        }
-        public static long add(long accumulator, int v) {
-            return accumulator + v;
-        }
-    }
-
-    public static class MySumFunction {
-        public MySumFunction() {
-        }
-        public long init() {
-            return 0L;
-        }
-        public long add(long accumulator, int v) {
-            return accumulator + v;
-        }
-        public long result(long accumulator) {
-            return accumulator;
-        }
-    }
-
-    public static CalciteState sqlOverDummyTable(String sql)
-            throws RelConversionException, ValidationException, SqlParseException {
-        SchemaPlus schema = Frameworks.createRootSchema(true);
-        JavaTypeFactory typeFactory = new JavaTypeFactoryImpl
-                (RelDataTypeSystem.DEFAULT);
-        StreamableTable streamableTable = new CompilerUtil.TableBuilderInfo(typeFactory)
-                .field("ID", SqlTypeName.INTEGER)
-                .field("NAME", typeFactory.createType(String.class))
-                .field("ADDR", typeFactory.createType(String.class))
-                .build();
-        Table table = streamableTable.stream();
-        schema.add("FOO", table);
-        schema.add("BAR", table);
-        schema.add("MYPLUS", ScalarFunctionImpl.create(MyPlus.class, "eval"));
-
-        List<SqlOperatorTable> sqlOperatorTables = new ArrayList<>();
-        sqlOperatorTables.add(SqlStdOperatorTable.instance());
-        sqlOperatorTables.add(new CalciteCatalogReader(CalciteSchema.from(schema),
-                false,
-                Collections.<String>emptyList(), typeFactory));
-        SqlOperatorTable chainedSqlOperatorTable = new ChainedSqlOperatorTable(sqlOperatorTables);
-        FrameworkConfig config = Frameworks.newConfigBuilder().defaultSchema(
-                schema).operatorTable(chainedSqlOperatorTable).build();
-        Planner planner = Frameworks.getPlanner(config);
-        SqlNode parse = planner.parse(sql);
-        SqlNode validate = planner.validate(parse);
-        RelNode tree = planner.convert(validate);
-        System.out.println(RelOptUtil.toString(tree, SqlExplainLevel.ALL_ATTRIBUTES));
-        return new CalciteState(schema, tree);
-    }
-
-    public static CalciteState sqlOverNestedTable(String sql)
-            throws RelConversionException, ValidationException, SqlParseException {
-        SchemaPlus schema = Frameworks.createRootSchema(true);
-        JavaTypeFactory typeFactory = new JavaTypeFactoryImpl
-                (RelDataTypeSystem.DEFAULT);
-
-        StreamableTable streamableTable = new CompilerUtil.TableBuilderInfo(typeFactory)
-                .field("ID", SqlTypeName.INTEGER)
-                .field("MAPFIELD",
-                        typeFactory.createTypeWithNullability(
-                                typeFactory.createMapType(
-                                        typeFactory.createTypeWithNullability(
-                                                typeFactory.createSqlType(SqlTypeName.VARCHAR), true),
-                                        typeFactory.createTypeWithNullability(
-                                                typeFactory.createSqlType(SqlTypeName.INTEGER), true))
-                                , true))
-                .field("NESTEDMAPFIELD",
-                        typeFactory.createTypeWithNullability(
-                            typeFactory.createMapType(
-                                    typeFactory.createTypeWithNullability(
-                                            typeFactory.createSqlType(SqlTypeName.VARCHAR), true),
-                                    typeFactory.createTypeWithNullability(
-                                            typeFactory.createMapType(
-                                                    typeFactory.createTypeWithNullability(
-                                                            typeFactory.createSqlType(SqlTypeName.VARCHAR), true),
-                                                    typeFactory.createTypeWithNullability(
-                                                            typeFactory.createSqlType(SqlTypeName.INTEGER), true))
-                                            , true))
-                                        , true))
-                .field("ARRAYFIELD", typeFactory.createTypeWithNullability(
-                        typeFactory.createArrayType(
-                            typeFactory.createTypeWithNullability(
-                                typeFactory.createSqlType(SqlTypeName.INTEGER), true), -1L)
-                        , true))
-                .build();
-        Table table = streamableTable.stream();
-        schema.add("FOO", table);
-        schema.add("BAR", table);
-        schema.add("MYPLUS", ScalarFunctionImpl.create(MyPlus.class, "eval"));
-        List<SqlOperatorTable> sqlOperatorTables = new ArrayList<>();
-        sqlOperatorTables.add(SqlStdOperatorTable.instance());
-        sqlOperatorTables.add(new CalciteCatalogReader(CalciteSchema.from(schema),
-                                                       false,
-                                                       Collections.<String>emptyList(), typeFactory));
-        SqlOperatorTable chainedSqlOperatorTable = new ChainedSqlOperatorTable(sqlOperatorTables);
-        FrameworkConfig config = Frameworks.newConfigBuilder().defaultSchema(
-                schema).operatorTable(chainedSqlOperatorTable).build();
-        Planner planner = Frameworks.getPlanner(config);
-        SqlNode parse = planner.parse(sql);
-        SqlNode validate = planner.validate(parse);
-        RelNode tree = planner.convert(validate);
-        System.out.println(RelOptUtil.toString(tree, SqlExplainLevel.ALL_ATTRIBUTES));
-        return new CalciteState(schema, tree);
-    }
-
-    public static class CalciteState {
-        final SchemaPlus schema;
-        final RelNode tree;
-
-        private CalciteState(SchemaPlus schema, RelNode tree) {
-            this.schema = schema;
-            this.tree = tree;
-        }
-
-        public SchemaPlus schema() {
-            return schema;
-        }
-
-        public RelNode tree() {
-            return tree;
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestPlanCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestPlanCompiler.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestPlanCompiler.java
deleted file mode 100644
index 3226810..0000000
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestPlanCompiler.java
+++ /dev/null
@@ -1,104 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.compiler.backends.standalone;
-
-import com.google.common.collect.ImmutableMap;
-import org.apache.storm.tuple.Values;
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.storm.sql.TestUtils;
-import org.apache.storm.sql.runtime.AbstractValuesProcessor;
-import org.apache.storm.sql.runtime.ChannelHandler;
-import org.apache.storm.sql.runtime.DataSource;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class TestPlanCompiler {
-  private final JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(
-      RelDataTypeSystem.DEFAULT);
-
-  @Test
-  public void testCompile() throws Exception {
-    String sql = "SELECT ID + 1 FROM FOO WHERE ID > 2";
-    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
-    PlanCompiler compiler = new PlanCompiler(typeFactory);
-    AbstractValuesProcessor proc = compiler.compile(state.tree());
-    Map<String, DataSource> data = new HashMap<>();
-    data.put("FOO", new TestUtils.MockDataSource());
-    List<Values> values = new ArrayList<>();
-    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
-    proc.initialize(data, h);
-    Assert.assertArrayEquals(new Values[] { new Values(4), new Values(5)},
-                             values.toArray());
-  }
-
-  @Test
-  public void testLogicalExpr() throws Exception {
-    String sql = "SELECT ID > 0 OR ID < 1, ID > 0 AND ID < 1, NOT (ID > 0 AND ID < 1) FROM FOO WHERE ID > 0 AND ID < 2";
-    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
-    PlanCompiler compiler = new PlanCompiler(typeFactory);
-    AbstractValuesProcessor proc = compiler.compile(state.tree());
-    Map<String, DataSource> data = new HashMap<>();
-    data.put("FOO", new TestUtils.MockDataSource());
-    List<Values> values = new ArrayList<>();
-    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
-    proc.initialize(data, h);
-    Assert.assertEquals(new Values(true, false, true), values.get(0));
-  }
-
-  @Test
-  public void testNested() throws Exception {
-    String sql = "SELECT ID, MAPFIELD['c'], NESTEDMAPFIELD, ARRAYFIELD " +
-            "FROM FOO " +
-            "WHERE NESTEDMAPFIELD['a']['b'] = 2 AND ARRAYFIELD[2] = 200";
-    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverNestedTable(sql);
-    PlanCompiler compiler = new PlanCompiler(typeFactory);
-    AbstractValuesProcessor proc = compiler.compile(state.tree());
-    Map<String, DataSource> data = new HashMap<>();
-    data.put("FOO", new TestUtils.MockNestedDataSource());
-    List<Values> values = new ArrayList<>();
-    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
-    proc.initialize(data, h);
-    Map<String, Integer> map = ImmutableMap.of("b", 2, "c", 4);
-    Map<String, Map<String, Integer>> nestedMap = ImmutableMap.of("a", map);
-    Assert.assertEquals(new Values(2, 4, nestedMap, Arrays.asList(100, 200, 300)), values.get(0));
-  }
-
-  @Test
-  public void testUdf() throws Exception {
-    String sql = "SELECT MYPLUS(ID, 3)" +
-            "FROM FOO " +
-            "WHERE ID = 2";
-    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverNestedTable(sql);
-    PlanCompiler compiler = new PlanCompiler(typeFactory);
-    AbstractValuesProcessor proc = compiler.compile(state.tree());
-    Map<String, DataSource> data = new HashMap<>();
-    data.put("FOO", new TestUtils.MockDataSource());
-    List<Values> values = new ArrayList<>();
-    ChannelHandler h = new TestUtils.CollectDataChannelHandler(values);
-    proc.initialize(data, h);
-    Assert.assertEquals(new Values(5), values.get(0));
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestRelNodeCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestRelNodeCompiler.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestRelNodeCompiler.java
deleted file mode 100644
index 4bee9aa..0000000
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestRelNodeCompiler.java
+++ /dev/null
@@ -1,64 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.compiler.backends.standalone;
-
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.calcite.rel.logical.LogicalFilter;
-import org.apache.calcite.rel.logical.LogicalProject;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.util.Collections;
-
-import static org.hamcrest.CoreMatchers.containsString;
-
-public class TestRelNodeCompiler {
-  @Test
-  public void testFilter() throws Exception {
-    String sql = "SELECT ID + 1 FROM FOO WHERE ID > 3";
-    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
-    JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(
-        RelDataTypeSystem.DEFAULT);
-    LogicalProject project = (LogicalProject) state.tree();
-    LogicalFilter filter = (LogicalFilter) project.getInput();
-
-    try (StringWriter sw = new StringWriter();
-         PrintWriter pw = new PrintWriter(sw)
-    ) {
-      RelNodeCompiler compiler = new RelNodeCompiler(pw, typeFactory);
-      // standalone mode doesn't use inputstreams argument
-      compiler.visitFilter(filter, Collections.EMPTY_LIST);
-      pw.flush();
-      Assert.assertThat(sw.toString(), containsString("> 3"));
-    }
-
-    try (StringWriter sw = new StringWriter();
-         PrintWriter pw = new PrintWriter(sw)
-    ) {
-      RelNodeCompiler compiler = new RelNodeCompiler(pw, typeFactory);
-      // standalone mode doesn't use inputstreams argument
-      compiler.visitProject(project, Collections.EMPTY_LIST);
-      pw.flush();
-      Assert.assertThat(sw.toString(), containsString(" + 1"));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestCompilerUtils.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestCompilerUtils.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestCompilerUtils.java
deleted file mode 100644
index f6ef1ca..0000000
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestCompilerUtils.java
+++ /dev/null
@@ -1,208 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * 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.sql.compiler.backends.trident;
-
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.schema.StreamableTable;
-import org.apache.calcite.schema.Table;
-import org.apache.calcite.schema.impl.AggregateFunctionImpl;
-import org.apache.calcite.schema.impl.ScalarFunctionImpl;
-import org.apache.calcite.sql.SqlExplainLevel;
-import org.apache.calcite.sql.parser.SqlParseException;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.tools.Frameworks;
-import org.apache.calcite.tools.RelConversionException;
-import org.apache.calcite.tools.ValidationException;
-import org.apache.storm.sql.compiler.CompilerUtil;
-import org.apache.storm.sql.planner.trident.QueryPlanner;
-import org.apache.storm.sql.planner.trident.rel.TridentRel;
-import org.apache.storm.sql.planner.StormRelUtils;
-
-public class TestCompilerUtils {
-
-    public static class MyPlus {
-        public static Integer eval(Integer x, Integer y) {
-            return x + y;
-        }
-    }
-
-    public static class MyStaticSumFunction {
-        public static long init() {
-            return 0L;
-        }
-        public static long add(long accumulator, int v) {
-            return accumulator + v;
-        }
-    }
-
-    public static class MySumFunction {
-        public MySumFunction() {
-        }
-        public long init() {
-            return 0L;
-        }
-        public long add(long accumulator, int v) {
-            return accumulator + v;
-        }
-        public long result(long accumulator) {
-            return accumulator;
-        }
-    }
-
-    public static CalciteState sqlOverDummyTable(String sql)
-            throws RelConversionException, ValidationException, SqlParseException {
-        SchemaPlus schema = Frameworks.createRootSchema(true);
-        JavaTypeFactory typeFactory = new JavaTypeFactoryImpl
-                (RelDataTypeSystem.DEFAULT);
-        StreamableTable streamableTable = new CompilerUtil.TableBuilderInfo(typeFactory)
-                .field("ID", SqlTypeName.INTEGER)
-                .field("NAME", typeFactory.createType(String.class))
-                .field("ADDR", typeFactory.createType(String.class))
-                .build();
-        Table table = streamableTable.stream();
-        schema.add("FOO", table);
-        schema.add("BAR", table);
-        schema.add("MYPLUS", ScalarFunctionImpl.create(MyPlus.class, "eval"));
-
-        QueryPlanner queryPlanner = new QueryPlanner(schema);
-        TridentRel tree = queryPlanner.getPlan(sql);
-        System.out.println(StormRelUtils.explain(tree, SqlExplainLevel.ALL_ATTRIBUTES));
-        return new CalciteState(schema, tree);
-    }
-
-    public static CalciteState sqlOverDummyGroupByTable(String sql)
-            throws RelConversionException, ValidationException, SqlParseException {
-        SchemaPlus schema = Frameworks.createRootSchema(true);
-        JavaTypeFactory typeFactory = new JavaTypeFactoryImpl
-                (RelDataTypeSystem.DEFAULT);
-        StreamableTable streamableTable = new CompilerUtil.TableBuilderInfo(typeFactory)
-                .field("ID", SqlTypeName.INTEGER)
-                .field("GRPID", SqlTypeName.INTEGER)
-                .field("NAME", typeFactory.createType(String.class))
-                .field("ADDR", typeFactory.createType(String.class))
-                .field("AGE", SqlTypeName.INTEGER)
-                .field("SCORE", SqlTypeName.INTEGER)
-                .build();
-        Table table = streamableTable.stream();
-        schema.add("FOO", table);
-        schema.add("BAR", table);
-        schema.add("MYSTATICSUM", AggregateFunctionImpl.create(MyStaticSumFunction.class));
-        schema.add("MYSUM", AggregateFunctionImpl.create(MySumFunction.class));
-
-        QueryPlanner queryPlanner = new QueryPlanner(schema);
-        TridentRel tree = queryPlanner.getPlan(sql);
-        System.out.println(StormRelUtils.explain(tree, SqlExplainLevel.ALL_ATTRIBUTES));
-        return new CalciteState(schema, tree);
-    }
-
-    public static CalciteState sqlOverNestedTable(String sql)
-            throws RelConversionException, ValidationException, SqlParseException {
-        SchemaPlus schema = Frameworks.createRootSchema(true);
-        JavaTypeFactory typeFactory = new JavaTypeFactoryImpl
-                (RelDataTypeSystem.DEFAULT);
-
-        StreamableTable streamableTable = new CompilerUtil.TableBuilderInfo(typeFactory)
-                .field("ID", SqlTypeName.INTEGER)
-                .field("MAPFIELD",
-                        typeFactory.createTypeWithNullability(
-                                typeFactory.createMapType(
-                                        typeFactory.createTypeWithNullability(
-                                                typeFactory.createSqlType(SqlTypeName.VARCHAR), true),
-                                        typeFactory.createTypeWithNullability(
-                                                typeFactory.createSqlType(SqlTypeName.INTEGER), true))
-                                , true))
-                .field("NESTEDMAPFIELD",
-                        typeFactory.createTypeWithNullability(
-                            typeFactory.createMapType(
-                                    typeFactory.createTypeWithNullability(
-                                            typeFactory.createSqlType(SqlTypeName.VARCHAR), true),
-                                    typeFactory.createTypeWithNullability(
-                                            typeFactory.createMapType(
-                                                    typeFactory.createTypeWithNullability(
-                                                            typeFactory.createSqlType(SqlTypeName.VARCHAR), true),
-                                                    typeFactory.createTypeWithNullability(
-                                                            typeFactory.createSqlType(SqlTypeName.INTEGER), true))
-                                            , true))
-                                        , true))
-                .field("ARRAYFIELD", typeFactory.createTypeWithNullability(
-                        typeFactory.createArrayType(
-                            typeFactory.createTypeWithNullability(
-                                typeFactory.createSqlType(SqlTypeName.INTEGER), true), -1L)
-                        , true))
-                .build();
-        Table table = streamableTable.stream();
-        schema.add("FOO", table);
-        schema.add("BAR", table);
-        schema.add("MYPLUS", ScalarFunctionImpl.create(MyPlus.class, "eval"));
-
-        QueryPlanner queryPlanner = new QueryPlanner(schema);
-        TridentRel tree = queryPlanner.getPlan(sql);
-        System.out.println(StormRelUtils.explain(tree, SqlExplainLevel.ALL_ATTRIBUTES));
-        return new CalciteState(schema, tree);
-    }
-
-    public static CalciteState sqlOverSimpleEquiJoinTables(String sql)
-            throws RelConversionException, ValidationException, SqlParseException {
-        SchemaPlus schema = Frameworks.createRootSchema(true);
-        JavaTypeFactory typeFactory = new JavaTypeFactoryImpl
-                (RelDataTypeSystem.DEFAULT);
-
-        StreamableTable streamableTable = new CompilerUtil.TableBuilderInfo(typeFactory)
-                .field("EMPID", SqlTypeName.INTEGER)
-                .field("EMPNAME", SqlTypeName.VARCHAR)
-                .field("DEPTID", SqlTypeName.INTEGER)
-                .build();
-        Table table = streamableTable.stream();
-
-        StreamableTable streamableTable2 = new CompilerUtil.TableBuilderInfo(typeFactory)
-                .field("DEPTID", SqlTypeName.INTEGER)
-                .field("DEPTNAME", SqlTypeName.VARCHAR)
-                .build();
-        Table table2 = streamableTable2.stream();
-
-        schema.add("EMP", table);
-        schema.add("DEPT", table2);
-
-        QueryPlanner queryPlanner = new QueryPlanner(schema);
-        TridentRel tree = queryPlanner.getPlan(sql);
-        System.out.println(StormRelUtils.explain(tree, SqlExplainLevel.ALL_ATTRIBUTES));
-        return new CalciteState(schema, tree);
-    }
-
-    public static class CalciteState {
-        final SchemaPlus schema;
-        final RelNode tree;
-
-        private CalciteState(SchemaPlus schema, RelNode tree) {
-            this.schema = schema;
-            this.tree = tree;
-        }
-
-        public SchemaPlus schema() {
-            return schema;
-        }
-
-        public RelNode tree() {
-            return tree;
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestPlanCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestPlanCompiler.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestPlanCompiler.java
deleted file mode 100644
index 9ba7267..0000000
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestPlanCompiler.java
+++ /dev/null
@@ -1,235 +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
- *  * <p>
- *  * http://www.apache.org/licenses/LICENSE-2.0
- *  * <p>
- *  * 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.sql.compiler.backends.trident;
-
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
-import org.apache.calcite.DataContext;
-import org.apache.calcite.avatica.util.DateTimeUtils;
-import org.apache.commons.collections4.CollectionUtils;
-import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.LocalCluster.LocalTopology;
-import org.apache.storm.sql.TestUtils;
-import org.apache.storm.sql.javac.CompilingClassLoader;
-import org.apache.storm.sql.planner.trident.QueryPlanner;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.sql.AbstractTridentProcessor;
-import org.apache.storm.trident.TridentTopology;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Values;
-import org.apache.storm.utils.Utils;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.time.ZoneOffset;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Callable;
-
-import static org.apache.storm.sql.TestUtils.MockState.getCollectedValues;
-
-public class TestPlanCompiler {
-  private static LocalCluster cluster;
-
-  @BeforeClass
-  public static void staticSetup() throws Exception {
-    cluster = new LocalCluster();
-  }
-
-  @AfterClass
-  public static void staticCleanup() {
-    if (cluster!= null) {
-      cluster.shutdown();
-      cluster = null;
-    }
-  }
-
-  @Before
-  public void setUp() {
-    getCollectedValues().clear();
-  }
-
-  @Test
-  public void testCompile() throws Exception {
-    final int EXPECTED_VALUE_SIZE = 2;
-    String sql = "SELECT ID FROM FOO WHERE ID > 2";
-    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
-    final Map<String, ISqlTridentDataSource> data = new HashMap<>();
-    data.put("FOO", new TestUtils.MockSqlTridentDataSource());
-    QueryPlanner planner = new QueryPlanner(state.schema());
-    AbstractTridentProcessor proc = planner.compile(data, sql);
-    final TridentTopology topo = proc.build();
-    Fields f = proc.outputStream().getOutputFields();
-    proc.outputStream().partitionPersist(new TestUtils.MockStateFactory(),
-            f, new TestUtils.MockStateUpdater(), new Fields());
-    runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo);
-    Assert.assertArrayEquals(new Values[] { new Values(3), new Values(4)}, getCollectedValues().toArray());
-  }
-
-  @Test
-  public void testInsert() throws Exception {
-    final int EXPECTED_VALUE_SIZE = 1;
-    String sql = "INSERT INTO BAR SELECT ID, NAME, ADDR FROM FOO WHERE ID > 3";
-    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
-    final Map<String, ISqlTridentDataSource> data = new HashMap<>();
-    data.put("FOO", new TestUtils.MockSqlTridentDataSource());
-    data.put("BAR", new TestUtils.MockSqlTridentDataSource());
-
-    QueryPlanner planner = new QueryPlanner(state.schema());
-    AbstractTridentProcessor proc = planner.compile(data, sql);
-    final TridentTopology topo = proc.build();
-    runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo);
-    Assert.assertArrayEquals(new Values[] { new Values(4, "abcde", "y")}, getCollectedValues().toArray());
-  }
-
-  @Test
-  public void testUdf() throws Exception {
-    int EXPECTED_VALUE_SIZE = 1;
-    String sql = "SELECT MYPLUS(ID, 3)" +
-            "FROM FOO " +
-            "WHERE ID = 2";
-    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
-    Map<String, ISqlTridentDataSource> data = new HashMap<>();
-    data.put("FOO", new TestUtils.MockSqlTridentDataSource());
-
-    QueryPlanner planner = new QueryPlanner(state.schema());
-    AbstractTridentProcessor proc = planner.compile(data, sql);
-    final TridentTopology topo = proc.build();
-    Fields f = proc.outputStream().getOutputFields();
-    proc.outputStream().partitionPersist(new TestUtils.MockStateFactory(),
-            f, new TestUtils.MockStateUpdater(), new Fields());
-    runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo);
-    Assert.assertArrayEquals(new Values[] { new Values(5) }, getCollectedValues().toArray());
-  }
-
-  @Test
-  public void testCaseStatement() throws Exception {
-    int EXPECTED_VALUE_SIZE = 5;
-    String sql = "SELECT CASE WHEN NAME IN ('a', 'abc', 'abcde') THEN UPPER('a') " +
-            "WHEN UPPER(NAME) = 'AB' THEN 'b' ELSE {fn CONCAT(NAME, '#')} END FROM FOO";
-    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
-
-    final Map<String, ISqlTridentDataSource> data = new HashMap<>();
-    data.put("FOO", new TestUtils.MockSqlTridentDataSource());
-
-    QueryPlanner planner = new QueryPlanner(state.schema());
-    AbstractTridentProcessor proc = planner.compile(data, sql);
-    final TridentTopology topo = proc.build();
-    Fields f = proc.outputStream().getOutputFields();
-    proc.outputStream().partitionPersist(new TestUtils.MockStateFactory(), f, new TestUtils.MockStateUpdater(), new Fields());
-    runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo);
-
-    Assert.assertArrayEquals(new Values[]{new Values("A"), new Values("b"), new Values("A"), new Values("abcd#"), new Values("A")}, getCollectedValues().toArray());
-  }
-
-  @Test
-  public void testNested() throws Exception {
-    int EXPECTED_VALUE_SIZE = 1;
-    String sql = "SELECT ID, MAPFIELD['c'], NESTEDMAPFIELD, ARRAYFIELD " +
-            "FROM FOO " +
-            "WHERE NESTEDMAPFIELD['a']['b'] = 2 AND ARRAYFIELD[2] = 200";
-    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverNestedTable(sql);
-
-    final Map<String, ISqlTridentDataSource> data = new HashMap<>();
-    data.put("FOO", new TestUtils.MockSqlTridentNestedDataSource());
-
-    QueryPlanner planner = new QueryPlanner(state.schema());
-    AbstractTridentProcessor proc = planner.compile(data, sql);
-    final TridentTopology topo = proc.build();
-    Fields f = proc.outputStream().getOutputFields();
-    proc.outputStream().partitionPersist(new TestUtils.MockStateFactory(), f, new TestUtils.MockStateUpdater(), new Fields());
-    runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo);
-
-    Map<String, Integer> map = ImmutableMap.of("b", 2, "c", 4);
-    Map<String, Map<String, Integer>> nestedMap = ImmutableMap.of("a", map);
-    Assert.assertArrayEquals(new Values[]{new Values(2, 4, nestedMap, Arrays.asList(100, 200, 300))}, getCollectedValues().toArray());
-  }
-
-  @Test
-  public void testDateKeywords() throws Exception {
-    int EXPECTED_VALUE_SIZE = 1;
-    String sql = "SELECT " +
-            "LOCALTIME, CURRENT_TIME, LOCALTIMESTAMP, CURRENT_TIMESTAMP, CURRENT_DATE " +
-            "FROM FOO " +
-            "WHERE ID > 0 AND ID < 2";
-    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
-
-    final Map<String, ISqlTridentDataSource> data = new HashMap<>();
-    data.put("FOO", new TestUtils.MockSqlTridentDataSource());
-    QueryPlanner planner = new QueryPlanner(state.schema());
-    AbstractTridentProcessor proc = planner.compile(data, sql);
-    final DataContext dataContext = proc.getDataContext();
-    final TridentTopology topo = proc.build();
-    Fields f = proc.outputStream().getOutputFields();
-    proc.outputStream().partitionPersist(new TestUtils.MockStateFactory(), f, new TestUtils.MockStateUpdater(), new Fields());
-    runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo);
-
-    long utcTimestamp = (long) dataContext.get(DataContext.Variable.UTC_TIMESTAMP.camelName);
-    long currentTimestamp = (long) dataContext.get(DataContext.Variable.CURRENT_TIMESTAMP.camelName);
-    long localTimestamp = (long) dataContext.get(DataContext.Variable.LOCAL_TIMESTAMP.camelName);
-
-    System.out.println(getCollectedValues());
-
-    java.sql.Timestamp timestamp = new java.sql.Timestamp(utcTimestamp);
-    int dateInt = (int) timestamp.toLocalDateTime().atOffset(ZoneOffset.UTC).toLocalDate().toEpochDay();
-    int localTimeInt = (int) (localTimestamp % DateTimeUtils.MILLIS_PER_DAY);
-    int currentTimeInt = (int) (currentTimestamp % DateTimeUtils.MILLIS_PER_DAY);
-
-    Assert.assertArrayEquals(new Values[]{new Values(localTimeInt, currentTimeInt, localTimestamp, currentTimestamp, dateInt)}, getCollectedValues().toArray());
-  }
-
-  private void runTridentTopology(final int expectedValueSize, AbstractTridentProcessor proc,
-                                  TridentTopology topo) throws Exception {
-    final Config conf = new Config();
-    conf.setMaxSpoutPending(20);
-
-    if (proc.getClassLoaders() != null && proc.getClassLoaders().size() > 0) {
-      CompilingClassLoader lastClassloader = proc.getClassLoaders().get(proc.getClassLoaders().size() - 1);
-      Utils.setClassLoaderForJavaDeSerialize(lastClassloader);
-    }
-
-    try (LocalTopology stormTopo = cluster.submitTopology("storm-sql", conf, topo.build())) {
-      waitForCompletion(1000 * 1000, new Callable<Boolean>() {
-        @Override
-        public Boolean call() throws Exception {
-          return getCollectedValues().size() < expectedValueSize;
-        }
-      });
-    } finally {
-      while(cluster.getClusterInfo().get_topologies_size() > 0) {
-        Thread.sleep(10);
-      }
-      Utils.resetClassLoaderForJavaDeSerialize();
-    }
-  }
-
-  private void waitForCompletion(long timeout, Callable<Boolean> cond) throws Exception {
-    long start = TestUtils.monotonicNow();
-    while (TestUtils.monotonicNow() - start < timeout && cond.call()) {
-      Thread.sleep(100);
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/test/org/apache/storm/sql/parser/TestSqlParser.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/parser/TestSqlParser.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/parser/TestSqlParser.java
deleted file mode 100644
index 68054d8..0000000
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/parser/TestSqlParser.java
+++ /dev/null
@@ -1,54 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.parser;
-
-import org.apache.calcite.sql.SqlNode;
-import org.apache.storm.sql.parser.impl.ParseException;
-import org.junit.Test;
-
-public class TestSqlParser {
-  @Test
-  public void testCreateTable() throws Exception {
-    String sql = "CREATE EXTERNAL TABLE foo (bar INT) LOCATION 'kafka:///foo'";
-    parse(sql);
-  }
-
-  @Test
-  public void testCreateTableWithPrimaryKey() throws Exception {
-    String sql = "CREATE EXTERNAL TABLE foo (bar INT PRIMARY KEY ASC) LOCATION 'kafka:///foo'";
-    parse(sql);
-  }
-
-  @Test(expected = ParseException.class)
-  public void testCreateTableWithoutLocation() throws Exception {
-    String sql = "CREATE EXTERNAL TABLE foo (bar INT)";
-    parse(sql);
-  }
-
-  @Test
-  public void testCreateFunction() throws Exception {
-    String sql = "CREATE FUNCTION foo AS 'org.apache.storm.sql.MyUDF'";
-    parse(sql);
-  }
-
-  private static SqlNode parse(String sql) throws Exception {
-    StormParser parser = new StormParser(sql);
-    return parser.impl().parseSqlStmtEof();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-hdfs/pom.xml b/external/sql/storm-sql-external/storm-sql-hdfs/pom.xml
deleted file mode 100644
index 9e0a599..0000000
--- a/external/sql/storm-sql-external/storm-sql-hdfs/pom.xml
+++ /dev/null
@@ -1,104 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements.  See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License.  You may obtain a copy of the License at
-
-     http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/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>
-        <relativePath>../../../../pom.xml</relativePath>
-    </parent>
-
-    <artifactId>storm-sql-hdfs</artifactId>
-
-    <developers>
-        <developer>
-            <id>vesense</id>
-            <name>Xin Wang</name>
-            <email>data.xinwang@gmail.com</email>
-        </developer>
-    </developers>
-
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>storm-core</artifactId>
-            <version>${project.version}</version>
-            <scope>provided</scope>
-            <exclusions>
-                <!--log4j-over-slf4j must be excluded for hadoop-minicluster
-                    see: http://stackoverflow.com/q/20469026/3542091 -->
-                <exclusion>
-                    <groupId>org.slf4j</groupId>
-                    <artifactId>log4j-over-slf4j</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>storm-sql-runtime</artifactId>
-            <version>${project.version}</version>
-            <scope>provided</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>storm-sql-runtime</artifactId>
-            <version>${project.version}</version>
-            <scope>test</scope>
-            <type>test-jar</type>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>storm-hdfs</artifactId>
-            <version>${project.version}</version>
-            <scope>provided</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-minicluster</artifactId>
-            <version>${hadoop.version}</version>
-            <exclusions>
-                <exclusion>
-                    <groupId>org.slf4j</groupId>
-                    <artifactId>slf4j-log4j12</artifactId>
-                </exclusion>
-            </exclusions>
-            <scope>test</scope>
-        </dependency>
-        <dependency>
-            <groupId>junit</groupId>
-            <artifactId>junit</artifactId>
-            <scope>test</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.mockito</groupId>
-            <artifactId>mockito-all</artifactId>
-            <scope>test</scope>
-        </dependency>
-    </dependencies>
-    <build>
-        <sourceDirectory>src/jvm</sourceDirectory>
-        <testSourceDirectory>src/test</testSourceDirectory>
-        <resources>
-            <resource>
-                <directory>${basedir}/src/resources</directory>
-            </resource>
-        </resources>
-    </build>
-</project>

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-hdfs/src/jvm/org/apache/storm/sql/hdfs/HdfsDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-hdfs/src/jvm/org/apache/storm/sql/hdfs/HdfsDataSourcesProvider.java b/external/sql/storm-sql-external/storm-sql-hdfs/src/jvm/org/apache/storm/sql/hdfs/HdfsDataSourcesProvider.java
deleted file mode 100644
index 38c3fcb..0000000
--- a/external/sql/storm-sql-external/storm-sql-hdfs/src/jvm/org/apache/storm/sql/hdfs/HdfsDataSourcesProvider.java
+++ /dev/null
@@ -1,135 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.hdfs;
-
-import com.google.common.base.Preconditions;
-import org.apache.storm.hdfs.trident.HdfsState;
-import org.apache.storm.hdfs.trident.HdfsStateFactory;
-import org.apache.storm.hdfs.trident.HdfsUpdater;
-import org.apache.storm.hdfs.trident.format.FileNameFormat;
-import org.apache.storm.hdfs.trident.format.RecordFormat;
-import org.apache.storm.hdfs.trident.format.SimpleFileNameFormat;
-import org.apache.storm.hdfs.trident.rotation.FileRotationPolicy;
-import org.apache.storm.hdfs.trident.rotation.FileSizeRotationPolicy;
-import org.apache.storm.hdfs.trident.rotation.TimedRotationPolicy;
-import org.apache.storm.sql.runtime.DataSource;
-import org.apache.storm.sql.runtime.DataSourcesProvider;
-import org.apache.storm.sql.runtime.FieldInfo;
-import org.apache.storm.sql.runtime.IOutputSerializer;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.sql.runtime.SimpleSqlTridentConsumer;
-import org.apache.storm.sql.runtime.utils.FieldInfoUtils;
-import org.apache.storm.sql.runtime.utils.SerdeUtils;
-import org.apache.storm.trident.spout.ITridentDataSource;
-import org.apache.storm.trident.state.StateFactory;
-import org.apache.storm.trident.state.StateUpdater;
-import org.apache.storm.trident.tuple.TridentTuple;
-
-import java.net.URI;
-import java.util.List;
-import java.util.Properties;
-
-/**
- * Create a HDFS sink based on the URI and properties. The URI has the format of hdfs://host:port/path-to-file
- * The properties are in JSON format which specifies the name / path of the hdfs file and etc.
- */
-public class HdfsDataSourcesProvider implements DataSourcesProvider {
-
-  private static class HdfsTridentDataSource implements ISqlTridentDataSource {
-    private final String url;
-    private final Properties props;
-    private final IOutputSerializer serializer;
-
-    private HdfsTridentDataSource(String url, Properties props, IOutputSerializer serializer) {
-      this.url = url;
-      this.props = props;
-      this.serializer = serializer;
-    }
-
-    @Override
-    public ITridentDataSource getProducer() {
-      throw new UnsupportedOperationException(this.getClass().getName() + " doesn't provide Producer");
-    }
-
-    @Override
-    public SqlTridentConsumer getConsumer() {
-      FileNameFormat fileNameFormat = new SimpleFileNameFormat()
-          .withPath(props.getProperty("hdfs.file.path", "/storm"))
-          .withName(props.getProperty("hdfs.file.name", "$TIME.$NUM.txt"));
-
-      RecordFormat recordFormat = new TridentRecordFormat(serializer);
-
-      FileRotationPolicy rotationPolicy;
-      String size = props.getProperty("hdfs.rotation.size.kb");
-      String interval = props.getProperty("hdfs.rotation.time.seconds");
-      Preconditions.checkArgument(size != null || interval != null, "Hdfs data source must contain file rotation config");
-
-      if (size != null) {
-        rotationPolicy = new FileSizeRotationPolicy(Float.parseFloat(size), FileSizeRotationPolicy.Units.KB);
-      } else {
-        rotationPolicy = new TimedRotationPolicy(Float.parseFloat(interval), TimedRotationPolicy.TimeUnit.SECONDS);
-      }
-
-      HdfsState.Options options = new HdfsState.HdfsFileOptions()
-          .withFileNameFormat(fileNameFormat)
-          .withRecordFormat(recordFormat)
-          .withRotationPolicy(rotationPolicy)
-          .withFsUrl(url);
-
-      StateFactory stateFactory = new HdfsStateFactory().withOptions(options);
-      StateUpdater stateUpdater = new HdfsUpdater();
-
-      return new SimpleSqlTridentConsumer(stateFactory, stateUpdater);
-    }
-  }
-
-  private static class TridentRecordFormat implements RecordFormat {
-    private final IOutputSerializer serializer;
-
-    private TridentRecordFormat(IOutputSerializer serializer) {
-      this.serializer = serializer;
-    }
-
-    @Override
-    public byte[] format(TridentTuple tuple) {
-      //TODO we should handle '\n'. ref DelimitedRecordFormat
-      return serializer.write(tuple.getValues(), null).array();
-    }
-
-  }
-
-  @Override
-  public String scheme() {
-    return "hdfs";
-  }
-
-  @Override
-  public DataSource construct(URI uri, String inputFormatClass, String outputFormatClass,
-                              List<FieldInfo> fields) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
-                                                Properties properties, List<FieldInfo> fields) {
-    List<String> fieldNames = FieldInfoUtils.getFieldNames(fields);
-    IOutputSerializer serializer = SerdeUtils.getSerializer(outputFormatClass, properties, fieldNames);
-    return new HdfsTridentDataSource(uri.toString(), properties, serializer);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-hdfs/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-hdfs/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider b/external/sql/storm-sql-external/storm-sql-hdfs/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
deleted file mode 100644
index 5fac84f..0000000
--- a/external/sql/storm-sql-external/storm-sql-hdfs/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
+++ /dev/null
@@ -1,16 +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.
-
-org.apache.storm.sql.hdfs.HdfsDataSourcesProvider
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-hdfs/src/test/org/apache/storm/sql/hdfs/TestHdfsDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-hdfs/src/test/org/apache/storm/sql/hdfs/TestHdfsDataSourcesProvider.java b/external/sql/storm-sql-external/storm-sql-hdfs/src/test/org/apache/storm/sql/hdfs/TestHdfsDataSourcesProvider.java
deleted file mode 100644
index 1473438..0000000
--- a/external/sql/storm-sql-external/storm-sql-hdfs/src/test/org/apache/storm/sql/hdfs/TestHdfsDataSourcesProvider.java
+++ /dev/null
@@ -1,129 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.hdfs;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.storm.hdfs.trident.HdfsState;
-import org.apache.storm.hdfs.trident.HdfsStateFactory;
-import org.apache.storm.hdfs.trident.HdfsUpdater;
-import org.apache.storm.sql.runtime.DataSourcesRegistry;
-import org.apache.storm.sql.runtime.FieldInfo;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.trident.state.StateUpdater;
-import org.apache.storm.trident.tuple.TridentTuple;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.internal.util.reflection.Whitebox;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Properties;
-
-import static org.apache.storm.hdfs.trident.HdfsState.HdfsFileOptions;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-
-public class TestHdfsDataSourcesProvider {
-  private static final List<FieldInfo> FIELDS = ImmutableList.of(
-      new FieldInfo("ID", int.class, true),
-      new FieldInfo("val", String.class, false));
-  private static final Properties TBL_PROPERTIES = new Properties();
-
-  private static String hdfsURI;
-  private static MiniDFSCluster hdfsCluster;
-
-  static {
-    TBL_PROPERTIES.put("hdfs.file.path", "/unittest");
-    TBL_PROPERTIES.put("hdfs.file.name", "test1.txt");
-    TBL_PROPERTIES.put("hdfs.rotation.time.seconds", "120");
-  }
-
-  @Before
-  public void setup() throws Exception {
-    Configuration conf = new Configuration();
-    conf.set("fs.trash.interval", "10");
-    conf.setBoolean("dfs.permissions", true);
-    File baseDir = new File("./target/hdfs/").getAbsoluteFile();
-    FileUtil.fullyDelete(baseDir);
-    conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath());
-
-    MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
-    hdfsCluster = builder.build();
-    hdfsURI = "hdfs://localhost:" + hdfsCluster.getNameNodePort() + "/";
-  }
-
-  @After
-  public void shutDown() throws IOException {
-    hdfsCluster.shutdown();
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testHdfsSink() {
-    ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(
-            URI.create(hdfsURI), null, null, TBL_PROPERTIES, FIELDS);
-    Assert.assertNotNull(ds);
-
-    ISqlTridentDataSource.SqlTridentConsumer consumer = ds.getConsumer();
-
-    Assert.assertEquals(HdfsStateFactory.class, consumer.getStateFactory().getClass());
-    Assert.assertEquals(HdfsUpdater.class, consumer.getStateUpdater().getClass());
-
-    HdfsState state = (HdfsState) consumer.getStateFactory().makeState(Collections.emptyMap(), null, 0, 1);
-    StateUpdater stateUpdater = consumer.getStateUpdater();
-
-    HdfsFileOptions options = mock(HdfsFileOptions.class);
-    Whitebox.setInternalState(state, "options", options);
-
-    List<TridentTuple> tupleList = mockTupleList();
-
-    for (TridentTuple t : tupleList) {
-      stateUpdater.updateState(state, Collections.singletonList(t), null);
-      try {
-        verify(options).execute(Collections.singletonList(t));
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-    }
-  }
-
-  private static List<TridentTuple> mockTupleList() {
-    List<TridentTuple> tupleList = new ArrayList<>();
-    TridentTuple t0 = mock(TridentTuple.class);
-    TridentTuple t1 = mock(TridentTuple.class);
-    doReturn(1).when(t0).get(0);
-    doReturn(2).when(t1).get(0);
-    doReturn(Lists.<Object>newArrayList(1, "2")).when(t0).getValues();
-    doReturn(Lists.<Object>newArrayList(2, "3")).when(t1).getValues();
-    tupleList.add(t0);
-    tupleList.add(t1);
-    return tupleList;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-external/storm-sql-kafka/pom.xml
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-external/storm-sql-kafka/pom.xml b/external/sql/storm-sql-external/storm-sql-kafka/pom.xml
deleted file mode 100644
index a7bdf7a..0000000
--- a/external/sql/storm-sql-external/storm-sql-kafka/pom.xml
+++ /dev/null
@@ -1,93 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements.  See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License.  You may obtain a copy of the License at
-
-     http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/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>
-        <relativePath>../../../../pom.xml</relativePath>
-    </parent>
-
-    <artifactId>storm-sql-kafka</artifactId>
-
-    <developers>
-        <developer>
-            <id>haohui</id>
-            <name>Haohui Mai</name>
-            <email>ricetons@gmail.com</email>
-        </developer>
-    </developers>
-
-    <dependencies>
-        <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-sql-runtime</artifactId>
-            <version>${project.version}</version>
-            <scope>provided</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>storm-sql-runtime</artifactId>
-            <version>${project.version}</version>
-            <scope>test</scope>
-            <type>test-jar</type>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>storm-kafka</artifactId>
-            <version>${project.version}</version>
-            <scope>provided</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.kafka</groupId>
-            <artifactId>${storm.kafka.artifact.id}</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.kafka</groupId>
-            <artifactId>kafka-clients</artifactId>
-            <version>${storm.kafka.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>junit</groupId>
-            <artifactId>junit</artifactId>
-            <scope>test</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.mockito</groupId>
-            <artifactId>mockito-all</artifactId>
-            <scope>test</scope>
-        </dependency>
-    </dependencies>
-    <build>
-        <sourceDirectory>src/jvm</sourceDirectory>
-        <testSourceDirectory>src/test</testSourceDirectory>
-        <resources>
-            <resource>
-                <directory>${basedir}/src/resources</directory>
-            </resource>
-        </resources>
-    </build>
-</project>


[02/23] storm git commit: STORM-2453 Move non-connectors into the top directory

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/Channels.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/Channels.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/Channels.java
new file mode 100644
index 0000000..3b5eedd
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/Channels.java
@@ -0,0 +1,109 @@
+/**
+ * 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.sql.runtime;
+
+import org.apache.storm.tuple.Values;
+
+public class Channels {
+  private static final ChannelContext VOID_CTX = new ChannelContext() {
+    @Override
+    public void emit(Values data) {}
+
+    @Override
+    public void fireChannelInactive() {}
+
+    @Override
+    public void flush() {
+
+    }
+
+    @Override
+    public void setSource(java.lang.Object source) {
+
+    }
+  };
+
+  private static class ChannelContextAdapter implements ChannelContext {
+    private final ChannelHandler handler;
+    private final ChannelContext next;
+
+    public ChannelContextAdapter(
+        ChannelContext next, ChannelHandler handler) {
+      this.handler = handler;
+      this.next = next;
+    }
+
+    @Override
+    public void emit(Values data) {
+      handler.dataReceived(next, data);
+    }
+
+    @Override
+    public void fireChannelInactive() {
+      handler.channelInactive(next);
+    }
+
+    @Override
+    public void flush() {
+      handler.flush(next);
+    }
+
+    @Override
+    public void setSource(java.lang.Object source) {
+      handler.setSource(next, source);
+      next.setSource(source); // propagate through the chain
+    }
+  }
+
+  private static class ForwardingChannelContext implements ChannelContext {
+    private final ChannelContext next;
+
+    public ForwardingChannelContext(ChannelContext next) {
+      this.next = next;
+    }
+
+    @Override
+    public void emit(Values data) {
+      next.emit(data);
+    }
+
+    @Override
+    public void fireChannelInactive() {
+      next.fireChannelInactive();
+    }
+
+    @Override
+    public void flush() {
+      next.flush();
+    }
+
+    @Override
+    public void setSource(Object source) {
+      next.setSource(source);
+    }
+  }
+
+  public static ChannelContext chain(
+      ChannelContext next, ChannelHandler handler) {
+    return new ChannelContextAdapter(next, handler);
+  }
+
+  public static ChannelContext voidContext() {
+    return VOID_CTX;
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSource.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSource.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSource.java
new file mode 100644
index 0000000..352af73
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSource.java
@@ -0,0 +1,27 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.sql.runtime;
+
+/**
+ * A DataSource ingests data in StormSQL. It provides a series of tuple to
+ * the downstream {@link ChannelHandler}.
+ *
+ */
+public interface DataSource {
+  void open(ChannelContext ctx);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesProvider.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesProvider.java
new file mode 100644
index 0000000..dbece9c
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesProvider.java
@@ -0,0 +1,47 @@
+/**
+ * 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.sql.runtime;
+
+import java.net.URI;
+import java.util.List;
+import java.util.Properties;
+
+public interface DataSourcesProvider {
+  /**
+   * @return the scheme of the data source
+   */
+  String scheme();
+
+  /**
+   * Construct a new data source.
+   * @param uri The URI that specifies the data source. The format of the URI
+   *            is fully customizable.
+   * @param inputFormatClass the name of the class that deserializes data.
+   *                         It is null when unspecified.
+   * @param outputFormatClass the name of the class that serializes data. It
+   *                          is null when unspecified.
+   * @param fields The name of the fields and the schema of the table.
+   */
+  DataSource construct(
+      URI uri, String inputFormatClass, String outputFormatClass,
+      List<FieldInfo> fields);
+
+  ISqlTridentDataSource constructTrident(
+          URI uri, String inputFormatClass, String outputFormatClass,
+          Properties properties, List<FieldInfo> fields);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesRegistry.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesRegistry.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesRegistry.java
new file mode 100644
index 0000000..dfefb01
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesRegistry.java
@@ -0,0 +1,76 @@
+/**
+ * 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.sql.runtime;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URI;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.ServiceLoader;
+
+public class DataSourcesRegistry {
+  private static final Logger LOG = LoggerFactory.getLogger(
+      DataSourcesRegistry.class);
+  private static final Map<String, DataSourcesProvider> providers;
+
+  static {
+    providers = new HashMap<>();
+    ServiceLoader<DataSourcesProvider> loader = ServiceLoader.load(
+        DataSourcesProvider.class);
+    for (DataSourcesProvider p : loader) {
+      LOG.info("Registering scheme {} with {}", p.scheme(), p);
+      providers.put(p.scheme(), p);
+    }
+  }
+
+  private DataSourcesRegistry() {
+  }
+
+  public static DataSource construct(
+      URI uri, String inputFormatClass, String outputFormatClass,
+      List<FieldInfo> fields) {
+    DataSourcesProvider provider = providers.get(uri.getScheme());
+    if (provider == null) {
+      return null;
+    }
+
+    return provider.construct(uri, inputFormatClass, outputFormatClass, fields);
+  }
+
+  public static ISqlTridentDataSource constructTridentDataSource(
+          URI uri, String inputFormatClass, String outputFormatClass,
+          Properties properties, List<FieldInfo> fields) {
+    DataSourcesProvider provider = providers.get(uri.getScheme());
+    if (provider == null) {
+      return null;
+    }
+
+    return provider.constructTrident(uri, inputFormatClass, outputFormatClass, properties, fields);
+  }
+
+  /**
+   * Allow unit tests to inject data sources.
+   */
+  public static Map<String, DataSourcesProvider> providerMap() {
+    return providers;
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/FieldInfo.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/FieldInfo.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/FieldInfo.java
new file mode 100644
index 0000000..03b030b
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/FieldInfo.java
@@ -0,0 +1,47 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.runtime;
+
+import java.io.Serializable;
+
+/**
+ * Describe each column of the field
+ */
+public class FieldInfo implements Serializable {
+  private final String name;
+  private final Class<?> type;
+  private final boolean isPrimary;
+
+  public FieldInfo(String name, Class<?> type, boolean isPrimary) {
+    this.name = name;
+    this.type = type;
+    this.isPrimary = isPrimary;
+  }
+
+  public String name() {
+    return name;
+  }
+
+  public Class<?> type() {
+    return type;
+  }
+
+  public boolean isPrimary() {
+    return isPrimary;
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/IOutputSerializer.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/IOutputSerializer.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/IOutputSerializer.java
new file mode 100644
index 0000000..b6670d9
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/IOutputSerializer.java
@@ -0,0 +1,31 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.runtime;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+public interface IOutputSerializer {
+  /**
+   * Serialize the data to a ByteBuffer. The caller can pass in a ByteBuffer so that the serializer can reuse the
+   * memory.
+   *
+   * @return A ByteBuffer contains the serialized result.
+   */
+  ByteBuffer write(List<Object> data, ByteBuffer buffer);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ISqlTridentDataSource.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ISqlTridentDataSource.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ISqlTridentDataSource.java
new file mode 100644
index 0000000..9eae5ae
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ISqlTridentDataSource.java
@@ -0,0 +1,65 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.runtime;
+
+import org.apache.storm.trident.spout.ITridentDataSource;
+import org.apache.storm.trident.state.StateFactory;
+import org.apache.storm.trident.state.StateUpdater;
+
+/**
+ * A ISqlTridentDataSource specifies how an external data source produces and consumes data.
+ */
+public interface ISqlTridentDataSource {
+  /**
+   * SqlTridentConsumer is a data structure containing StateFactory and StateUpdater for consuming tuples with State.
+   *
+   * Please note that StateFactory and StateUpdater should use same class which implements State.
+   *
+   * @see org.apache.storm.trident.state.StateFactory
+   * @see org.apache.storm.trident.state.StateUpdater
+   */
+  interface SqlTridentConsumer {
+    StateFactory getStateFactory();
+    StateUpdater getStateUpdater();
+  }
+
+  /**
+   * Provides instance of ITridentDataSource which can be used as producer in Trident.
+   *
+   * Since ITridentDataSource is a marker interface for Trident Spout interfaces, this method should effectively
+   * return an instance of one of these interfaces (can be changed if Trident API evolves) or descendants:
+   * - IBatchSpout
+   * - ITridentSpout
+   * - IPartitionedTridentSpout
+   * - IOpaquePartitionedTridentSpout
+   *
+   * @see org.apache.storm.trident.spout.ITridentDataSource
+   * @see org.apache.storm.trident.spout.IBatchSpout
+   * @see org.apache.storm.trident.spout.ITridentSpout
+   * @see org.apache.storm.trident.spout.IPartitionedTridentSpout
+   * @see org.apache.storm.trident.spout.IOpaquePartitionedTridentSpout
+   */
+  ITridentDataSource getProducer();
+
+  /**
+   * Provides instance of SqlTridentConsumer which can be used as consumer (State) in Trident.
+   *
+   * @see SqlTridentConsumer
+   */
+  SqlTridentConsumer getConsumer();
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/SimpleSqlTridentConsumer.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/SimpleSqlTridentConsumer.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/SimpleSqlTridentConsumer.java
new file mode 100644
index 0000000..c9abd16
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/SimpleSqlTridentConsumer.java
@@ -0,0 +1,41 @@
+/**
+ * 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.sql.runtime;
+
+import org.apache.storm.trident.state.StateFactory;
+import org.apache.storm.trident.state.StateUpdater;
+
+public class SimpleSqlTridentConsumer implements ISqlTridentDataSource.SqlTridentConsumer {
+    private final StateFactory stateFactory;
+    private final StateUpdater stateUpdater;
+
+    public SimpleSqlTridentConsumer(StateFactory stateFactory, StateUpdater stateUpdater) {
+        this.stateFactory = stateFactory;
+        this.stateUpdater = stateUpdater;
+    }
+
+    @Override
+    public StateFactory getStateFactory() {
+        return stateFactory;
+    }
+
+    @Override
+    public StateUpdater getStateUpdater() {
+        return stateUpdater;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/StormSqlFunctions.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/StormSqlFunctions.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/StormSqlFunctions.java
new file mode 100644
index 0000000..a373483
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/StormSqlFunctions.java
@@ -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.
+ */
+package org.apache.storm.sql.runtime;
+
+public class StormSqlFunctions {
+  public static Boolean eq(Object b0, Object b1) {
+    if (b0 == null || b1 == null) {
+      return null;
+    }
+    return b0.equals(b1);
+  }
+
+  public static Boolean ne(Object b0, Object b1) {
+    if (b0 == null || b1 == null) {
+      return null;
+    }
+    return !b0.equals(b1);
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/DebuggableExecutableExpression.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/DebuggableExecutableExpression.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/DebuggableExecutableExpression.java
new file mode 100644
index 0000000..e78f354
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/DebuggableExecutableExpression.java
@@ -0,0 +1,45 @@
+/**
+ * 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.sql.runtime.calcite;
+
+import org.apache.calcite.interpreter.Context;
+
+public class DebuggableExecutableExpression implements ExecutableExpression {
+    private ExecutableExpression delegate;
+    private String delegateCode;
+
+    public DebuggableExecutableExpression(ExecutableExpression delegate, String delegateCode) {
+        this.delegate = delegate;
+        this.delegateCode = delegateCode;
+    }
+
+    @Override
+    public Object execute(Context context) {
+        return delegate.execute(context);
+    }
+
+    @Override
+    public void execute(Context context, Object[] results) {
+        delegate.execute(context, results);
+    }
+
+    public String getDelegateCode() {
+        return delegateCode;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/ExecutableExpression.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/ExecutableExpression.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/ExecutableExpression.java
new file mode 100644
index 0000000..8416945
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/ExecutableExpression.java
@@ -0,0 +1,31 @@
+/**
+ * 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.sql.runtime.calcite;
+
+import org.apache.calcite.interpreter.Context;
+
+import java.io.Serializable;
+
+/**
+ * Compiled executable expression.
+ */
+public interface ExecutableExpression extends Serializable {
+    Object execute(Context context);
+    void execute(Context context, Object[] results);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/StormDataContext.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/StormDataContext.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/StormDataContext.java
new file mode 100644
index 0000000..4861b43
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/calcite/StormDataContext.java
@@ -0,0 +1,79 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.runtime.calcite;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.util.Holder;
+
+import java.io.Serializable;
+import java.util.Calendar;
+import java.util.TimeZone;
+
+/**
+ * This is based on SlimDataContext in Calcite, and borrow some from DataContextImpl in Calcite.
+ */
+public class StormDataContext implements DataContext, Serializable {
+    private final ImmutableMap<Object, Object> map;
+
+    public StormDataContext() {
+        // Store the time at which the query started executing. The SQL
+        // standard says that functions such as CURRENT_TIMESTAMP return the
+        // same value throughout the query.
+
+        final Holder<Long> timeHolder = Holder.of(System.currentTimeMillis());
+
+        // Give a hook chance to alter the clock.
+        Hook.CURRENT_TIME.run(timeHolder);
+        final long time = timeHolder.get();
+        final TimeZone timeZone = Calendar.getInstance().getTimeZone();
+        final long localOffset = timeZone.getOffset(time);
+        final long currentOffset = localOffset;
+
+        ImmutableMap.Builder<Object, Object> builder = ImmutableMap.builder();
+        builder.put(Variable.UTC_TIMESTAMP.camelName, time)
+                .put(Variable.CURRENT_TIMESTAMP.camelName, time + currentOffset)
+                .put(Variable.LOCAL_TIMESTAMP.camelName, time + localOffset)
+                .put(Variable.TIME_ZONE.camelName, timeZone);
+        map = builder.build();
+    }
+
+    @Override
+    public SchemaPlus getRootSchema() {
+        return null;
+    }
+
+    @Override
+    public JavaTypeFactory getTypeFactory() {
+        return null;
+    }
+
+    @Override
+    public QueryProvider getQueryProvider() {
+        return null;
+    }
+
+    @Override
+    public Object get(String name) {
+        return map.get(name);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/SocketDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/SocketDataSourcesProvider.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/SocketDataSourcesProvider.java
new file mode 100644
index 0000000..0e65220
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/SocketDataSourcesProvider.java
@@ -0,0 +1,100 @@
+/**
+ * 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.sql.runtime.datasource.socket;
+
+import org.apache.storm.spout.Scheme;
+import org.apache.storm.sql.runtime.DataSource;
+import org.apache.storm.sql.runtime.DataSourcesProvider;
+import org.apache.storm.sql.runtime.FieldInfo;
+import org.apache.storm.sql.runtime.IOutputSerializer;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.sql.runtime.SimpleSqlTridentConsumer;
+import org.apache.storm.sql.runtime.datasource.socket.trident.SocketState;
+import org.apache.storm.sql.runtime.datasource.socket.trident.SocketStateUpdater;
+import org.apache.storm.sql.runtime.datasource.socket.trident.TridentSocketSpout;
+import org.apache.storm.sql.runtime.utils.FieldInfoUtils;
+import org.apache.storm.sql.runtime.utils.SerdeUtils;
+import org.apache.storm.trident.spout.ITridentDataSource;
+import org.apache.storm.trident.state.StateFactory;
+import org.apache.storm.trident.state.StateUpdater;
+
+import java.net.URI;
+import java.util.List;
+import java.util.Properties;
+
+/**
+ * Create a Socket data source based on the URI and properties. The URI has the format of
+ * socket://[host]:[port]. Both of host and port are mandatory.
+ *
+ * Note that it connects to given host and port, and receive the message if it's used for input source,
+ * and send the message if it's used for output data source.
+ */
+public class SocketDataSourcesProvider implements DataSourcesProvider {
+    @Override
+    public String scheme() {
+        return "socket";
+    }
+
+    private static class SocketTridentDataSource implements ISqlTridentDataSource {
+
+        private final String host;
+        private final int port;
+        private final Scheme scheme;
+        private final IOutputSerializer serializer;
+
+        SocketTridentDataSource(Scheme scheme, IOutputSerializer serializer, String host, int port) {
+            this.scheme = scheme;
+            this.serializer = serializer;
+            this.host = host;
+            this.port = port;
+        }
+
+        @Override
+        public ITridentDataSource getProducer() {
+            return new TridentSocketSpout(scheme, host, port);
+        }
+
+        @Override
+        public SqlTridentConsumer getConsumer() {
+            StateFactory stateFactory = new SocketState.Factory(host, port);
+            StateUpdater<SocketState> stateUpdater = new SocketStateUpdater(serializer);
+            return new SimpleSqlTridentConsumer(stateFactory, stateUpdater);
+        }
+    }
+
+    @Override
+    public DataSource construct(URI uri, String inputFormatClass, String outputFormatClass, List<FieldInfo> fields) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass, Properties properties, List<FieldInfo> fields) {
+        String host = uri.getHost();
+        int port = uri.getPort();
+        if (port == -1) {
+            throw new RuntimeException("Port information is not available. URI: " + uri);
+        }
+
+        List<String> fieldNames = FieldInfoUtils.getFieldNames(fields);
+        Scheme scheme = SerdeUtils.getScheme(inputFormatClass, properties, fieldNames);
+        IOutputSerializer serializer = SerdeUtils.getSerializer(outputFormatClass, properties, fieldNames);
+
+        return new SocketTridentDataSource(scheme, serializer, host, port);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketState.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketState.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketState.java
new file mode 100644
index 0000000..3f85756
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketState.java
@@ -0,0 +1,89 @@
+/**
+ * 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.sql.runtime.datasource.socket.trident;
+
+import org.apache.storm.task.IMetricsContext;
+import org.apache.storm.trident.state.State;
+import org.apache.storm.trident.state.StateFactory;
+
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.net.Socket;
+import java.util.Map;
+
+/**
+ * Trident State implementation of Socket. It only supports writing.
+ */
+public class SocketState implements State {
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public void beginCommit(Long txid) {
+        // no-op
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public void commit(Long txid) {
+        // no-op
+    }
+
+    public static class Factory implements StateFactory {
+        private final String host;
+        private final int port;
+
+        public Factory(String host, int port) {
+            this.host = host;
+            this.port = port;
+        }
+
+        @Override
+        public State makeState(Map conf, IMetricsContext metrics, int partitionIndex, int numPartitions) {
+            BufferedWriter out;
+            try {
+                Socket socket = new Socket(host, port);
+                out = new BufferedWriter(new OutputStreamWriter(socket.getOutputStream()));
+            } catch (IOException e) {
+                throw new RuntimeException("Exception while initializing socket for State. host " +
+                        host + " port " + port, e);
+            }
+
+            // State doesn't have close() and Storm actually doesn't guarantee so we can't release socket resource anyway
+            return new SocketState(out);
+        }
+    }
+
+    private BufferedWriter out;
+
+    private SocketState(BufferedWriter out) {
+        this.out = out;
+    }
+
+    public void write(String str) throws IOException {
+        out.write(str);
+    }
+
+    public void flush() throws IOException {
+        out.flush();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketStateUpdater.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketStateUpdater.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketStateUpdater.java
new file mode 100644
index 0000000..3062a90
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/SocketStateUpdater.java
@@ -0,0 +1,59 @@
+/*
+ * 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.sql.runtime.datasource.socket.trident;
+
+import org.apache.storm.sql.runtime.IOutputSerializer;
+import org.apache.storm.trident.operation.TridentCollector;
+import org.apache.storm.trident.state.BaseStateUpdater;
+import org.apache.storm.trident.tuple.TridentTuple;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * StateUpdater for SocketState. Serializes tuple one by one and writes to socket, and finally flushes.
+ */
+public class SocketStateUpdater extends BaseStateUpdater<SocketState> {
+    private static final Logger LOG = LoggerFactory.getLogger(SocketStateUpdater.class);
+
+    private final IOutputSerializer outputSerializer;
+
+    public SocketStateUpdater(IOutputSerializer outputSerializer) {
+        this.outputSerializer = outputSerializer;
+    }
+
+    @Override
+    public void updateState(SocketState state, List<TridentTuple> tuples, TridentCollector collector) {
+        try {
+            for (TridentTuple tuple : tuples) {
+                byte[] array = outputSerializer.write(tuple.getValues(), null).array();
+                String data = new String(array);
+                state.write(data + "\n");
+            }
+            state.flush();
+        } catch (IOException e) {
+            LOG.error("Error while updating state.", e);
+            collector.reportError(e);
+            throw new RuntimeException("Error while updating state.", e);
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/TridentSocketSpout.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/TridentSocketSpout.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/TridentSocketSpout.java
new file mode 100644
index 0000000..97f63a7
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/datasource/socket/trident/TridentSocketSpout.java
@@ -0,0 +1,177 @@
+/**
+ * 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.sql.runtime.datasource.socket.trident;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.storm.Config;
+import org.apache.storm.spout.Scheme;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.trident.operation.TridentCollector;
+import org.apache.storm.trident.spout.IBatchSpout;
+import org.apache.storm.tuple.Fields;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.net.Socket;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.BlockingDeque;
+import java.util.concurrent.LinkedBlockingDeque;
+
+/**
+ * Trident Spout for Socket data. Only available for Storm SQL, and only use for test purposes.
+ */
+public class TridentSocketSpout implements IBatchSpout {
+    private static final Logger LOG = LoggerFactory.getLogger(TridentSocketSpout.class);
+
+    private final String host;
+    private final int port;
+    private final Scheme scheme;
+
+    private volatile boolean _running = true;
+
+    private BlockingDeque<String> queue;
+    private Socket socket;
+    private Thread readerThread;
+    private BufferedReader in;
+    private ObjectMapper objectMapper;
+
+    private Map<Long, List<List<Object>>> batches;
+
+    public TridentSocketSpout(Scheme scheme, String host, int port) {
+        this.scheme = scheme;
+        this.host = host;
+        this.port = port;
+    }
+
+    @Override
+    public void open(Map conf, TopologyContext context) {
+        this.queue = new LinkedBlockingDeque<>();
+        this.objectMapper = new ObjectMapper();
+        this.batches = new HashMap<>();
+
+        try {
+            socket = new Socket(host, port);
+            in = new BufferedReader(new InputStreamReader(socket.getInputStream()));
+        } catch (IOException e) {
+            throw new RuntimeException("Error opening socket: host " + host + " port " + port);
+        }
+
+        readerThread = new Thread(new SocketReaderRunnable());
+        readerThread.start();
+    }
+
+    @Override
+    public void emitBatch(long batchId, TridentCollector collector) {
+        // read line and parse it to json
+        List<List<Object>> batch = this.batches.get(batchId);
+        if (batch == null) {
+            batch = new ArrayList<>();
+
+            while(queue.peek() != null) {
+                String line = queue.poll();
+                List<Object> values = convertLineToTuple(line);
+                if (values == null) {
+                    continue;
+                }
+
+                batch.add(values);
+            }
+
+            this.batches.put(batchId, batch);
+        }
+
+        for (List<Object> list : batch) {
+            collector.emit(list);
+        }
+    }
+
+    private List<Object> convertLineToTuple(String line) {
+        return scheme.deserialize(ByteBuffer.wrap(line.getBytes()));
+    }
+
+    @Override
+    public void ack(long batchId) {
+        this.batches.remove(batchId);
+    }
+
+    @Override
+    public void close() {
+        _running = false;
+        readerThread.interrupt();
+        queue.clear();
+
+        closeQuietly(in);
+        closeQuietly(socket);
+    }
+
+    @Override
+    public Map<String, Object> getComponentConfiguration() {
+        Config conf = new Config();
+        conf.setMaxTaskParallelism(1);
+        return conf;
+    }
+
+    @Override
+    public Fields getOutputFields() {
+        return scheme.getOutputFields();
+    }
+
+    private class SocketReaderRunnable implements Runnable {
+        public void run() {
+            while (_running) {
+                try {
+                    String line = in.readLine();
+                    if (line == null) {
+                        throw new RuntimeException("EOF reached from the socket. We can't read the data any more.");
+                    }
+
+                    queue.push(line.trim());
+                } catch (Throwable t) {
+                    // This spout is added to test purpose, so just failing fast doesn't hurt much
+                    die(t);
+                }
+            }
+        }
+    }
+
+    private void die(Throwable t) {
+        LOG.error("Halting process: TridentSocketSpout died.", t);
+        if (_running || (t instanceof Error)) { //don't exit if not running, unless it is an Error
+            System.exit(11);
+        }
+    }
+
+    private void closeQuietly(final Closeable closeable) {
+        try {
+            if (closeable != null) {
+                closeable.close();
+            }
+        } catch (final IOException ioe) {
+            // ignore
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroScheme.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroScheme.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroScheme.java
new file mode 100644
index 0000000..3bf1a23
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroScheme.java
@@ -0,0 +1,74 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.runtime.serde.avro;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.BinaryDecoder;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.storm.spout.Scheme;
+import org.apache.storm.sql.runtime.utils.SerdeUtils;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.utils.Utils;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * AvroScheme uses generic(without code generation) instead of specific(with code generation) readers.
+ */
+public class AvroScheme implements Scheme {
+  private final String schemaString;
+  private final List<String> fieldNames;
+  private final CachedSchemas schemas;
+
+  public AvroScheme(String schemaString, List<String> fieldNames) {
+    this.schemaString = schemaString;
+    this.fieldNames = fieldNames;
+    this.schemas = new CachedSchemas();
+  }
+
+  @Override
+  public List<Object> deserialize(ByteBuffer ser) {
+    try {
+      Schema schema = schemas.getSchema(schemaString);
+      DatumReader<GenericRecord> reader = new GenericDatumReader<>(schema);
+      BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(Utils.toByteArray(ser), null);
+      GenericRecord record = reader.read(null, decoder);
+
+      ArrayList<Object> list = new ArrayList<>(fieldNames.size());
+      for (String field : fieldNames) {
+        Object value = record.get(field);
+        // Avro strings are stored using a special Avro Utf8 type instead of using Java primitives
+        list.add(SerdeUtils.convertAvroUtf8(value));
+      }
+      return list;
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public Fields getOutputFields() {
+    return new Fields(fieldNames);
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroSerializer.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroSerializer.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroSerializer.java
new file mode 100644
index 0000000..5dc3393
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/AvroSerializer.java
@@ -0,0 +1,72 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.runtime.serde.avro;
+
+import com.google.common.base.Preconditions;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.DatumWriter;
+import org.apache.avro.io.Encoder;
+import org.apache.avro.io.EncoderFactory;
+import org.apache.storm.sql.runtime.IOutputSerializer;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+/**
+ * AvroSerializer uses generic(without code generation) instead of specific(with code generation) writers.
+ */
+public class AvroSerializer implements IOutputSerializer, Serializable {
+  private final String schemaString;
+  private final List<String> fieldNames;
+  private final CachedSchemas schemas;
+
+  public AvroSerializer(String schemaString, List<String> fieldNames) {
+    this.schemaString = schemaString;
+    this.fieldNames = fieldNames;
+    this.schemas = new CachedSchemas();
+  }
+
+  @Override
+  public ByteBuffer write(List<Object> data, ByteBuffer buffer) {
+    Preconditions.checkArgument(data != null && data.size() == fieldNames.size(), "Invalid schemas");
+    try {
+      Schema schema = schemas.getSchema(schemaString);
+      GenericRecord record = new GenericData.Record(schema);
+      for (int i = 0; i < fieldNames.size(); i++) {
+        record.put(fieldNames.get(i), data.get(i));
+      }
+
+      ByteArrayOutputStream out = new ByteArrayOutputStream();
+      DatumWriter<GenericRecord> writer = new GenericDatumWriter<>(record.getSchema());
+      Encoder encoder = EncoderFactory.get().directBinaryEncoder(out, null);
+      writer.write(record, encoder);
+      encoder.flush();
+      byte[] bytes = out.toByteArray();
+      out.close();
+      return ByteBuffer.wrap(bytes);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/CachedSchemas.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/CachedSchemas.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/CachedSchemas.java
new file mode 100644
index 0000000..4f0e747
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/avro/CachedSchemas.java
@@ -0,0 +1,41 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.runtime.serde.avro;
+
+import org.apache.avro.Schema;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+
+// TODO this class is reserved for supporting messages with different schemas.
+// current only one schema in the cache
+public class CachedSchemas implements Serializable{
+
+    private final Map<String, Schema> cache = new HashMap<>();
+
+    public Schema getSchema(String schemaString) {
+        Schema schema = cache.get(schemaString);
+        if (schema == null) {
+            schema = new Schema.Parser().parse(schemaString);
+            cache.put(schemaString, schema);
+        }
+        return schema;
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvScheme.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvScheme.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvScheme.java
new file mode 100644
index 0000000..34fb1bb
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvScheme.java
@@ -0,0 +1,70 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.runtime.serde.csv;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.csv.CSVFormat;
+import org.apache.commons.csv.CSVParser;
+import org.apache.commons.csv.CSVRecord;
+import org.apache.storm.spout.Scheme;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.utils.Utils;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * CsvScheme uses the standard RFC4180 CSV Parser
+ * One of the difference from Tsv format is that fields with embedded commas will be quoted.
+ * eg: a,"b,c",d is allowed.
+ *
+ * @see <a href="https://tools.ietf.org/html/rfc4180">RFC4180</a>
+ */
+public class CsvScheme implements Scheme {
+  private final List<String> fieldNames;
+
+  public CsvScheme(List<String> fieldNames) {
+    this.fieldNames = fieldNames;
+  }
+
+  @Override
+  public List<Object> deserialize(ByteBuffer ser) {
+    try {
+      String data = new String(Utils.toByteArray(ser), StandardCharsets.UTF_8);
+      CSVParser parser = CSVParser.parse(data, CSVFormat.RFC4180);
+      CSVRecord record = parser.getRecords().get(0);
+      Preconditions.checkArgument(record.size() == fieldNames.size(), "Invalid schema");
+
+      ArrayList<Object> list = new ArrayList<>(fieldNames.size());
+      for (int i = 0; i < record.size(); i++) {
+        list.add(record.get(i));
+      }
+      return list;
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public Fields getOutputFields() {
+    return new Fields(fieldNames);
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvSerializer.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvSerializer.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvSerializer.java
new file mode 100644
index 0000000..0d3bd74
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/csv/CsvSerializer.java
@@ -0,0 +1,59 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.runtime.serde.csv;
+
+import org.apache.commons.csv.CSVFormat;
+import org.apache.commons.csv.CSVPrinter;
+import org.apache.storm.sql.runtime.IOutputSerializer;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.io.StringWriter;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+
+/**
+ * CsvSerializer uses the standard RFC4180 CSV Parser
+ * One of the difference from Tsv format is that fields with embedded commas will be quoted.
+ * eg: a,"b,c",d is allowed.
+ *
+ * @see <a href="https://tools.ietf.org/html/rfc4180">RFC4180</a>
+ */
+public class CsvSerializer implements IOutputSerializer, Serializable {
+  private final List<String> fields; //reserved for future
+
+  public CsvSerializer(List<String> fields) {
+        this.fields = fields;
+    }
+
+  @Override
+  public ByteBuffer write(List<Object> data, ByteBuffer buffer) {
+    try {
+      StringWriter writer = new StringWriter();
+      CSVPrinter printer = new CSVPrinter(writer, CSVFormat.RFC4180);
+      for (Object o : data) {
+        printer.print(o);
+      }
+      //since using StringWriter, we do not need to close it.
+      return ByteBuffer.wrap(writer.getBuffer().toString().getBytes(StandardCharsets.UTF_8));
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonScheme.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonScheme.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonScheme.java
new file mode 100644
index 0000000..d288fa1
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonScheme.java
@@ -0,0 +1,58 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.runtime.serde.json;
+
+import org.apache.storm.spout.Scheme;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.utils.Utils;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+public class JsonScheme implements Scheme {
+  private final List<String> fields;
+
+  public JsonScheme(List<String> fields) {
+    this.fields = fields;
+  }
+
+  @Override
+  public List<Object> deserialize(ByteBuffer ser) {
+    ObjectMapper mapper = new ObjectMapper();
+    try {
+      @SuppressWarnings("unchecked")
+      HashMap<String, Object> map = mapper.readValue(Utils.toByteArray(ser), HashMap.class);
+      ArrayList<Object> list = new ArrayList<>(fields.size());
+      for (String f : fields) {
+        list.add(map.get(f));
+      }
+      return list;
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public Fields getOutputFields() {
+    return new Fields(fields);
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonSerializer.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonSerializer.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonSerializer.java
new file mode 100644
index 0000000..1e825c4
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/json/JsonSerializer.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.runtime.serde.json;
+
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.google.common.base.Preconditions;
+import org.apache.storm.sql.runtime.IOutputSerializer;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.io.StringWriter;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+
+public class JsonSerializer implements IOutputSerializer, Serializable {
+  private final List<String> fieldNames;
+  private final JsonFactory jsonFactory;
+
+  public JsonSerializer(List<String> fieldNames) {
+    this.fieldNames = fieldNames;
+    jsonFactory = new JsonFactory();
+  }
+
+  @Override
+  public ByteBuffer write(List<Object> data, ByteBuffer buffer) {
+    Preconditions.checkArgument(data != null && data.size() == fieldNames.size(), "Invalid schema");
+    StringWriter sw = new StringWriter();
+    try (JsonGenerator jg = jsonFactory.createGenerator(sw)) {
+      jg.writeStartObject();
+      for (int i = 0; i < fieldNames.size(); ++i) {
+        jg.writeFieldName(fieldNames.get(i));
+        jg.writeObject(data.get(i));
+      }
+      jg.writeEndObject();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+    return ByteBuffer.wrap(sw.toString().getBytes(StandardCharsets.UTF_8));
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvScheme.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvScheme.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvScheme.java
new file mode 100644
index 0000000..310494c
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvScheme.java
@@ -0,0 +1,58 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.runtime.serde.tsv;
+
+import com.google.common.base.Preconditions;
+import org.apache.storm.spout.Scheme;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.utils.Utils;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * TsvScheme uses a simple delimited format implemention by splitting string,
+ * and it supports user defined delimiter.
+ */
+public class TsvScheme implements Scheme {
+  private final List<String> fieldNames;
+  private final char delimiter;
+
+  public TsvScheme(List<String> fieldNames, char delimiter) {
+    this.fieldNames = fieldNames;
+    this.delimiter = delimiter;
+  }
+
+  @Override
+  public List<Object> deserialize(ByteBuffer ser) {
+    String data = new String(Utils.toByteArray(ser), StandardCharsets.UTF_8);
+    List<String> parts = org.apache.storm.sql.runtime.utils.Utils.split(data, delimiter);
+    Preconditions.checkArgument(parts.size() == fieldNames.size(), "Invalid schema");
+
+    ArrayList<Object> list = new ArrayList<>(fieldNames.size());
+    list.addAll(parts);
+    return list;
+  }
+
+  @Override
+  public Fields getOutputFields() {
+    return new Fields(fieldNames);
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvSerializer.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvSerializer.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvSerializer.java
new file mode 100644
index 0000000..1cf1c76
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/serde/tsv/TsvSerializer.java
@@ -0,0 +1,54 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.runtime.serde.tsv;
+
+import org.apache.storm.sql.runtime.IOutputSerializer;
+
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+
+/**
+ * TsvSerializer uses a simple delimited format implemention by splitting string,
+ * and it supports user defined delimiter.
+ */
+public class TsvSerializer implements IOutputSerializer, Serializable {
+  private final List<String> fields; //reserved for future
+  private final char delimiter;
+
+  public TsvSerializer(List<String> fields, char delimiter) {
+    this.fields = fields;
+    this.delimiter = delimiter;
+    }
+
+  @Override
+  public ByteBuffer write(List<Object> data, ByteBuffer buffer) {
+    StringBuilder sb = new StringBuilder(512); // 512: for most scenes to avoid inner array resizing
+    for (int i = 0; i < data.size(); i++) {
+      Object o = data.get(i);
+      if (i == 0) {
+        sb.append(o);
+      } else {
+        sb.append(delimiter);
+        sb.append(o);
+      }
+    }
+    return ByteBuffer.wrap(sb.toString().getBytes(StandardCharsets.UTF_8));
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationCalc.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationCalc.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationCalc.java
new file mode 100644
index 0000000..6c76481
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationCalc.java
@@ -0,0 +1,86 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.sql.runtime.trident.functions;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.interpreter.Context;
+import org.apache.calcite.interpreter.StormContext;
+import org.apache.storm.sql.runtime.calcite.DebuggableExecutableExpression;
+import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
+import org.apache.storm.trident.operation.OperationAwareFlatMapFunction;
+import org.apache.storm.trident.operation.TridentOperationContext;
+import org.apache.storm.trident.tuple.TridentTuple;
+import org.apache.storm.tuple.Values;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Map;
+
+public class EvaluationCalc implements OperationAwareFlatMapFunction {
+    private static final Logger LOG = LoggerFactory.getLogger(EvaluationCalc.class);
+
+    private final ExecutableExpression filterInstance;
+    private final ExecutableExpression projectionInstance;
+    private final Object[] outputValues;
+    private final DataContext dataContext;
+
+    public EvaluationCalc(ExecutableExpression filterInstance, ExecutableExpression projectionInstance, int outputCount, DataContext dataContext) {
+        this.filterInstance = filterInstance;
+        this.projectionInstance = projectionInstance;
+        this.outputValues = new Object[outputCount];
+        this.dataContext = dataContext;
+    }
+
+    @Override
+    public void prepare(Map conf, TridentOperationContext context) {
+        if (projectionInstance != null && projectionInstance instanceof DebuggableExecutableExpression) {
+            LOG.info("Expression code for projection: \n{}", ((DebuggableExecutableExpression) projectionInstance).getDelegateCode());
+        }
+        if (filterInstance != null && filterInstance instanceof DebuggableExecutableExpression) {
+            LOG.info("Expression code for filter: \n{}", ((DebuggableExecutableExpression) filterInstance).getDelegateCode());
+        }
+    }
+
+    @Override
+    public void cleanup() {
+
+    }
+
+    @Override
+    public Iterable<Values> execute(TridentTuple input) {
+        Context calciteContext = new StormContext(dataContext);
+        calciteContext.values = input.getValues().toArray();
+
+        if (filterInstance != null) {
+            filterInstance.execute(calciteContext, outputValues);
+            // filtered out
+            if (outputValues[0] == null || !((Boolean) outputValues[0])) {
+                return Collections.emptyList();
+            }
+        }
+
+        if (projectionInstance != null) {
+            projectionInstance.execute(calciteContext, outputValues);
+            return Collections.singletonList(new Values(outputValues));
+        } else {
+            return Collections.singletonList(new Values(input.getValues()));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFilter.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFilter.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFilter.java
new file mode 100644
index 0000000..9314852
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFilter.java
@@ -0,0 +1,61 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.sql.runtime.trident.functions;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.interpreter.Context;
+import org.apache.calcite.interpreter.StormContext;
+import org.apache.storm.sql.runtime.calcite.DebuggableExecutableExpression;
+import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
+import org.apache.storm.trident.operation.BaseFilter;
+import org.apache.storm.trident.operation.TridentOperationContext;
+import org.apache.storm.trident.tuple.TridentTuple;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+
+public class EvaluationFilter extends BaseFilter {
+    private static final Logger LOG = LoggerFactory.getLogger(EvaluationFilter.class);
+
+    private final ExecutableExpression filterInstance;
+    private final DataContext dataContext;
+    private final Object[] outputValues;
+
+    public EvaluationFilter(ExecutableExpression filterInstance, DataContext dataContext) {
+        this.filterInstance = filterInstance;
+        this.dataContext = dataContext;
+        this.outputValues = new Object[1];
+    }
+
+    @Override
+    public void prepare(Map conf, TridentOperationContext context) {
+        if (filterInstance != null && filterInstance instanceof DebuggableExecutableExpression) {
+            LOG.info("Expression code for filter: \n{}", ((DebuggableExecutableExpression) filterInstance).getDelegateCode());
+        }
+    }
+
+    @Override
+    public boolean isKeep(TridentTuple tuple) {
+        Context calciteContext = new StormContext(dataContext);
+        calciteContext.values = tuple.getValues().toArray();
+        filterInstance.execute(calciteContext, outputValues);
+        return (outputValues[0] != null && (boolean) outputValues[0]);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFunction.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFunction.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFunction.java
new file mode 100644
index 0000000..2608104
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/EvaluationFunction.java
@@ -0,0 +1,67 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.sql.runtime.trident.functions;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.interpreter.Context;
+import org.apache.calcite.interpreter.StormContext;
+import org.apache.storm.sql.runtime.calcite.DebuggableExecutableExpression;
+import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
+import org.apache.storm.trident.operation.OperationAwareMapFunction;
+import org.apache.storm.trident.operation.TridentOperationContext;
+import org.apache.storm.trident.tuple.TridentTuple;
+import org.apache.storm.tuple.Values;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+
+public class EvaluationFunction implements OperationAwareMapFunction {
+    private static final Logger LOG = LoggerFactory.getLogger(EvaluationFunction.class);
+
+    private final ExecutableExpression projectionInstance;
+    private final Object[] outputValues;
+    private final DataContext dataContext;
+
+    public EvaluationFunction(ExecutableExpression projectionInstance, int outputCount, DataContext dataContext) {
+        this.projectionInstance = projectionInstance;
+        this.outputValues = new Object[outputCount];
+        this.dataContext = dataContext;
+    }
+
+    @Override
+    public void prepare(Map conf, TridentOperationContext context) {
+        if (projectionInstance instanceof DebuggableExecutableExpression) {
+            LOG.info("Expression code: {}", ((DebuggableExecutableExpression) projectionInstance).getDelegateCode());
+        }
+    }
+
+    @Override
+    public void cleanup() {
+
+    }
+
+    @Override
+    public Values execute(TridentTuple input) {
+        Context calciteContext = new StormContext(dataContext);
+        calciteContext.values = input.getValues().toArray();
+        projectionInstance.execute(calciteContext, outputValues);
+        return new Values(outputValues);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/ForwardFunction.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/ForwardFunction.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/ForwardFunction.java
new file mode 100644
index 0000000..4c3a266
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/functions/ForwardFunction.java
@@ -0,0 +1,30 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.sql.runtime.trident.functions;
+
+import org.apache.storm.trident.operation.BaseFunction;
+import org.apache.storm.trident.operation.TridentCollector;
+import org.apache.storm.trident.tuple.TridentTuple;
+
+public class ForwardFunction extends BaseFunction {
+    @Override
+    public void execute(TridentTuple tuple, TridentCollector collector) {
+        collector.emit(tuple.getValues());
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/FieldInfoUtils.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/FieldInfoUtils.java b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/FieldInfoUtils.java
new file mode 100644
index 0000000..efd5d25
--- /dev/null
+++ b/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/utils/FieldInfoUtils.java
@@ -0,0 +1,39 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.runtime.utils;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Lists;
+import org.apache.storm.sql.runtime.FieldInfo;
+
+import java.io.Serializable;
+import java.util.List;
+
+public final class FieldInfoUtils {
+
+    public static List<String> getFieldNames(List<FieldInfo> fields) {
+        return Lists.transform(fields, new FieldNameExtractor());
+    }
+
+    private static class FieldNameExtractor implements Function<FieldInfo, String>, Serializable {
+        @Override
+        public String apply(FieldInfo fieldInfo) {
+            return fieldInfo.name();
+        }
+    }
+}


[21/23] storm git commit: STORM-2453 Move non-connectors into the top directory

Posted by ka...@apache.org.
STORM-2453 Move non-connectors into the top directory

* move storm-sql, flux, storm-submit-tools to the top directory
* rearrange target directories for them in binary dist.
  * create "lib-tools" directory for storing non-storm-core libs
  * sql: lib-tools/sql/core and lib-tools/sql/runtime
  * storm-submit-tools: lib-tools/submit-tools
  * storm-kafka-monitor: lib-tools/storm-kafka-monitor


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

Branch: refs/heads/master
Commit: e9d783386e6d964df925546beb4db55106e61312
Parents: 28c8711
Author: Jungtaek Lim <ka...@gmail.com>
Authored: Tue Apr 4 17:27:32 2017 +0900
Committer: Jungtaek Lim <ka...@gmail.com>
Committed: Tue Apr 4 23:04:18 2017 +0900

----------------------------------------------------------------------
 bin/storm-kafka-monitor                         |   2 +-
 bin/storm.py                                    |   9 +-
 external/flux/README.md                         | 872 -------------------
 external/flux/flux-core/pom.xml                 | 112 ---
 .../main/java/org/apache/storm/flux/Flux.java   | 261 ------
 .../java/org/apache/storm/flux/FluxBuilder.java | 630 --------------
 .../apache/storm/flux/api/TopologySource.java   |  39 -
 .../org/apache/storm/flux/model/BeanDef.java    |  39 -
 .../storm/flux/model/BeanListReference.java     |  37 -
 .../apache/storm/flux/model/BeanReference.java  |  39 -
 .../org/apache/storm/flux/model/BoltDef.java    |  24 -
 .../storm/flux/model/ConfigMethodDef.java       |  67 --
 .../storm/flux/model/ExecutionContext.java      |  77 --
 .../apache/storm/flux/model/GroupingDef.java    |  77 --
 .../org/apache/storm/flux/model/IncludeDef.java |  54 --
 .../org/apache/storm/flux/model/ObjectDef.java  |  95 --
 .../apache/storm/flux/model/PropertyDef.java    |  58 --
 .../org/apache/storm/flux/model/SpoutDef.java   |  24 -
 .../org/apache/storm/flux/model/StreamDef.java  |  64 --
 .../apache/storm/flux/model/TopologyDef.java    | 216 -----
 .../storm/flux/model/TopologySourceDef.java     |  36 -
 .../org/apache/storm/flux/model/VertexDef.java  |  36 -
 .../apache/storm/flux/parser/FluxParser.java    | 208 -----
 .../flux-core/src/main/resources/splash.txt     |   9 -
 .../org/apache/storm/flux/FluxBuilderTest.java  |  31 -
 .../org/apache/storm/flux/IntegrationTest.java  |  39 -
 .../java/org/apache/storm/flux/TCKTest.java     | 256 ------
 .../multilang/MultilangEnvirontmentTest.java    |  89 --
 .../apache/storm/flux/test/SimpleTopology.java  |  59 --
 .../storm/flux/test/SimpleTopologySource.java   |  52 --
 .../test/SimpleTopologyWithConfigParam.java     |  55 --
 .../org/apache/storm/flux/test/TestBolt.java    | 131 ---
 .../storm/flux/test/TridentTopologySource.java  |  71 --
 .../src/test/resources/configs/bad_hbase.yaml   |  98 ---
 .../test/resources/configs/bad_shell_test.yaml  | 118 ---
 .../resources/configs/config-methods-test.yaml  |  92 --
 .../resources/configs/diamond-topology.yaml     |  87 --
 .../existing-topology-method-override.yaml      |  25 -
 .../existing-topology-reflection-config.yaml    |  24 -
 .../configs/existing-topology-reflection.yaml   |  24 -
 .../configs/existing-topology-trident.yaml      |  24 -
 .../resources/configs/existing-topology.yaml    |  23 -
 .../src/test/resources/configs/hdfs_test.yaml   |  97 ---
 .../test/resources/configs/include_test.yaml    |  25 -
 .../configs/invalid-existing-topology.yaml      |  33 -
 .../src/test/resources/configs/kafka_test.yaml  | 126 ---
 .../src/test/resources/configs/shell_test.yaml  | 118 ---
 .../test/resources/configs/simple_hbase.yaml    | 120 ---
 .../resources/configs/substitution-test.yaml    | 106 ---
 .../src/test/resources/configs/tck.yaml         |  95 --
 .../src/test/resources/configs/test.properties  |  18 -
 .../flux-core/src/test/resources/log4j2.xml     |  34 -
 external/flux/flux-examples/README.md           |  93 --
 external/flux/flux-examples/pom.xml             | 146 ----
 .../flux/examples/StatefulWordCounter.java      |  64 --
 .../storm/flux/examples/TestPrintBolt.java      |  39 -
 .../storm/flux/examples/TestWindowBolt.java     |  47 -
 .../storm/flux/examples/WordCountClient.java    |  74 --
 .../apache/storm/flux/examples/WordCounter.java |  71 --
 .../src/main/resources/hbase_bolt.properties    |  18 -
 .../src/main/resources/hdfs_bolt.properties     |  26 -
 .../src/main/resources/kafka_spout.yaml         | 136 ---
 .../src/main/resources/multilang.yaml           |  89 --
 .../src/main/resources/simple_hbase.yaml        |  92 --
 .../src/main/resources/simple_hdfs.yaml         | 105 ---
 .../resources/simple_stateful_wordcount.yaml    |  60 --
 .../src/main/resources/simple_windowing.yaml    |  69 --
 .../src/main/resources/simple_wordcount.yaml    |  68 --
 external/flux/flux-ui/README.md                 |   3 -
 external/flux/flux-wrappers/pom.xml             |  51 --
 .../flux/wrappers/bolts/FluxShellBolt.java      | 175 ----
 .../storm/flux/wrappers/bolts/LogInfoBolt.java  |  44 -
 .../flux/wrappers/spouts/FluxShellSpout.java    | 175 ----
 .../main/resources/resources/randomsentence.js  |  93 --
 .../main/resources/resources/splitsentence.py   |  24 -
 external/flux/pom.xml                           |  85 --
 external/sql/README.md                          | 207 -----
 external/sql/pom.xml                            |  47 -
 external/sql/storm-sql-core/pom.xml             | 279 ------
 .../sql/storm-sql-core/src/codegen/config.fmpp  |  23 -
 .../storm-sql-core/src/codegen/data/Parser.tdd  |  80 --
 .../src/codegen/includes/license.ftl            |  17 -
 .../src/codegen/includes/parserImpls.ftl        | 113 ---
 .../storm/sql/AbstractTridentProcessor.java     |  58 --
 .../src/jvm/org/apache/storm/sql/StormSql.java  |  59 --
 .../jvm/org/apache/storm/sql/StormSqlImpl.java  | 290 ------
 .../org/apache/storm/sql/StormSqlRunner.java    |  85 --
 .../sql/calcite/ParallelStreamableTable.java    |  35 -
 .../apache/storm/sql/compiler/CompilerUtil.java | 183 ----
 .../sql/compiler/RexNodeToJavaCodeCompiler.java | 231 -----
 .../sql/compiler/StormSqlTypeFactoryImpl.java   |  51 --
 .../standalone/BuiltinAggregateFunctions.java   | 238 -----
 .../backends/standalone/PlanCompiler.java       | 139 ---
 .../standalone/PostOrderRelNodeVisitor.java     | 132 ---
 .../backends/standalone/RelNodeCompiler.java    | 484 ----------
 .../storm/sql/javac/CompilingClassLoader.java   | 225 -----
 .../storm/sql/parser/ColumnConstraint.java      |  42 -
 .../storm/sql/parser/ColumnDefinition.java      |  44 -
 .../storm/sql/parser/SqlCreateFunction.java     |  94 --
 .../apache/storm/sql/parser/SqlCreateTable.java | 166 ----
 .../apache/storm/sql/parser/SqlDDLKeywords.java |  27 -
 .../apache/storm/sql/parser/StormParser.java    |  47 -
 .../apache/storm/sql/parser/UnparseUtil.java    |  60 --
 .../sql/planner/StormRelDataTypeSystem.java     |  37 -
 .../apache/storm/sql/planner/StormRelUtils.java |  68 --
 .../planner/UnsupportedOperatorsVisitor.java    |  24 -
 .../storm/sql/planner/rel/StormCalcRelBase.java |  32 -
 .../sql/planner/rel/StormFilterRelBase.java     |  32 -
 .../storm/sql/planner/rel/StormJoinRelBase.java |  36 -
 .../sql/planner/rel/StormProjectRelBase.java    |  35 -
 .../storm/sql/planner/rel/StormRelNode.java     |  24 -
 .../planner/rel/StormStreamInsertRelBase.java   |  36 -
 .../sql/planner/rel/StormStreamScanRelBase.java |  34 -
 .../storm/sql/planner/trident/QueryPlanner.java | 156 ----
 .../sql/planner/trident/TridentPlanCreator.java | 125 ---
 .../planner/trident/TridentStormRuleSets.java   | 110 ---
 .../sql/planner/trident/rel/TridentCalcRel.java |  97 ---
 .../planner/trident/rel/TridentFilterRel.java   |  65 --
 .../trident/rel/TridentLogicalConvention.java   |  67 --
 .../planner/trident/rel/TridentProjectRel.java  |  69 --
 .../sql/planner/trident/rel/TridentRel.java     |  26 -
 .../trident/rel/TridentStreamInsertRel.java     |  76 --
 .../trident/rel/TridentStreamScanRel.java       |  55 --
 .../trident/rules/TridentAggregateRule.java     |  39 -
 .../planner/trident/rules/TridentCalcRule.java  |  45 -
 .../trident/rules/TridentFilterRule.java        |  45 -
 .../planner/trident/rules/TridentJoinRule.java  |  37 -
 .../trident/rules/TridentModifyRule.java        |  71 --
 .../trident/rules/TridentProjectRule.java       |  45 -
 .../planner/trident/rules/TridentScanRule.java  |  58 --
 .../test/org/apache/storm/sql/TestStormSql.java | 492 -----------
 .../storm/sql/compiler/TestExprSemantic.java    | 410 ---------
 .../backends/standalone/TestCompilerUtils.java  | 183 ----
 .../backends/standalone/TestPlanCompiler.java   | 104 ---
 .../standalone/TestRelNodeCompiler.java         |  64 --
 .../backends/trident/TestCompilerUtils.java     | 208 -----
 .../backends/trident/TestPlanCompiler.java      | 235 -----
 .../apache/storm/sql/parser/TestSqlParser.java  |  54 --
 .../storm-sql-external/storm-sql-hdfs/pom.xml   | 104 ---
 .../storm/sql/hdfs/HdfsDataSourcesProvider.java | 135 ---
 ...apache.storm.sql.runtime.DataSourcesProvider |  16 -
 .../sql/hdfs/TestHdfsDataSourcesProvider.java   | 129 ---
 .../storm-sql-external/storm-sql-kafka/pom.xml  |  93 --
 .../sql/kafka/KafkaDataSourcesProvider.java     | 170 ----
 ...apache.storm.sql.runtime.DataSourcesProvider |  16 -
 .../sql/kafka/TestKafkaDataSourcesProvider.java | 125 ---
 .../storm-sql-mongodb/pom.xml                   |  84 --
 .../sql/mongodb/MongoDataSourcesProvider.java   | 126 ---
 ...apache.storm.sql.runtime.DataSourcesProvider |  16 -
 .../mongodb/TestMongoDataSourcesProvider.java   | 122 ---
 .../storm-sql-external/storm-sql-redis/pom.xml  |  76 --
 .../sql/redis/RedisDataSourcesProvider.java     | 228 -----
 ...apache.storm.sql.runtime.DataSourcesProvider |  16 -
 .../sql/redis/TestRedisDataSourcesProvider.java | 158 ----
 external/sql/storm-sql-runtime/pom.xml          | 136 ---
 .../calcite/interpreter/StormContext.java       |  31 -
 .../sql/runtime/AbstractChannelHandler.java     |  52 --
 .../sql/runtime/AbstractValuesProcessor.java    |  46 -
 .../storm/sql/runtime/ChannelContext.java       |  30 -
 .../storm/sql/runtime/ChannelHandler.java       |  41 -
 .../org/apache/storm/sql/runtime/Channels.java  | 109 ---
 .../apache/storm/sql/runtime/DataSource.java    |  27 -
 .../storm/sql/runtime/DataSourcesProvider.java  |  47 -
 .../storm/sql/runtime/DataSourcesRegistry.java  |  76 --
 .../org/apache/storm/sql/runtime/FieldInfo.java |  47 -
 .../storm/sql/runtime/IOutputSerializer.java    |  31 -
 .../sql/runtime/ISqlTridentDataSource.java      |  65 --
 .../sql/runtime/SimpleSqlTridentConsumer.java   |  41 -
 .../storm/sql/runtime/StormSqlFunctions.java    |  34 -
 .../calcite/DebuggableExecutableExpression.java |  45 -
 .../runtime/calcite/ExecutableExpression.java   |  31 -
 .../sql/runtime/calcite/StormDataContext.java   |  79 --
 .../socket/SocketDataSourcesProvider.java       | 101 ---
 .../datasource/socket/trident/SocketState.java  |  89 --
 .../socket/trident/SocketStateUpdater.java      |  59 --
 .../socket/trident/TridentSocketSpout.java      | 177 ----
 .../sql/runtime/serde/avro/AvroScheme.java      |  74 --
 .../sql/runtime/serde/avro/AvroSerializer.java  |  72 --
 .../sql/runtime/serde/avro/CachedSchemas.java   |  41 -
 .../storm/sql/runtime/serde/csv/CsvScheme.java  |  70 --
 .../sql/runtime/serde/csv/CsvSerializer.java    |  59 --
 .../sql/runtime/serde/json/JsonScheme.java      |  58 --
 .../sql/runtime/serde/json/JsonSerializer.java  |  57 --
 .../storm/sql/runtime/serde/tsv/TsvScheme.java  |  58 --
 .../sql/runtime/serde/tsv/TsvSerializer.java    |  54 --
 .../trident/functions/EvaluationCalc.java       |  86 --
 .../trident/functions/EvaluationFilter.java     |  61 --
 .../trident/functions/EvaluationFunction.java   |  67 --
 .../trident/functions/ForwardFunction.java      |  30 -
 .../storm/sql/runtime/utils/FieldInfoUtils.java |  39 -
 .../storm/sql/runtime/utils/SerdeUtils.java     | 123 ---
 .../apache/storm/sql/runtime/utils/Utils.java   |  55 --
 ...apache.storm.sql.runtime.DataSourcesProvider |  16 -
 .../apache/storm/sql/TestAvroSerializer.java    |  72 --
 .../org/apache/storm/sql/TestCsvSerializer.java |  54 --
 .../storm/sql/TestJsonRepresentation.java       |  52 --
 .../org/apache/storm/sql/TestTsvSerializer.java |  46 -
 .../test/org/apache/storm/sql/TestUtils.java    | 584 -------------
 .../socket/TestSocketDataSourceProvider.java    |  94 --
 external/storm-submit-tools/pom.xml             | 216 -----
 .../submit/command/DependencyResolverMain.java  | 158 ----
 .../storm/submit/dependency/AetherUtils.java    |  91 --
 .../apache/storm/submit/dependency/Booter.java  |  51 --
 .../submit/dependency/DependencyResolver.java   |  98 ---
 .../dependency/RepositorySystemFactory.java     |  67 --
 .../submit/dependency/AetherUtilsTest.java      | 102 ---
 .../dependency/DependencyResolverTest.java      |  83 --
 flux/README.md                                  | 872 +++++++++++++++++++
 flux/flux-core/pom.xml                          | 112 +++
 .../main/java/org/apache/storm/flux/Flux.java   | 259 ++++++
 .../java/org/apache/storm/flux/FluxBuilder.java | 630 ++++++++++++++
 .../apache/storm/flux/api/TopologySource.java   |  39 +
 .../org/apache/storm/flux/model/BeanDef.java    |  34 +
 .../storm/flux/model/BeanListReference.java     |  37 +
 .../apache/storm/flux/model/BeanReference.java  |  39 +
 .../org/apache/storm/flux/model/BoltDef.java    |  24 +
 .../storm/flux/model/ConfigMethodDef.java       |  67 ++
 .../storm/flux/model/ExecutionContext.java      |  77 ++
 .../apache/storm/flux/model/GroupingDef.java    |  77 ++
 .../org/apache/storm/flux/model/IncludeDef.java |  54 ++
 .../org/apache/storm/flux/model/ObjectDef.java  |  95 ++
 .../apache/storm/flux/model/PropertyDef.java    |  58 ++
 .../org/apache/storm/flux/model/SpoutDef.java   |  24 +
 .../org/apache/storm/flux/model/StreamDef.java  |  64 ++
 .../apache/storm/flux/model/TopologyDef.java    | 216 +++++
 .../storm/flux/model/TopologySourceDef.java     |  36 +
 .../org/apache/storm/flux/model/VertexDef.java  |  36 +
 .../apache/storm/flux/parser/FluxParser.java    | 206 +++++
 flux/flux-core/src/main/resources/splash.txt    |   9 +
 .../org/apache/storm/flux/FluxBuilderTest.java  |  31 +
 .../org/apache/storm/flux/IntegrationTest.java  |  39 +
 .../java/org/apache/storm/flux/TCKTest.java     | 254 ++++++
 .../multilang/MultilangEnvirontmentTest.java    |  89 ++
 .../apache/storm/flux/test/SimpleTopology.java  |  58 ++
 .../storm/flux/test/SimpleTopologySource.java   |  52 ++
 .../test/SimpleTopologyWithConfigParam.java     |  53 ++
 .../org/apache/storm/flux/test/TestBolt.java    | 131 +++
 .../storm/flux/test/TridentTopologySource.java  |  71 ++
 .../src/test/resources/configs/bad_hbase.yaml   |  98 +++
 .../test/resources/configs/bad_shell_test.yaml  | 118 +++
 .../resources/configs/config-methods-test.yaml  |  92 ++
 .../resources/configs/diamond-topology.yaml     |  87 ++
 .../existing-topology-method-override.yaml      |  25 +
 .../existing-topology-reflection-config.yaml    |  24 +
 .../configs/existing-topology-reflection.yaml   |  24 +
 .../configs/existing-topology-trident.yaml      |  24 +
 .../resources/configs/existing-topology.yaml    |  23 +
 .../src/test/resources/configs/hdfs_test.yaml   |  97 +++
 .../test/resources/configs/include_test.yaml    |  25 +
 .../configs/invalid-existing-topology.yaml      |  33 +
 .../src/test/resources/configs/kafka_test.yaml  | 126 +++
 .../src/test/resources/configs/shell_test.yaml  | 118 +++
 .../test/resources/configs/simple_hbase.yaml    | 120 +++
 .../resources/configs/substitution-test.yaml    | 106 +++
 .../src/test/resources/configs/tck.yaml         |  95 ++
 .../src/test/resources/configs/test.properties  |  18 +
 flux/flux-core/src/test/resources/log4j2.xml    |  34 +
 flux/flux-examples/README.md                    |  93 ++
 flux/flux-examples/pom.xml                      | 146 ++++
 .../flux/examples/StatefulWordCounter.java      |  64 ++
 .../storm/flux/examples/TestPrintBolt.java      |  39 +
 .../storm/flux/examples/TestWindowBolt.java     |  47 +
 .../storm/flux/examples/WordCountClient.java    |  74 ++
 .../apache/storm/flux/examples/WordCounter.java |  71 ++
 .../src/main/resources/hbase_bolt.properties    |  18 +
 .../src/main/resources/hdfs_bolt.properties     |  26 +
 .../src/main/resources/kafka_spout.yaml         | 136 +++
 .../src/main/resources/multilang.yaml           |  89 ++
 .../src/main/resources/simple_hbase.yaml        |  92 ++
 .../src/main/resources/simple_hdfs.yaml         | 105 +++
 .../resources/simple_stateful_wordcount.yaml    |  60 ++
 .../src/main/resources/simple_windowing.yaml    |  69 ++
 .../src/main/resources/simple_wordcount.yaml    |  68 ++
 flux/flux-ui/README.md                          |   3 +
 flux/flux-wrappers/pom.xml                      |  51 ++
 .../flux/wrappers/bolts/FluxShellBolt.java      | 175 ++++
 .../storm/flux/wrappers/bolts/LogInfoBolt.java  |  44 +
 .../flux/wrappers/spouts/FluxShellSpout.java    | 175 ++++
 .../main/resources/resources/randomsentence.js  |  93 ++
 .../main/resources/resources/splitsentence.py   |  24 +
 flux/pom.xml                                    |  85 ++
 pom.xml                                         |   6 +-
 sql/README.md                                   | 207 +++++
 sql/pom.xml                                     |  47 +
 sql/storm-sql-core/pom.xml                      | 279 ++++++
 sql/storm-sql-core/src/codegen/config.fmpp      |  23 +
 sql/storm-sql-core/src/codegen/data/Parser.tdd  |  80 ++
 .../src/codegen/includes/license.ftl            |  17 +
 .../src/codegen/includes/parserImpls.ftl        | 113 +++
 .../storm/sql/AbstractTridentProcessor.java     |  56 ++
 .../src/jvm/org/apache/storm/sql/StormSql.java  |  59 ++
 .../jvm/org/apache/storm/sql/StormSqlImpl.java  | 290 ++++++
 .../org/apache/storm/sql/StormSqlRunner.java    |  85 ++
 .../sql/calcite/ParallelStreamableTable.java    |  35 +
 .../apache/storm/sql/compiler/CompilerUtil.java | 183 ++++
 .../sql/compiler/RexNodeToJavaCodeCompiler.java | 231 +++++
 .../sql/compiler/StormSqlTypeFactoryImpl.java   |  51 ++
 .../standalone/BuiltinAggregateFunctions.java   | 238 +++++
 .../backends/standalone/PlanCompiler.java       | 139 +++
 .../standalone/PostOrderRelNodeVisitor.java     | 132 +++
 .../backends/standalone/RelNodeCompiler.java    | 484 ++++++++++
 .../storm/sql/javac/CompilingClassLoader.java   | 225 +++++
 .../storm/sql/parser/ColumnConstraint.java      |  42 +
 .../storm/sql/parser/ColumnDefinition.java      |  44 +
 .../storm/sql/parser/SqlCreateFunction.java     |  94 ++
 .../apache/storm/sql/parser/SqlCreateTable.java | 166 ++++
 .../apache/storm/sql/parser/SqlDDLKeywords.java |  27 +
 .../apache/storm/sql/parser/StormParser.java    |  47 +
 .../apache/storm/sql/parser/UnparseUtil.java    |  59 ++
 .../sql/planner/StormRelDataTypeSystem.java     |  37 +
 .../apache/storm/sql/planner/StormRelUtils.java |  68 ++
 .../planner/UnsupportedOperatorsVisitor.java    |  24 +
 .../storm/sql/planner/rel/StormCalcRelBase.java |  31 +
 .../sql/planner/rel/StormFilterRelBase.java     |  31 +
 .../storm/sql/planner/rel/StormJoinRelBase.java |  35 +
 .../sql/planner/rel/StormProjectRelBase.java    |  34 +
 .../storm/sql/planner/rel/StormRelNode.java     |  24 +
 .../planner/rel/StormStreamInsertRelBase.java   |  35 +
 .../sql/planner/rel/StormStreamScanRelBase.java |  33 +
 .../storm/sql/planner/trident/QueryPlanner.java | 156 ++++
 .../sql/planner/trident/TridentPlanCreator.java | 123 +++
 .../planner/trident/TridentStormRuleSets.java   | 110 +++
 .../sql/planner/trident/rel/TridentCalcRel.java |  97 +++
 .../planner/trident/rel/TridentFilterRel.java   |  65 ++
 .../trident/rel/TridentLogicalConvention.java   |  67 ++
 .../planner/trident/rel/TridentProjectRel.java  |  69 ++
 .../sql/planner/trident/rel/TridentRel.java     |  25 +
 .../trident/rel/TridentStreamInsertRel.java     |  76 ++
 .../trident/rel/TridentStreamScanRel.java       |  55 ++
 .../trident/rules/TridentAggregateRule.java     |  39 +
 .../planner/trident/rules/TridentCalcRule.java  |  45 +
 .../trident/rules/TridentFilterRule.java        |  45 +
 .../planner/trident/rules/TridentJoinRule.java  |  37 +
 .../trident/rules/TridentModifyRule.java        |  71 ++
 .../trident/rules/TridentProjectRule.java       |  45 +
 .../planner/trident/rules/TridentScanRule.java  |  58 ++
 .../test/org/apache/storm/sql/TestStormSql.java | 492 +++++++++++
 .../storm/sql/compiler/TestExprSemantic.java    | 410 +++++++++
 .../backends/standalone/TestCompilerUtils.java  | 183 ++++
 .../backends/standalone/TestPlanCompiler.java   | 104 +++
 .../standalone/TestRelNodeCompiler.java         |  64 ++
 .../backends/trident/TestCompilerUtils.java     | 208 +++++
 .../backends/trident/TestPlanCompiler.java      | 232 +++++
 .../apache/storm/sql/parser/TestSqlParser.java  |  54 ++
 sql/storm-sql-external/storm-sql-hdfs/pom.xml   | 104 +++
 .../storm/sql/hdfs/HdfsDataSourcesProvider.java | 135 +++
 ...apache.storm.sql.runtime.DataSourcesProvider |  16 +
 .../sql/hdfs/TestHdfsDataSourcesProvider.java   | 129 +++
 sql/storm-sql-external/storm-sql-kafka/pom.xml  |  93 ++
 .../sql/kafka/KafkaDataSourcesProvider.java     | 170 ++++
 ...apache.storm.sql.runtime.DataSourcesProvider |  16 +
 .../sql/kafka/TestKafkaDataSourcesProvider.java | 125 +++
 .../storm-sql-mongodb/pom.xml                   |  84 ++
 .../sql/mongodb/MongoDataSourcesProvider.java   | 126 +++
 ...apache.storm.sql.runtime.DataSourcesProvider |  16 +
 .../mongodb/TestMongoDataSourcesProvider.java   | 122 +++
 sql/storm-sql-external/storm-sql-redis/pom.xml  |  76 ++
 .../sql/redis/RedisDataSourcesProvider.java     | 228 +++++
 ...apache.storm.sql.runtime.DataSourcesProvider |  16 +
 .../sql/redis/TestRedisDataSourcesProvider.java | 158 ++++
 sql/storm-sql-runtime/pom.xml                   | 136 +++
 .../calcite/interpreter/StormContext.java       |  31 +
 .../sql/runtime/AbstractChannelHandler.java     |  52 ++
 .../sql/runtime/AbstractValuesProcessor.java    |  44 +
 .../storm/sql/runtime/ChannelContext.java       |  30 +
 .../storm/sql/runtime/ChannelHandler.java       |  41 +
 .../org/apache/storm/sql/runtime/Channels.java  | 109 +++
 .../apache/storm/sql/runtime/DataSource.java    |  27 +
 .../storm/sql/runtime/DataSourcesProvider.java  |  47 +
 .../storm/sql/runtime/DataSourcesRegistry.java  |  76 ++
 .../org/apache/storm/sql/runtime/FieldInfo.java |  47 +
 .../storm/sql/runtime/IOutputSerializer.java    |  31 +
 .../sql/runtime/ISqlTridentDataSource.java      |  65 ++
 .../sql/runtime/SimpleSqlTridentConsumer.java   |  41 +
 .../storm/sql/runtime/StormSqlFunctions.java    |  34 +
 .../calcite/DebuggableExecutableExpression.java |  45 +
 .../runtime/calcite/ExecutableExpression.java   |  31 +
 .../sql/runtime/calcite/StormDataContext.java   |  79 ++
 .../socket/SocketDataSourcesProvider.java       | 100 +++
 .../datasource/socket/trident/SocketState.java  |  89 ++
 .../socket/trident/SocketStateUpdater.java      |  59 ++
 .../socket/trident/TridentSocketSpout.java      | 177 ++++
 .../sql/runtime/serde/avro/AvroScheme.java      |  74 ++
 .../sql/runtime/serde/avro/AvroSerializer.java  |  72 ++
 .../sql/runtime/serde/avro/CachedSchemas.java   |  41 +
 .../storm/sql/runtime/serde/csv/CsvScheme.java  |  70 ++
 .../sql/runtime/serde/csv/CsvSerializer.java    |  59 ++
 .../sql/runtime/serde/json/JsonScheme.java      |  58 ++
 .../sql/runtime/serde/json/JsonSerializer.java  |  57 ++
 .../storm/sql/runtime/serde/tsv/TsvScheme.java  |  58 ++
 .../sql/runtime/serde/tsv/TsvSerializer.java    |  54 ++
 .../trident/functions/EvaluationCalc.java       |  86 ++
 .../trident/functions/EvaluationFilter.java     |  61 ++
 .../trident/functions/EvaluationFunction.java   |  67 ++
 .../trident/functions/ForwardFunction.java      |  30 +
 .../storm/sql/runtime/utils/FieldInfoUtils.java |  39 +
 .../storm/sql/runtime/utils/SerdeUtils.java     | 123 +++
 .../apache/storm/sql/runtime/utils/Utils.java   |  55 ++
 ...apache.storm.sql.runtime.DataSourcesProvider |  16 +
 .../apache/storm/sql/TestAvroSerializer.java    |  72 ++
 .../org/apache/storm/sql/TestCsvSerializer.java |  54 ++
 .../storm/sql/TestJsonRepresentation.java       |  52 ++
 .../org/apache/storm/sql/TestTsvSerializer.java |  46 +
 .../test/org/apache/storm/sql/TestUtils.java    | 584 +++++++++++++
 .../socket/TestSocketDataSourceProvider.java    |  94 ++
 .../final-package/src/main/assembly/binary.xml  |  86 +-
 storm-submit-tools/pom.xml                      | 216 +++++
 .../submit/command/DependencyResolverMain.java  | 158 ++++
 .../storm/submit/dependency/AetherUtils.java    |  91 ++
 .../apache/storm/submit/dependency/Booter.java  |  51 ++
 .../submit/dependency/DependencyResolver.java   |  98 +++
 .../dependency/RepositorySystemFactory.java     |  67 ++
 .../submit/dependency/AetherUtilsTest.java      | 102 +++
 .../dependency/DependencyResolverTest.java      |  83 ++
 414 files changed, 19540 insertions(+), 19605 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/bin/storm-kafka-monitor
----------------------------------------------------------------------
diff --git a/bin/storm-kafka-monitor b/bin/storm-kafka-monitor
index a51052d..2958e85 100755
--- a/bin/storm-kafka-monitor
+++ b/bin/storm-kafka-monitor
@@ -49,4 +49,4 @@ if [ -z "$JAVA_HOME" ]; then
 else
   JAVA="$JAVA_HOME/bin/java"
 fi
-exec $JAVA $STORM_JAAS_CONF_PARAM -cp $STORM_BASE_DIR/toollib/storm-kafka-monitor-*.jar org.apache.storm.kafka.monitor.KafkaOffsetLagUtil "$@"
\ No newline at end of file
+exec $JAVA $STORM_JAAS_CONF_PARAM -cp $STORM_BASE_DIR/lib-tools/storm-kafka-monitor/storm-kafka-monitor-*.jar org.apache.storm.kafka.monitor.KafkaOffsetLagUtil "$@"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/bin/storm.py
----------------------------------------------------------------------
diff --git a/bin/storm.py b/bin/storm.py
index 7ab1f9c..a2d9243 100755
--- a/bin/storm.py
+++ b/bin/storm.py
@@ -82,6 +82,7 @@ if (not os.path.isfile(os.path.join(USER_CONF_DIR, "storm.yaml"))):
 
 STORM_LIB_DIR = os.path.join(STORM_DIR, "lib")
 STORM_DRPC_LIB_DIR = os.path.join(STORM_DIR, "lib-drpc-server")
+STORM_TOOLS_LIB_DIR = os.path.join(STORM_DIR, "lib-tools")
 STORM_BIN_DIR = os.path.join(STORM_DIR, "bin")
 STORM_LOG4J2_CONF_DIR = os.path.join(STORM_DIR, "log4j2")
 STORM_SUPERVISOR_LOG_FILE = os.getenv('STORM_SUPERVISOR_LOG_FILE', "supervisor.log")
@@ -167,10 +168,8 @@ def resolve_dependencies(artifacts, artifact_repositories):
     print("Resolving dependencies on demand: artifacts (%s) with repositories (%s)" % (artifacts, artifact_repositories))
     sys.stdout.flush()
 
-    # TODO: should we move some external modules to outer place?
-
     # storm-submit module doesn't rely on storm-core and relevant libs
-    extrajars = get_jars_full(STORM_DIR + "/external/storm-submit-tools")
+    extrajars = get_jars_full(os.path.join(STORM_TOOLS_LIB_DIR, "submit-tools"))
     classpath = normclasspath(os.pathsep.join(extrajars))
 
     command = [
@@ -343,8 +342,8 @@ def sql(sql_file, topology_name):
     local_jars = DEP_JARS_OPTS
     artifact_to_file_jars = resolve_dependencies(DEP_ARTIFACTS_OPTS, DEP_ARTIFACTS_REPOSITORIES_OPTS)
 
-    sql_core_jars = get_jars_full(STORM_DIR + "/external/sql/storm-sql-core")
-    sql_runtime_jars = get_jars_full(STORM_DIR + "/external/sql/storm-sql-runtime")
+    sql_core_jars = get_jars_full(os.path.join(STORM_TOOLS_LIB_DIR, "sql", "core"))
+    sql_runtime_jars = get_jars_full(os.path.join(STORM_TOOLS_LIB_DIR, "sql", "runtime"))
 
     # include storm-sql-runtime jar(s) to local jar list
     local_jars.extend(sql_runtime_jars)

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/README.md
----------------------------------------------------------------------
diff --git a/external/flux/README.md b/external/flux/README.md
deleted file mode 100644
index 206ae22..0000000
--- a/external/flux/README.md
+++ /dev/null
@@ -1,872 +0,0 @@
-# flux
-A framework for creating and deploying Apache Storm streaming computations with less friction.
-
-## Definition
-**flux** |fl\u0259ks| _noun_
-
-1. The action or process of flowing or flowing out
-2. Continuous change
-3. In physics, the rate of flow of a fluid, radiant energy, or particles across a given area
-4. A substance mixed with a solid to lower its melting point
-
-## Rationale
-Bad things happen when configuration is hard-coded. No one should have to recompile or repackage an application in
-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,
-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.
-
-## Features
-
- * Easily configure and deploy Storm topologies (Both Storm core and Microbatch API) without embedding configuration
-   in your topology code
- * Support for existing topology code (see below)
- * Define Storm Core API (Spouts/Bolts) using a flexible YAML DSL
- * YAML DSL support for most Storm components (storm-kafka, storm-hdfs, storm-hbase, etc.)
- * Convenient support for multi-lang components
- * External property substitution/filtering for easily switching between configurations/environments (similar to Maven-style
-   `${variable.name}` substitution)
-
-## Usage
-
-To use Flux, add it as a dependency and package all your Storm components in a fat jar, then create a YAML document
-to define your topology (see below for YAML configuration options).
-
-### Building from Source
-The easiest way to use Flux, is to add it as a Maven dependency in you project as described below.
-
-If you would like to build Flux from source and run the unit/integration tests, you will need the following installed
-on your system:
-
-* Python 2.6.x or later
-* Node.js 0.10.x or later
-
-#### Building with unit tests enabled:
-
-```
-mvn clean install
-```
-
-#### Building with unit tests disabled:
-If you would like to build Flux without installing Python or Node.js you can simply skip the unit tests:
-
-```
-mvn clean install -DskipTests=true
-```
-
-Note that if you plan on using Flux to deploy topologies to a remote cluster, you will still need to have Python
-installed since it is required by Apache Storm.
-
-
-#### Building with integration tests enabled:
-
-```
-mvn clean install -DskipIntegration=false
-```
-
-
-### Packaging with Maven
-To enable Flux for your Storm components, you need to add it as a dependency such that it's included in the Storm
-topology jar. This can be accomplished with the Maven shade plugin (preferred) or the Maven assembly plugin (not
-recommended).
-
-#### Flux Maven Dependency
-The current version of Flux is available in Maven Central at the following coordinates:
-```xml
-<dependency>
-    <groupId>org.apache.storm</groupId>
-    <artifactId>flux-core</artifactId>
-    <version>${storm.version}</version>
-</dependency>
-```
-
-Using shell spouts and bolts requires additional Flux Wrappers library:
-```xml
-<dependency>
-    <groupId>org.apache.storm</groupId>
-    <artifactId>flux-wrappers</artifactId>
-    <version>${storm.version}</version>
-</dependency>
-```
-
-#### Creating a Flux-Enabled Topology JAR
-The example below illustrates Flux usage with the Maven shade plugin:
-
- ```xml
-<!-- include Flux and user dependencies in the shaded jar -->
-<dependencies>
-    <!-- Flux include -->
-    <dependency>
-        <groupId>org.apache.storm</groupId>
-        <artifactId>flux-core</artifactId>
-        <version>${storm.version}</version>
-    </dependency>
-    <!-- Flux Wrappers include -->
-    <dependency>
-        <groupId>org.apache.storm</groupId>
-        <artifactId>flux-wrappers</artifactId>
-        <version>${storm.version}</version>
-    </dependency>
-
-    <!-- add user dependencies here... -->
-
-</dependencies>
-<!-- create a fat jar that includes all dependencies -->
-<build>
-    <plugins>
-        <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-shade-plugin</artifactId>
-            <version>1.4</version>
-            <configuration>
-                <createDependencyReducedPom>true</createDependencyReducedPom>
-            </configuration>
-            <executions>
-                <execution>
-                    <phase>package</phase>
-                    <goals>
-                        <goal>shade</goal>
-                    </goals>
-                    <configuration>
-                        <transformers>
-                            <transformer
-                                    implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
-                            <transformer
-                                    implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
-                                <mainClass>org.apache.storm.flux.Flux</mainClass>
-                            </transformer>
-                        </transformers>
-                    </configuration>
-                </execution>
-            </executions>
-        </plugin>
-    </plugins>
-</build>
- ```
-
-### Deploying and Running a Flux Topology
-Once your topology components are packaged with the Flux dependency, you can run different topologies either locally
-or remotely using the `storm jar` command. For example, if your fat jar is named `myTopology-0.1.0-SNAPSHOT.jar` you
-could run it locally with the command:
-
-
-```bash
-storm jar myTopology-0.1.0-SNAPSHOT.jar org.apache.storm.flux.Flux --local my_config.yaml
-
-```
-
-### Command line options
-```
-usage: storm jar <my_topology_uber_jar.jar> org.apache.storm.flux.Flux
-             [options] <topology-config.yaml>
- -d,--dry-run                 Do not run or deploy the topology. Just
-                              build, validate, and print information about
-                              the topology.
- -e,--env-filter              Perform environment variable substitution.
-                              Replace keys identified with `${ENV-[NAME]}`
-                              will be replaced with the corresponding
-                              `NAME` environment value
- -f,--filter <file>           Perform property substitution. Use the
-                              specified file as a source of properties,
-                              and replace keys identified with {$[property
-                              name]} with the value defined in the
-                              properties file.
- -i,--inactive                Deploy the topology, but do not activate it.
- -l,--local                   Run the topology in local mode.
- -n,--no-splash               Suppress the printing of the splash screen.
- -q,--no-detail               Suppress the printing of topology details.
- -r,--remote                  Deploy the topology to a remote cluster.
- -R,--resource                Treat the supplied path as a classpath
-                              resource instead of a file.
- -s,--sleep <ms>              When running locally, the amount of time to
-                              sleep (in ms.) before killing the topology
-                              and shutting down the local cluster.
- -z,--zookeeper <host:port>   When running in local mode, use the
-                              ZooKeeper at the specified <host>:<port>
-                              instead of the in-process ZooKeeper.
-                              (requires Storm 0.9.3 or later)
-```
-
-**NOTE:** Flux tries to avoid command line switch collision with the `storm` command, and allows any other command line
-switches to pass through to the `storm` command.
-
-For example, you can use the `storm` command switch `-c` to override a topology configuration property. The following
-example command will run Flux and override the `nimus.host` configuration:
-
-```bash
-storm jar myTopology-0.1.0-SNAPSHOT.jar org.apache.storm.flux.Flux --remote my_config.yaml -c nimbus.host=localhost
-```
-
-### Sample output
-```
-\u2588\u2588\u2588\u2588\u2588\u2588\u2588\u2557\u2588\u2588\u2557     \u2588\u2588\u2557   \u2588\u2588\u2557\u2588\u2588\u2557  \u2588\u2588\u2557
-\u2588\u2588\u2554\u2550\u2550\u2550\u2550\u255d\u2588\u2588\u2551     \u2588\u2588\u2551   \u2588\u2588\u2551\u255a\u2588\u2588\u2557\u2588\u2588\u2554\u255d
-\u2588\u2588\u2588\u2588\u2588\u2557  \u2588\u2588\u2551     \u2588\u2588\u2551   \u2588\u2588\u2551 \u255a\u2588\u2588\u2588\u2554\u255d
-\u2588\u2588\u2554\u2550\u2550\u255d  \u2588\u2588\u2551     \u2588\u2588\u2551   \u2588\u2588\u2551 \u2588\u2588\u2554\u2588\u2588\u2557
-\u2588\u2588\u2551     \u2588\u2588\u2588\u2588\u2588\u2588\u2588\u2557\u255a\u2588\u2588\u2588\u2588\u2588\u2588\u2554\u255d\u2588\u2588\u2554\u255d \u2588\u2588\u2557
-\u255a\u2550\u255d     \u255a\u2550\u2550\u2550\u2550\u2550\u2550\u255d \u255a\u2550\u2550\u2550\u2550\u2550\u255d \u255a\u2550\u255d  \u255a\u2550\u255d
-+-         Apache Storm        -+
-+-  data FLow User eXperience  -+
-Version: 0.3.0
-Parsing file: /Users/hsimpson/Projects/donut_domination/storm/shell_test.yaml
----------- TOPOLOGY DETAILS ----------
-Name: shell-topology
---------------- SPOUTS ---------------
-sentence-spout[1](org.apache.storm.flux.wrappers.spouts.FluxShellSpout)
----------------- BOLTS ---------------
-splitsentence[1](org.apache.storm.flux.wrappers.bolts.FluxShellBolt)
-log[1](org.apache.storm.flux.wrappers.bolts.LogInfoBolt)
-count[1](org.apache.storm.testing.TestWordCounter)
---------------- STREAMS ---------------
-sentence-spout --SHUFFLE--> splitsentence
-splitsentence --FIELDS--> count
-count --SHUFFLE--> log
---------------------------------------
-Submitting topology: 'shell-topology' to remote cluster...
-```
-
-## YAML Configuration
-Flux topologies are defined in a YAML file that describes a topology. A Flux topology
-definition consists of the following:
-
-  1. A topology name
-  2. A list of topology "components" (named Java objects that will be made available in the environment)
-  3. **EITHER** (A DSL topology definition):
-      * A list of spouts, each identified by a unique ID
-      * A list of bolts, each identified by a unique ID
-      * A list of "stream" objects representing a flow of tuples between spouts and bolts
-  4. **OR** (A JVM class that can produce a `org.apache.storm.generated.StormTopology` instance:
-      * A `topologySource` definition.
-
-
-
-For example, here is a simple definition of a wordcount topology using the YAML DSL:
-
-```yaml
-name: "yaml-topology"
-config:
-  topology.workers: 1
-
-# spout definitions
-spouts:
-  - id: "spout-1"
-    className: "org.apache.storm.testing.TestWordSpout"
-    parallelism: 1
-
-# bolt definitions
-bolts:
-  - id: "bolt-1"
-    className: "org.apache.storm.testing.TestWordCounter"
-    parallelism: 1
-  - id: "bolt-2"
-    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
-    parallelism: 1
-
-#stream definitions
-streams:
-  - name: "spout-1 --> bolt-1" # name isn't used (placeholder for logging, UI, etc.)
-    from: "spout-1"
-    to: "bolt-1"
-    grouping:
-      type: FIELDS
-      args: ["word"]
-
-  - name: "bolt-1 --> bolt2"
-    from: "bolt-1"
-    to: "bolt-2"
-    grouping:
-      type: SHUFFLE
-
-
-```
-## Property Substitution/Filtering
-It's common for developers to want to easily switch between configurations, for example switching deployment between
-a development environment and a production environment. This can be accomplished by using separate YAML configuration
-files, but that approach would lead to unnecessary duplication, especially in situations where the Storm topology
-does not change, but configuration settings such as host names, ports, and parallelism paramters do.
-
-For this case, Flux offers properties filtering to allow you two externalize values to a `.properties` file and have
-them substituted before the `.yaml` file is parsed.
-
-To enable property filtering, use the `--filter` command line option and specify a `.properties` file. For example,
-if you invoked flux like so:
-
-```bash
-storm jar myTopology-0.1.0-SNAPSHOT.jar org.apache.storm.flux.Flux --local my_config.yaml --filter dev.properties
-```
-With the following `dev.properties` file:
-
-```properties
-kafka.zookeeper.hosts: localhost:2181
-```
-
-You would then be able to reference those properties by key in your `.yaml` file using `${}` syntax:
-
-```yaml
-  - id: "zkHosts"
-    className: "org.apache.storm.kafka.ZkHosts"
-    constructorArgs:
-      - "${kafka.zookeeper.hosts}"
-```
-
-In this case, Flux would replace `${kafka.zookeeper.hosts}` with `localhost:2181` before parsing the YAML contents.
-
-### Environment Variable Substitution/Filtering
-Flux also allows environment variable substitution. For example, if an environment variable named `ZK_HOSTS` if defined,
-you can reference it in a Flux YAML file with the following syntax:
-
-```
-${ENV-ZK_HOSTS}
-```
-
-## Components
-Components are essentially named object instances that are made available as configuration options for spouts and
-bolts. If you are familiar with the Spring framework, components are roughly analagous to Spring beans.
-
-Every component is identified, at a minimum, by a unique identifier (String) and a class name (String). For example,
-the following will make an instance of the `org.apache.storm.kafka.StringScheme` class available as a reference under the key
-`"stringScheme"` . This assumes the `org.apache.storm.kafka.StringScheme` has a default constructor.
-
-```yaml
-components:
-  - id: "stringScheme"
-    className: "org.apache.storm.kafka.StringScheme"
-```
-
-### Contructor Arguments, References, Properties and Configuration Methods
-
-####Constructor Arguments
-Arguments to a class constructor can be configured by adding a `contructorArgs` element to a components.
-`constructorArgs` is a list of objects that will be passed to the class' constructor. The following example creates an
-object by calling the constructor that takes a single string as an argument:
-
-```yaml
-  - id: "zkHosts"
-    className: "org.apache.storm.kafka.ZkHosts"
-    constructorArgs:
-      - "localhost:2181"
-      - true
-```
-
-####References
-Each component instance is identified by a unique id that allows it to be used/reused by other components. To
-reference an existing component, you specify the id of the component with the `ref` tag.
-
-In the following example, a component with the id `"stringScheme"` is created, and later referenced, as a an argument
-to another component's constructor:
-
-```yaml
-components:
-  - id: "stringScheme"
-    className: "org.apache.storm.kafka.StringScheme"
-
-  - id: "stringMultiScheme"
-    className: "org.apache.storm.spout.SchemeAsMultiScheme"
-    constructorArgs:
-      - ref: "stringScheme" # component with id "stringScheme" must be declared above.
-```
-
-You can also reference existing components in list via specifying the id of the components with the `reflist` tag.
-The type of the reflist will be `List<Object>`, but Flux can automatically convert List to Array (also varargs), 
-so you can use reflist on argument which type is `List<Type>`, or `Type[]`, or `Type...`.
-
-Please note that all components in the list must be same type.
-
-```yaml
-components:
-  - id: "boundCQLStatementMapperBuilder"
-    className: "org.apache.storm.cassandra.query.builder.BoundCQLStatementMapperBuilder"
-    constructorArgs:
-    - "INSERT INTO sink_cassandra (eventKey, driverId, truckId, driverName) VALUES (?, ?, ?, ?)"
-    configMethods:
-    - name: "bind"
-      args:
-      - reflist: ["FieldSelector-1", "FieldSelector-2", "FieldSelector-3", "FieldSelector-4"]
-```
-
-**N.B.:** References can only be used after (below) the object they point to has been declared.
-
-####Properties
-In addition to calling constructors with different arguments, Flux also allows you to configure components using
-JavaBean-like setter methods and fields declared as `public`:
-
-```yaml
-  - id: "spoutConfig"
-    className: "org.apache.storm.kafka.SpoutConfig"
-    constructorArgs:
-      # brokerHosts
-      - ref: "zkHosts"
-      # topic
-      - "myKafkaTopic"
-      # zkRoot
-      - "/kafkaSpout"
-      # id
-      - "myId"
-    properties:
-      - name: "ignoreZkOffsets"
-        value: true
-      - name: "scheme"
-        ref: "stringMultiScheme"
-```
-
-In the example above, the `properties` declaration will cause Flux to look for a public method in the `SpoutConfig` with
-the signature `setForceFromStart(boolean b)` and attempt to invoke it. If a setter method is not found, Flux will then
-look for a public instance variable with the name `ignoreZkOffsets` and attempt to set its value.
-
-References may also be used as property values.
-
-####Configuration Methods
-Conceptually, configuration methods are similar to Properties and Constructor Args -- they allow you to invoke an
-arbitrary method on an object after it is constructed. Configuration methods are useful for working with classes that
-don't expose JavaBean methods or have constructors that can fully configure the object. Common examples include classes
-that use the builder pattern for configuration/composition.
-
-The following YAML example creates a bolt and configures it by calling several methods:
-
-```yaml
-bolts:
-  - id: "bolt-1"
-    className: "org.apache.storm.flux.test.TestBolt"
-    parallelism: 1
-    configMethods:
-      - name: "withFoo"
-        args:
-          - "foo"
-      - name: "withNone"
-      # no args needed, so no "args" line
-      - name: "withBar"
-        args:
-          - "bar"
-      - name: "withFooBar"
-        args:
-          - "foo"
-          - "bar"
-```
-
-The signatures of the corresponding methods are as follows:
-
-```java
-    public void withFoo(String foo);
-    public void withNone(); // method with zero arguments
-    public void withBar(String bar);
-    public void withFooBar(String foo, String bar);
-```
-
-Arguments passed to configuration methods work much the same way as constructor arguments, and support references as
-well.
-
-### Using Java `enum`s in Contructor Arguments, References, Properties and Configuration Methods
-You can easily use Java `enum` values as arguments in a Flux YAML file, simply by referencing the name of the `enum`.
-
-For example, [Storm's HDFS module]() includes the following `enum` definition (simplified for brevity):
-
-```java
-public static enum Units {
-    KB, MB, GB, TB
-}
-```
-
-And the `org.apache.storm.hdfs.bolt.rotation.FileSizeRotationPolicy` class has the following constructor:
-
-```java
-public FileSizeRotationPolicy(float count, Units units)
-
-```
-The following Flux `component` definition could be used to call the constructor:
-
-```yaml
-  - id: "rotationPolicy"
-    className: "org.apache.storm.hdfs.bolt.rotation.FileSizeRotationPolicy"
-    constructorArgs:
-      - 5.0
-      - MB
-```
-
-The above definition is functionally equivalent to the following Java code:
-
-```java
-// rotate files when they reach 5MB
-FileRotationPolicy rotationPolicy = new FileSizeRotationPolicy(5.0f, Units.MB);
-```
-
-## Topology Config
-The `config` section is simply a map of Storm topology configuration parameters that will be passed to the
-`org.apache.storm.StormSubmitter` as an instance of the `org.apache.storm.Config` class:
-
-```yaml
-config:
-  topology.workers: 4
-  topology.max.spout.pending: 1000
-  topology.message.timeout.secs: 30
-```
-
-# Existing Topologies
-If you have existing Storm topologies, you can still use Flux to deploy/run/test them. This feature allows you to
-leverage Flux Constructor Arguments, References, Properties, and Topology Config declarations for existing topology
-classes.
-
-The easiest way to use an existing topology class is to define
-a `getTopology()` instance method with one of the following signatures:
-
-```java
-public StormTopology getTopology(Map<String, Object> config)
-```
-or:
-
-```java
-public StormTopology getTopology(Config config)
-```
-
-You could then use the following YAML to configure your topology:
-
-```yaml
-name: "existing-topology"
-topologySource:
-  className: "org.apache.storm.flux.test.SimpleTopology"
-```
-
-If the class you would like to use as a topology source has a different method name (i.e. not `getTopology`), you can
-override it:
-
-```yaml
-name: "existing-topology"
-topologySource:
-  className: "org.apache.storm.flux.test.SimpleTopology"
-  methodName: "getTopologyWithDifferentMethodName"
-```
-
-__N.B.:__ The specified method must accept a single argument of type `java.util.Map<String, Object>` or
-`org.apache.storm.Config`, and return a `org.apache.storm.generated.StormTopology` object.
-
-# YAML DSL
-## Spouts and Bolts
-Spout and Bolts are configured in their own respective section of the YAML configuration. Spout and Bolt definitions
-are extensions to the `component` definition that add a `parallelism` parameter that sets the parallelism  for a
-component when the topology is deployed.
-
-Because spout and bolt definitions extend `component` they support constructor arguments, references, and properties as
-well.
-
-Shell spout example:
-
-```yaml
-spouts:
-  - id: "sentence-spout"
-    className: "org.apache.storm.flux.wrappers.spouts.FluxShellSpout"
-    # shell spout constructor takes 2 arguments: String[], String[]
-    constructorArgs:
-      # command line
-      - ["node", "randomsentence.js"]
-      # output fields
-      - ["word"]
-    parallelism: 1
-```
-
-Kafka spout example:
-
-```yaml
-components:
-  - id: "stringScheme"
-    className: "org.apache.storm.kafka.StringScheme"
-
-  - id: "stringMultiScheme"
-    className: "org.apache.storm.spout.SchemeAsMultiScheme"
-    constructorArgs:
-      - ref: "stringScheme"
-
-  - id: "zkHosts"
-    className: "org.apache.storm.kafka.ZkHosts"
-    constructorArgs:
-      - "localhost:2181"
-
-# Alternative kafka config
-#  - id: "kafkaConfig"
-#    className: "org.apache.storm.kafka.KafkaConfig"
-#    constructorArgs:
-#      # brokerHosts
-#      - ref: "zkHosts"
-#      # topic
-#      - "myKafkaTopic"
-#      # clientId (optional)
-#      - "myKafkaClientId"
-
-  - id: "spoutConfig"
-    className: "org.apache.storm.kafka.SpoutConfig"
-    constructorArgs:
-      # brokerHosts
-      - ref: "zkHosts"
-      # topic
-      - "myKafkaTopic"
-      # zkRoot
-      - "/kafkaSpout"
-      # id
-      - "myId"
-    properties:
-      - name: "ignoreZkOffsets"
-        value: true
-      - name: "scheme"
-        ref: "stringMultiScheme"
-
-config:
-  topology.workers: 1
-
-# spout definitions
-spouts:
-  - id: "kafka-spout"
-    className: "org.apache.storm.kafka.KafkaSpout"
-    constructorArgs:
-      - ref: "spoutConfig"
-
-```
-
-Bolt Examples:
-
-```yaml
-# bolt definitions
-bolts:
-  - id: "splitsentence"
-    className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
-    constructorArgs:
-      # command line
-      - ["python", "splitsentence.py"]
-      # output fields
-      - ["word"]
-    parallelism: 1
-    # ...
-
-  - id: "log"
-    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
-    parallelism: 1
-    # ...
-
-  - id: "count"
-    className: "org.apache.storm.testing.TestWordCounter"
-    parallelism: 1
-    # ...
-```
-## Streams and Stream Groupings
-Streams in Flux are represented as a list of connections (Graph edges, data flow, etc.) between the Spouts and Bolts in
-a topology, with an associated Grouping definition.
-
-A Stream definition has the following properties:
-
-**`name`:** A name for the connection (optional, currently unused)
-
-**`from`:** The `id` of a Spout or Bolt that is the source (publisher)
-
-**`to`:** The `id` of a Spout or Bolt that is the destination (subscriber)
-
-**`grouping`:** The stream grouping definition for the Stream
-
-A Grouping definition has the following properties:
-
-**`type`:** The type of grouping. One of `ALL`,`CUSTOM`,`DIRECT`,`SHUFFLE`,`LOCAL_OR_SHUFFLE`,`FIELDS`,`GLOBAL`, or `NONE`.
-
-**`streamId`:** The Storm stream ID (Optional. If unspecified will use the default stream)
-
-**`args`:** For the `FIELDS` grouping, a list of field names.
-
-**`customClass`** For the `CUSTOM` grouping, a definition of custom grouping class instance
-
-The `streams` definition example below sets up a topology with the following wiring:
-
-```
-    kafka-spout --> splitsentence --> count --> log
-```
-
-
-```yaml
-#stream definitions
-# stream definitions define connections between spouts and bolts.
-# note that such connections can be cyclical
-# custom stream groupings are also supported
-
-streams:
-  - name: "kafka --> split" # name isn't used (placeholder for logging, UI, etc.)
-    from: "kafka-spout"
-    to: "splitsentence"
-    grouping:
-      type: SHUFFLE
-
-  - name: "split --> count"
-    from: "splitsentence"
-    to: "count"
-    grouping:
-      type: FIELDS
-      args: ["word"]
-
-  - name: "count --> log"
-    from: "count"
-    to: "log"
-    grouping:
-      type: SHUFFLE
-```
-
-### Custom Stream Groupings
-Custom stream groupings are defined by setting the grouping type to `CUSTOM` and defining a `customClass` parameter
-that tells Flux how to instantiate the custom class. The `customClass` definition extends `component`, so it supports
-constructor arguments, references, and properties as well.
-
-The example below creates a Stream with an instance of the `org.apache.storm.testing.NGrouping` custom stream grouping
-class.
-
-```yaml
-  - name: "bolt-1 --> bolt2"
-    from: "bolt-1"
-    to: "bolt-2"
-    grouping:
-      type: CUSTOM
-      customClass:
-        className: "org.apache.storm.testing.NGrouping"
-        constructorArgs:
-          - 1
-```
-
-## Includes and Overrides
-Flux allows you to include the contents of other YAML files, and have them treated as though they were defined in the
-same file. Includes may be either files, or classpath resources.
-
-Includes are specified as a list of maps:
-
-```yaml
-includes:
-  - resource: false
-    file: "src/test/resources/configs/shell_test.yaml"
-    override: false
-```
-
-If the `resource` property is set to `true`, the include will be loaded as a classpath resource from the value of the
-`file` attribute, otherwise it will be treated as a regular file.
-
-The `override` property controls how includes affect the values defined in the current file. If `override` is set to
-`true`, values in the included file will replace values in the current file being parsed. If `override` is set to
-`false`, values in the current file being parsed will take precedence, and the parser will refuse to replace them.
-
-**N.B.:** Includes are not yet recursive. Includes from included files will be ignored.
-
-
-## Basic Word Count Example
-
-This example uses a spout implemented in JavaScript, a bolt implemented in Python, and a bolt implemented in Java
-
-Topology YAML config:
-
-```yaml
----
-name: "shell-topology"
-config:
-  topology.workers: 1
-
-# spout definitions
-spouts:
-  - id: "sentence-spout"
-    className: "org.apache.storm.flux.wrappers.spouts.FluxShellSpout"
-    # shell spout constructor takes 2 arguments: String[], String[]
-    constructorArgs:
-      # command line
-      - ["node", "randomsentence.js"]
-      # output fields
-      - ["word"]
-    parallelism: 1
-
-# bolt definitions
-bolts:
-  - id: "splitsentence"
-    className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
-    constructorArgs:
-      # command line
-      - ["python", "splitsentence.py"]
-      # output fields
-      - ["word"]
-    parallelism: 1
-
-  - id: "log"
-    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
-    parallelism: 1
-
-  - id: "count"
-    className: "org.apache.storm.testing.TestWordCounter"
-    parallelism: 1
-
-#stream definitions
-# stream definitions define connections between spouts and bolts.
-# note that such connections can be cyclical
-# custom stream groupings are also supported
-
-streams:
-  - name: "spout --> split" # name isn't used (placeholder for logging, UI, etc.)
-    from: "sentence-spout"
-    to: "splitsentence"
-    grouping:
-      type: SHUFFLE
-
-  - name: "split --> count"
-    from: "splitsentence"
-    to: "count"
-    grouping:
-      type: FIELDS
-      args: ["word"]
-
-  - name: "count --> log"
-    from: "count"
-    to: "log"
-    grouping:
-      type: SHUFFLE
-```
-
-
-## Micro-Batching (Trident) API Support
-Currenty, the Flux YAML DSL only supports the Core Storm API, but support for Storm's micro-batching API is planned.
-
-To use Flux with a Trident topology, define a topology getter method and reference it in your YAML config:
-
-```yaml
-name: "my-trident-topology"
-
-config:
-  topology.workers: 1
-
-topologySource:
-  className: "org.apache.storm.flux.test.TridentTopologySource"
-  # Flux will look for "getTopology", this will override that.
-  methodName: "getTopologyWithDifferentMethodName"
-```
-
-## Committer Sponsors
-
- * P. Taylor Goetz ([ptgoetz@apache.org](mailto:ptgoetz@apache.org))

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/pom.xml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/pom.xml b/external/flux/flux-core/pom.xml
deleted file mode 100644
index 1cf4a9b..0000000
--- a/external/flux/flux-core/pom.xml
+++ /dev/null
@@ -1,112 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements.  See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License.  You may obtain a copy of the License at
-
-     http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <modelVersion>4.0.0</modelVersion>
-
-    <parent>
-        <groupId>org.apache.storm</groupId>
-        <artifactId>flux</artifactId>
-        <version>2.0.0-SNAPSHOT</version>
-        <relativePath>../pom.xml</relativePath>
-    </parent>
-
-    <artifactId>flux-core</artifactId>
-    <packaging>jar</packaging>
-
-    <name>flux-core</name>
-
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>flux-wrappers</artifactId>
-            <version>${project.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>storm-kafka</artifactId>
-            <version>${project.version}</version>
-            <scope>test</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>storm-hdfs</artifactId>
-            <version>${project.version}</version>
-            <scope>test</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>storm-hbase</artifactId>
-            <version>${project.version}</version>
-            <scope>test</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.yaml</groupId>
-            <artifactId>snakeyaml</artifactId>
-        </dependency>
-    </dependencies>
-    <build>
-        <resources>
-            <resource>
-                <directory>src/main/resources</directory>
-                <filtering>true</filtering>
-            </resource>
-        </resources>
-        <plugins>
-        <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-shade-plugin</artifactId>
-            <configuration>
-                <createDependencyReducedPom>true</createDependencyReducedPom>
-                <filters>
-                    <filter>
-                        <artifact>*:*</artifact>
-                        <excludes>
-                            <exclude>META-INF/*.SF</exclude>
-                            <exclude>META-INF/*.sf</exclude>
-                            <exclude>META-INF/*.DSA</exclude>
-                            <exclude>META-INF/*.dsa</exclude>
-                            <exclude>META-INF/*.RSA</exclude>
-                            <exclude>META-INF/*.rsa</exclude>
-                            <exclude>META-INF/*.EC</exclude>
-                            <exclude>META-INF/*.ec</exclude>
-                            <exclude>META-INF/MSFTSIG.SF</exclude>
-                            <exclude>META-INF/MSFTSIG.RSA</exclude>
-                        </excludes>
-                    </filter>
-                </filters>
-            </configuration>
-            <executions>
-                <execution>
-                    <phase>package</phase>
-                    <goals>
-                        <goal>shade</goal>
-                    </goals>
-                    <configuration>
-                        <transformers>
-                            <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
-                            <transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
-                                <mainClass>org.apache.storm.flux.Flux</mainClass>
-                            </transformer>
-                        </transformers>
-                    </configuration>
-                </execution>
-            </executions>
-        </plugin>
-        </plugins>
-    </build>
-</project>

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/main/java/org/apache/storm/flux/Flux.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/main/java/org/apache/storm/flux/Flux.java b/external/flux/flux-core/src/main/java/org/apache/storm/flux/Flux.java
deleted file mode 100644
index 982765f..0000000
--- a/external/flux/flux-core/src/main/java/org/apache/storm/flux/Flux.java
+++ /dev/null
@@ -1,261 +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.flux;
-
-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.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.*;
-import java.util.Map;
-import java.util.Properties;
-
-/**
- * Flux entry point.
- *
- */
-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";
-    private static final String OPTION_SLEEP = "sleep";
-    private static final String OPTION_DRY_RUN = "dry-run";
-    private static final String OPTION_NO_DETAIL = "no-detail";
-    private static final String OPTION_NO_SPLASH = "no-splash";
-    private static final String OPTION_INACTIVE = "inactive";
-    private static final String OPTION_ZOOKEEPER = "zookeeper";
-    private static final String OPTION_FILTER = "filter";
-    private static final String OPTION_ENV_FILTER = "env-filter";
-
-    public static void main(String[] args) throws Exception {
-        Options options = new Options();
-
-        options.addOption(option(0, "l", OPTION_LOCAL, "Run the topology in local mode."));
-
-        options.addOption(option(0, "r", OPTION_REMOTE, "Deploy the topology to a remote cluster."));
-
-        options.addOption(option(0, "R", OPTION_RESOURCE, "Treat the supplied path as a classpath resource instead of a file."));
-
-        options.addOption(option(1, "s", OPTION_SLEEP, "ms", "When running locally, the amount of time to sleep (in ms.) " +
-                "before killing the topology and shutting down the local cluster."));
-
-        options.addOption(option(0, "d", OPTION_DRY_RUN, "Do not run or deploy the topology. Just build, validate, " +
-                "and print information about the topology."));
-
-        options.addOption(option(0, "q", OPTION_NO_DETAIL, "Suppress the printing of topology details."));
-
-        options.addOption(option(0, "n", OPTION_NO_SPLASH, "Suppress the printing of the splash screen."));
-
-        options.addOption(option(0, "i", OPTION_INACTIVE, "Deploy the topology, but do not activate it."));
-
-        options.addOption(option(1, "z", OPTION_ZOOKEEPER, "host:port", "When running in local mode, use the ZooKeeper at the " +
-                "specified <host>:<port> instead of the in-process ZooKeeper. (requires Storm 0.9.3 or later)"));
-
-        options.addOption(option(1, "f", OPTION_FILTER, "file", "Perform property substitution. Use the specified file " +
-                "as a source of properties, and replace keys identified with {$[property name]} with the value defined " +
-                "in the properties file."));
-
-        options.addOption(option(0, "e", OPTION_ENV_FILTER, "Perform environment variable substitution. Replace keys" +
-                "identified with `${ENV-[NAME]}` will be replaced with the corresponding `NAME` environment value"));
-
-        CommandLineParser parser = new BasicParser();
-        CommandLine cmd = parser.parse(options, args);
-
-        if (cmd.getArgs().length != 1) {
-            usage(options);
-            System.exit(1);
-        }
-        runCli(cmd);
-    }
-
-    private static Option option(int argCount, String shortName, String longName, String description){
-       return option(argCount, shortName, longName, longName, description);
-    }
-
-    private static Option option(int argCount, String shortName, String longName, String argName, String description){
-        Option option = OptionBuilder.hasArgs(argCount)
-                .withArgName(argName)
-                .withLongOpt(longName)
-                .withDescription(description)
-                .create(shortName);
-        return option;
-    }
-
-    private static void usage(Options options) {
-        HelpFormatter formatter = new HelpFormatter();
-        formatter.printHelp("storm jar <my_topology_uber_jar.jar> " +
-                Flux.class.getName() +
-                " [options] <topology-config.yaml>", options);
-    }
-
-    private static void runCli(CommandLine cmd)throws Exception {
-        if(!cmd.hasOption(OPTION_NO_SPLASH)) {
-            printSplash();
-        }
-
-        boolean dumpYaml = cmd.hasOption("dump-yaml");
-
-        TopologyDef topologyDef = null;
-        String filePath = (String)cmd.getArgList().get(0);
-
-        // TODO conditionally load properties from a file our resource
-        String filterProps = null;
-        if(cmd.hasOption(OPTION_FILTER)){
-            filterProps = cmd.getOptionValue(OPTION_FILTER);
-        }
-
-
-        boolean envFilter = cmd.hasOption(OPTION_ENV_FILTER);
-        if(cmd.hasOption(OPTION_RESOURCE)){
-            printf("Parsing classpath resource: %s", filePath);
-            topologyDef = FluxParser.parseResource(filePath, dumpYaml, true, filterProps, envFilter);
-        } else {
-            printf("Parsing file: %s",
-                    new File(filePath).getAbsolutePath());
-            topologyDef = FluxParser.parseFile(filePath, dumpYaml, true, filterProps, envFilter);
-        }
-
-
-        String topologyName = topologyDef.getName();
-        // merge contents of `config` into topology config
-        Config conf = FluxBuilder.buildConfig(topologyDef);
-        ExecutionContext context = new ExecutionContext(topologyDef, conf);
-        StormTopology topology = FluxBuilder.buildTopology(context);
-
-        if(!cmd.hasOption(OPTION_NO_DETAIL)){
-            printTopologyInfo(context);
-        }
-
-        if(!cmd.hasOption(OPTION_DRY_RUN)) {
-            if (cmd.hasOption(OPTION_REMOTE)) {
-                LOG.info("Running remotely...");
-                // should the topology be active or inactive
-                SubmitOptions submitOptions = null;
-                if(cmd.hasOption(OPTION_INACTIVE)){
-                    LOG.info("Deploying topology in an INACTIVE state...");
-                    submitOptions = new SubmitOptions(TopologyInitialStatus.INACTIVE);
-                } else {
-                    LOG.info("Deploying topology in an ACTIVE state...");
-                    submitOptions = new SubmitOptions(TopologyInitialStatus.ACTIVE);
-                }
-                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();
-                }
-            }
-        }
-    }
-
-    static void printTopologyInfo(ExecutionContext ctx){
-        TopologyDef t = ctx.getTopologyDef();
-        if(t.isDslTopology()) {
-            print("---------- TOPOLOGY DETAILS ----------");
-
-            printf("Topology Name: %s", t.getName());
-            print("--------------- SPOUTS ---------------");
-            for (SpoutDef s : t.getSpouts()) {
-                printf("%s [%d] (%s)", s.getId(), s.getParallelism(), s.getClassName());
-            }
-            print("---------------- BOLTS ---------------");
-            for (BoltDef b : t.getBolts()) {
-                printf("%s [%d] (%s)", b.getId(), b.getParallelism(), b.getClassName());
-            }
-
-            print("--------------- STREAMS ---------------");
-            for (StreamDef sd : t.getStreams()) {
-                printf("%s --%s--> %s", sd.getFrom(), sd.getGrouping().getType(), sd.getTo());
-            }
-            print("--------------------------------------");
-        }
-    }
-
-    // save a little typing
-    private static void printf(String format, Object... args){
-        print(String.format(format, args));
-    }
-
-    private static void print(String string){
-        System.out.println(string);
-    }
-
-    private static void printSplash() throws IOException {
-        // banner
-        InputStream is = Flux.class.getResourceAsStream("/splash.txt");
-        if(is != null){
-            InputStreamReader isr = new InputStreamReader(is, "UTF-8");
-            BufferedReader br = new BufferedReader(isr);
-            String line = null;
-            while((line = br.readLine()) != null){
-                System.out.println(line);
-            }
-        }
-    }
-}


[23/23] storm git commit: STORM-2453: CHANGELOG

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


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

Branch: refs/heads/master
Commit: c652d3ffc2e9a701f0fd67f787f9e257499d6cbb
Parents: 1dd4bbc
Author: Jungtaek Lim <ka...@gmail.com>
Authored: Thu Apr 6 08:18:49 2017 +0900
Committer: Jungtaek Lim <ka...@gmail.com>
Committed: Thu Apr 6 08:18:49 2017 +0900

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


http://git-wip-us.apache.org/repos/asf/storm/blob/c652d3ff/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 846823d..7ff80c1 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
 \ufeff## 2.0.0
+ * STORM-2453: Move non-connectors into the top directory
  * STORM-2363: Provide configuration to set the number of RollingWindow
  * STORM-2058: Fix Maven warnings about missing reporting.plugins.plugin.version
  * STORM-1369: Add MapState implementation to storm-cassandra


[19/23] storm git commit: STORM-2453 Move non-connectors into the top directory

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/java/org/apache/storm/flux/TCKTest.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/java/org/apache/storm/flux/TCKTest.java b/external/flux/flux-core/src/test/java/org/apache/storm/flux/TCKTest.java
deleted file mode 100644
index bb5d7ec..0000000
--- a/external/flux/flux-core/src/test/java/org/apache/storm/flux/TCKTest.java
+++ /dev/null
@@ -1,256 +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.flux;
-
-import org.apache.storm.Config;
-import org.apache.storm.generated.StormTopology;
-import org.apache.storm.flux.model.ExecutionContext;
-import org.apache.storm.flux.model.TopologyDef;
-import org.apache.storm.flux.parser.FluxParser;
-import org.apache.storm.flux.test.TestBolt;
-import org.junit.Test;
-
-import java.io.File;
-
-import static org.junit.Assert.*;
-
-public class TCKTest {
-    @Test
-    public void testTCK() throws Exception {
-        TopologyDef topologyDef = FluxParser.parseResource("/configs/tck.yaml", false, true, null, false);
-        Config conf = FluxBuilder.buildConfig(topologyDef);
-        ExecutionContext context = new ExecutionContext(topologyDef, conf);
-        StormTopology topology = FluxBuilder.buildTopology(context);
-        assertNotNull(topology);
-        topology.validate();
-    }
-
-    @Test
-    public void testShellComponents() throws Exception {
-        TopologyDef topologyDef = FluxParser.parseResource("/configs/shell_test.yaml", false, true, null, false);
-        Config conf = FluxBuilder.buildConfig(topologyDef);
-        ExecutionContext context = new ExecutionContext(topologyDef, conf);
-        StormTopology topology = FluxBuilder.buildTopology(context);
-        assertNotNull(topology);
-        topology.validate();
-    }
-
-    @Test(expected = IllegalArgumentException.class)
-    public void testBadShellComponents() throws Exception {
-        TopologyDef topologyDef = FluxParser.parseResource("/configs/bad_shell_test.yaml", false, true, null, false);
-        Config conf = FluxBuilder.buildConfig(topologyDef);
-        ExecutionContext context = new ExecutionContext(topologyDef, conf);
-        StormTopology topology = FluxBuilder.buildTopology(context);
-        assertNotNull(topology);
-        topology.validate();
-    }
-
-    @Test
-    public void testKafkaSpoutConfig() throws Exception {
-        TopologyDef topologyDef = FluxParser.parseResource("/configs/kafka_test.yaml", false, true, null, false);
-        Config conf = FluxBuilder.buildConfig(topologyDef);
-        ExecutionContext context = new ExecutionContext(topologyDef, conf);
-        StormTopology topology = FluxBuilder.buildTopology(context);
-        assertNotNull(topology);
-        topology.validate();
-    }
-
-    @Test
-    public void testLoadFromResource() throws Exception {
-        TopologyDef topologyDef = FluxParser.parseResource("/configs/kafka_test.yaml", false, true, null, false);
-        Config conf = FluxBuilder.buildConfig(topologyDef);
-        ExecutionContext context = new ExecutionContext(topologyDef, conf);
-        StormTopology topology = FluxBuilder.buildTopology(context);
-        assertNotNull(topology);
-        topology.validate();
-    }
-
-
-    @Test
-    public void testHdfs() throws Exception {
-        TopologyDef topologyDef = FluxParser.parseResource("/configs/hdfs_test.yaml", false, true, null, false);
-        Config conf = FluxBuilder.buildConfig(topologyDef);
-        ExecutionContext context = new ExecutionContext(topologyDef, conf);
-        StormTopology topology = FluxBuilder.buildTopology(context);
-        assertNotNull(topology);
-        topology.validate();
-    }
-
-    @Test
-    public void testDiamondTopology() throws Exception {
-        TopologyDef topologyDef = FluxParser.parseResource("/configs/diamond-topology.yaml", false, true, null, false);
-        Config conf = FluxBuilder.buildConfig(topologyDef);
-        ExecutionContext context = new ExecutionContext(topologyDef, conf);
-        StormTopology topology = FluxBuilder.buildTopology(context);
-        assertNotNull(topology);
-        topology.validate();
-    }
-
-
-    @Test
-    public void testHbase() throws Exception {
-        TopologyDef topologyDef = FluxParser.parseResource("/configs/simple_hbase.yaml", false, true, null, false);
-        Config conf = FluxBuilder.buildConfig(topologyDef);
-        ExecutionContext context = new ExecutionContext(topologyDef, conf);
-        StormTopology topology = FluxBuilder.buildTopology(context);
-        assertNotNull(topology);
-        topology.validate();
-    }
-
-    @Test(expected = IllegalArgumentException.class)
-    public void testBadHbase() throws Exception {
-        TopologyDef topologyDef = FluxParser.parseResource("/configs/bad_hbase.yaml", false, true, null, false);
-        Config conf = FluxBuilder.buildConfig(topologyDef);
-        ExecutionContext context = new ExecutionContext(topologyDef, conf);
-        StormTopology topology = FluxBuilder.buildTopology(context);
-        assertNotNull(topology);
-        topology.validate();
-    }
-
-    @Test
-    public void testIncludes() throws Exception {
-        TopologyDef topologyDef = FluxParser.parseResource("/configs/include_test.yaml", false, true, null, false);
-        Config conf = FluxBuilder.buildConfig(topologyDef);
-        ExecutionContext context = new ExecutionContext(topologyDef, conf);
-        StormTopology topology = FluxBuilder.buildTopology(context);
-        assertNotNull(topology);
-        assertTrue(topologyDef.getName().equals("include-topology"));
-        assertTrue(topologyDef.getBolts().size() > 0);
-        assertTrue(topologyDef.getSpouts().size() > 0);
-        topology.validate();
-    }
-
-    @Test
-    public void testTopologySource() throws Exception {
-        TopologyDef topologyDef = FluxParser.parseResource("/configs/existing-topology.yaml", false, true, null, false);
-        assertTrue(topologyDef.validate());
-        Config conf = FluxBuilder.buildConfig(topologyDef);
-        ExecutionContext context = new ExecutionContext(topologyDef, conf);
-        StormTopology topology = FluxBuilder.buildTopology(context);
-        assertNotNull(topology);
-        topology.validate();
-    }
-
-    @Test
-    public void testTopologySourceWithReflection() throws Exception {
-        TopologyDef topologyDef = FluxParser.parseResource("/configs/existing-topology-reflection.yaml", false, true, null, false);
-        assertTrue(topologyDef.validate());
-        Config conf = FluxBuilder.buildConfig(topologyDef);
-        ExecutionContext context = new ExecutionContext(topologyDef, conf);
-        StormTopology topology = FluxBuilder.buildTopology(context);
-        assertNotNull(topology);
-        topology.validate();
-    }
-
-    @Test
-    public void testTopologySourceWithConfigParam() throws Exception {
-        TopologyDef topologyDef = FluxParser.parseResource("/configs/existing-topology-reflection-config.yaml", false, true, null, false);
-        assertTrue(topologyDef.validate());
-        Config conf = FluxBuilder.buildConfig(topologyDef);
-        ExecutionContext context = new ExecutionContext(topologyDef, conf);
-        StormTopology topology = FluxBuilder.buildTopology(context);
-        assertNotNull(topology);
-        topology.validate();
-    }
-
-    @Test
-    public void testTopologySourceWithMethodName() throws Exception {
-        TopologyDef topologyDef = FluxParser.parseResource("/configs/existing-topology-method-override.yaml", false, true, null, false);
-        assertTrue(topologyDef.validate());
-        Config conf = FluxBuilder.buildConfig(topologyDef);
-        ExecutionContext context = new ExecutionContext(topologyDef, conf);
-        StormTopology topology = FluxBuilder.buildTopology(context);
-        assertNotNull(topology);
-        topology.validate();
-    }
-
-
-    @Test
-    public void testTridentTopologySource() throws Exception {
-        TopologyDef topologyDef = FluxParser.parseResource("/configs/existing-topology-trident.yaml", false, true, null, false);
-        assertTrue(topologyDef.validate());
-        Config conf = FluxBuilder.buildConfig(topologyDef);
-        ExecutionContext context = new ExecutionContext(topologyDef, conf);
-        StormTopology topology = FluxBuilder.buildTopology(context);
-        assertNotNull(topology);
-        topology.validate();
-    }
-
-    @Test(expected = IllegalArgumentException.class)
-    public void testInvalidTopologySource() throws Exception {
-        TopologyDef topologyDef = FluxParser.parseResource("/configs/invalid-existing-topology.yaml", false, true, null, false);
-        assertFalse("Topology config is invalid.", topologyDef.validate());
-        Config conf = FluxBuilder.buildConfig(topologyDef);
-        ExecutionContext context = new ExecutionContext(topologyDef, conf);
-        StormTopology topology = FluxBuilder.buildTopology(context);
-    }
-
-
-    @Test
-    public void testTopologySourceWithGetMethodName() throws Exception {
-        TopologyDef topologyDef = FluxParser.parseResource("/configs/existing-topology-reflection.yaml", false, true, null, false);
-        assertTrue(topologyDef.validate());
-        Config conf = FluxBuilder.buildConfig(topologyDef);
-        ExecutionContext context = new ExecutionContext(topologyDef, conf);
-        StormTopology topology = FluxBuilder.buildTopology(context);
-        assertNotNull(topology);
-        topology.validate();
-    }
-
-    @Test
-    public void testTopologySourceWithConfigMethods() throws Exception {
-        TopologyDef topologyDef = FluxParser.parseResource("/configs/config-methods-test.yaml", false, true, null, false);
-        assertTrue(topologyDef.validate());
-        Config conf = FluxBuilder.buildConfig(topologyDef);
-        ExecutionContext context = new ExecutionContext(topologyDef, conf);
-        StormTopology topology = FluxBuilder.buildTopology(context);
-        assertNotNull(topology);
-        topology.validate();
-
-        // make sure the property was actually set
-        TestBolt bolt = (TestBolt)context.getBolt("bolt-1");
-        assertTrue(bolt.getFoo().equals("foo"));
-        assertTrue(bolt.getBar().equals("bar"));
-        assertTrue(bolt.getFooBar().equals("foobar"));
-        assertArrayEquals(new TestBolt.TestClass[] {new TestBolt.TestClass("foo"), new TestBolt.TestClass("bar"), new TestBolt.TestClass("baz")}, bolt.getClasses());
-    }
-
-    @Test
-    public void testVariableSubstitution() throws Exception {
-        TopologyDef topologyDef = FluxParser.parseResource("/configs/substitution-test.yaml", false, true, "src/test/resources/configs/test.properties", true);
-        assertTrue(topologyDef.validate());
-        Config conf = FluxBuilder.buildConfig(topologyDef);
-        ExecutionContext context = new ExecutionContext(topologyDef, conf);
-        StormTopology topology = FluxBuilder.buildTopology(context);
-        assertNotNull(topology);
-        topology.validate();
-
-        // test basic substitution
-        assertEquals("Property not replaced.",
-                "substitution-topology",
-                context.getTopologyDef().getName());
-
-        // test environment variable substitution
-        // $PATH should be defined on most systems
-        String envPath = System.getenv().get("PATH");
-        assertEquals("ENV variable not replaced.",
-                envPath,
-                context.getTopologyDef().getConfig().get("test.env.value"));
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/java/org/apache/storm/flux/multilang/MultilangEnvirontmentTest.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/java/org/apache/storm/flux/multilang/MultilangEnvirontmentTest.java b/external/flux/flux-core/src/test/java/org/apache/storm/flux/multilang/MultilangEnvirontmentTest.java
deleted file mode 100644
index dcded17..0000000
--- a/external/flux/flux-core/src/test/java/org/apache/storm/flux/multilang/MultilangEnvirontmentTest.java
+++ /dev/null
@@ -1,89 +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.flux.multilang;
-
-
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.ByteArrayOutputStream;
-import java.io.InputStream;
-import java.io.OutputStream;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * Sanity checks to make sure we can at least invoke the shells used.
- */
-public class MultilangEnvirontmentTest {
-    private static final Logger LOG = LoggerFactory.getLogger(MultilangEnvirontmentTest.class);
-
-    @Test
-    public void testInvokePython() throws Exception {
-        String[] command = new String[]{"python", "--version"};
-        int exitVal = invokeCommand(command);
-        assertEquals("Exit value for python is 0.", 0, exitVal);
-    }
-
-    @Test
-    public void testInvokeNode() throws Exception {
-        String[] command = new String[]{"node", "--version"};
-        int exitVal = invokeCommand(command);
-        assertEquals("Exit value for node is 0.", 0, exitVal);
-    }
-
-    private static class StreamRedirect implements Runnable {
-        private InputStream in;
-        private OutputStream out;
-
-        public StreamRedirect(InputStream in, OutputStream out) {
-            this.in = in;
-            this.out = out;
-        }
-
-        @Override
-        public void run() {
-            try {
-                int i = -1;
-                while ((i = this.in.read()) != -1) {
-                    out.write(i);
-                }
-                this.in.close();
-                this.out.close();
-            } catch (Exception e) {
-                e.printStackTrace();
-            }
-        }
-    }
-
-    private int invokeCommand(String[] args) throws Exception {
-        LOG.debug("Invoking command: {}", args);
-
-        ProcessBuilder pb = new ProcessBuilder(args);
-        pb.redirectErrorStream(true);
-        final Process proc = pb.start();
-
-        ByteArrayOutputStream out = new ByteArrayOutputStream();
-        Thread t = new Thread(new StreamRedirect(proc.getInputStream(), out));
-        t.start();
-        int exitVal = proc.waitFor();
-        LOG.debug("Command result: {}", out.toString());
-        return exitVal;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopology.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopology.java b/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopology.java
deleted file mode 100644
index 8e3cda2..0000000
--- a/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopology.java
+++ /dev/null
@@ -1,59 +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.flux.test;
-
-import org.apache.storm.generated.StormTopology;
-import org.apache.storm.topology.TopologyBuilder;
-import org.apache.storm.flux.api.TopologySource;
-import org.apache.storm.flux.wrappers.bolts.LogInfoBolt;
-import org.apache.storm.flux.wrappers.spouts.FluxShellSpout;
-
-import java.util.Map;
-
-/**
- * Test topology source that does not implement TopologySource, but has the same
- * `getTopology()` method.
- */
-public class SimpleTopology{
-
-
-    public SimpleTopology(){}
-
-    public SimpleTopology(String foo, String bar){}
-
-    public StormTopology getTopologyWithDifferentMethodName(Map<String, Object> config){
-        return getTopology(config);
-    }
-
-
-    public StormTopology getTopology(Map<String, Object> config) {
-        TopologyBuilder builder = new TopologyBuilder();
-
-        // spouts
-        FluxShellSpout spout = new FluxShellSpout(
-                new String[]{"node", "randomsentence.js"},
-                new String[]{"word"});
-        builder.setSpout("sentence-spout", spout, 1);
-
-        // bolts
-        builder.setBolt("log-bolt", new LogInfoBolt(), 1)
-                .shuffleGrouping("sentence-spout");
-
-        return builder.createTopology();
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologySource.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologySource.java b/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologySource.java
deleted file mode 100644
index 2fadacf..0000000
--- a/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologySource.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.test;
-
-import org.apache.storm.generated.StormTopology;
-import org.apache.storm.topology.TopologyBuilder;
-import org.apache.storm.flux.api.TopologySource;
-import org.apache.storm.flux.wrappers.bolts.LogInfoBolt;
-import org.apache.storm.flux.wrappers.spouts.FluxShellSpout;
-
-import java.util.Map;
-
-public class SimpleTopologySource implements TopologySource {
-
-
-    public SimpleTopologySource(){}
-
-    public SimpleTopologySource(String foo, String bar){}
-
-
-    @Override
-    public StormTopology getTopology(Map<String, Object> config) {
-        TopologyBuilder builder = new TopologyBuilder();
-
-        // spouts
-        FluxShellSpout spout = new FluxShellSpout(
-                new String[]{"node", "randomsentence.js"},
-                new String[]{"word"});
-        builder.setSpout("sentence-spout", spout, 1);
-
-        // bolts
-        builder.setBolt("log-bolt", new LogInfoBolt(), 1)
-                .shuffleGrouping("sentence-spout");
-
-        return builder.createTopology();
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologyWithConfigParam.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologyWithConfigParam.java b/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologyWithConfigParam.java
deleted file mode 100644
index 8b0aa05..0000000
--- a/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/SimpleTopologyWithConfigParam.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.storm.flux.test;
-
-import org.apache.storm.Config;
-import org.apache.storm.generated.StormTopology;
-import org.apache.storm.topology.TopologyBuilder;
-import org.apache.storm.flux.wrappers.bolts.LogInfoBolt;
-import org.apache.storm.flux.wrappers.spouts.FluxShellSpout;
-
-import java.util.Map;
-
-/**
- * Test topology source that does not implement TopologySource, but has the same
- * `getTopology()` method.
- */
-public class SimpleTopologyWithConfigParam {
-
-
-    public SimpleTopologyWithConfigParam(){}
-
-    public SimpleTopologyWithConfigParam(String foo, String bar){}
-
-
-    public StormTopology getTopology(Config config) {
-        TopologyBuilder builder = new TopologyBuilder();
-
-        // spouts
-        FluxShellSpout spout = new FluxShellSpout(
-                new String[]{"node", "randomsentence.js"},
-                new String[]{"word"});
-        builder.setSpout("sentence-spout", spout, 1);
-
-        // bolts
-        builder.setBolt("log-bolt", new LogInfoBolt(), 1)
-                .shuffleGrouping("sentence-spout");
-
-        return builder.createTopology();
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/TestBolt.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/TestBolt.java b/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/TestBolt.java
deleted file mode 100644
index 28d11b6..0000000
--- a/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/TestBolt.java
+++ /dev/null
@@ -1,131 +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.flux.test;
-
-import org.apache.storm.topology.BasicOutputCollector;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.topology.base.BaseBasicBolt;
-import org.apache.storm.tuple.Tuple;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Serializable;
-
-
-public class TestBolt extends BaseBasicBolt {
-    private static final Logger LOG = LoggerFactory.getLogger(TestBolt.class);
-
-    private String foo;
-    private String bar;
-    private String fooBar;
-    private String none;
-    private TestClass[] classes;
-
-    public static class TestClass implements Serializable {
-        private String field;
-
-        public TestClass(String field) {
-            this.field = field;
-        }
-
-        public String getField() {
-            return field;
-        }
-
-        @Override
-        public boolean equals(Object o) {
-            if (this == o) return true;
-            if (!(o instanceof TestClass)) return false;
-
-            TestClass testClass = (TestClass) o;
-
-            return getField() != null ? getField().equals(testClass.getField()) : testClass.getField() == null;
-        }
-
-        @Override
-        public int hashCode() {
-            return getField() != null ? getField().hashCode() : 0;
-        }
-    }
-
-
-    public static enum TestEnum {
-        FOO,
-        BAR
-    }
-
-    public TestBolt(TestEnum te){
-
-    }
-
-    public TestBolt(TestEnum te, float f){
-
-    }
-
-    public TestBolt(TestEnum te, float f, boolean b){
-
-    }
-
-    public TestBolt(TestEnum te, float f, boolean b, TestClass... str) {
-
-    }
-
-    @Override
-    public void execute(Tuple tuple, BasicOutputCollector basicOutputCollector) {
-        LOG.info("{}", tuple);
-    }
-
-    @Override
-    public void declareOutputFields(OutputFieldsDeclarer outputFieldsDeclarer) {
-
-    }
-
-    // config methods
-    public void withFoo(String foo){
-        this.foo = foo;
-    }
-    public void withNone(){
-        this.none = "hit";
-    }
-    public void withBar(String bar){
-        this.bar = bar;
-    }
-
-    public void withFooBar(String foo, String bar){
-        this.fooBar = foo + bar;
-    }
-
-    public void withClasses(TestClass...classes) {
-        this.classes = classes;
-    }
-
-    public String getFoo(){
-        return this.foo;
-    }
-    public String getBar(){
-        return this.bar;
-    }
-
-    public String getFooBar(){
-        return this.fooBar;
-    }
-
-    public TestClass[] getClasses() {
-        return classes;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/TridentTopologySource.java
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/TridentTopologySource.java b/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/TridentTopologySource.java
deleted file mode 100644
index 36b272b..0000000
--- a/external/flux/flux-core/src/test/java/org/apache/storm/flux/test/TridentTopologySource.java
+++ /dev/null
@@ -1,71 +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.flux.test;
-
-import org.apache.storm.Config;
-import org.apache.storm.generated.StormTopology;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Values;
-import org.apache.storm.kafka.StringScheme;
-import org.apache.storm.trident.TridentTopology;
-import org.apache.storm.trident.operation.BaseFunction;
-import org.apache.storm.trident.operation.TridentCollector;
-import org.apache.storm.trident.operation.builtin.Count;
-import org.apache.storm.trident.testing.FixedBatchSpout;
-import org.apache.storm.trident.testing.MemoryMapState;
-import org.apache.storm.trident.tuple.TridentTuple;
-
-/**
- * Basic Trident example that will return a `StormTopology` from a `getTopology()` method.
- */
-public class TridentTopologySource {
-
-    private FixedBatchSpout spout;
-
-    public StormTopology getTopology(Config config) {
-
-        this.spout = new FixedBatchSpout(new Fields("sentence"), 20,
-                new Values("one two"),
-                new Values("two three"),
-                new Values("three four"),
-                new Values("four five"),
-                new Values("five six")
-        );
-
-
-        TridentTopology trident = new TridentTopology();
-
-        trident.newStream("wordcount", spout).name("sentence").parallelismHint(1).shuffle()
-                .each(new Fields("sentence"), new Split(), new Fields("word"))
-                .parallelismHint(1)
-                .groupBy(new Fields("word"))
-                .persistentAggregate(new MemoryMapState.Factory(), new Count(), new Fields("count"))
-                .parallelismHint(1);
-        return trident.build();
-    }
-
-    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));
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/bad_hbase.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/bad_hbase.yaml b/external/flux/flux-core/src/test/resources/configs/bad_hbase.yaml
deleted file mode 100644
index a29e314..0000000
--- a/external/flux/flux-core/src/test/resources/configs/bad_hbase.yaml
+++ /dev/null
@@ -1,98 +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.
-
-# Test ability to wire together shell spouts/bolts
----
-
-# topology definition
-# name to be used when submitting
-name: "hbase-wordcount"
-
-# Components
-# Components are analagous to Spring beans. They are meant to be used as constructor,
-# property(setter), and builder arguments.
-#
-# for the time being, components must be declared in the order they are referenced
-
-components:
-  - id: "columnFields"
-    className: "org.apache.storm.tuple.Fields"
-    constructorArgs:
-      - ["word"]
-
-  - id: "counterFields"
-    className: "org.apache.storm.tuple.Fields"
-    constructorArgs:
-      # !!! the following won't work, and should thow an IllegalArgumentException...
-      - "count"
-
-  - id: "mapper"
-    className: "org.apache.storm.hbase.bolt.mapper.SimpleHBaseMapper"
-    configMethods:
-      - name: "withRowKeyField"
-        args: ["word"]
-      - name: "withColumnFields"
-        args: [ref: "columnFields"]
-      - name: "withCounterFields"
-        args: [ref: "counterFields"]
-      - name: "withColumnFamily"
-        args: ["cf"]
-
-# topology configuration
-# this will be passed to the submitter as a map of config options
-#
-config:
-  topology.workers: 1
-  hbase.conf:
-    hbase.rootdir: "hdfs://hadoop:54310/hbase"
-    hbase.zookeeper.quorum: "hadoop"
-
-# spout definitions
-spouts:
-  - id: "word-spout"
-    className: "org.apache.storm.testing.TestWordSpout"
-    parallelism: 1
-
-# bolt definitions
-
-bolts:
-  - id: "count-bolt"
-    className: "org.apache.storm.testing.TestWordCounter"
-
-  - id: "hbase-bolt"
-    className: "org.apache.storm.hbase.bolt.HBaseBolt"
-    constructorArgs:
-      - "WordCount" # HBase table name
-      - ref: "mapper"
-    configMethods:
-      - name: "withConfigKey"
-        args: ["hbase.conf"]
-    parallelism: 1
-
-
-streams:
-  - name: "" # name isn't used (placeholder for logging, UI, etc.)
-    from: "word-spout"
-    to: "count-bolt"
-    grouping:
-      type: SHUFFLE
-
-  - name: "" # name isn't used (placeholder for logging, UI, etc.)
-    from: "count-bolt"
-    to: "hbase-bolt"
-    grouping:
-      type: FIELDS
-      args: ["word"]

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/bad_shell_test.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/bad_shell_test.yaml b/external/flux/flux-core/src/test/resources/configs/bad_shell_test.yaml
deleted file mode 100644
index 0892ce7..0000000
--- a/external/flux/flux-core/src/test/resources/configs/bad_shell_test.yaml
+++ /dev/null
@@ -1,118 +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.
-
-# Test ability to wire together shell spouts/bolts
----
-
-# topology definition
-# name to be used when submitting
-name: "shell-topology"
-
-# Components
-# Components are analagous to Spring beans. They are meant to be used as constructor,
-# property(setter), and builder arguments.
-#components:
-#  - id: "myComponent"
-#    className: "com.foo.bar.MyComponent"
-#    constructorArgs:
-#      - ...
-#    properties:
-#      foo: "bar"
-#      bar: "foo"
-
-# NOTE: We may want to consider some level of spring integration. For example, allowing component references
-# to a spring `ApplicationContext`.
-
-# topology configuration
-# this will be passed to the submitter as a map of config options
-#
-config:
-  topology.workers: 1
-  # ...
-
-# spout definitions
-spouts:
-  - id: "sentence-spout"
-    className: "org.apache.storm.flux.wrappers.spouts.FluxShellSpout"
-    # shell spout constructor takes 2 arguments: String[], String[]
-    constructorArgs:
-      # command line
-      - ["node", "randomsentence.js"]
-      # output fields
-      - ["word"]
-    configMethods:
-      - name: "addComponentConfig"
-        args: ["rabbitmq.configfile", "etc/rabbit.yml", "hello"]
-      - name: "addComponentConfig"
-        args:
-        - "publisher.data_paths"
-        - ["actions", "hello"]
-    parallelism: 1
-    # ...
-
-# bolt definitions
-bolts:
-  - id: "splitsentence"
-    className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
-    constructorArgs:
-      # command line
-      - ["python", "splitsentence.py"]
-      # output fields
-      - ["word"]
-    configMethods:
-      - name: "addComponentConfig"
-        args: ["rabbitmq.configfile", "etc/rabbit.yml", "hello"]
-      - name: "addComponentConfig"
-        args:
-        - "publisher.data_paths"
-        - ["actions", "hello"]
-    parallelism: 1
-    # ...
-
-  - id: "log"
-    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
-    parallelism: 1
-    # ...
-
-  - id: "count"
-    className: "org.apache.storm.testing.TestWordCounter"
-    parallelism: 1
-    # ...
-
-#stream definitions
-# stream definitions define connections between spouts and bolts.
-# note that such connections can be cyclical
-# custom stream groupings are also supported
-
-streams:
-  - name: "spout --> split" # name isn't used (placeholder for logging, UI, etc.)
-    from: "sentence-spout"
-    to: "splitsentence"
-    grouping:
-      type: SHUFFLE
-
-  - name: "split --> count"
-    from: "splitsentence"
-    to: "count"
-    grouping:
-      type: FIELDS
-      args: ["word"]
-
-  - name: "count --> log"
-    from: "count"
-    to: "log"
-    grouping:
-      type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/config-methods-test.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/config-methods-test.yaml b/external/flux/flux-core/src/test/resources/configs/config-methods-test.yaml
deleted file mode 100644
index 7c4ffb3..0000000
--- a/external/flux/flux-core/src/test/resources/configs/config-methods-test.yaml
+++ /dev/null
@@ -1,92 +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.
----
-name: "yaml-topology"
-
-#
-config:
-  topology.workers: 1
-  # ...
-
-components:
-  - id: "foo"
-    className: "org.apache.storm.flux.test.TestBolt$TestClass"
-    constructorArgs:
-      - "foo"
-  - id: "bar"
-    className: "org.apache.storm.flux.test.TestBolt$TestClass"
-    constructorArgs:
-      - "bar"
-  - id: "baz"
-    className: "org.apache.storm.flux.test.TestBolt$TestClass"
-    constructorArgs:
-      - "baz"
-
-# spout definitions
-spouts:
-  - id: "spout-1"
-    className: "org.apache.storm.testing.TestWordSpout"
-    parallelism: 1
-    # ...
-
-# bolt definitions
-bolts:
-  - id: "bolt-1"
-    className: "org.apache.storm.flux.test.TestBolt"
-    parallelism: 1
-    constructorArgs:
-      - FOO # enum class
-      - 1.0
-      - true
-      - reflist: ["foo", "bar"]
-    configMethods:
-      - name: "withFoo"
-        args:
-          - "foo"
-      - name: "withNone"
-      - name: "withBar"
-        args:
-          - "bar"
-      - name: "withFooBar"
-        args:
-          - "foo"
-          - "bar"
-      - name: "withClasses"
-        args:
-          - reflist:
-            - "foo"
-            - "bar"
-            - "baz"
-
-
-#stream definitions
-# stream definitions define connections between spouts and bolts.
-# note that such connections can be cyclical
-streams:
-  - name: "spout-1 --> bolt-1" # name isn't used (placeholder for logging, UI, etc.)
-#    id: "connection-1"
-    from: "spout-1"
-    to: "bolt-1"
-    grouping:
-      type: SHUFFLE
-
-
-
-
-
-
-
-

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/diamond-topology.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/diamond-topology.yaml b/external/flux/flux-core/src/test/resources/configs/diamond-topology.yaml
deleted file mode 100644
index 957c258..0000000
--- a/external/flux/flux-core/src/test/resources/configs/diamond-topology.yaml
+++ /dev/null
@@ -1,87 +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.
-
----
-
-# topology definition
-# name to be used when submitting
-name: "diamond-topology"
-
-# topology configuration
-# this will be passed to the submitter as a map of config options
-#
-config:
-  topology.workers: 1
-
-# spout definitions
-spouts:
-  - id: "spout-1"
-    className: "org.apache.storm.testing.TestWordSpout"
-    parallelism: 1
-
-# bolt definitions
-bolts:
-  - id: "A"
-    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
-    parallelism: 1
-
-  - id: "B"
-    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
-    parallelism: 1
-
-  - id: "C"
-    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
-    parallelism: 1
-
-  - id: "D"
-    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
-    parallelism: 1
-
-#stream definitions
-# stream definitions define connections between spouts and bolts.
-# note that such connections can be cyclical
-streams:
-  - name: "spout-1 --> bolt-1" # name isn't used (placeholder for logging, UI, etc.)
-#    id: "connection-1"
-    from: "spout-1"
-    to: "A"
-    grouping:
-      type: FIELDS
-      args: ["word"]
-
-  - from: "A"
-    to: "B"
-    grouping:
-      type: SHUFFLE
-
-  - from: "A"
-    to: "C"
-    grouping:
-      type: SHUFFLE
-
-  - from: "C"
-    to: "D"
-    grouping:
-      type: SHUFFLE
-
-  - from: "B"
-    to: "D"
-    grouping:
-      type: SHUFFLE
-
-
-
-

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/existing-topology-method-override.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/existing-topology-method-override.yaml b/external/flux/flux-core/src/test/resources/configs/existing-topology-method-override.yaml
deleted file mode 100644
index fceeeed..0000000
--- a/external/flux/flux-core/src/test/resources/configs/existing-topology-method-override.yaml
+++ /dev/null
@@ -1,25 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
----
-
-# configuration that uses an existing topology that does not implement TopologySource
-name: "existing-topology"
-topologySource:
-  className: "org.apache.storm.flux.test.SimpleTopology"
-  methodName: "getTopologyWithDifferentMethodName"
-  constructorArgs:
-    - "foo"
-    - "bar"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/existing-topology-reflection-config.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/existing-topology-reflection-config.yaml b/external/flux/flux-core/src/test/resources/configs/existing-topology-reflection-config.yaml
deleted file mode 100644
index 440fe4d..0000000
--- a/external/flux/flux-core/src/test/resources/configs/existing-topology-reflection-config.yaml
+++ /dev/null
@@ -1,24 +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.
----
-
-# configuration that uses an existing topology that does not implement TopologySource
-name: "existing-topology"
-topologySource:
-  className: "org.apache.storm.flux.test.SimpleTopologyWithConfigParam"
-  constructorArgs:
-    - "foo"
-    - "bar"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/existing-topology-reflection.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/existing-topology-reflection.yaml b/external/flux/flux-core/src/test/resources/configs/existing-topology-reflection.yaml
deleted file mode 100644
index 975885b..0000000
--- a/external/flux/flux-core/src/test/resources/configs/existing-topology-reflection.yaml
+++ /dev/null
@@ -1,24 +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.
----
-
-# configuration that uses an existing topology that does not implement TopologySource
-name: "existing-topology"
-topologySource:
-  className: "org.apache.storm.flux.test.SimpleTopology"
-  constructorArgs:
-    - "foo"
-    - "bar"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/existing-topology-trident.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/existing-topology-trident.yaml b/external/flux/flux-core/src/test/resources/configs/existing-topology-trident.yaml
deleted file mode 100644
index 978181b..0000000
--- a/external/flux/flux-core/src/test/resources/configs/existing-topology-trident.yaml
+++ /dev/null
@@ -1,24 +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.
----
-
-name: "existing-topology"
-
-config:
-  topology.workers: 1
-
-topologySource:
-  className: "org.apache.storm.flux.test.TridentTopologySource"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/existing-topology.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/existing-topology.yaml b/external/flux/flux-core/src/test/resources/configs/existing-topology.yaml
deleted file mode 100644
index e112c0f..0000000
--- a/external/flux/flux-core/src/test/resources/configs/existing-topology.yaml
+++ /dev/null
@@ -1,23 +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.
----
-
-name: "existing-topology"
-topologySource:
-  className: "org.apache.storm.flux.test.SimpleTopologySource"
-  constructorArgs:
-    - "foo"
-    - "bar"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/hdfs_test.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/hdfs_test.yaml b/external/flux/flux-core/src/test/resources/configs/hdfs_test.yaml
deleted file mode 100644
index 2bccb33..0000000
--- a/external/flux/flux-core/src/test/resources/configs/hdfs_test.yaml
+++ /dev/null
@@ -1,97 +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.
-
-# Test ability to wire together shell spouts/bolts
----
-
-# topology definition
-# name to be used when submitting
-name: "hdfs-topology"
-
-# Components
-# Components are analagous to Spring beans. They are meant to be used as constructor,
-# property(setter), and builder arguments.
-#
-# for the time being, components must be declared in the order they are referenced
-components:
-  - id: "syncPolicy"
-    className: "org.apache.storm.hdfs.bolt.sync.CountSyncPolicy"
-    constructorArgs:
-      - 1000
-  - id: "rotationPolicy"
-    className: "org.apache.storm.hdfs.bolt.rotation.FileSizeRotationPolicy"
-    constructorArgs:
-      - 5.0
-      - MB
-
-  - id: "fileNameFormat"
-    className: "org.apache.storm.hdfs.bolt.format.DefaultFileNameFormat"
-    configMethods:
-      - name: "withPath"
-        args: ["/tmp/foo/"]
-      - name: "withExtension"
-        args: [".txt"]
-
-  - id: "recordFormat"
-    className: "org.apache.storm.hdfs.bolt.format.DelimitedRecordFormat"
-    configMethods:
-      - name: "withFieldDelimiter"
-        args: ["|"]
-
-  - id: "rotationAction"
-    className: "org.apache.storm.hdfs.common.rotation.MoveFileAction"
-    configMethods:
-      - name: "toDestination"
-        args: ["/tmp/dest2"]
-
-# spout definitions
-spouts:
-  - id: "spout-1"
-    className: "org.apache.storm.testing.TestWordSpout"
-    parallelism: 1
-    # ...
-
-# bolt definitions
-
-#        HdfsBolt bolt = new HdfsBolt()
-#                .withConfigKey("hdfs.config")
-#                .withFsUrl(args[0])
-#                .withFileNameFormat(fileNameFormat)
-#                .withRecordFormat(format)
-#                .withRotationPolicy(rotationPolicy)
-#                .withSyncPolicy(syncPolicy)
-#                .addRotationAction(new MoveFileAction().toDestination("/tmp/dest2/"));
-bolts:
-  - id: "bolt-1"
-    className: "org.apache.storm.hdfs.bolt.HdfsBolt"
-    configMethods:
-      - name: "withConfigKey"
-        args: ["hdfs.config"]
-      - name: "withFsUrl"
-        args: ["hdfs://hadoop:54310"]
-      - name: "withFileNameFormat"
-        args: [ref: "fileNameFormat"]
-      - name: "withRecordFormat"
-        args: [ref: "recordFormat"]
-      - name: "withRotationPolicy"
-        args: [ref: "rotationPolicy"]
-      - name: "withSyncPolicy"
-        args: [ref: "syncPolicy"]
-      - name: "addRotationAction"
-        args: [ref: "rotationAction"]
-    parallelism: 1
-    # ...
-

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/include_test.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/include_test.yaml b/external/flux/flux-core/src/test/resources/configs/include_test.yaml
deleted file mode 100644
index 702f590..0000000
--- a/external/flux/flux-core/src/test/resources/configs/include_test.yaml
+++ /dev/null
@@ -1,25 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-# Test includes by defining nothing, and simply override the topology name
----
-
-name: "include-topology"
-
-includes:
-  - resource: true
-    file: "/configs/shell_test.yaml"
-    override: false #otherwise subsequent includes that define 'name' would override
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/invalid-existing-topology.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/invalid-existing-topology.yaml b/external/flux/flux-core/src/test/resources/configs/invalid-existing-topology.yaml
deleted file mode 100644
index c2dfac0..0000000
--- a/external/flux/flux-core/src/test/resources/configs/invalid-existing-topology.yaml
+++ /dev/null
@@ -1,33 +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.
-
-# This is an invalid config. It defines both a topologySource and a list of spouts.
----
-
-name: "existing-topology"
-topologySource:
-  className: "org.apache.storm.flux.test.SimpleTopologySource"
-
-spouts:
-  - id: "sentence-spout"
-    className: "org.apache.storm.flux.wrappers.spouts.FluxShellSpout"
-    # shell spout constructor takes 2 arguments: String[], String[]
-    constructorArgs:
-      # command line
-      - ["node", "randomsentence.js"]
-      # output fields
-      - ["word"]
-    parallelism: 1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/kafka_test.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/kafka_test.yaml b/external/flux/flux-core/src/test/resources/configs/kafka_test.yaml
deleted file mode 100644
index 1fb59ca..0000000
--- a/external/flux/flux-core/src/test/resources/configs/kafka_test.yaml
+++ /dev/null
@@ -1,126 +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.
----
-
-# topology definition
-# name to be used when submitting
-name: "kafka-topology"
-
-# Components
-# Components are analagous to Spring beans. They are meant to be used as constructor,
-# property(setter), and builder arguments.
-#
-# for the time being, components must be declared in the order they are referenced
-components:
-  - id: "stringScheme"
-    className: "org.apache.storm.kafka.StringScheme"
-
-  - id: "stringMultiScheme"
-    className: "org.apache.storm.spout.SchemeAsMultiScheme"
-    constructorArgs:
-      - ref: "stringScheme"
-
-  - id: "zkHosts"
-    className: "org.apache.storm.kafka.ZkHosts"
-    constructorArgs:
-      - "localhost:2181"
-
-# Alternative kafka config
-#  - id: "kafkaConfig"
-#    className: "org.apache.storm.kafka.KafkaConfig"
-#    constructorArgs:
-#      # brokerHosts
-#      - ref: "zkHosts"
-#      # topic
-#      - "myKafkaTopic"
-#      # clientId (optional)
-#      - "myKafkaClientId"
-
-  - id: "spoutConfig"
-    className: "org.apache.storm.kafka.SpoutConfig"
-    constructorArgs:
-      # brokerHosts
-      - ref: "zkHosts"
-      # topic
-      - "myKafkaTopic"
-      # zkRoot
-      - "/kafkaSpout"
-      # id
-      - "myId"
-    properties:
-      - name: "ignoreZkOffsets"
-        value: true
-      - name: "scheme"
-        ref: "stringMultiScheme"
-
-# topology configuration
-# this will be passed to the submitter as a map of config options
-#
-config:
-  topology.workers: 1
-  # ...
-
-# spout definitions
-spouts:
-  - id: "kafka-spout"
-    className: "org.apache.storm.kafka.KafkaSpout"
-    constructorArgs:
-      - ref: "spoutConfig"
-
-# bolt definitions
-bolts:
-  - id: "splitsentence"
-    className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
-    constructorArgs:
-      # command line
-      - ["python", "splitsentence.py"]
-      # output fields
-      - ["word"]
-    parallelism: 1
-
-  - id: "log"
-    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
-    parallelism: 1
-    # ...
-
-  - id: "count"
-    className: "org.apache.storm.testing.TestWordCounter"
-    parallelism: 1
-
-#stream definitions
-# stream definitions define connections between spouts and bolts.
-# note that such connections can be cyclical
-# custom stream groupings are also supported
-
-streams:
-  - name: "kafka --> split" # name isn't used (placeholder for logging, UI, etc.)
-    from: "kafka-spout"
-    to: "splitsentence"
-    grouping:
-      type: SHUFFLE
-
-  - name: "split --> count"
-    from: "splitsentence"
-    to: "count"
-    grouping:
-      type: FIELDS
-      args: ["word"]
-
-  - name: "count --> log"
-    from: "count"
-    to: "log"
-    grouping:
-      type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/shell_test.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/shell_test.yaml b/external/flux/flux-core/src/test/resources/configs/shell_test.yaml
deleted file mode 100644
index dfab397..0000000
--- a/external/flux/flux-core/src/test/resources/configs/shell_test.yaml
+++ /dev/null
@@ -1,118 +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.
-
-# Test ability to wire together shell spouts/bolts
----
-
-# topology definition
-# name to be used when submitting
-name: "shell-topology"
-
-# Components
-# Components are analagous to Spring beans. They are meant to be used as constructor,
-# property(setter), and builder arguments.
-#components:
-#  - id: "myComponent"
-#    className: "com.foo.bar.MyComponent"
-#    constructorArgs:
-#      - ...
-#    properties:
-#      foo: "bar"
-#      bar: "foo"
-
-# NOTE: We may want to consider some level of spring integration. For example, allowing component references
-# to a spring `ApplicationContext`.
-
-# topology configuration
-# this will be passed to the submitter as a map of config options
-#
-config:
-  topology.workers: 1
-  # ...
-
-# spout definitions
-spouts:
-  - id: "sentence-spout"
-    className: "org.apache.storm.flux.wrappers.spouts.FluxShellSpout"
-    # shell spout constructor takes 2 arguments: String[], String[]
-    constructorArgs:
-      # command line
-      - ["node", "randomsentence.js"]
-      # output fields
-      - ["word"]
-    configMethods:
-      - name: "addComponentConfig"
-        args: ["rabbitmq.configfile", "etc/rabbit.yml"]
-      - name: "addComponentConfig"
-        args:
-        - "publisher.data_paths"
-        - ["actions"]
-    parallelism: 1
-    # ...
-
-# bolt definitions
-bolts:
-  - id: "splitsentence"
-    className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
-    constructorArgs:
-      # command line
-      - ["python", "splitsentence.py"]
-      # output fields
-      - ["word"]
-    configMethods:
-      - name: "addComponentConfig"
-        args: ["rabbitmq.configfile", "etc/rabbit.yml"]
-      - name: "addComponentConfig"
-        args:
-        - "publisher.data_paths"
-        - ["actions"]
-    parallelism: 1
-    # ...
-
-  - id: "log"
-    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
-    parallelism: 1
-    # ...
-
-  - id: "count"
-    className: "org.apache.storm.testing.TestWordCounter"
-    parallelism: 1
-    # ...
-
-#stream definitions
-# stream definitions define connections between spouts and bolts.
-# note that such connections can be cyclical
-# custom stream groupings are also supported
-
-streams:
-  - name: "spout --> split" # name isn't used (placeholder for logging, UI, etc.)
-    from: "sentence-spout"
-    to: "splitsentence"
-    grouping:
-      type: SHUFFLE
-
-  - name: "split --> count"
-    from: "splitsentence"
-    to: "count"
-    grouping:
-      type: FIELDS
-      args: ["word"]
-
-  - name: "count --> log"
-    from: "count"
-    to: "log"
-    grouping:
-      type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/simple_hbase.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/simple_hbase.yaml b/external/flux/flux-core/src/test/resources/configs/simple_hbase.yaml
deleted file mode 100644
index b841b53..0000000
--- a/external/flux/flux-core/src/test/resources/configs/simple_hbase.yaml
+++ /dev/null
@@ -1,120 +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.
-
-# Test ability to wire together shell spouts/bolts
----
-
-# topology definition
-# name to be used when submitting
-name: "hbase-wordcount"
-
-# Components
-# Components are analagous to Spring beans. They are meant to be used as constructor,
-# property(setter), and builder arguments.
-#
-# for the time being, components must be declared in the order they are referenced
-
-#        WordSpout spout = new WordSpout();
-#        WordCounter bolt = new WordCounter();
-#
-#        SimpleHBaseMapper mapper = new SimpleHBaseMapper()
-#                .withRowKeyField("word")
-#                .withColumnFields(new Fields("word"))
-#                .withCounterFields(new Fields("count"))
-#                .withColumnFamily("cf");
-#
-#        HBaseBolt hbase = new HBaseBolt("WordCount", mapper)
-#                .withConfigKey("hbase.conf");
-#
-#
-#        // wordSpout ==> countBolt ==> HBaseBolt
-#        TopologyBuilder builder = new TopologyBuilder();
-#
-#        builder.setSpout(WORD_SPOUT, spout, 1);
-#        builder.setBolt(COUNT_BOLT, bolt, 1).shuffleGrouping(WORD_SPOUT);
-#        builder.setBolt(HBASE_BOLT, hbase, 1).fieldsGrouping(COUNT_BOLT, new Fields("word"));
-
-
-
-
-components:
-  - id: "columnFields"
-    className: "org.apache.storm.tuple.Fields"
-    constructorArgs:
-      - ["word"]
-
-  - id: "counterFields"
-    className: "org.apache.storm.tuple.Fields"
-    constructorArgs:
-      - ["count"]
-
-  - id: "mapper"
-    className: "org.apache.storm.hbase.bolt.mapper.SimpleHBaseMapper"
-    configMethods:
-      - name: "withRowKeyField"
-        args: ["word"]
-      - name: "withColumnFields"
-        args: [ref: "columnFields"]
-      - name: "withCounterFields"
-        args: [ref: "counterFields"]
-      - name: "withColumnFamily"
-        args: ["cf"]
-
-# topology configuration
-# this will be passed to the submitter as a map of config options
-#
-config:
-  topology.workers: 1
-  hbase.conf:
-    hbase.rootdir: "hdfs://hadoop:54310/hbase"
-    hbase.zookeeper.quorum: "hadoop"
-
-# spout definitions
-spouts:
-  - id: "word-spout"
-    className: "org.apache.storm.testing.TestWordSpout"
-    parallelism: 1
-
-# bolt definitions
-
-bolts:
-  - id: "count-bolt"
-    className: "org.apache.storm.testing.TestWordCounter"
-
-  - id: "hbase-bolt"
-    className: "org.apache.storm.hbase.bolt.HBaseBolt"
-    constructorArgs:
-      - "WordCount" # HBase table name
-      - ref: "mapper"
-    configMethods:
-      - name: "withConfigKey"
-        args: ["hbase.conf"]
-    parallelism: 1
-
-
-streams:
-  - name: "" # name isn't used (placeholder for logging, UI, etc.)
-    from: "word-spout"
-    to: "count-bolt"
-    grouping:
-      type: SHUFFLE
-
-  - name: "" # name isn't used (placeholder for logging, UI, etc.)
-    from: "count-bolt"
-    to: "hbase-bolt"
-    grouping:
-      type: FIELDS
-      args: ["word"]

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/substitution-test.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/substitution-test.yaml b/external/flux/flux-core/src/test/resources/configs/substitution-test.yaml
deleted file mode 100644
index ce9e62d..0000000
--- a/external/flux/flux-core/src/test/resources/configs/substitution-test.yaml
+++ /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.
-
-# Test ability to wire together shell spouts/bolts
----
-
-# topology definition
-# name to be used when submitting
-name: "${topology.name}"
-
-# Components
-# Components are analagous to Spring beans. They are meant to be used as constructor,
-# property(setter), and builder arguments.
-#components:
-#  - id: "myComponent"
-#    className: "com.foo.bar.MyComponent"
-#    constructorArgs:
-#      - ...
-#    properties:
-#      foo: "bar"
-#      bar: "foo"
-
-# NOTE: We may want to consider some level of spring integration. For example, allowing component references
-# to a spring `ApplicationContext`.
-
-# topology configuration
-# this will be passed to the submitter as a map of config options
-#
-config:
-  topology.workers: 1
-  # test environent variable substitution
-  test.env.value: ${ENV-PATH}
-  # ...
-
-# spout definitions
-spouts:
-  - id: "sentence-spout"
-    className: "org.apache.storm.flux.wrappers.spouts.FluxShellSpout"
-    # shell spout constructor takes 2 arguments: String[], String[]
-    constructorArgs:
-      # command line
-      - ["node", "randomsentence.js"]
-      # output fields
-      - ["word"]
-    parallelism: 1
-    # ...
-
-# bolt definitions
-bolts:
-  - id: "splitsentence"
-    className: "org.apache.storm.flux.wrappers.bolts.FluxShellBolt"
-    constructorArgs:
-      # command line
-      - ["python", "splitsentence.py"]
-      # output fields
-      - ["word"]
-    parallelism: 1
-    # ...
-
-  - id: "log"
-    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
-    parallelism: 1
-    # ...
-
-  - id: "count"
-    className: "org.apache.storm.testing.TestWordCounter"
-    parallelism: 1
-    # ...
-
-#stream definitions
-# stream definitions define connections between spouts and bolts.
-# note that such connections can be cyclical
-# custom stream groupings are also supported
-
-streams:
-  - name: "spout --> split" # name isn't used (placeholder for logging, UI, etc.)
-    from: "sentence-spout"
-    to: "splitsentence"
-    grouping:
-      type: SHUFFLE
-
-  - name: "split --> count"
-    from: "splitsentence"
-    to: "count"
-    grouping:
-      type: FIELDS
-      args: ["word"]
-
-  - name: "count --> log"
-    from: "count"
-    to: "log"
-    grouping:
-      type: SHUFFLE

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/tck.yaml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/tck.yaml b/external/flux/flux-core/src/test/resources/configs/tck.yaml
deleted file mode 100644
index 5d40445..0000000
--- a/external/flux/flux-core/src/test/resources/configs/tck.yaml
+++ /dev/null
@@ -1,95 +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.
-
-
-# YAML configuration to serve as a basic smoke test for what is supported.
-#
-# We should support comments, so if we've failed so far, things aren't good.
-
-# we shouldn't choke if we see a document separator...
----
-
-# topology definition
-# name to be used when submitting
-name: "yaml-topology"
-
-# Components
-# Components are analagous to Spring beans. They are meant to be used as constructor,
-# property(setter), and builder arguments.
-#components:
-#  - id: "myComponent"
-#    className: "com.foo.bar.MyComponent"
-#    properties:
-#      foo: "bar"
-#      bar: "foo"
-
-# NOTE: We may want to consider some level of spring integration. For example, allowing component references
-# to a spring `ApplicationContext`.
-
-# topology configuration
-# this will be passed to the submitter as a map of config options
-#
-config:
-  topology.workers: 1
-  # ...
-
-# spout definitions
-spouts:
-  - id: "spout-1"
-    className: "org.apache.storm.testing.TestWordSpout"
-    parallelism: 1
-    # ...
-
-# bolt definitions
-bolts:
-  - id: "bolt-1"
-    className: "org.apache.storm.testing.TestWordCounter"
-    parallelism: 1
-    # ...
-
-  - id: "bolt-2"
-    className: "org.apache.storm.flux.wrappers.bolts.LogInfoBolt"
-    parallelism: 1
-    # ...
-
-#stream definitions
-# stream definitions define connections between spouts and bolts.
-# note that such connections can be cyclical
-streams:
-  - name: "spout-1 --> bolt-1" # name isn't used (placeholder for logging, UI, etc.)
-#    id: "connection-1"
-    from: "spout-1"
-    to: "bolt-1"
-    grouping:
-      type: FIELDS
-      args: ["word"]
-
-  - name: "bolt-1 --> bolt2"
-    from: "bolt-1"
-    to: "bolt-2"
-    grouping:
-      type: CUSTOM
-      customClass:
-        className: "org.apache.storm.testing.NGrouping"
-        constructorArgs:
-          - 1
-
-
-
-
-
-
-

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/configs/test.properties
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/configs/test.properties b/external/flux/flux-core/src/test/resources/configs/test.properties
deleted file mode 100644
index ecd89d9..0000000
--- a/external/flux/flux-core/src/test/resources/configs/test.properties
+++ /dev/null
@@ -1,18 +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.
-
-topology.name: substitution-topology
-some.other.property: foo bar
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-core/src/test/resources/log4j2.xml
----------------------------------------------------------------------
diff --git a/external/flux/flux-core/src/test/resources/log4j2.xml b/external/flux/flux-core/src/test/resources/log4j2.xml
deleted file mode 100644
index d3333f3..0000000
--- a/external/flux/flux-core/src/test/resources/log4j2.xml
+++ /dev/null
@@ -1,34 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements.  See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License.  You may obtain a copy of the License at
-
-     http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<Configuration scan="true" monitorInterval="30">
-  <Appenders>
-    <Console name="A1" target="SYSTEM_OUT">
-      <PatternLayout>
-        <pattern>%-4r [%t] %-5level %c{1} - %msg%n</pattern>
-      </PatternLayout>
-    </Console>
-  </Appenders>
-  <Loggers>
-    <Logger name="org.apache.storm.curator" level="warn"/>
-    <Logger name="org.apache.storm.flux" level="debug"/>
-    <Logger name="org.apache.storm.zookeeper" level="warn"/>
-    <Root level="debug">
-      <appender-ref ref="A1"/>
-    </Root>
-  </Loggers>
-</Configuration>

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/flux/flux-examples/README.md
----------------------------------------------------------------------
diff --git a/external/flux/flux-examples/README.md b/external/flux/flux-examples/README.md
deleted file mode 100644
index 3d610b4..0000000
--- a/external/flux/flux-examples/README.md
+++ /dev/null
@@ -1,93 +0,0 @@
-# Flux Examples
-A collection of examples illustrating various capabilities.
-
-## Building From Source and Running
-
-Checkout the projects source and perform a top level Maven build (i.e. from the `flux` directory):
-
-```bash
-git clone https://github.com/apache/storm.git
-cd storm
-mvn install -DskipTests=true
-```
-
-This will create a shaded (i.e. "fat" or "uber") jar in the `external/flux/flux-examples/target` directory that can run/deployed with
-the `storm` command:
-
-```bash
-cd flux-examples
-storm jar ./target/flux-examples-*-SNAPSHOT.jar org.apache.storm.flux.Flux --local ./src/main/resources/simple_wordcount.yaml
-```
-
-The example YAML files are also packaged in the examples jar, so they can also be referenced with Flux's `--resource`
-command line switch:
-
-```bash
-storm jar ./target/flux-examples-*.jar org.apache.storm.flux.Flux --local --resource /simple_wordcount.yaml
-```
-
-## Available Examples
-
-### [simple_wordcount.yaml](src/main/resources/simple_wordcount.yaml)
-
-This is a very basic wordcount example using Java spouts and bolts. It simply logs the running count of each word
-received.
-
-### [multilang.yaml](src/main/resources/multilang.yaml)
-
-Another wordcount example that uses a spout written in JavaScript (node.js), a bolt written in Python, and two bolts
-written in java.
-
-### [kafka_spout.yaml](src/main/resources/kafka_spout.yaml)
-
-This example illustrates how to configure Storm's `storm-kafka` spout using Flux YAML DSL `components`, `references`,
-and `constructor arguments` constructs.
-
-### [simple_hdfs.yaml](src/main/resources/simple_hdfs.yaml)
-
-This example demonstrates using Flux to setup a storm-hdfs bolt to write to an HDFS cluster. It also demonstrates Flux's
-variable substitution/filtering feature.
-
-To run the `simple_hdfs.yaml` example, copy the `hdfs_bolt.properties` file to a convenient location and change, at
-least, the property `hdfs.url` to point to a HDFS cluster. Then you can run the example something like:
-
-```bash
-storm jar ./target/flux-examples-*.jar org.apache.storm.flux.Flux --local ./src/main/resources/simple_hdfs.yaml --filter my_hdfs_bolt.properties
-```
-
-### [simple_hbase.yaml](src/main/resources/simple_hbase.yaml)
-
-This example illustrates how to use Flux to setup a storm-hbase bolt to write to HBase.
-
-To run the `simple_hbase.yaml` example, copy the `hbase_bolt.properties` file to a convenient location and change the properties
- `hbase.rootdir` and `hbase.zookeeper.quorum`. Then you can run the example something like:
-
-```bash
-storm jar ./target/flux-examples-*.jar org.apache.storm.flux.Flux --local ./src/main/resources/simple_hbase.yaml --filter my_hbase_bolt.properties
-```
-
-### [simple_windowing.yaml](src/main/resources/simple_windowing.yaml)
-
-This example illustrates how to use Flux to set up a storm topology that contains windowing operations.
-
-To run,
-
-```bash
-storm jar ./target/flux-examples-*.jar org.apache.storm.flux.Flux --local ./src/main/resources/simple_windowing.yaml
-```
-
-### [simple_stateful_wordcount.yaml](src/main/resources/simple_stateful_wordcount.yaml)
-
-Flux also supports stateful bolts which is illustrated with this example. It is basically an extension of the basic wordcount example.
-The state is periodically saved (checkpointed) and restored when the topology is restarted.
-
-```bash
-storm jar ./target/flux-examples-*.jar org.apache.storm.flux.Flux --local ./src/main/resources/simple_stateful_wordcount.yaml
-```
-
-By default the state is stored in-memory only. As such you won't see a resumed state unless you configure to use Redis as the state backend.
-Ensure that you have Redis running at `localhost:6379` and that `storm-redis-*.jar` is in the classpath.
-
-```bash
-STORM_EXT_CLASSPATH=../../storm-redis/target storm jar ./target/flux-examples-*.jar -c topology.state.provider=org.apache.storm.redis.state.RedisKeyValueStateProvider org.apache.storm.flux.Flux --local ./src/main/resources/simple_stateful_wordcount.yaml
-```


[05/23] storm git commit: STORM-2453 Move non-connectors into the top directory

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelDataTypeSystem.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelDataTypeSystem.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelDataTypeSystem.java
new file mode 100644
index 0000000..437877c
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelDataTypeSystem.java
@@ -0,0 +1,37 @@
+/**
+ * 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.sql.planner;
+
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rel.type.RelDataTypeSystemImpl;
+
+public class StormRelDataTypeSystem extends RelDataTypeSystemImpl {
+    public static final RelDataTypeSystem STORM_REL_DATATYPE_SYSTEM = new StormRelDataTypeSystem();
+
+    @Override
+    public int getMaxNumericScale() {
+        return 38;
+    }
+
+    @Override
+    public int getMaxNumericPrecision() {
+        return 38;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelUtils.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelUtils.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelUtils.java
new file mode 100644
index 0000000..40bbacd
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/StormRelUtils.java
@@ -0,0 +1,68 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.planner;
+
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.storm.sql.planner.trident.rel.TridentRel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class StormRelUtils {
+    private static final Logger LOG = LoggerFactory.getLogger(StormRelUtils.class);
+
+    private static final AtomicInteger sequence = new AtomicInteger(0);
+    private static final AtomicInteger classSequence = new AtomicInteger(0);
+
+    public static String getStageName(TridentRel relNode) {
+        return relNode.getClass().getSimpleName().toUpperCase() + "_" + relNode.getId() + "_" + sequence.getAndIncrement();
+    }
+
+    public static String getClassName(TridentRel relNode) {
+        return "Generated_" + relNode.getClass().getSimpleName().toUpperCase() + "_" + relNode.getId() + "_" +
+                classSequence.getAndIncrement();
+    }
+
+    public static TridentRel getStormRelInput(RelNode input) {
+        if (input instanceof RelSubset) {
+            // go with known best input
+            input = ((RelSubset) input).getBest();
+        }
+        return (TridentRel) input;
+    }
+
+    public static String explain(final RelNode rel) {
+        return explain(rel, SqlExplainLevel.EXPPLAN_ATTRIBUTES);
+    }
+
+    public static String explain(final RelNode rel, SqlExplainLevel detailLevel) {
+        String explain = "";
+        try {
+            explain = RelOptUtil.toString(rel);
+        } catch (StackOverflowError e) {
+            LOG.error("StackOverflowError occurred while extracting plan. Please report it to the dev@ mailing list.");
+            LOG.error("RelNode " + rel + " ExplainLevel " + detailLevel, e);
+            LOG.error("Forcing plan to empty string and continue... SQL Runner may not working properly after.");
+        }
+        return explain;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/UnsupportedOperatorsVisitor.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/UnsupportedOperatorsVisitor.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/UnsupportedOperatorsVisitor.java
new file mode 100644
index 0000000..258fe72
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/UnsupportedOperatorsVisitor.java
@@ -0,0 +1,24 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.planner;
+
+import org.apache.calcite.sql.util.SqlShuttle;
+
+public class UnsupportedOperatorsVisitor extends SqlShuttle {
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormCalcRelBase.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormCalcRelBase.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormCalcRelBase.java
new file mode 100644
index 0000000..29ca08b
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormCalcRelBase.java
@@ -0,0 +1,31 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.planner.rel;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Calc;
+import org.apache.calcite.rex.RexProgram;
+
+public abstract class StormCalcRelBase extends Calc implements StormRelNode {
+    protected StormCalcRelBase(RelOptCluster cluster, RelTraitSet traits, RelNode child, RexProgram program) {
+        super(cluster, traits, child, program);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormFilterRelBase.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormFilterRelBase.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormFilterRelBase.java
new file mode 100644
index 0000000..fa2ac65
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormFilterRelBase.java
@@ -0,0 +1,31 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.planner.rel;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rex.RexNode;
+
+public abstract class StormFilterRelBase extends Filter implements StormRelNode {
+    protected StormFilterRelBase(RelOptCluster cluster, RelTraitSet traits, RelNode child, RexNode condition) {
+        super(cluster, traits, child, condition);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormJoinRelBase.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormJoinRelBase.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormJoinRelBase.java
new file mode 100644
index 0000000..d8e82c5
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormJoinRelBase.java
@@ -0,0 +1,35 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.planner.rel;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.CorrelationId;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rex.RexNode;
+
+import java.util.Set;
+
+public abstract class StormJoinRelBase extends Join implements StormRelNode {
+    protected StormJoinRelBase(RelOptCluster cluster, RelTraitSet traitSet, RelNode left, RelNode right, RexNode condition, Set<CorrelationId> variablesSet, JoinRelType joinType) {
+        super(cluster, traitSet, left, right, condition, variablesSet, joinType);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormProjectRelBase.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormProjectRelBase.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormProjectRelBase.java
new file mode 100644
index 0000000..fe32ba5
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormProjectRelBase.java
@@ -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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.planner.rel;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+
+import java.util.List;
+
+public abstract class StormProjectRelBase extends Project implements StormRelNode {
+    protected StormProjectRelBase(RelOptCluster cluster, RelTraitSet traits, RelNode input, List<? extends RexNode> projects, RelDataType rowType) {
+        super(cluster, traits, input, projects, rowType);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormRelNode.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormRelNode.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormRelNode.java
new file mode 100644
index 0000000..9327868
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormRelNode.java
@@ -0,0 +1,24 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.planner.rel;
+
+import org.apache.calcite.rel.RelNode;
+
+public interface StormRelNode extends RelNode {
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamInsertRelBase.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamInsertRelBase.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamInsertRelBase.java
new file mode 100644
index 0000000..59694fc
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamInsertRelBase.java
@@ -0,0 +1,35 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.planner.rel;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.TableModify;
+import org.apache.calcite.rex.RexNode;
+
+import java.util.List;
+
+public abstract class StormStreamInsertRelBase extends TableModify implements StormRelNode {
+    protected StormStreamInsertRelBase(RelOptCluster cluster, RelTraitSet traits, RelOptTable table, Prepare.CatalogReader catalogReader, RelNode child, Operation operation, List<String> updateColumnList, List<RexNode> sourceExpressionList, boolean flattened) {
+        super(cluster, traits, table, catalogReader, child, operation, updateColumnList, sourceExpressionList, flattened);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamScanRelBase.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamScanRelBase.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamScanRelBase.java
new file mode 100644
index 0000000..32f1ac2
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/rel/StormStreamScanRelBase.java
@@ -0,0 +1,33 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.planner.rel;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.core.TableScan;
+
+public abstract class StormStreamScanRelBase extends TableScan implements StormRelNode {
+
+    // FIXME: define Table class and table.unwrap() to get it
+
+    protected StormStreamScanRelBase(RelOptCluster cluster, RelTraitSet traitSet, RelOptTable table) {
+        super(cluster, traitSet, table);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/QueryPlanner.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/QueryPlanner.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/QueryPlanner.java
new file mode 100644
index 0000000..f98fb02
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/QueryPlanner.java
@@ -0,0 +1,156 @@
+/*
+ *
+ *  * 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.sql.planner.trident;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.plan.Contexts;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.prepare.CalciteCatalogReader;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
+import org.apache.calcite.tools.FrameworkConfig;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.tools.Planner;
+import org.apache.calcite.tools.RelConversionException;
+import org.apache.calcite.tools.ValidationException;
+import org.apache.storm.sql.javac.CompilingClassLoader;
+import org.apache.storm.sql.planner.StormRelDataTypeSystem;
+import org.apache.storm.sql.planner.UnsupportedOperatorsVisitor;
+import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
+import org.apache.storm.sql.planner.trident.rel.TridentRel;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.sql.AbstractTridentProcessor;
+import org.apache.storm.trident.Stream;
+import org.apache.storm.trident.TridentTopology;
+import org.apache.storm.trident.fluent.IAggregatableStream;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public class QueryPlanner {
+
+    public static final int STORM_REL_CONVERSION_RULES = 1;
+
+    private final Planner planner;
+
+    private final JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(
+            RelDataTypeSystem.DEFAULT);
+
+    public QueryPlanner(SchemaPlus schema) {
+        final List<RelTraitDef> traitDefs = new ArrayList<RelTraitDef>();
+
+        traitDefs.add(ConventionTraitDef.INSTANCE);
+        traitDefs.add(RelCollationTraitDef.INSTANCE);
+
+        List<SqlOperatorTable> sqlOperatorTables = new ArrayList<>();
+        sqlOperatorTables.add(SqlStdOperatorTable.instance());
+        sqlOperatorTables.add(new CalciteCatalogReader(CalciteSchema.from(schema),
+                false,
+                Collections.<String>emptyList(), typeFactory));
+
+        FrameworkConfig config = Frameworks.newConfigBuilder()
+                .defaultSchema(schema)
+                .operatorTable(new ChainedSqlOperatorTable(sqlOperatorTables))
+                .traitDefs(traitDefs)
+                .context(Contexts.EMPTY_CONTEXT)
+                .ruleSets(TridentStormRuleSets.getRuleSets())
+                .costFactory(null)
+                .typeSystem(StormRelDataTypeSystem.STORM_REL_DATATYPE_SYSTEM)
+                .build();
+        this.planner = Frameworks.getPlanner(config);
+    }
+
+    public AbstractTridentProcessor compile(Map<String, ISqlTridentDataSource> sources, String query) throws Exception {
+        TridentRel relNode = getPlan(query);
+
+        TridentPlanCreator tridentPlanCreator = new TridentPlanCreator(sources, new RexBuilder(typeFactory));
+        relNode.tridentPlan(tridentPlanCreator);
+
+        final TridentTopology topology = tridentPlanCreator.getTopology();
+        final IAggregatableStream lastStream = tridentPlanCreator.pop();
+        final DataContext dc = tridentPlanCreator.getDataContext();
+        final List<CompilingClassLoader> cls = tridentPlanCreator.getClassLoaders();
+
+        return new AbstractTridentProcessor() {
+            @Override
+            public TridentTopology build() {
+                return topology;
+            }
+
+            @Override
+            public Stream outputStream() {
+                return lastStream.toStream();
+            }
+
+            @Override
+            public DataContext getDataContext() {
+                return dc;
+            }
+
+            @Override
+            public List<CompilingClassLoader> getClassLoaders() {
+                return cls;
+            }
+        };
+    }
+
+    public TridentRel getPlan(String query) throws ValidationException, RelConversionException, SqlParseException {
+        return (TridentRel) validateAndConvert(planner.parse(query));
+    }
+
+    private RelNode validateAndConvert(SqlNode sqlNode) throws ValidationException, RelConversionException {
+        SqlNode validated = validateNode(sqlNode);
+        RelNode relNode = convertToRelNode(validated);
+        return convertToStormRel(relNode);
+    }
+
+    private RelNode convertToStormRel(RelNode relNode) throws RelConversionException {
+        RelTraitSet traitSet = relNode.getTraitSet();
+        traitSet = traitSet.simplify();
+
+        // PlannerImpl.transform() optimizes RelNode with ruleset
+        return planner.transform(STORM_REL_CONVERSION_RULES, traitSet.plus(TridentLogicalConvention.INSTANCE), relNode);
+    }
+
+    private RelNode convertToRelNode(SqlNode sqlNode) throws RelConversionException {
+        return planner.rel(sqlNode).rel;
+    }
+
+    private SqlNode validateNode(SqlNode sqlNode) throws ValidationException {
+        SqlNode validatedSqlNode = planner.validate(sqlNode);
+        validatedSqlNode.accept(new UnsupportedOperatorsVisitor());
+        return validatedSqlNode;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentPlanCreator.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentPlanCreator.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentPlanCreator.java
new file mode 100644
index 0000000..30ebf7e
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentPlanCreator.java
@@ -0,0 +1,123 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.planner.trident;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.storm.sql.compiler.RexNodeToJavaCodeCompiler;
+import org.apache.storm.sql.javac.CompilingClassLoader;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.sql.runtime.calcite.DebuggableExecutableExpression;
+import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
+import org.apache.storm.sql.runtime.calcite.StormDataContext;
+import org.apache.storm.trident.TridentTopology;
+import org.apache.storm.trident.fluent.IAggregatableStream;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.List;
+import java.util.Map;
+
+public class TridentPlanCreator {
+    private final Map<String, ISqlTridentDataSource> sources;
+    private final JavaTypeFactory typeFactory;
+    private final RexNodeToJavaCodeCompiler rexCompiler;
+    private final DataContext dataContext;
+    private final TridentTopology topology;
+
+    private final Deque<IAggregatableStream> streamStack = new ArrayDeque<>();
+    private final List<CompilingClassLoader> classLoaders = new ArrayList<>();
+
+    public TridentPlanCreator(Map<String, ISqlTridentDataSource> sources, RexBuilder rexBuilder) {
+        this.sources = sources;
+        this.rexCompiler = new RexNodeToJavaCodeCompiler(rexBuilder);
+        this.typeFactory = (JavaTypeFactory) rexBuilder.getTypeFactory();
+
+        this.topology = new TridentTopology();
+        this.dataContext = new StormDataContext();
+    }
+
+    public void addStream(IAggregatableStream stream) throws Exception {
+        push(stream);
+    }
+
+    public IAggregatableStream pop() {
+        return streamStack.pop();
+    }
+
+    public Map<String, ISqlTridentDataSource> getSources() {
+        return sources;
+    }
+
+    public DataContext getDataContext() {
+        return dataContext;
+    }
+
+    public JavaTypeFactory getTypeFactory() {
+        return typeFactory;
+    }
+
+    public TridentTopology getTopology() {
+        return topology;
+    }
+
+    public ExecutableExpression createScalarInstance(List<RexNode> nodes, RelDataType inputRowType, String className)
+            throws CompilingClassLoader.CompilerException, ClassNotFoundException, IllegalAccessException, InstantiationException {
+        String expr = rexCompiler.compile(nodes, inputRowType, className);
+        CompilingClassLoader classLoader = new CompilingClassLoader(
+                getLastClassLoader(), className, expr, null);
+        ExecutableExpression instance = (ExecutableExpression) classLoader.loadClass(className).newInstance();
+        addClassLoader(classLoader);
+        return new DebuggableExecutableExpression(instance, expr);
+    }
+
+    public ExecutableExpression createScalarInstance(RexProgram program, String className)
+            throws CompilingClassLoader.CompilerException, ClassNotFoundException, IllegalAccessException, InstantiationException {
+        String expr = rexCompiler.compile(program, className);
+        CompilingClassLoader classLoader = new CompilingClassLoader(
+                getLastClassLoader(), className, expr, null);
+        ExecutableExpression instance = (ExecutableExpression) classLoader.loadClass(className).newInstance();
+        addClassLoader(classLoader);
+        return new DebuggableExecutableExpression(instance, expr);
+    }
+
+    private void push(IAggregatableStream stream) {
+        streamStack.push(stream);
+    }
+
+    public void addClassLoader(CompilingClassLoader compilingClassLoader) {
+        this.classLoaders.add(compilingClassLoader);
+    }
+
+    public ClassLoader getLastClassLoader() {
+        if (this.classLoaders.size() > 0) {
+            return this.classLoaders.get(this.classLoaders.size() - 1);
+        } else {
+            return this.getClass().getClassLoader();
+        }
+    }
+
+    public List<CompilingClassLoader> getClassLoaders() {
+        return classLoaders;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentStormRuleSets.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentStormRuleSets.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentStormRuleSets.java
new file mode 100644
index 0000000..e146069
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/TridentStormRuleSets.java
@@ -0,0 +1,110 @@
+/*
+ *
+ *  * 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.sql.planner.trident;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.rel.rules.CalcMergeRule;
+import org.apache.calcite.rel.rules.FilterCalcMergeRule;
+import org.apache.calcite.rel.rules.FilterProjectTransposeRule;
+import org.apache.calcite.rel.rules.FilterToCalcRule;
+import org.apache.calcite.rel.rules.ProjectCalcMergeRule;
+import org.apache.calcite.rel.rules.ProjectFilterTransposeRule;
+import org.apache.calcite.rel.rules.ProjectRemoveRule;
+import org.apache.calcite.rel.rules.ProjectToCalcRule;
+import org.apache.calcite.rel.rules.PruneEmptyRules;
+import org.apache.calcite.rel.rules.ReduceExpressionsRule;
+import org.apache.calcite.rel.rules.SortRemoveRule;
+import org.apache.calcite.rel.rules.UnionEliminatorRule;
+import org.apache.calcite.rel.stream.StreamRules;
+import org.apache.calcite.tools.RuleSet;
+import org.apache.storm.sql.planner.trident.rules.TridentCalcRule;
+import org.apache.storm.sql.planner.trident.rules.TridentFilterRule;
+import org.apache.storm.sql.planner.trident.rules.TridentScanRule;
+import org.apache.storm.sql.planner.trident.rules.TridentAggregateRule;
+import org.apache.storm.sql.planner.trident.rules.TridentJoinRule;
+import org.apache.storm.sql.planner.trident.rules.TridentModifyRule;
+import org.apache.storm.sql.planner.trident.rules.TridentProjectRule;
+
+import java.util.Iterator;
+
+public class TridentStormRuleSets {
+    private static final ImmutableSet<RelOptRule> calciteToStormConversionRules =
+            ImmutableSet.<RelOptRule>builder().add(
+                    SortRemoveRule.INSTANCE,
+
+                    FilterToCalcRule.INSTANCE,
+                    ProjectToCalcRule.INSTANCE,
+                    FilterCalcMergeRule.INSTANCE,
+                    ProjectCalcMergeRule.INSTANCE,
+                    CalcMergeRule.INSTANCE,
+
+                    PruneEmptyRules.FILTER_INSTANCE,
+                    PruneEmptyRules.PROJECT_INSTANCE,
+                    PruneEmptyRules.UNION_INSTANCE,
+
+                    ProjectFilterTransposeRule.INSTANCE,
+                    FilterProjectTransposeRule.INSTANCE,
+                    ProjectRemoveRule.INSTANCE,
+
+                    ReduceExpressionsRule.FILTER_INSTANCE,
+                    ReduceExpressionsRule.PROJECT_INSTANCE,
+                    ReduceExpressionsRule.CALC_INSTANCE,
+
+                    // merge and push unions rules
+                    UnionEliminatorRule.INSTANCE,
+
+                    TridentScanRule.INSTANCE,
+                    TridentFilterRule.INSTANCE,
+                    TridentProjectRule.INSTANCE,
+                    TridentAggregateRule.INSTANCE,
+                    TridentJoinRule.INSTANCE,
+                    TridentModifyRule.INSTANCE,
+                    TridentCalcRule.INSTANCE
+            ).build();
+
+    public static RuleSet[] getRuleSets() {
+        return new RuleSet[]{
+                new StormRuleSet(StreamRules.RULES),
+                new StormRuleSet(ImmutableSet.<RelOptRule>builder().addAll(StreamRules.RULES).addAll(calciteToStormConversionRules).build())
+        };
+    }
+
+    private static class StormRuleSet implements RuleSet {
+        final ImmutableSet<RelOptRule> rules;
+
+        public StormRuleSet(ImmutableSet<RelOptRule> rules) {
+            this.rules = rules;
+        }
+
+        public StormRuleSet(ImmutableList<RelOptRule> rules) {
+            this.rules = ImmutableSet.<RelOptRule>builder()
+                    .addAll(rules)
+                    .build();
+        }
+
+        @Override
+        public Iterator<RelOptRule> iterator() {
+            return rules.iterator();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentCalcRel.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentCalcRel.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentCalcRel.java
new file mode 100644
index 0000000..482e841
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentCalcRel.java
@@ -0,0 +1,97 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.planner.trident.rel;
+
+import com.google.common.collect.Lists;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Calc;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.storm.sql.planner.StormRelUtils;
+import org.apache.storm.sql.planner.rel.StormCalcRelBase;
+import org.apache.storm.sql.planner.trident.TridentPlanCreator;
+import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
+import org.apache.storm.sql.runtime.trident.functions.EvaluationCalc;
+import org.apache.storm.trident.Stream;
+import org.apache.storm.tuple.Fields;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class TridentCalcRel extends StormCalcRelBase implements TridentRel {
+    public TridentCalcRel(RelOptCluster cluster, RelTraitSet traits, RelNode child, RexProgram program) {
+        super(cluster, traits, child, program);
+    }
+
+    @Override
+    public Calc copy(RelTraitSet traitSet, RelNode child, RexProgram program) {
+        return new TridentCalcRel(getCluster(), traitSet, child, program);
+    }
+
+    @Override
+    public void tridentPlan(TridentPlanCreator planCreator) throws Exception {
+        // SingleRel
+        RelNode input = getInput();
+        StormRelUtils.getStormRelInput(input).tridentPlan(planCreator);
+        Stream inputStream = planCreator.pop().toStream();
+
+        String stageName = StormRelUtils.getStageName(this);
+
+        RelDataType inputRowType = getInput(0).getRowType();
+
+        List<String> outputFieldNames = getRowType().getFieldNames();
+        int outputCount = outputFieldNames.size();
+
+        // filter
+        ExecutableExpression filterInstance = null;
+        RexLocalRef condition = program.getCondition();
+        if (condition != null) {
+            RexNode conditionNode = program.expandLocalRef(condition);
+            filterInstance = planCreator.createScalarInstance(Lists.newArrayList(conditionNode), inputRowType,
+                    StormRelUtils.getClassName(this));
+        }
+
+        // projection
+        ExecutableExpression projectionInstance = null;
+        List<RexLocalRef> projectList = program.getProjectList();
+        if (projectList != null && !projectList.isEmpty()) {
+            List<RexNode> expandedNodes = new ArrayList<>();
+            for (RexLocalRef project : projectList) {
+                expandedNodes.add(program.expandLocalRef(project));
+            }
+
+            projectionInstance = planCreator.createScalarInstance(expandedNodes, inputRowType,
+                    StormRelUtils.getClassName(this));
+        }
+
+        if (projectionInstance == null && filterInstance == null) {
+            // it shouldn't be happen
+            throw new IllegalStateException("Either projection or condition, or both should be provided.");
+        }
+
+        final Stream finalStream = inputStream
+                .flatMap(new EvaluationCalc(filterInstance, projectionInstance, outputCount, planCreator.getDataContext()), new Fields(outputFieldNames))
+                .name(stageName);
+
+        planCreator.addStream(finalStream);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentFilterRel.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentFilterRel.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentFilterRel.java
new file mode 100644
index 0000000..1fe0927
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentFilterRel.java
@@ -0,0 +1,65 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.planner.trident.rel;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+import org.apache.storm.sql.planner.StormRelUtils;
+import org.apache.storm.sql.planner.rel.StormFilterRelBase;
+import org.apache.storm.sql.planner.trident.TridentPlanCreator;
+import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
+import org.apache.storm.sql.runtime.trident.functions.EvaluationFilter;
+import org.apache.storm.trident.Stream;
+import org.apache.storm.trident.fluent.IAggregatableStream;
+
+import java.util.List;
+
+public class TridentFilterRel extends StormFilterRelBase implements TridentRel {
+    public TridentFilterRel(RelOptCluster cluster, RelTraitSet traits, RelNode child, RexNode condition) {
+        super(cluster, traits, child, condition);
+    }
+
+    @Override
+    public Filter copy(RelTraitSet traitSet, RelNode input, RexNode condition) {
+        return new TridentFilterRel(getCluster(), traitSet, input, condition);
+    }
+
+    @Override
+    public void tridentPlan(TridentPlanCreator planCreator) throws Exception {
+        // SingleRel
+        RelNode input = getInput();
+        StormRelUtils.getStormRelInput(input).tridentPlan(planCreator);
+        Stream inputStream = planCreator.pop().toStream();
+
+        String stageName = StormRelUtils.getStageName(this);
+
+        List<RexNode> childExps = getChildExps();
+        RelDataType inputRowType = getInput(0).getRowType();
+
+        String filterClassName = StormRelUtils.getClassName(this);
+        ExecutableExpression filterInstance = planCreator.createScalarInstance(childExps, inputRowType, filterClassName);
+
+        IAggregatableStream finalStream = inputStream.filter(new EvaluationFilter(filterInstance, planCreator.getDataContext()))
+                .name(stageName);
+        planCreator.addStream(finalStream);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentLogicalConvention.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentLogicalConvention.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentLogicalConvention.java
new file mode 100644
index 0000000..d221498
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentLogicalConvention.java
@@ -0,0 +1,67 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.planner.trident.rel;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitDef;
+import org.apache.calcite.plan.RelTraitSet;
+
+public enum TridentLogicalConvention implements Convention {
+  INSTANCE;
+
+  @Override
+  public Class getInterface() {
+    return TridentRel.class;
+  }
+
+  @Override
+  public String getName() {
+    return "STORM_LOGICAL";
+  }
+
+  @Override
+  public RelTraitDef getTraitDef() {
+    return ConventionTraitDef.INSTANCE;
+  }
+
+  @Override
+  public boolean satisfies(RelTrait trait) {
+    return this == trait;
+  }
+
+  @Override
+  public void register(RelOptPlanner planner) {}
+
+  @Override
+  public String toString() {
+    return getName();
+  }
+
+  @Override
+  public boolean canConvertConvention(Convention toConvention) {
+    return false;
+  }
+
+  @Override
+  public boolean useAbstractConvertersForConversion(RelTraitSet fromTraits, RelTraitSet toTraits) {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentProjectRel.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentProjectRel.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentProjectRel.java
new file mode 100644
index 0000000..06be5d7
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentProjectRel.java
@@ -0,0 +1,69 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.planner.trident.rel;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+import org.apache.storm.sql.planner.StormRelUtils;
+import org.apache.storm.sql.planner.rel.StormProjectRelBase;
+import org.apache.storm.sql.planner.trident.TridentPlanCreator;
+import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
+import org.apache.storm.sql.runtime.trident.functions.EvaluationFunction;
+import org.apache.storm.trident.Stream;
+import org.apache.storm.tuple.Fields;
+
+import java.util.List;
+
+public class TridentProjectRel extends StormProjectRelBase implements TridentRel {
+    public TridentProjectRel(RelOptCluster cluster, RelTraitSet traits, RelNode input, List<? extends RexNode> projects, RelDataType rowType) {
+        super(cluster, traits, input, projects, rowType);
+    }
+
+    @Override
+    public Project copy(RelTraitSet traitSet, RelNode input, List<RexNode> projects, RelDataType rowType) {
+        return new TridentProjectRel(getCluster(), traitSet, input, projects, rowType);
+    }
+
+    @Override
+    public void tridentPlan(TridentPlanCreator planCreator) throws Exception {
+        // SingleRel
+        RelNode input = getInput();
+        StormRelUtils.getStormRelInput(input).tridentPlan(planCreator);
+        Stream inputStream = planCreator.pop().toStream();
+
+        String stageName = StormRelUtils.getStageName(this);
+        String projectionClassName = StormRelUtils.getClassName(this);
+
+        List<String> outputFieldNames = getRowType().getFieldNames();
+        int outputCount = outputFieldNames.size();
+
+        List<RexNode> childExps = getChildExps();
+        RelDataType inputRowType = getInput(0).getRowType();
+
+        ExecutableExpression projectionInstance = planCreator.createScalarInstance(childExps, inputRowType, projectionClassName);
+        Stream finalStream = inputStream
+                .map(new EvaluationFunction(projectionInstance, outputCount, planCreator.getDataContext()), new Fields(outputFieldNames))
+                .name(stageName);
+
+        planCreator.addStream(finalStream);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentRel.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentRel.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentRel.java
new file mode 100644
index 0000000..8b8e949
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentRel.java
@@ -0,0 +1,25 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.planner.trident.rel;
+
+import org.apache.storm.sql.planner.rel.StormRelNode;
+import org.apache.storm.sql.planner.trident.TridentPlanCreator;
+
+public interface TridentRel extends StormRelNode {
+    void tridentPlan(TridentPlanCreator planCreator) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamInsertRel.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamInsertRel.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamInsertRel.java
new file mode 100644
index 0000000..e92c29b
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamInsertRel.java
@@ -0,0 +1,76 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.planner.trident.rel;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rex.RexNode;
+import org.apache.storm.sql.planner.StormRelUtils;
+import org.apache.storm.sql.planner.rel.StormStreamInsertRelBase;
+import org.apache.storm.sql.planner.trident.TridentPlanCreator;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.trident.Stream;
+import org.apache.storm.trident.fluent.IAggregatableStream;
+import org.apache.storm.tuple.Fields;
+
+import java.util.List;
+
+public class TridentStreamInsertRel extends StormStreamInsertRelBase implements TridentRel {
+    public TridentStreamInsertRel(RelOptCluster cluster, RelTraitSet traits, RelOptTable table, Prepare.CatalogReader catalogReader, RelNode child, Operation operation, List<String> updateColumnList, List<RexNode> sourceExpressionList, boolean flattened) {
+        super(cluster, traits, table, catalogReader, child, operation, updateColumnList, sourceExpressionList, flattened);
+    }
+
+    @Override
+    public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+        return new TridentStreamInsertRel(getCluster(), traitSet, getTable(), getCatalogReader(),
+                sole(inputs), getOperation(), getUpdateColumnList(), getSourceExpressionList(), isFlattened());
+    }
+
+    @Override
+    public void tridentPlan(TridentPlanCreator planCreator) throws Exception {
+        // SingleRel
+        RelNode input = getInput();
+        StormRelUtils.getStormRelInput(input).tridentPlan(planCreator);
+        Stream inputStream = planCreator.pop().toStream();
+
+        String stageName = StormRelUtils.getStageName(this);
+
+        Preconditions.checkArgument(isInsert(), "Only INSERT statement is supported.");
+
+        List<String> inputFields = this.input.getRowType().getFieldNames();
+        List<String> outputFields = getRowType().getFieldNames();
+
+        // FIXME: this should be really different...
+        String tableName = Joiner.on('.').join(getTable().getQualifiedName());
+        ISqlTridentDataSource.SqlTridentConsumer consumer = planCreator.getSources().get(tableName).getConsumer();
+
+        // In fact this is normally the end of stream, but I'm still not sure so I open new streams based on State values
+        IAggregatableStream finalStream = inputStream
+                .partitionPersist(consumer.getStateFactory(), new Fields(inputFields), consumer.getStateUpdater(),
+                        new Fields(outputFields))
+                .newValuesStream().name(stageName);
+
+        planCreator.addStream(finalStream);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamScanRel.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamScanRel.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamScanRel.java
new file mode 100644
index 0000000..c563d73
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rel/TridentStreamScanRel.java
@@ -0,0 +1,55 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.planner.trident.rel;
+
+import com.google.common.base.Joiner;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.storm.sql.planner.StormRelUtils;
+import org.apache.storm.sql.planner.rel.StormStreamScanRelBase;
+import org.apache.storm.sql.planner.trident.TridentPlanCreator;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.trident.fluent.IAggregatableStream;
+
+import java.util.Map;
+
+public class TridentStreamScanRel extends StormStreamScanRelBase implements TridentRel {
+    private final int parallelismHint;
+
+    public TridentStreamScanRel(RelOptCluster cluster, RelTraitSet traitSet, RelOptTable table, int parallelismHint) {
+        super(cluster, traitSet, table);
+        this.parallelismHint = parallelismHint;
+    }
+
+    @Override
+    public void tridentPlan(TridentPlanCreator planCreator) throws Exception {
+        String sourceName = Joiner.on('.').join(getTable().getQualifiedName());
+
+        // FIXME: this should be really different...
+        Map<String, ISqlTridentDataSource> sources = planCreator.getSources();
+        if (!sources.containsKey(sourceName)) {
+            throw new RuntimeException("Cannot find table " + sourceName);
+        }
+
+        String stageName = StormRelUtils.getStageName(this);
+        IAggregatableStream finalStream = planCreator.getTopology().newStream(stageName, sources.get(sourceName).getProducer())
+                .parallelismHint(parallelismHint);
+        planCreator.addStream(finalStream);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentAggregateRule.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentAggregateRule.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentAggregateRule.java
new file mode 100644
index 0000000..ac35414
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentAggregateRule.java
@@ -0,0 +1,39 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.planner.trident.rules;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
+
+public class TridentAggregateRule extends ConverterRule {
+    public static final RelOptRule INSTANCE = new TridentAggregateRule();
+
+    private TridentAggregateRule() {
+        super(LogicalAggregate.class, Convention.NONE, TridentLogicalConvention.INSTANCE, "TridentAggregateRule");
+    }
+
+    @Override
+    public RelNode convert(RelNode rel) {
+        throw new UnsupportedOperationException("Aggregate operation is not supported.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentCalcRule.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentCalcRule.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentCalcRule.java
new file mode 100644
index 0000000..25126ec
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentCalcRule.java
@@ -0,0 +1,45 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.planner.trident.rules;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Calc;
+import org.apache.calcite.rel.logical.LogicalCalc;
+import org.apache.storm.sql.planner.trident.rel.TridentCalcRel;
+import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
+
+public class TridentCalcRule extends ConverterRule {
+  public static final TridentCalcRule INSTANCE = new TridentCalcRule();
+
+  private TridentCalcRule() {
+    super(LogicalCalc.class, Convention.NONE, TridentLogicalConvention.INSTANCE, "TridentCalcRule");
+  }
+
+  @Override
+  public RelNode convert(RelNode rel) {
+    final Calc calc = (Calc) rel;
+    final RelNode input = calc.getInput();
+
+    return new TridentCalcRel(calc.getCluster(), calc.getTraitSet().replace(TridentLogicalConvention.INSTANCE),
+            convert(input, input.getTraitSet().replace(TridentLogicalConvention.INSTANCE)),
+            calc.getProgram());
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentFilterRule.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentFilterRule.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentFilterRule.java
new file mode 100644
index 0000000..7f9c41f
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentFilterRule.java
@@ -0,0 +1,45 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.planner.trident.rules;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.storm.sql.planner.trident.rel.TridentFilterRel;
+import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
+
+public class TridentFilterRule extends ConverterRule {
+  public static TridentFilterRule INSTANCE = new TridentFilterRule();
+
+  private TridentFilterRule() {
+    super(LogicalFilter.class, Convention.NONE, TridentLogicalConvention.INSTANCE, "TridentFilterRule");
+  }
+
+  @Override
+  public RelNode convert(RelNode rel) {
+    final Filter filter = (Filter) rel;
+    final RelNode input = filter.getInput();
+
+    return new TridentFilterRel(filter.getCluster(),
+        filter.getTraitSet().replace(TridentLogicalConvention.INSTANCE),
+        convert(input, input.getTraitSet().replace(TridentLogicalConvention.INSTANCE)),
+        filter.getCondition());
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentJoinRule.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentJoinRule.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentJoinRule.java
new file mode 100644
index 0000000..90f5083
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentJoinRule.java
@@ -0,0 +1,37 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.planner.trident.rules;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
+
+public class TridentJoinRule extends ConverterRule {
+  public static final TridentJoinRule INSTANCE = new TridentJoinRule();
+
+  private TridentJoinRule() {
+    super(LogicalJoin.class, Convention.NONE, TridentLogicalConvention.INSTANCE, "TridentJoinRule");
+  }
+
+  @Override
+  public RelNode convert(RelNode rel) {
+    throw new UnsupportedOperationException("Join operation is not supported.");
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentModifyRule.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentModifyRule.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentModifyRule.java
new file mode 100644
index 0000000..2155451
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentModifyRule.java
@@ -0,0 +1,71 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.planner.trident.rules;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.TableModify;
+import org.apache.calcite.rel.logical.LogicalTableModify;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.schema.Table;
+import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
+import org.apache.storm.sql.planner.trident.rel.TridentStreamInsertRel;
+
+import java.util.List;
+
+public class TridentModifyRule extends ConverterRule {
+  public static final TridentModifyRule INSTANCE = new TridentModifyRule();
+
+  private TridentModifyRule() {
+    super(LogicalTableModify.class, Convention.NONE, TridentLogicalConvention.INSTANCE, "TridentModifyRule");
+  }
+
+  @Override
+  public RelNode convert(RelNode rel) {
+    final TableModify tableModify = (TableModify) rel;
+    final RelNode input = tableModify.getInput();
+
+    final RelOptCluster cluster = tableModify.getCluster();
+    final RelTraitSet traitSet = tableModify.getTraitSet().replace(TridentLogicalConvention.INSTANCE);
+    final RelOptTable relOptTable = tableModify.getTable();
+    final Prepare.CatalogReader catalogReader = tableModify.getCatalogReader();
+    final RelNode convertedInput = convert(input, input.getTraitSet().replace(TridentLogicalConvention.INSTANCE));
+    final TableModify.Operation operation = tableModify.getOperation();
+    final List<String> updateColumnList = tableModify.getUpdateColumnList();
+    final List<RexNode> sourceExpressionList = tableModify.getSourceExpressionList();
+    final boolean flattened = tableModify.isFlattened();
+
+    final Table table = tableModify.getTable().unwrap(Table.class);
+
+    switch (table.getJdbcTableType()) {
+      case STREAM:
+        if (operation != TableModify.Operation.INSERT) {
+          throw new UnsupportedOperationException(String.format("Streams doesn't support %s modify operation", operation));
+        }
+        return new TridentStreamInsertRel(cluster, traitSet, relOptTable, catalogReader, convertedInput, operation,
+            updateColumnList, sourceExpressionList, flattened);
+      default:
+        throw new IllegalArgumentException(String.format("Unsupported table type: %s", table.getJdbcTableType()));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentProjectRule.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentProjectRule.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentProjectRule.java
new file mode 100644
index 0000000..2922725
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentProjectRule.java
@@ -0,0 +1,45 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.planner.trident.rules;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
+import org.apache.storm.sql.planner.trident.rel.TridentProjectRel;
+
+public class TridentProjectRule extends ConverterRule {
+  public static final TridentProjectRule INSTANCE = new TridentProjectRule();
+
+  private TridentProjectRule() {
+    super(LogicalProject.class, Convention.NONE, TridentLogicalConvention.INSTANCE,
+        "TridentProjectRule");
+  }
+
+  @Override
+  public RelNode convert(RelNode rel) {
+    final Project project = (Project) rel;
+    final RelNode input = project.getInput();
+
+    return new TridentProjectRel(project.getCluster(),
+        project.getTraitSet().replace(TridentLogicalConvention.INSTANCE),
+        convert(input, input.getTraitSet().replace(TridentLogicalConvention.INSTANCE)), project.getProjects(), project.getRowType());
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentScanRule.java
----------------------------------------------------------------------
diff --git a/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentScanRule.java b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentScanRule.java
new file mode 100644
index 0000000..abbd680
--- /dev/null
+++ b/sql/storm-sql-core/src/jvm/org/apache/storm/sql/planner/trident/rules/TridentScanRule.java
@@ -0,0 +1,58 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sql.planner.trident.rules;
+
+import org.apache.calcite.adapter.enumerable.EnumerableConvention;
+import org.apache.calcite.adapter.enumerable.EnumerableTableScan;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.schema.Table;
+import org.apache.storm.sql.calcite.ParallelStreamableTable;
+import org.apache.storm.sql.planner.trident.rel.TridentLogicalConvention;
+import org.apache.storm.sql.planner.trident.rel.TridentStreamScanRel;
+
+public class TridentScanRule extends ConverterRule {
+  public static final TridentScanRule INSTANCE = new TridentScanRule();
+  public static final int DEFAULT_PARALLELISM_HINT = 1;
+
+  private TridentScanRule() {
+    super(EnumerableTableScan.class, EnumerableConvention.INSTANCE, TridentLogicalConvention.INSTANCE, "TridentScanRule");
+  }
+
+  @Override
+  public RelNode convert(RelNode rel) {
+    final TableScan scan = (TableScan) rel;
+    int parallelismHint = DEFAULT_PARALLELISM_HINT;
+
+    final ParallelStreamableTable parallelTable = scan.getTable().unwrap(ParallelStreamableTable.class);
+    if (parallelTable != null && parallelTable.parallelismHint() != null) {
+      parallelismHint = parallelTable.parallelismHint();
+    }
+
+    final Table table = scan.getTable().unwrap(Table.class);
+    switch (table.getJdbcTableType()) {
+      case STREAM:
+        return new TridentStreamScanRel(scan.getCluster(),
+            scan.getTraitSet().replace(TridentLogicalConvention.INSTANCE),
+            scan.getTable(), parallelismHint);
+      default:
+        throw new IllegalArgumentException(String.format("Unsupported table type: %s", table.getJdbcTableType()));
+    }
+  }
+}


[17/23] storm git commit: STORM-2453 Move non-connectors into the top directory

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/pom.xml
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/pom.xml b/external/sql/storm-sql-core/pom.xml
deleted file mode 100644
index 776a54c..0000000
--- a/external/sql/storm-sql-core/pom.xml
+++ /dev/null
@@ -1,279 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements.  See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License.  You may obtain a copy of the License at
-
-     http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/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>
-        <relativePath>../../../pom.xml</relativePath>
-    </parent>
-
-    <artifactId>storm-sql-core</artifactId>
-
-    <developers>
-        <developer>
-            <id>haohui</id>
-            <name>Haohui Mai</name>
-            <email>ricetons@gmail.com</email>
-        </developer>
-    </developers>
-
-    <dependencies>
-        <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-sql-runtime</artifactId>
-            <version>${project.version}</version>
-            <scope>provided</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.storm</groupId>
-            <artifactId>storm-sql-runtime</artifactId>
-            <version>${project.version}</version>
-            <scope>test</scope>
-            <type>test-jar</type>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.calcite</groupId>
-            <artifactId>calcite-core</artifactId>
-            <version>${calcite.version}</version>
-            <exclusions>
-                <exclusion>
-                    <groupId>com.fasterxml.jackson.core</groupId>
-                    <artifactId>jackson-annotations</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-        <dependency>
-            <groupId>com.fasterxml.jackson.core</groupId>
-            <artifactId>jackson-annotations</artifactId>
-            <version>${jackson.version}</version>
-        </dependency>
-        <dependency>
-            <groupId>commons-cli</groupId>
-            <artifactId>commons-cli</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>commons-lang</groupId>
-            <artifactId>commons-lang</artifactId>
-            <scope>runtime</scope>
-        </dependency>
-        <dependency>
-            <groupId>junit</groupId>
-            <artifactId>junit</artifactId>
-            <scope>test</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.mockito</groupId>
-            <artifactId>mockito-all</artifactId>
-            <scope>test</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.commons</groupId>
-            <artifactId>commons-collections4</artifactId>
-            <version>4.1</version>
-            <scope>test</scope>
-        </dependency>
-    </dependencies>
-    <build>
-        <sourceDirectory>src/jvm</sourceDirectory>
-        <testSourceDirectory>src/test</testSourceDirectory>
-        <plugins>
-            <plugin>
-                <artifactId>maven-resources-plugin</artifactId>
-                <executions>
-                    <execution>
-                        <id>copy-fmpp-resources</id>
-                        <phase>initialize</phase>
-                        <goals>
-                            <goal>copy-resources</goal>
-                        </goals>
-                        <configuration>
-                            <outputDirectory>${project.build.directory}/codegen</outputDirectory>
-                            <resources>
-                                <resource>
-                                    <directory>src/codegen</directory>
-                                    <filtering>false</filtering>
-                                </resource>
-                            </resources>
-                        </configuration>
-                    </execution>
-                    <execution>
-                        <id>copy-java-sources</id>
-                        <phase>process-sources</phase>
-                        <goals>
-                            <goal>copy-resources</goal>
-                        </goals>
-                        <configuration>
-                            <outputDirectory>${basedir}/target/classes/</outputDirectory>
-                            <resources>
-                                <resource>
-                                    <directory>src/jvm</directory>
-                                    <filtering>true</filtering>
-                                </resource>
-                                <resource>
-                                    <directory>src/test</directory>
-                                    <filtering>true</filtering>
-                                </resource>
-                                <resource>
-                                    <directory>target/generated-sources</directory>
-                                    <!-- <include>*/org</include> -->
-                                    <filtering>true</filtering>
-                                </resource>
-                            </resources>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-jar-plugin</artifactId>
-                <version>2.2</version>
-                <executions>
-                    <execution>
-                        <goals>
-                            <goal>test-jar</goal>
-                        </goals>
-                    </execution>
-                </executions>
-            </plugin>
-            <plugin>
-                <!-- Extract parser grammar template from calcite-core.jar and put
-                     it under ${project.build.directory} where all freemarker templates are. -->
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-dependency-plugin</artifactId>
-                <version>2.8</version>
-                <executions>
-                    <execution>
-                        <id>unpack-parser-template</id>
-                        <phase>initialize</phase>
-                        <goals>
-                            <goal>unpack</goal>
-                        </goals>
-                        <configuration>
-                            <artifactItems>
-                                <artifactItem>
-                                    <groupId>org.apache.calcite</groupId>
-                                    <artifactId>calcite-core</artifactId>
-                                    <type>jar</type>
-                                    <overWrite>true</overWrite>
-                                    <outputDirectory>${project.build.directory}/</outputDirectory>
-                                    <includes>**/Parser.jj</includes>
-                                </artifactItem>
-                            </artifactItems>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-            <!-- using appassembler-maven-plugin instead of maven-dependency-plugin to copy dependencies
-            as copy and unpack goal are not working together -->
-            <plugin>
-                <groupId>org.codehaus.mojo</groupId>
-                <artifactId>appassembler-maven-plugin</artifactId>
-                <version>1.9</version>
-                <executions>
-                    <execution>
-                        <id>create-repo</id>
-                        <goals>
-                            <goal>create-repository</goal>
-                        </goals>
-                        <configuration>
-                            <assembleDirectory>${project.build.directory}/app-assembler</assembleDirectory>
-                            <repositoryLayout>flat</repositoryLayout>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-            <plugin>
-                <groupId>com.googlecode.fmpp-maven-plugin</groupId>
-                <artifactId>fmpp-maven-plugin</artifactId>
-                <version>1.0</version>
-                <dependencies>
-                    <dependency>
-                        <groupId>org.freemarker</groupId>
-                        <artifactId>freemarker</artifactId>
-                        <version>2.3.25-incubating</version>
-                    </dependency>
-                </dependencies>
-                <executions>
-                    <execution>
-                        <id>generate-fmpp-sources</id>
-                        <phase>generate-sources</phase>
-                        <goals>
-                            <goal>generate</goal>
-                        </goals>
-                        <configuration>
-                            <cfgFile>${project.build.directory}/codegen/config.fmpp</cfgFile>
-                            <outputDirectory>target/generated-sources</outputDirectory>
-                            <templateDirectory>${project.build.directory}/codegen/templates</templateDirectory>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-            <plugin>
-                <groupId>org.codehaus.mojo</groupId>
-                <artifactId>build-helper-maven-plugin</artifactId>
-                <version>1.5</version>
-                <executions>
-                    <execution>
-                        <id>add-generated-sources</id>
-                        <phase>process-sources</phase>
-                        <goals>
-                            <goal>add-source</goal>
-                        </goals>
-                        <configuration>
-                            <sources>
-                                <source>${project.build.directory}/generated-sources</source>
-                            </sources>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-            <plugin>
-                <groupId>org.codehaus.mojo</groupId>
-                <artifactId>javacc-maven-plugin</artifactId>
-                <version>2.4</version>
-                <executions>
-                    <execution>
-                        <phase>generate-sources</phase>
-                        <id>javacc</id>
-                        <goals>
-                            <goal>javacc</goal>
-                        </goals>
-                        <configuration>
-                            <sourceDirectory>${project.build.directory}/generated-sources/</sourceDirectory>
-                            <includes>
-                                <include>**/Parser.jj</include>
-                            </includes>
-                            <lookAhead>2</lookAhead>
-                            <isStatic>false</isStatic>
-                            <outputDirectory>${project.build.directory}/generated-sources/</outputDirectory>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-        </plugins>
-    </build>
-</project>

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/codegen/config.fmpp
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/codegen/config.fmpp b/external/sql/storm-sql-core/src/codegen/config.fmpp
deleted file mode 100644
index be5a792..0000000
--- a/external/sql/storm-sql-core/src/codegen/config.fmpp
+++ /dev/null
@@ -1,23 +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.
-
-data: {
-  parser:                   tdd(../data/Parser.tdd)
-}
-
-freemarkerLinks: {
-  includes: includes/
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/codegen/data/Parser.tdd
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/codegen/data/Parser.tdd b/external/sql/storm-sql-core/src/codegen/data/Parser.tdd
deleted file mode 100644
index b0dccb6..0000000
--- a/external/sql/storm-sql-core/src/codegen/data/Parser.tdd
+++ /dev/null
@@ -1,80 +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.
-
-{
-  # Generated parser implementation class package and name
-  package: "org.apache.storm.sql.parser.impl",
-  class: "StormParserImpl",
-
-  # List of import statements.
-  imports: [
-    "org.apache.calcite.sql.validate.*",
-    "org.apache.calcite.util.*",
-    "org.apache.storm.sql.parser.*",
-    "java.util.*"
-  ]
-
-  # List of keywords.
-  keywords: [
-    "LOCATION",
-    "INPUTFORMAT",
-    "OUTPUTFORMAT",
-    "PARALLELISM",
-    "STORED",
-    "TBLPROPERTIES",
-    "JAR"
-  ]
-
-  # List of methods for parsing custom SQL statements.
-  statementParserMethods: [
-    "SqlCreateTable()",
-    "SqlCreateFunction()"
-  ]
-
-  # List of methods for parsing custom literals.
-  # Example: ParseJsonLiteral().
-  literalParserMethods: [
-  ]
-
-  # List of methods for parsing custom data types.
-  dataTypeParserMethods: [
-  ]
-
-  nonReservedKeywords: [
-  ]
-
-  createStatementParserMethods: [
-  ]
-
-  alterStatementParserMethods: [
-  ]
-
-  dropStatementParserMethods: [
-  ]
-
-  # List of files in @includes directory that have parser method
-  # implementations for custom SQL statements, literals or types
-  # given as part of "statementParserMethods", "literalParserMethods" or
-  # "dataTypeParserMethods".
-  implementationFiles: [
-    "parserImpls.ftl"
-  ]
-
-  includeCompoundIdentifier: true,
-  includeBraces: true,
-  includeAdditionalDeclarations: false,
-  allowBangEqual: false
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/codegen/includes/license.ftl
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/codegen/includes/license.ftl b/external/sql/storm-sql-core/src/codegen/includes/license.ftl
deleted file mode 100644
index 7e66353..0000000
--- a/external/sql/storm-sql-core/src/codegen/includes/license.ftl
+++ /dev/null
@@ -1,17 +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.
- */

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/codegen/includes/parserImpls.ftl
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/codegen/includes/parserImpls.ftl b/external/sql/storm-sql-core/src/codegen/includes/parserImpls.ftl
deleted file mode 100644
index 4143840..0000000
--- a/external/sql/storm-sql-core/src/codegen/includes/parserImpls.ftl
+++ /dev/null
@@ -1,113 +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. -->
-
-
-private void ColumnDef(List<ColumnDefinition> list) :
-{
-    SqlParserPos pos;
-    SqlIdentifier name;
-    SqlDataTypeSpec type;
-    ColumnConstraint constraint = null;
-    SqlMonotonicity monotonicity = SqlMonotonicity.NOT_MONOTONIC;
-}
-{
-    name = SimpleIdentifier() { pos = getPos(); }
-    type = DataType()
-    [
-      <PRIMARY> <KEY>
-      [ <ASC>   { monotonicity = SqlMonotonicity.INCREASING; }
-      | <DESC>  { monotonicity = SqlMonotonicity.DECREASING; }
-      ]
-      { constraint = new ColumnConstraint.PrimaryKey(monotonicity, getPos()); }
-    ]
-    {
-        list.add(new ColumnDefinition(name, type, constraint, pos));
-    }
-}
-
-SqlNodeList ColumnDefinitionList() :
-{
-    SqlParserPos pos;
-    List<ColumnDefinition> list = Lists.newArrayList();
-}
-{
-    <LPAREN> { pos = getPos(); }
-    ColumnDef(list)
-    ( <COMMA> ColumnDef(list) )*
-    <RPAREN> {
-        return new SqlNodeList(list, pos.plus(getPos()));
-    }
-}
-
-/**
- * CREATE EXTERNAL TABLE ( IF NOT EXISTS )?
- *   ( database_name '.' )? table_name ( '(' column_def ( ',' column_def )* ')'
- *   ( STORED AS INPUTFORMAT input_format_classname OUTPUTFORMAT output_format_classname )?
- *   LOCATION location_uri
- *   ( TBLPROPERTIES tbl_properties )?
- *   ( AS select_stmt )
- */
-SqlNode SqlCreateTable() :
-{
-    SqlParserPos pos;
-    SqlIdentifier tblName;
-    SqlNodeList fieldList;
-    SqlNode location;
-    SqlNode parallelism = null;
-    SqlNode input_format_class_name = null, output_format_class_name = null;
-    SqlNode tbl_properties = null;
-    SqlNode select = null;
-}
-{
-    <CREATE> { pos = getPos(); }
-    <EXTERNAL> <TABLE>
-    tblName = CompoundIdentifier()
-    fieldList = ColumnDefinitionList()
-    [
-      <STORED> <AS>
-      <INPUTFORMAT> input_format_class_name = StringLiteral()
-      <OUTPUTFORMAT> output_format_class_name = StringLiteral()
-    ]
-    <LOCATION>
-    location = StringLiteral()
-    [ <PARALLELISM> parallelism = UnsignedNumericLiteral() ]
-    [ <TBLPROPERTIES> tbl_properties = StringLiteral() ]
-    [ <AS> select = OrderedQueryOrExpr(ExprContext.ACCEPT_QUERY) ] {
-        return new SqlCreateTable(pos, tblName, fieldList,
-        input_format_class_name, output_format_class_name, location,
-        parallelism, tbl_properties, select);
-    }
-}
-
-/**
- * CREATE FUNCTION functionname AS 'classname'
- */
-SqlNode SqlCreateFunction() :
-{
-    SqlParserPos pos;
-    SqlIdentifier functionName;
-    SqlNode className;
-    SqlNode jarName = null;
-}
-{
-    <CREATE> { pos = getPos(); }
-    <FUNCTION>
-        functionName = CompoundIdentifier()
-    <AS>
-        className = StringLiteral()
-    [
-      <USING> <JAR>
-      jarName = StringLiteral()
-    ]
-    {
-      return new SqlCreateFunction(pos, functionName, className, jarName);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/AbstractTridentProcessor.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/AbstractTridentProcessor.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/AbstractTridentProcessor.java
deleted file mode 100644
index 6af71d4..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/AbstractTridentProcessor.java
+++ /dev/null
@@ -1,58 +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.sql;
-
-import org.apache.calcite.DataContext;
-import org.apache.storm.sql.javac.CompilingClassLoader;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.trident.Stream;
-import org.apache.storm.trident.TridentTopology;
-
-import java.util.List;
-import java.util.Map;
-
-public abstract class AbstractTridentProcessor {
-  protected Stream outputStream;
-  protected DataContext dataContext;
-  protected List<CompilingClassLoader> classLoaders;
-  /**
-   * @return the output stream of the SQL
-   */
-  public Stream outputStream() {
-    return outputStream;
-  }
-
-  /**
-   * Construct the trident topology based on the SQL.
-   */
-  public abstract TridentTopology build();
-
-  /**
-   * @return DataContext instance which is used with execution of query
-   */
-  public DataContext getDataContext() {
-    return dataContext;
-  }
-
-  /**
-   * @return Classloaders to compile. They're all chaining so the last classloader can access all classes.
-   */
-  public List<CompilingClassLoader> getClassLoaders() {
-    return classLoaders;
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSql.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSql.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSql.java
deleted file mode 100644
index 5dec4af..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSql.java
+++ /dev/null
@@ -1,59 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql;
-
-import org.apache.storm.StormSubmitter;
-import org.apache.storm.generated.SubmitOptions;
-import org.apache.storm.sql.runtime.ChannelHandler;
-
-import java.util.Map;
-
-/**
- * The StormSql class provides standalone, interactive interfaces to execute
- * SQL statements over streaming data.
- * <p>
- * The StormSql class is stateless. The user needs to submit the data
- * definition language (DDL) statements and the query statements in the same
- * batch.
- */
-public abstract class StormSql {
-  /**
-   * Execute the SQL statements in stand-alone mode. The user can retrieve the result by passing in an instance
-   * of {@see ChannelHandler}.
-   */
-  public abstract void execute(Iterable<String> statements,
-                               ChannelHandler handler) throws Exception;
-
-  /**
-   * Submit the SQL statements to Nimbus and run it as a topology.
-   */
-  public abstract void submit(
-      String name, Iterable<String> statements, Map<String, ?> stormConf, SubmitOptions opts,
-      StormSubmitter.ProgressListener progressListener, String asUser)
-      throws Exception;
-
-  /**
-   * Print out query plan for each query.
-   */
-  public abstract void explain(Iterable<String> statements) throws Exception;
-
-  public static StormSql construct() {
-    return new StormSqlImpl();
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlImpl.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlImpl.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlImpl.java
deleted file mode 100644
index 007daa7..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlImpl.java
+++ /dev/null
@@ -1,290 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql;
-
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.calcite.prepare.CalciteCatalogReader;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.calcite.schema.Function;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.schema.Table;
-import org.apache.calcite.schema.impl.AggregateFunctionImpl;
-import org.apache.calcite.schema.impl.ScalarFunctionImpl;
-import org.apache.calcite.sql.SqlExplainLevel;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlOperatorTable;
-import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
-import org.apache.calcite.tools.FrameworkConfig;
-import org.apache.calcite.tools.Frameworks;
-import org.apache.calcite.tools.Planner;
-import org.apache.storm.StormSubmitter;
-import org.apache.storm.generated.SubmitOptions;
-import org.apache.storm.sql.compiler.StormSqlTypeFactoryImpl;
-import org.apache.storm.sql.compiler.backends.standalone.PlanCompiler;
-import org.apache.storm.sql.javac.CompilingClassLoader;
-import org.apache.storm.sql.parser.ColumnConstraint;
-import org.apache.storm.sql.parser.ColumnDefinition;
-import org.apache.storm.sql.parser.SqlCreateFunction;
-import org.apache.storm.sql.parser.SqlCreateTable;
-import org.apache.storm.sql.parser.StormParser;
-import org.apache.storm.sql.planner.StormRelUtils;
-import org.apache.storm.sql.planner.trident.QueryPlanner;
-import org.apache.storm.sql.runtime.AbstractValuesProcessor;
-import org.apache.storm.sql.runtime.ChannelHandler;
-import org.apache.storm.sql.runtime.DataSource;
-import org.apache.storm.sql.runtime.DataSourcesRegistry;
-import org.apache.storm.sql.runtime.FieldInfo;
-import org.apache.storm.sql.runtime.ISqlTridentDataSource;
-import org.apache.storm.trident.TridentTopology;
-
-import java.io.BufferedOutputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.lang.reflect.Method;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.jar.Attributes;
-import java.util.jar.JarOutputStream;
-import java.util.jar.Manifest;
-import java.util.zip.ZipEntry;
-
-import static org.apache.storm.sql.compiler.CompilerUtil.TableBuilderInfo;
-
-class StormSqlImpl extends StormSql {
-  private final JavaTypeFactory typeFactory = new StormSqlTypeFactoryImpl(
-      RelDataTypeSystem.DEFAULT);
-  private final SchemaPlus schema = Frameworks.createRootSchema(true);
-  private boolean hasUdf = false;
-
-  @Override
-  public void execute(
-      Iterable<String> statements, ChannelHandler result)
-      throws Exception {
-    Map<String, DataSource> dataSources = new HashMap<>();
-    for (String sql : statements) {
-      StormParser parser = new StormParser(sql);
-      SqlNode node = parser.impl().parseSqlStmtEof();
-      if (node instanceof SqlCreateTable) {
-        handleCreateTable((SqlCreateTable) node, dataSources);
-      } else if (node instanceof SqlCreateFunction) {
-        handleCreateFunction((SqlCreateFunction) node);
-      } else {
-        FrameworkConfig config = buildFrameWorkConfig();
-        Planner planner = Frameworks.getPlanner(config);
-        SqlNode parse = planner.parse(sql);
-        SqlNode validate = planner.validate(parse);
-        RelNode tree = planner.convert(validate);
-        PlanCompiler compiler = new PlanCompiler(typeFactory);
-        AbstractValuesProcessor proc = compiler.compile(tree);
-        proc.initialize(dataSources, result);
-      }
-    }
-  }
-
-  @Override
-  public void submit(
-      String name, Iterable<String> statements, Map<String, ?> stormConf, SubmitOptions opts,
-      StormSubmitter.ProgressListener progressListener, String asUser)
-      throws Exception {
-    Map<String, ISqlTridentDataSource> dataSources = new HashMap<>();
-    for (String sql : statements) {
-      StormParser parser = new StormParser(sql);
-      SqlNode node = parser.impl().parseSqlStmtEof();
-      if (node instanceof SqlCreateTable) {
-        handleCreateTableForTrident((SqlCreateTable) node, dataSources);
-      } else if (node instanceof SqlCreateFunction) {
-        handleCreateFunction((SqlCreateFunction) node);
-      }  else {
-        QueryPlanner planner = new QueryPlanner(schema);
-        AbstractTridentProcessor processor = planner.compile(dataSources, sql);
-        TridentTopology topo = processor.build();
-
-        Path jarPath = null;
-        try {
-          // QueryPlanner on Trident mode configures the topology with compiled classes,
-          // so we need to add new classes into topology jar
-          // Topology will be serialized and sent to Nimbus, and deserialized and executed in workers.
-
-          jarPath = Files.createTempFile("storm-sql", ".jar");
-          System.setProperty("storm.jar", jarPath.toString());
-          packageTopology(jarPath, processor);
-          StormSubmitter.submitTopologyAs(name, stormConf, topo.build(), opts, progressListener, asUser);
-        } finally {
-          if (jarPath != null) {
-            Files.delete(jarPath);
-          }
-        }
-      }
-    }
-  }
-
-  @Override
-  public void explain(Iterable<String> statements) throws Exception {
-    Map<String, ISqlTridentDataSource> dataSources = new HashMap<>();
-    for (String sql : statements) {
-      StormParser parser = new StormParser(sql);
-      SqlNode node = parser.impl().parseSqlStmtEof();
-
-      System.out.println("===========================================================");
-      System.out.println("query>");
-      System.out.println(sql);
-      System.out.println("-----------------------------------------------------------");
-
-      if (node instanceof SqlCreateTable) {
-        handleCreateTableForTrident((SqlCreateTable) node, dataSources);
-        System.out.println("No plan presented on DDL");
-      } else if (node instanceof SqlCreateFunction) {
-        handleCreateFunction((SqlCreateFunction) node);
-        System.out.println("No plan presented on DDL");
-      } else {
-        FrameworkConfig config = buildFrameWorkConfig();
-        Planner planner = Frameworks.getPlanner(config);
-        SqlNode parse = planner.parse(sql);
-        SqlNode validate = planner.validate(parse);
-        RelNode tree = planner.convert(validate);
-
-        String plan = StormRelUtils.explain(tree, SqlExplainLevel.ALL_ATTRIBUTES);
-        System.out.println("plan>");
-        System.out.println(plan);
-      }
-
-      System.out.println("===========================================================");
-    }
-  }
-
-  private void packageTopology(Path jar, AbstractTridentProcessor processor) throws IOException {
-    Manifest manifest = new Manifest();
-    Attributes attr = manifest.getMainAttributes();
-    attr.put(Attributes.Name.MANIFEST_VERSION, "1.0");
-    attr.put(Attributes.Name.MAIN_CLASS, processor.getClass().getCanonicalName());
-    try (JarOutputStream out = new JarOutputStream(
-            new BufferedOutputStream(new FileOutputStream(jar.toFile())), manifest)) {
-      List<CompilingClassLoader> classLoaders = processor.getClassLoaders();
-      if (classLoaders != null && !classLoaders.isEmpty()) {
-        for (CompilingClassLoader classLoader : classLoaders) {
-          for (Map.Entry<String, ByteArrayOutputStream> e : classLoader.getClasses().entrySet()) {
-            out.putNextEntry(new ZipEntry(e.getKey().replace(".", "/") + ".class"));
-            out.write(e.getValue().toByteArray());
-            out.closeEntry();
-          }
-        }
-      }
-    }
-  }
-
-  private void handleCreateTable(
-      SqlCreateTable n, Map<String, DataSource> dataSources) {
-    List<FieldInfo> fields = updateSchema(n);
-    DataSource ds = DataSourcesRegistry.construct(n.location(), n
-        .inputFormatClass(), n.outputFormatClass(), fields);
-    if (ds == null) {
-      throw new RuntimeException("Cannot construct data source for " + n
-          .tableName());
-    } else if (dataSources.containsKey(n.tableName())) {
-      throw new RuntimeException("Duplicated definition for table " + n
-          .tableName());
-    }
-    dataSources.put(n.tableName(), ds);
-  }
-
-  private void handleCreateFunction(SqlCreateFunction sqlCreateFunction) throws ClassNotFoundException {
-    if(sqlCreateFunction.jarName() != null) {
-      throw new UnsupportedOperationException("UDF 'USING JAR' not implemented");
-    }
-    Method method;
-    Function function;
-    if ((method=findMethod(sqlCreateFunction.className(), "evaluate")) != null) {
-      function = ScalarFunctionImpl.create(method);
-    } else if (findMethod(sqlCreateFunction.className(), "add") != null) {
-      function = AggregateFunctionImpl.create(Class.forName(sqlCreateFunction.className()));
-    } else {
-      throw new RuntimeException("Invalid scalar or aggregate function");
-    }
-    schema.add(sqlCreateFunction.functionName().toUpperCase(), function);
-    hasUdf = true;
-  }
-
-  private Method findMethod(String clazzName, String methodName) throws ClassNotFoundException {
-    Class<?> clazz = Class.forName(clazzName);
-    for (Method method : clazz.getMethods()) {
-      if (method.getName().equals(methodName)) {
-        return method;
-      }
-    }
-    return null;
-  }
-
-  private void handleCreateTableForTrident(
-      SqlCreateTable n, Map<String, ISqlTridentDataSource> dataSources) {
-    List<FieldInfo> fields = updateSchema(n);
-    ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(n.location(), n
-        .inputFormatClass(), n.outputFormatClass(), n.properties(), fields);
-    if (ds == null) {
-      throw new RuntimeException("Failed to find data source for " + n
-          .tableName() + " URI: " + n.location());
-    } else if (dataSources.containsKey(n.tableName())) {
-      throw new RuntimeException("Duplicated definition for table " + n
-          .tableName());
-    }
-    dataSources.put(n.tableName(), ds);
-  }
-
-  private List<FieldInfo> updateSchema(SqlCreateTable n) {
-    TableBuilderInfo builder = new TableBuilderInfo(typeFactory);
-    List<FieldInfo> fields = new ArrayList<>();
-    for (ColumnDefinition col : n.fieldList()) {
-      builder.field(col.name(), col.type(), col.constraint());
-      RelDataType dataType = col.type().deriveType(typeFactory);
-      Class<?> javaType = (Class<?>)typeFactory.getJavaClass(dataType);
-      ColumnConstraint constraint = col.constraint();
-      boolean isPrimary = constraint != null && constraint instanceof ColumnConstraint.PrimaryKey;
-      fields.add(new FieldInfo(col.name(), javaType, isPrimary));
-    }
-
-    if (n.parallelism() != null) {
-      builder.parallelismHint(n.parallelism());
-    }
-    Table table = builder.build();
-    schema.add(n.tableName(), table);
-    return fields;
-  }
-
-  private FrameworkConfig buildFrameWorkConfig() {
-    if (hasUdf) {
-      List<SqlOperatorTable> sqlOperatorTables = new ArrayList<>();
-      sqlOperatorTables.add(SqlStdOperatorTable.instance());
-      sqlOperatorTables.add(new CalciteCatalogReader(CalciteSchema.from(schema),
-                                                     false,
-                                                     Collections.<String>emptyList(), typeFactory));
-      return Frameworks.newConfigBuilder().defaultSchema(schema)
-              .operatorTable(new ChainedSqlOperatorTable(sqlOperatorTables)).build();
-    } else {
-      return Frameworks.newConfigBuilder().defaultSchema(schema).build();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlRunner.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlRunner.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlRunner.java
deleted file mode 100644
index 5618647..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlRunner.java
+++ /dev/null
@@ -1,85 +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.sql;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.CommandLineParser;
-import org.apache.commons.cli.DefaultParser;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Options;
-import org.apache.storm.generated.SubmitOptions;
-import org.apache.storm.generated.TopologyInitialStatus;
-import org.apache.storm.utils.Utils;
-
-import java.nio.charset.StandardCharsets;
-import java.nio.file.Files;
-import java.nio.file.Paths;
-import java.util.List;
-import java.util.Map;
-
-public class StormSqlRunner {
-    private static final String OPTION_SQL_FILE_SHORT = "f";
-    private static final String OPTION_SQL_FILE_LONG = "file";
-    private static final String OPTION_SQL_TOPOLOGY_NAME_SHORT = "t";
-    private static final String OPTION_SQL_TOPOLOGY_NAME_LONG = "topology";
-    private static final String OPTION_SQL_EXPLAIN_SHORT = "e";
-    private static final String OPTION_SQL_EXPLAIN_LONG = "explain";
-
-    public static void main(String[] args) throws Exception {
-        Options options = buildOptions();
-        CommandLineParser parser = new DefaultParser();
-        CommandLine commandLine = parser.parse(options, args);
-
-        if (!commandLine.hasOption(OPTION_SQL_FILE_LONG)) {
-            printUsageAndExit(options, OPTION_SQL_FILE_LONG + " is required");
-        }
-
-        String filePath = commandLine.getOptionValue(OPTION_SQL_FILE_LONG);
-        List<String> stmts = Files.readAllLines(Paths.get(filePath), StandardCharsets.UTF_8);
-        StormSql sql = StormSql.construct();
-        @SuppressWarnings("unchecked")
-        Map<String, ?> conf = Utils.readStormConfig();
-
-        if (commandLine.hasOption(OPTION_SQL_EXPLAIN_LONG)) {
-            sql.explain(stmts);
-        } else if (commandLine.hasOption(OPTION_SQL_TOPOLOGY_NAME_LONG)) {
-            String topoName = commandLine.getOptionValue(OPTION_SQL_TOPOLOGY_NAME_LONG);
-            SubmitOptions submitOptions = new SubmitOptions(TopologyInitialStatus.ACTIVE);
-            sql.submit(topoName, stmts, conf, submitOptions, null, null);
-        } else {
-            printUsageAndExit(options, "Either " + OPTION_SQL_TOPOLOGY_NAME_LONG + " or " + OPTION_SQL_EXPLAIN_LONG +
-                    " must be presented");
-        }
-    }
-
-    private static void printUsageAndExit(Options options, String message) {
-        System.out.println(message);
-        HelpFormatter formatter = new HelpFormatter();
-        formatter.printHelp("storm-sql-runner ", options);
-        System.exit(1);
-    }
-
-    private static Options buildOptions() {
-        Options options = new Options();
-        options.addOption(OPTION_SQL_FILE_SHORT, OPTION_SQL_FILE_LONG, true, "REQUIRED SQL file which has sql statements");
-        options.addOption(OPTION_SQL_TOPOLOGY_NAME_SHORT, OPTION_SQL_TOPOLOGY_NAME_LONG, true, "Topology name to submit");
-        options.addOption(OPTION_SQL_EXPLAIN_SHORT, OPTION_SQL_EXPLAIN_LONG, false, "Activate explain mode (topology name will be ignored)");
-        return options;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/calcite/ParallelStreamableTable.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/calcite/ParallelStreamableTable.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/calcite/ParallelStreamableTable.java
deleted file mode 100644
index c6b584d..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/calcite/ParallelStreamableTable.java
+++ /dev/null
@@ -1,35 +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.sql.calcite;
-
-import org.apache.calcite.rel.stream.Delta;
-import org.apache.calcite.schema.StreamableTable;
-
-/**
- * Table that can be converted to a stream. This table also has its parallelism information.
- *
- * @see Delta
- */
-public interface ParallelStreamableTable extends StreamableTable {
-
-    /**
-     * Returns parallelism hint of this table. Returns null if don't know.
-     */
-    Integer parallelismHint();
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/CompilerUtil.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/CompilerUtil.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/CompilerUtil.java
deleted file mode 100644
index 2e237c0..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/CompilerUtil.java
+++ /dev/null
@@ -1,183 +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.sql.compiler;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableList;
-import org.apache.calcite.rel.RelCollations;
-import org.apache.calcite.rel.RelFieldCollation;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.schema.*;
-import org.apache.calcite.sql.SqlDataTypeSpec;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.sql.validate.SqlMonotonicity;
-import org.apache.calcite.util.ImmutableBitSet;
-import org.apache.calcite.util.Util;
-import org.apache.storm.sql.calcite.ParallelStreamableTable;
-import org.apache.storm.sql.parser.ColumnConstraint;
-
-import java.util.ArrayList;
-
-import static org.apache.calcite.rel.RelFieldCollation.Direction;
-import static org.apache.calcite.rel.RelFieldCollation.Direction.ASCENDING;
-import static org.apache.calcite.rel.RelFieldCollation.Direction.DESCENDING;
-import static org.apache.calcite.rel.RelFieldCollation.NullDirection;
-import static org.apache.calcite.sql.validate.SqlMonotonicity.INCREASING;
-
-public class CompilerUtil {
-  public static String escapeJavaString(String s, boolean nullMeansNull) {
-      if(s == null) {
-        return nullMeansNull ? "null" : "\"\"";
-      } else {
-        String s1 = Util.replace(s, "\\", "\\\\");
-        String s2 = Util.replace(s1, "\"", "\\\"");
-        String s3 = Util.replace(s2, "\n\r", "\\n");
-        String s4 = Util.replace(s3, "\n", "\\n");
-        String s5 = Util.replace(s4, "\r", "\\r");
-        return "\"" + s5 + "\"";
-      }
-  }
-
-  public static class TableBuilderInfo {
-    private final RelDataTypeFactory typeFactory;
-
-    public TableBuilderInfo(RelDataTypeFactory typeFactory) {
-      this.typeFactory = typeFactory;
-    }
-
-    private static class FieldType {
-      private final String name;
-      private final RelDataType relDataType;
-
-      private FieldType(String name, RelDataType relDataType) {
-        this.name = name;
-        this.relDataType = relDataType;
-      }
-
-    }
-
-    private final ArrayList<FieldType> fields = new ArrayList<>();
-    private final ArrayList<Object[]> rows = new ArrayList<>();
-    private int primaryKey = -1;
-    private Integer parallelismHint;
-    private SqlMonotonicity primaryKeyMonotonicity;
-    private Statistic stats;
-
-    public TableBuilderInfo field(String name, SqlTypeName type) {
-      return field(name, typeFactory.createSqlType(type));
-    }
-
-    public TableBuilderInfo field(String name, RelDataType type) {
-      fields.add(new FieldType(name, type));
-      return this;
-    }
-
-    public TableBuilderInfo field(String name, SqlDataTypeSpec type, ColumnConstraint constraint) {
-      RelDataType dataType = type.deriveType(typeFactory);
-      if (constraint instanceof ColumnConstraint.PrimaryKey) {
-        ColumnConstraint.PrimaryKey pk = (ColumnConstraint.PrimaryKey) constraint;
-        Preconditions.checkState(primaryKey == -1, "There are more than one primary key in the table");
-        primaryKey = fields.size();
-        primaryKeyMonotonicity = pk.monotonicity();
-      }
-      fields.add(new FieldType(name, dataType));
-      return this;
-    }
-
-    public TableBuilderInfo statistics(Statistic stats) {
-      this.stats = stats;
-      return this;
-    }
-
-    @VisibleForTesting
-    public TableBuilderInfo rows(Object[] data) {
-      rows.add(data);
-      return this;
-    }
-
-    public TableBuilderInfo parallelismHint(int parallelismHint) {
-      this.parallelismHint = parallelismHint;
-      return this;
-    }
-
-    public StreamableTable build() {
-      final Statistic stat = buildStatistic();
-      final Table tbl = new Table() {
-        @Override
-        public RelDataType getRowType(
-            RelDataTypeFactory relDataTypeFactory) {
-          RelDataTypeFactory.FieldInfoBuilder b = relDataTypeFactory.builder();
-          for (FieldType f : fields) {
-            b.add(f.name, f.relDataType);
-          }
-          return b.build();
-        }
-
-        @Override
-        public Statistic getStatistic() {
-          return stat != null ? stat : Statistics.of(rows.size(),
-                                                     ImmutableList.<ImmutableBitSet>of());
-        }
-
-        @Override
-        public Schema.TableType getJdbcTableType() {
-          return Schema.TableType.STREAM;
-        }
-      };
-
-      return new ParallelStreamableTable() {
-        @Override
-        public Integer parallelismHint() {
-          return parallelismHint;
-        }
-
-        @Override
-        public Table stream() {
-          return tbl;
-        }
-
-        @Override
-        public RelDataType getRowType(RelDataTypeFactory relDataTypeFactory) {
-          return tbl.getRowType(relDataTypeFactory);
-        }
-
-        @Override
-        public Statistic getStatistic() {
-          return tbl.getStatistic();
-        }
-
-        @Override
-        public Schema.TableType getJdbcTableType() {
-          return Schema.TableType.STREAM;
-        }
-      };
-    }
-
-    private Statistic buildStatistic() {
-      if (stats != null || primaryKey == -1) {
-        return stats;
-      }
-      Direction dir = primaryKeyMonotonicity == INCREASING ? ASCENDING : DESCENDING;
-      RelFieldCollation collation = new RelFieldCollation(primaryKey, dir, NullDirection.UNSPECIFIED);
-      return Statistics.of(fields.size(), ImmutableList.of(ImmutableBitSet.of(primaryKey)),
-          ImmutableList.of(RelCollations.of(collation)));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RexNodeToJavaCodeCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RexNodeToJavaCodeCompiler.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RexNodeToJavaCodeCompiler.java
deleted file mode 100644
index 5ac95e0..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RexNodeToJavaCodeCompiler.java
+++ /dev/null
@@ -1,231 +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.sql.compiler;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
-import org.apache.calcite.adapter.enumerable.JavaRowFormat;
-import org.apache.calcite.adapter.enumerable.PhysType;
-import org.apache.calcite.adapter.enumerable.PhysTypeImpl;
-import org.apache.calcite.adapter.enumerable.RexToLixTranslator;
-import org.apache.calcite.interpreter.Context;
-import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.calcite.linq4j.function.Function1;
-import org.apache.calcite.linq4j.tree.BlockBuilder;
-import org.apache.calcite.linq4j.tree.BlockStatement;
-import org.apache.calcite.linq4j.tree.ClassDeclaration;
-import org.apache.calcite.linq4j.tree.Expression;
-import org.apache.calcite.linq4j.tree.Expressions;
-import org.apache.calcite.linq4j.tree.MemberDeclaration;
-import org.apache.calcite.linq4j.tree.ParameterExpression;
-import org.apache.calcite.linq4j.tree.Types;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexProgram;
-import org.apache.calcite.rex.RexProgramBuilder;
-import org.apache.calcite.util.BuiltInMethod;
-import org.apache.calcite.util.Pair;
-import org.apache.storm.sql.runtime.calcite.ExecutableExpression;
-
-import java.lang.reflect.Constructor;
-import java.lang.reflect.Field;
-import java.lang.reflect.Method;
-import java.lang.reflect.Modifier;
-import java.lang.reflect.Type;
-import java.util.List;
-
-/**
- * Compiles a scalar expression ({@link org.apache.calcite.rex.RexNode}) to Java source code String.
- *
- * This code is inspired by JaninoRexCompiler in Calcite, but while it is returning {@link org.apache.calcite.interpreter.Scalar} which is executable,
- * we need to pass the source code to compile and serialize instance so that it can be executed on worker efficiently.
- */
-public class RexNodeToJavaCodeCompiler {
-  private final RexBuilder rexBuilder;
-
-  public RexNodeToJavaCodeCompiler(RexBuilder rexBuilder) {
-    this.rexBuilder = rexBuilder;
-  }
-
-  public BlockStatement compileToBlock(List<RexNode> nodes, RelDataType inputRowType) {
-    final RexProgramBuilder programBuilder =
-        new RexProgramBuilder(inputRowType, rexBuilder);
-    for (RexNode node : nodes) {
-      programBuilder.addProject(node, null);
-    }
-
-    return compileToBlock(programBuilder.getProgram());
-  }
-
-  public BlockStatement compileToBlock(final RexProgram program) {
-    final ParameterExpression context_ =
-            Expressions.parameter(Context.class, "context");
-    final ParameterExpression outputValues_ =
-            Expressions.parameter(Object[].class, "outputValues");
-
-    return compileToBlock(program, context_, outputValues_).toBlock();
-  }
-
-  public String compile(List<RexNode> nodes, RelDataType inputRowType, String className) {
-    final RexProgramBuilder programBuilder =
-            new RexProgramBuilder(inputRowType, rexBuilder);
-    for (RexNode node : nodes) {
-      programBuilder.addProject(node, null);
-    }
-
-    return compile(programBuilder.getProgram(), className);
-  }
-
-  public String compile(final RexProgram program, String className) {
-    final ParameterExpression context_ =
-            Expressions.parameter(Context.class, "context");
-    final ParameterExpression outputValues_ =
-            Expressions.parameter(Object[].class, "outputValues");
-
-    BlockBuilder builder = compileToBlock(program, context_, outputValues_);
-    return baz(context_, outputValues_, builder.toBlock(), className);
-  }
-
-  private BlockBuilder compileToBlock(final RexProgram program, ParameterExpression context_,
-                                        ParameterExpression outputValues_) {
-    RelDataType inputRowType = program.getInputRowType();
-    final BlockBuilder builder = new BlockBuilder();
-    final JavaTypeFactoryImpl javaTypeFactory =
-            new JavaTypeFactoryImpl(rexBuilder.getTypeFactory().getTypeSystem());
-
-    final RexToLixTranslator.InputGetter inputGetter =
-            new RexToLixTranslator.InputGetterImpl(
-                    ImmutableList.of(
-                            Pair.<Expression, PhysType>of(
-                                    Expressions.field(context_,
-                                            BuiltInMethod.CONTEXT_VALUES.field),
-                                    PhysTypeImpl.of(javaTypeFactory, inputRowType,
-                                            JavaRowFormat.ARRAY, false))));
-    final Function1<String, RexToLixTranslator.InputGetter> correlates =
-            new Function1<String, RexToLixTranslator.InputGetter>() {
-              public RexToLixTranslator.InputGetter apply(String a0) {
-                throw new UnsupportedOperationException();
-              }
-            };
-    final Expression root =
-            Expressions.field(context_, BuiltInMethod.CONTEXT_ROOT.field);
-    final List<Expression> list =
-            RexToLixTranslator.translateProjects(program, javaTypeFactory, builder,
-                    null, root, inputGetter, correlates);
-    for (int i = 0; i < list.size(); i++) {
-      builder.add(
-              Expressions.statement(
-                      Expressions.assign(
-                              Expressions.arrayIndex(outputValues_,
-                                      Expressions.constant(i)),
-                              list.get(i))));
-    }
-
-    return builder;
-  }
-
-  /** Given a method that implements {@link ExecutableExpression#execute(Context, Object[])},
-   * adds a bridge method that implements {@link ExecutableExpression#execute(Context)}, and
-   * compiles. */
-  static String baz(ParameterExpression context_,
-                    ParameterExpression outputValues_, BlockStatement block, String className) {
-    final List<MemberDeclaration> declarations = Lists.newArrayList();
-
-    // public void execute(Context, Object[] outputValues)
-    declarations.add(
-            Expressions.methodDecl(Modifier.PUBLIC, void.class,
-                    StormBuiltInMethod.EXPR_EXECUTE2.method.getName(),
-                    ImmutableList.of(context_, outputValues_), block));
-
-    // public Object execute(Context)
-    final BlockBuilder builder = new BlockBuilder();
-    final Expression values_ = builder.append("values",
-            Expressions.newArrayBounds(Object.class, 1,
-                    Expressions.constant(1)));
-    builder.add(
-            Expressions.statement(
-                    Expressions.call(
-                            Expressions.parameter(ExecutableExpression.class, "this"),
-                            StormBuiltInMethod.EXPR_EXECUTE2.method, context_, values_)));
-    builder.add(
-            Expressions.return_(null,
-                    Expressions.arrayIndex(values_, Expressions.constant(0))));
-    declarations.add(
-            Expressions.methodDecl(Modifier.PUBLIC, Object.class,
-                    StormBuiltInMethod.EXPR_EXECUTE1.method.getName(),
-                    ImmutableList.of(context_), builder.toBlock()));
-
-    final ClassDeclaration classDeclaration =
-            Expressions.classDecl(Modifier.PUBLIC, className, null,
-                    ImmutableList.<Type>of(ExecutableExpression.class), declarations);
-
-    return Expressions.toString(Lists.newArrayList(classDeclaration), "\n", false);
-  }
-
-  enum StormBuiltInMethod {
-    EXPR_EXECUTE1(ExecutableExpression.class, "execute", Context.class),
-    EXPR_EXECUTE2(ExecutableExpression.class, "execute", Context.class, Object[].class);
-
-    public final Method method;
-    public final Constructor constructor;
-    public final Field field;
-
-    public static final ImmutableMap<Method, BuiltInMethod> MAP;
-
-    static {
-      final ImmutableMap.Builder<Method, BuiltInMethod> builder =
-              ImmutableMap.builder();
-      for (BuiltInMethod value : BuiltInMethod.values()) {
-        if (value.method != null) {
-          builder.put(value.method, value);
-        }
-      }
-      MAP = builder.build();
-    }
-
-    private StormBuiltInMethod(Method method, Constructor constructor, Field field) {
-      this.method = method;
-      this.constructor = constructor;
-      this.field = field;
-    }
-
-    /**
-     * Defines a method.
-     */
-    StormBuiltInMethod(Class clazz, String methodName, Class... argumentTypes) {
-      this(Types.lookupMethod(clazz, methodName, argumentTypes), null, null);
-    }
-
-    /**
-     * Defines a constructor.
-     */
-    StormBuiltInMethod(Class clazz, Class... argumentTypes) {
-      this(null, Types.lookupConstructor(clazz, argumentTypes), null);
-    }
-
-    /**
-     * Defines a field.
-     */
-    StormBuiltInMethod(Class clazz, String fieldName, boolean dummy) {
-      this(null, null, Types.lookupField(clazz, fieldName));
-      assert dummy : "dummy value for method overloading must be true";
-    }
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/StormSqlTypeFactoryImpl.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/StormSqlTypeFactoryImpl.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/StormSqlTypeFactoryImpl.java
deleted file mode 100644
index 21ca063..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/StormSqlTypeFactoryImpl.java
+++ /dev/null
@@ -1,51 +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.sql.compiler;
-
-import com.google.common.base.Function;
-import com.google.common.collect.Lists;
-import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.calcite.rel.type.RelRecordType;
-import org.apache.calcite.sql.type.JavaToSqlTypeConversionRules;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-public class StormSqlTypeFactoryImpl extends JavaTypeFactoryImpl {
-
-    public StormSqlTypeFactoryImpl() {
-    }
-
-    public StormSqlTypeFactoryImpl(RelDataTypeSystem typeSystem) {
-        super(typeSystem);
-    }
-
-    @Override
-    public RelDataType toSql(RelDataType type) {
-        if (type instanceof JavaType) {
-            JavaType javaType = (JavaType) type;
-            SqlTypeName sqlTypeName = JavaToSqlTypeConversionRules.instance().lookup(javaType.getJavaClass());
-            if (sqlTypeName == null) {
-                sqlTypeName = SqlTypeName.ANY;
-            }
-            return createTypeWithNullability(createSqlType(sqlTypeName), type.isNullable());
-        }
-        return super.toSql(type);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/BuiltinAggregateFunctions.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/BuiltinAggregateFunctions.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/BuiltinAggregateFunctions.java
deleted file mode 100644
index 9dc4ba8..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/BuiltinAggregateFunctions.java
+++ /dev/null
@@ -1,238 +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.sql.compiler.backends.standalone;
-
-import com.google.common.collect.ImmutableList;
-import org.apache.storm.tuple.Values;
-
-import java.lang.reflect.Type;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Built-in implementations for some of the standard aggregation operations.
- * Aggregations can be implemented as a class with the following methods viz. init, add and result.
- * The class could contain only static methods, only non-static methods or be generic.
- */
-public class BuiltinAggregateFunctions {
-    // binds the type information and the class implementing the aggregation
-    public static class TypeClass {
-        public static class GenericType {
-        }
-
-        public final Type ty;
-        public final Class<?> clazz;
-
-        private TypeClass(Type ty, Class<?> clazz) {
-            this.ty = ty;
-            this.clazz = clazz;
-        }
-
-        static TypeClass of(Type ty, Class<?> clazz) {
-            return new TypeClass(ty, clazz);
-        }
-    }
-
-    static final Map<String, List<TypeClass>> TABLE = new HashMap<>();
-
-    public static class ByteSum {
-        public static Byte init() {
-            return 0;
-        }
-
-        public static Byte add(Byte accumulator, Byte val) {
-            return (byte) (accumulator + val);
-        }
-
-        public static Byte result(Byte accumulator) {
-            return accumulator;
-        }
-    }
-
-    public static class ShortSum {
-        public static Short init() {
-            return 0;
-        }
-
-        public static Short add(Short accumulator, Short val) {
-            return (short) (accumulator + val);
-        }
-
-        public static Short result(Short accumulator) {
-            return accumulator;
-        }
-    }
-
-    public static class IntSum {
-        public static Integer init() {
-            return 0;
-        }
-
-        public static Integer add(Integer accumulator, Integer val) {
-            return accumulator + val;
-        }
-
-        public static Integer result(Integer accumulator) {
-            return accumulator;
-        }
-    }
-
-    public static class LongSum {
-        public static Long init() {
-            return 0L;
-        }
-
-        public static Long add(Long accumulator, Long val) {
-            return accumulator + val;
-        }
-
-        public static Long result(Long accumulator) {
-            return accumulator;
-        }
-    }
-
-    public static class FloatSum {
-        public static Float init() {
-            return 0.0f;
-        }
-
-        public static Float add(Float accumulator, Float val) {
-            return accumulator + val;
-        }
-
-        public static Float result(Float accumulator) {
-            return accumulator;
-        }
-    }
-
-    public static class DoubleSum {
-        public static Double init() {
-            return 0.0;
-        }
-
-        public static Double add(Double accumulator, Double val) {
-            return accumulator + val;
-        }
-
-        public static Double result(Double accumulator) {
-            return accumulator;
-        }
-    }
-
-    public static class Max<T extends Comparable<T>> {
-        public T init() {
-            return null;
-        }
-
-        public T add(T accumulator, T val) {
-            return (accumulator == null || accumulator.compareTo(val) < 0) ? val : accumulator;
-        }
-
-        public T result(T accumulator) {
-            return accumulator;
-        }
-    }
-
-    public static class Min<T extends Comparable<T>> {
-        public T init() {
-            return null;
-        }
-
-        public T add(T accumulator, T val) {
-            return (accumulator == null || accumulator.compareTo(val) > 0) ? val : accumulator;
-        }
-
-        public T result(T accumulator) {
-            return accumulator;
-        }
-    }
-
-    public static class IntAvg {
-        private int count;
-
-        public Integer init() {
-            return 0;
-        }
-
-        public Integer add(Integer accumulator, Integer val) {
-            ++count;
-            return accumulator + val;
-        }
-
-        public Integer result(Integer accumulator) {
-            Integer result = accumulator / count;
-            count = 0;
-            return result;
-        }
-    }
-
-    public static class DoubleAvg {
-        private int count;
-
-        public Double init() {
-            return 0.0;
-        }
-
-        public Double add(Double accumulator, Double val) {
-            ++count;
-            return accumulator + val;
-        }
-
-        public Double result(Double accumulator) {
-            Double result = accumulator / count;
-            count = 0;
-            return result;
-        }
-    }
-
-    public static class Count {
-        public static Long init() {
-            return 0L;
-        }
-
-        public static Long add(Long accumulator, Values vals) {
-            for (Object val : vals) {
-                if (val == null) {
-                    return accumulator;
-                }
-            }
-            return accumulator + 1;
-        }
-
-        public static Long result(Long accumulator) {
-            return accumulator;
-        }
-    }
-
-    static {
-        TABLE.put("SUM", ImmutableList.of(
-                TypeClass.of(float.class, FloatSum.class),
-                TypeClass.of(double.class, DoubleSum.class),
-                TypeClass.of(byte.class, ByteSum.class),
-                TypeClass.of(short.class, ShortSum.class),
-                TypeClass.of(long.class, LongSum.class),
-                TypeClass.of(int.class, IntSum.class)));
-        TABLE.put("AVG", ImmutableList.of(
-                TypeClass.of(double.class, DoubleAvg.class),
-                TypeClass.of(int.class, IntAvg.class)));
-        TABLE.put("COUNT", ImmutableList.of(TypeClass.of(long.class, Count.class)));
-        TABLE.put("MAX", ImmutableList.of(TypeClass.of(TypeClass.GenericType.class, Max.class)));
-        TABLE.put("MIN", ImmutableList.of(TypeClass.of(TypeClass.GenericType.class, Min.class)));
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PlanCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PlanCompiler.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PlanCompiler.java
deleted file mode 100644
index 01546ed..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PlanCompiler.java
+++ /dev/null
@@ -1,139 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.compiler.backends.standalone;
-
-import com.google.common.base.Joiner;
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.TableScan;
-import org.apache.storm.sql.compiler.CompilerUtil;
-import org.apache.storm.sql.javac.CompilingClassLoader;
-import org.apache.storm.sql.runtime.AbstractValuesProcessor;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.util.HashSet;
-import java.util.Set;
-
-public class PlanCompiler {
-  private static final Logger LOG = LoggerFactory.getLogger(PlanCompiler.class);
-
-  private static final Joiner NEW_LINE_JOINER = Joiner.on("\n");
-  private static final String PACKAGE_NAME = "org.apache.storm.sql.generated";
-  private static final String PROLOGUE = NEW_LINE_JOINER.join(
-      "// GENERATED CODE", "package " + PACKAGE_NAME + ";", "",
-      "import java.util.Iterator;", "import java.util.Map;", "import java.util.HashMap;",
-      "import java.util.List;", "import java.util.ArrayList;",
-      "import java.util.LinkedHashMap;",
-      "import org.apache.storm.tuple.Values;",
-      "import org.apache.storm.sql.runtime.AbstractChannelHandler;",
-      "import org.apache.storm.sql.runtime.Channels;",
-      "import org.apache.storm.sql.runtime.ChannelContext;",
-      "import org.apache.storm.sql.runtime.ChannelHandler;",
-      "import org.apache.storm.sql.runtime.DataSource;",
-      "import org.apache.storm.sql.runtime.AbstractValuesProcessor;",
-      "import com.google.common.collect.ArrayListMultimap;",
-      "import com.google.common.collect.Multimap;",
-      "import org.apache.calcite.interpreter.Context;",
-      "import org.apache.calcite.interpreter.StormContext;",
-      "import org.apache.calcite.DataContext;",
-      "import org.apache.storm.sql.runtime.calcite.StormDataContext;",
-      "public final class Processor extends AbstractValuesProcessor {",
-      "  public final static DataContext dataContext = new StormDataContext();",
-      "");
-  private static final String INITIALIZER_PROLOGUE = NEW_LINE_JOINER.join(
-      "  @Override",
-      "  public void initialize(Map<String, DataSource> data,",
-      "                         ChannelHandler result) {",
-      "    ChannelContext r = Channels.chain(Channels.voidContext(), result);",
-      ""
-  );
-
-  private final JavaTypeFactory typeFactory;
-
-  public PlanCompiler(JavaTypeFactory typeFactory) {
-    this.typeFactory = typeFactory;
-  }
-
-  private String generateJavaSource(RelNode root) throws Exception {
-    StringWriter sw = new StringWriter();
-    try (PrintWriter pw = new PrintWriter(sw)) {
-      RelNodeCompiler compiler = new RelNodeCompiler(pw, typeFactory);
-      printPrologue(pw);
-      compiler.traverse(root);
-      printMain(pw, root);
-      printEpilogue(pw);
-    }
-    return sw.toString();
-  }
-
-  private void printMain(PrintWriter pw, RelNode root) {
-    Set<TableScan> tables = new HashSet<>();
-    pw.print(INITIALIZER_PROLOGUE);
-    chainOperators(pw, root, tables);
-    for (TableScan n : tables) {
-      String escaped = CompilerUtil.escapeJavaString(
-          Joiner.on('.').join(n.getTable().getQualifiedName()), true);
-      String r = NEW_LINE_JOINER.join(
-          "    if (!data.containsKey(%1$s))",
-          "      throw new RuntimeException(\"Cannot find table \" + %1$s);",
-          "  data.get(%1$s).open(CTX_%2$d);",
-          "");
-      pw.print(String.format(r, escaped, n.getId()));
-    }
-    pw.print("  }\n");
-  }
-
-  private void chainOperators(PrintWriter pw, RelNode root, Set<TableScan> tables) {
-    doChainOperators(pw, root, tables, "r");
-  }
-
-  private void doChainOperators(PrintWriter pw, RelNode node, Set<TableScan> tables, String parentCtx) {
-    pw.print(
-            String.format("    ChannelContext CTX_%d = Channels.chain(%2$s, %3$s);\n",
-                          node.getId(), parentCtx, RelNodeCompiler.getStageName(node)));
-    String currentCtx = String.format("CTX_%d", node.getId());
-    if (node instanceof TableScan) {
-      tables.add((TableScan) node);
-    }
-    for (RelNode i : node.getInputs()) {
-      doChainOperators(pw, i, tables, currentCtx);
-    }
-  }
-
-  public AbstractValuesProcessor compile(RelNode plan) throws Exception {
-    String javaCode = generateJavaSource(plan);
-    LOG.debug("Compiling... source code {}", javaCode);
-    ClassLoader cl = new CompilingClassLoader(getClass().getClassLoader(),
-                                              PACKAGE_NAME + ".Processor",
-                                              javaCode, null);
-    return (AbstractValuesProcessor) cl.loadClass(
-        PACKAGE_NAME + ".Processor").newInstance();
-  }
-
-  private static void printEpilogue(
-      PrintWriter pw) throws Exception {
-    pw.print("}\n");
-  }
-
-  private static void printPrologue(PrintWriter pw) {
-    pw.append(PROLOGUE);
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/e9d78338/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PostOrderRelNodeVisitor.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PostOrderRelNodeVisitor.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PostOrderRelNodeVisitor.java
deleted file mode 100644
index afed8a9..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PostOrderRelNodeVisitor.java
+++ /dev/null
@@ -1,132 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.sql.compiler.backends.standalone;
-
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.*;
-import org.apache.calcite.rel.stream.Delta;
-
-import java.util.ArrayList;
-import java.util.List;
-
-public abstract class PostOrderRelNodeVisitor<T> {
-  public final T traverse(RelNode n) throws Exception {
-    List<T> inputStreams = new ArrayList<>();
-    for (RelNode input : n.getInputs()) {
-      inputStreams.add(traverse(input));
-    }
-
-    if (n instanceof Aggregate) {
-      return visitAggregate((Aggregate) n, inputStreams);
-    } else if (n instanceof Calc) {
-      return visitCalc((Calc) n, inputStreams);
-    } else if (n instanceof Collect) {
-      return visitCollect((Collect) n, inputStreams);
-    } else if (n instanceof Correlate) {
-      return visitCorrelate((Correlate) n, inputStreams);
-    } else if (n instanceof Delta) {
-      return visitDelta((Delta) n, inputStreams);
-    } else if (n instanceof Exchange) {
-      return visitExchange((Exchange) n, inputStreams);
-    } else if (n instanceof Project) {
-      return visitProject((Project) n, inputStreams);
-    } else if (n instanceof Filter) {
-      return visitFilter((Filter) n, inputStreams);
-    } else if (n instanceof Sample) {
-      return visitSample((Sample) n, inputStreams);
-    } else if (n instanceof Sort) {
-      return visitSort((Sort) n, inputStreams);
-    } else if (n instanceof TableModify) {
-      return visitTableModify((TableModify) n, inputStreams);
-    } else if (n instanceof TableScan) {
-      return visitTableScan((TableScan) n, inputStreams);
-    } else if (n instanceof Uncollect) {
-      return visitUncollect((Uncollect) n, inputStreams);
-    } else if (n instanceof Window) {
-      return visitWindow((Window) n, inputStreams);
-    } else if (n instanceof Join) {
-      return visitJoin((Join) n, inputStreams);
-    } else {
-      return defaultValue(n, inputStreams);
-    }
-  }
-
-  public T visitAggregate(Aggregate aggregate, List<T> inputStreams) throws Exception {
-    return defaultValue(aggregate, inputStreams);
-  }
-
-  public T visitCalc(Calc calc, List<T> inputStreams) throws Exception {
-    return defaultValue(calc, inputStreams);
-  }
-
-  public T visitCollect(Collect collect, List<T> inputStreams) throws Exception {
-    return defaultValue(collect, inputStreams);
-  }
-
-  public T visitCorrelate(Correlate correlate, List<T> inputStreams) throws Exception {
-    return defaultValue(correlate, inputStreams);
-  }
-
-  public T visitDelta(Delta delta, List<T> inputStreams) throws Exception {
-    return defaultValue(delta, inputStreams);
-  }
-
-  public T visitExchange(Exchange exchange, List<T> inputStreams) throws Exception {
-    return defaultValue(exchange, inputStreams);
-  }
-
-  public T visitProject(Project project, List<T> inputStreams) throws Exception {
-    return defaultValue(project, inputStreams);
-  }
-
-  public T visitFilter(Filter filter, List<T> inputStreams) throws Exception {
-    return defaultValue(filter, inputStreams);
-  }
-
-  public T visitSample(Sample sample, List<T> inputStreams) throws Exception {
-    return defaultValue(sample, inputStreams);
-  }
-
-  public T visitSort(Sort sort, List<T> inputStreams) throws Exception {
-    return defaultValue(sort, inputStreams);
-  }
-
-  public T visitTableModify(TableModify modify, List<T> inputStreams) throws Exception {
-    return defaultValue(modify, inputStreams);
-  }
-
-  public T visitTableScan(TableScan scan, List<T> inputStreams) throws Exception {
-    return defaultValue(scan, inputStreams);
-  }
-
-  public T visitUncollect(Uncollect uncollect, List<T> inputStreams) throws Exception {
-    return defaultValue(uncollect, inputStreams);
-  }
-
-  public T visitWindow(Window window, List<T> inputStreams) throws Exception {
-    return defaultValue(window, inputStreams);
-  }
-
-  public T visitJoin(Join join, List<T> inputStreams) throws Exception {
-    return defaultValue(join, inputStreams);
-  }
-
-  public T defaultValue(RelNode n, List<T> inputStreams) {
-    return null;
-  }
-}