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

[01/20] storm git commit: [StormSQL] Implement compiler for expressions.

Repository: storm
Updated Branches:
  refs/heads/master 2b7a75839 -> 39163bfce


[StormSQL] Implement compiler for expressions.


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

Branch: refs/heads/master
Commit: 4e2fe47c3b322552f7dc8f2861ef3e496054006b
Parents: b6fa601
Author: Haohui Mai <wh...@apache.org>
Authored: Thu Oct 22 16:42:24 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Fri Dec 4 11:09:01 2015 -0800

----------------------------------------------------------------------
 .../apache/storm/sql/compiler/CompilerUtil.java |  35 +++
 .../apache/storm/sql/compiler/ExprCompiler.java | 216 +++++++++++++++++++
 .../storm/sql/compiler/TestExprCompiler.java    |  90 ++++++++
 .../apache/storm/sql/compiler/TestUtils.java    |  33 +++
 4 files changed, 374 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/4e2fe47c/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
new file mode 100644
index 0000000..5e7453a
--- /dev/null
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/CompilerUtil.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.compiler;
+
+import org.apache.calcite.util.Util;
+
+class CompilerUtil {
+  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 + "\"";
+      }
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4e2fe47c/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java
new file mode 100644
index 0000000..b9d8f88
--- /dev/null
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.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.sql.compiler;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexCorrelVariable;
+import org.apache.calcite.rex.RexDynamicParam;
+import org.apache.calcite.rex.RexFieldAccess;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexOver;
+import org.apache.calcite.rex.RexRangeRef;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.util.NlsString;
+import org.apache.calcite.util.Util;
+
+import java.io.PrintWriter;
+import java.lang.reflect.Type;
+import java.math.BigDecimal;
+import java.util.AbstractMap;
+import java.util.IdentityHashMap;
+import java.util.Map;
+
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.DIVIDE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.DIVIDE_INTEGER;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.GREATER_THAN;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.GREATER_THAN_OR_EQUAL;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LESS_THAN;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LESS_THAN_OR_EQUAL;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MINUS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MULTIPLY;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.PLUS;
+
+/**
+ * Compile RexNode on top of the Tuple abstraction.
+ */
+class ExprCompiler implements RexVisitor<String> {
+  private final PrintWriter pw;
+  private final Map<RexNode, String> expr = new IdentityHashMap<>();
+  private final JavaTypeFactory typeFactory;
+  private static final ImpTable IMP_TABLE = new ImpTable();
+
+  ExprCompiler(PrintWriter pw, JavaTypeFactory typeFactory) {
+    this.pw = pw;
+    this.typeFactory = typeFactory;
+  }
+
+  @Override
+  public String visitInputRef(RexInputRef rexInputRef) {
+    if (expr.containsKey(rexInputRef)) {
+      return expr.get(rexInputRef);
+    }
+    String name = printExpr(rexInputRef, String.format("_data.get(%d)",
+                            rexInputRef.getIndex()));
+    expr.put(rexInputRef, name);
+    return name;
+  }
+
+  @Override
+  public String visitLocalRef(RexLocalRef rexLocalRef) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public String visitLiteral(RexLiteral rexLiteral) {
+    Object v = rexLiteral.getValue();
+    RelDataType ty = rexLiteral.getType();
+    switch(rexLiteral.getTypeName()) {
+      case BOOLEAN:
+        return v.toString();
+      case CHAR:
+        return CompilerUtil.escapeJavaString(((NlsString) v).getValue(), true);
+      case NULL:
+        return "null";
+      case DOUBLE:
+      case BIGINT:
+      case DECIMAL:
+        switch (ty.getSqlTypeName()) {
+          case TINYINT:
+          case SMALLINT:
+          case INTEGER:
+            return Long.toString(((BigDecimal) v).longValueExact());
+          case BIGINT:
+            return Long.toString(((BigDecimal)v).longValueExact()) + 'L';
+          case DECIMAL:
+          case FLOAT:
+          case REAL:
+          case DOUBLE:
+            return Util.toScientificNotation((BigDecimal) v);
+        }
+        break;
+      default:
+        throw new UnsupportedOperationException();
+    }
+    return null;
+  }
+
+  @Override
+  public String visitCall(RexCall rexCall) {
+    return IMP_TABLE.compile(this, rexCall);
+  }
+
+  @Override
+  public String visitOver(RexOver rexOver) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public String visitCorrelVariable(
+      RexCorrelVariable rexCorrelVariable) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public String visitDynamicParam(
+      RexDynamicParam rexDynamicParam) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public String visitRangeRef(RexRangeRef rexRangeRef) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public String visitFieldAccess(
+      RexFieldAccess rexFieldAccess) {
+    throw new UnsupportedOperationException();
+  }
+
+  private String printExpr(RexNode node, String definition) {
+    String name = "t" + expr.size();
+    Type ty = typeFactory.getJavaClass(node.getType());
+    String typeName = ((Class<?>)ty).getCanonicalName();
+    pw.append(
+        String.format("%s %s = (%s)(%s);\n", typeName, name, typeName, definition));
+    return name;
+  }
+
+  private interface CallExprPrinter {
+    String translate(ExprCompiler compiler, RexCall call);
+  }
+
+  /**
+   * Inspired by Calcite's RexImpTable, the ImpTable class maps the operators
+   * to their corresponding implementation that generates the expressions in
+   * the format of Java source code.
+   */
+  private static class ImpTable {
+    private final Map<SqlOperator, CallExprPrinter> translators;
+
+    private ImpTable() {
+      ImmutableMap.Builder<SqlOperator, CallExprPrinter> builder =
+          ImmutableMap.builder();
+      builder.put(infixBinary(LESS_THAN, "<"))
+          .put(infixBinary(LESS_THAN_OR_EQUAL, "<="))
+          .put(infixBinary(GREATER_THAN, ">"))
+          .put(infixBinary(GREATER_THAN_OR_EQUAL, ">="))
+          .put(infixBinary(PLUS, "+"))
+          .put(infixBinary(MINUS, "-"))
+          .put(infixBinary(MULTIPLY, "*"))
+          .put(infixBinary(DIVIDE, "/"))
+          .put(infixBinary(DIVIDE_INTEGER, "/"));
+      this.translators = builder.build();
+    }
+
+    private String compile(ExprCompiler compiler, RexCall call) {
+      SqlOperator op = call.getOperator();
+      CallExprPrinter printer = translators.get(op);
+      if (printer == null) {
+        throw new UnsupportedOperationException();
+      } else {
+        return printer.translate(compiler, call);
+      }
+    }
+
+    private Map.Entry<SqlOperator, CallExprPrinter> infixBinary
+        (SqlOperator op, final String javaOperator) {
+      CallExprPrinter trans = new CallExprPrinter() {
+        @Override
+        public String translate(
+            ExprCompiler compiler, RexCall call) {
+          int size = call.getOperands().size();
+          assert size == 2;
+          String[] ops = new String[size];
+          for (int i = 0; i < size; ++i) {
+            ops[i] = call.getOperands().get(i).accept(compiler);
+          }
+          return String.format("%s %s %s", ops[0], javaOperator, ops[1]);
+        }
+      };
+      return new AbstractMap.SimpleImmutableEntry<>(op, trans);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4e2fe47c/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprCompiler.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprCompiler.java
new file mode 100644
index 0000000..6409d63
--- /dev/null
+++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprCompiler.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.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 org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.junit.Test;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+public class TestExprCompiler {
+  @Test
+  public void testLiteral() throws Exception {
+    String sql = "SELECT 1,1.0,TRUE,'FOO' FROM FOO";
+    TestUtils.CalciteState state = TestUtils.sqlOverDummyTable(sql);
+    JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
+    LogicalProject project = (LogicalProject) state.tree;
+    String[] res = new String[project.getChildExps().size()];
+    for (int i = 0; i < project.getChildExps().size(); ++i) {
+      StringWriter sw = new StringWriter();
+      try (PrintWriter pw = new PrintWriter(sw)) {
+        ExprCompiler compiler = new ExprCompiler(pw, typeFactory);
+        res[i] = project.getChildExps().get(i).accept(compiler);
+      }
+    }
+
+    assertArrayEquals(new String[] {"1", "1.0E0", "true", "\"FOO\""}, res);
+  }
+
+  @Test
+  public void testInputRef() throws Exception {
+    String sql = "SELECT ID FROM FOO";
+    TestUtils.CalciteState state = TestUtils.sqlOverDummyTable(sql);
+    JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
+    LogicalProject project = (LogicalProject) state.tree;
+    StringWriter sw = new StringWriter();
+    try (PrintWriter pw = new PrintWriter(sw)) {
+      ExprCompiler compiler = new ExprCompiler(pw, typeFactory);
+      project.getChildExps().get(0).accept(compiler);
+    }
+
+    assertEquals("int t0 = (int)(_data.get(0));\n", sw.toString());
+  }
+
+  @Test
+  public void testCallExpr() throws Exception {
+    String sql = "SELECT 1>2, 3+5, 1-1.0, 3+ID FROM FOO";
+    TestUtils.CalciteState state = TestUtils.sqlOverDummyTable(sql);
+    JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
+    LogicalProject project = (LogicalProject) state.tree;
+    String[] res = new String[project.getChildExps().size()];
+    List<StringWriter> sw = new ArrayList<>();
+    for (int i = 0; i < project.getChildExps().size(); ++i) {
+      sw.add(new StringWriter());
+    }
+
+    for (int i = 0; i < project.getChildExps().size(); ++i) {
+      try (PrintWriter pw = new PrintWriter(sw.get(i))) {
+        ExprCompiler compiler = new ExprCompiler(pw, typeFactory);
+        res[i] = project.getChildExps().get(i).accept(compiler);
+      }
+    }
+    assertArrayEquals(new String[]{"1 > 2", "3 + 5", "1 - 1.0E0", "3 + t0"},
+                      res);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/4e2fe47c/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestUtils.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestUtils.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestUtils.java
index c16cc49..ae4300a 100644
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestUtils.java
+++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestUtils.java
@@ -1,18 +1,41 @@
 package org.apache.storm.sql.compiler;
 
 import com.google.common.collect.ImmutableList;
+import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Statistic;
 import org.apache.calcite.schema.Statistics;
 import org.apache.calcite.schema.Table;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.parser.SqlParseException;
 import org.apache.calcite.sql.type.SqlTypeName;
+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.calcite.util.ImmutableBitSet;
 
 import java.util.ArrayList;
 
 class TestUtils {
+  static CalciteState sqlOverDummyTable(String sql)
+      throws RelConversionException, ValidationException, SqlParseException {
+    SchemaPlus schema = Frameworks.createRootSchema(true);
+    Table table = newTable().field("ID", SqlTypeName.INTEGER).build();
+    schema.add("FOO", table);
+    FrameworkConfig config = Frameworks.newConfigBuilder().defaultSchema(
+        schema).build();
+    Planner planner = Frameworks.getPlanner(config);
+    SqlNode parse = planner.parse(sql);
+    SqlNode validate = planner.validate(parse);
+    RelNode tree = planner.convert(validate);
+    return new CalciteState(schema, tree);
+  }
+
   static class TableBuilderInfo {
     private static class FieldType {
       private static final int NO_PRECISION = -1;
@@ -89,4 +112,14 @@ class TestUtils {
   static TableBuilderInfo newTable() {
     return new TableBuilderInfo();
   }
+
+  static class CalciteState {
+    final SchemaPlus schema;
+    final RelNode tree;
+
+    private CalciteState(SchemaPlus schema, RelNode tree) {
+      this.schema = schema;
+      this.tree = tree;
+    }
+  }
 }


[18/20] storm git commit: [StormSQL] STORM-1357. Support writing to Kafka streams in Storm SQL.

Posted by sr...@apache.org.
[StormSQL] STORM-1357. Support writing to Kafka streams in Storm SQL.


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

Branch: refs/heads/master
Commit: b5efe2c4433c0245c2fe0c1119e64baf6ecca776
Parents: b60712f
Author: Haohui Mai <wh...@apache.org>
Authored: Mon Nov 30 18:58:29 2015 -0800
Committer: Haohui Mai <wh...@apache.org>
Committed: Fri Dec 4 11:09:28 2015 -0800

----------------------------------------------------------------------
 .../apache/storm/sql/parser/SqlCreateTable.java | 14 +++++--
 .../test/org/apache/storm/sql/TestStormSql.java |  2 +-
 .../apache/storm/sql/kafka/JsonSerializer.java  |  6 +--
 .../sql/kafka/KafkaDataSourcesProvider.java     | 35 ++++++++++++----
 .../sql/kafka/TestKafkaDataSourcesProvider.java | 17 ++++++--
 .../storm/sql/runtime/DataSourcesProvider.java  |  3 +-
 .../storm/sql/runtime/DataSourcesRegistry.java  |  4 +-
 .../jvm/storm/kafka/ByteBufferSerializer.java   | 41 +++++++++++++++++++
 .../src/jvm/storm/kafka/IntSerializer.java      | 42 ++++++++++++++++++++
 9 files changed, 140 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/b5efe2c4/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
index 8fe4160..8ac52ed 100644
--- 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
@@ -113,13 +113,19 @@ public class SqlCreateTable extends SqlCall {
   }
 
   public String inputFormatClass() {
-    return inputFormatClass == null ? null : SqlLiteral.stringValue(
-        inputFormatClass);
+    return getString(inputFormatClass);
   }
 
   public String outputFormatClass() {
-    return outputFormatClass == null ? null : SqlLiteral.stringValue
-        (outputFormatClass);
+    return getString(outputFormatClass);
+  }
+
+  public String properties() {
+    return getString(properties);
+  }
+
+  private String getString(SqlNode n) {
+    return n == null ? null : SqlLiteral.stringValue(n);
   }
 
   @SuppressWarnings("unchecked")

http://git-wip-us.apache.org/repos/asf/storm/blob/b5efe2c4/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
index b238e18..add8da5 100644
--- 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
@@ -44,7 +44,7 @@ public class TestStormSql {
 
     @Override
     public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
-         List<FieldInfo> fields) {
+                                                  String properties, List<FieldInfo> fields) {
       throw new UnsupportedOperationException();
     }
   }

http://git-wip-us.apache.org/repos/asf/storm/blob/b5efe2c4/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/JsonSerializer.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/JsonSerializer.java b/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/JsonSerializer.java
index 7c5aa57..e3d5d01 100644
--- a/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/JsonSerializer.java
+++ b/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/JsonSerializer.java
@@ -20,14 +20,12 @@ package org.apache.storm.sql.kafka;
 import com.fasterxml.jackson.core.JsonFactory;
 import com.fasterxml.jackson.core.JsonGenerator;
 import com.google.common.base.Preconditions;
-import org.apache.commons.io.Charsets;
-import org.apache.commons.lang.CharSet;
 import org.apache.storm.sql.runtime.IOutputSerializer;
 
 import java.io.IOException;
 import java.io.StringWriter;
 import java.nio.ByteBuffer;
-import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
 import java.util.List;
 
 class JsonSerializer implements IOutputSerializer {
@@ -53,6 +51,6 @@ class JsonSerializer implements IOutputSerializer {
     } catch (IOException e) {
       throw new RuntimeException(e);
     }
-    return ByteBuffer.wrap(sw.toString().getBytes(Charsets.UTF_8));
+    return ByteBuffer.wrap(sw.toString().getBytes(StandardCharsets.UTF_8));
   }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b5efe2c4/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/KafkaDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/KafkaDataSourcesProvider.java b/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/KafkaDataSourcesProvider.java
index e57e4d3..7da57ba 100644
--- a/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/KafkaDataSourcesProvider.java
+++ b/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/KafkaDataSourcesProvider.java
@@ -18,6 +18,7 @@
 package org.apache.storm.sql.kafka;
 
 import backtype.storm.spout.SchemeAsMultiScheme;
+import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Preconditions;
 import org.apache.storm.sql.runtime.*;
 import storm.kafka.ZkHosts;
@@ -33,13 +34,15 @@ import storm.trident.operation.TridentOperationContext;
 import storm.trident.spout.ITridentDataSource;
 import storm.trident.tuple.TridentTuple;
 
+import java.io.IOException;
 import java.net.URI;
 import java.nio.ByteBuffer;
 import java.util.*;
 
 /**
- * Create a Kafka spout based on the URI. The URI has the format of
- * kafka://zkhost:port/broker_path?topic=topic.
+ * 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;
@@ -80,11 +83,14 @@ public class KafkaDataSourcesProvider implements DataSourcesProvider {
     private transient TridentKafkaState state;
     private final String topic;
     private final int primaryKeyIndex;
+    private final Properties producerProperties;
     private final List<String> fieldNames;
 
-    private KafkaTridentSink(String topic, int primaryKeyIndex, List<String> fieldNames) {
+    private KafkaTridentSink(String topic, int primaryKeyIndex, Properties producerProperties,
+                             List<String> fieldNames) {
       this.topic = topic;
       this.primaryKeyIndex = primaryKeyIndex;
+      this.producerProperties = producerProperties;
       this.fieldNames = fieldNames;
     }
 
@@ -100,6 +106,7 @@ public class KafkaDataSourcesProvider implements DataSourcesProvider {
       state = new TridentKafkaState()
           .withKafkaTopicSelector(new StaticTopicSelector(topic))
           .withTridentTupleToKafkaMapper(m);
+      state.prepare(producerProperties);
     }
 
     @Override
@@ -113,11 +120,13 @@ public class KafkaDataSourcesProvider implements DataSourcesProvider {
     private final String topic;
     private final int primaryKeyIndex;
     private final List<String> fields;
+    private final Properties producerProperties;
     private KafkaTridentDataSource(TridentKafkaConfig conf, String topic, int primaryKeyIndex,
-                                   List<String> fields) {
+                                   Properties producerProperties, List<String> fields) {
       this.conf = conf;
       this.topic = topic;
       this.primaryKeyIndex = primaryKeyIndex;
+      this.producerProperties = producerProperties;
       this.fields = fields;
     }
 
@@ -128,7 +137,7 @@ public class KafkaDataSourcesProvider implements DataSourcesProvider {
 
     @Override
     public Function getConsumer() {
-      return new KafkaTridentSink(topic, primaryKeyIndex, fields);
+      return new KafkaTridentSink(topic, primaryKeyIndex, producerProperties, fields);
     }
   }
 
@@ -145,7 +154,7 @@ public class KafkaDataSourcesProvider implements DataSourcesProvider {
 
   @Override
   public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
-                                                List<FieldInfo> fields) {
+                                                String 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());
@@ -163,7 +172,19 @@ public class KafkaDataSourcesProvider implements DataSourcesProvider {
     }
     Preconditions.checkState(primaryIndex != -1, "Kafka stream table must have a primary key");
     conf.scheme = new SchemeAsMultiScheme(new JsonScheme(fieldNames));
-    return new KafkaTridentDataSource(conf, topic, primaryIndex, fieldNames);
+    ObjectMapper mapper = new ObjectMapper();
+    Properties producerProp = new Properties();
+    try {
+      @SuppressWarnings("unchecked")
+      HashMap<String, Object> map = mapper.readValue(properties, HashMap.class);
+      @SuppressWarnings("unchecked")
+      HashMap<String, Object> producerConfig = (HashMap<String, Object>) map.get("producer");
+      Preconditions.checkNotNull(producerConfig, "Kafka Table must contain producer config");
+      producerProp.putAll(producerConfig);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+    return new KafkaTridentDataSource(conf, topic, primaryIndex, producerProp, fieldNames);
   }
 
   private static Map<String, String> parseURIParams(String query) {

http://git-wip-us.apache.org/repos/asf/storm/blob/b5efe2c4/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java b/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java
index 531f764..9380b66 100644
--- a/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java
+++ b/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java
@@ -17,10 +17,11 @@
  */
 package org.apache.storm.sql.kafka;
 
+import com.google.common.base.Joiner;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
-import kafka.javaapi.producer.Producer;
-import kafka.producer.KeyedMessage;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.storm.sql.kafka.KafkaDataSourcesProvider.KafkaTridentSink;
 import org.apache.storm.sql.runtime.DataSourcesRegistry;
 import org.apache.storm.sql.runtime.FieldInfo;
@@ -46,12 +47,20 @@ public class TestKafkaDataSourcesProvider {
       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 String TBL_PROPERTIES = Joiner.on('\n').join(
+      "{\"producer\": {",
+      "\"bootstrap.servers\": \"localhost:9092\",",
+      "\"acks\": \"1\",",
+      "\"key.serializer\": \"org.apache.kafka.common.serialization.StringSerializer\",",
+      "\"value.serializer\": \"org.apache.kafka.common.serialization.StringSerializer\"",
+      "}",
+      "}"
+  );
   @SuppressWarnings("unchecked")
   @Test
   public void testKafkaSink() {
     ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(
-        URI.create("kafka://mock?topic=foo"), null, null, FIELDS);
+        URI.create("kafka://mock?topic=foo"), null, null, TBL_PROPERTIES, FIELDS);
     Assert.assertNotNull(ds);
     KafkaTridentSink sink = (KafkaTridentSink) ds.getConsumer();
     sink.prepare(null, null);

http://git-wip-us.apache.org/repos/asf/storm/blob/b5efe2c4/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
index eaabc8d..0ce2d45 100644
--- 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
@@ -22,7 +22,6 @@ package org.apache.storm.sql.runtime;
 
 import java.net.URI;
 import java.util.List;
-import java.util.Map;
 
 public interface DataSourcesProvider {
   /**
@@ -46,5 +45,5 @@ public interface DataSourcesProvider {
 
   ISqlTridentDataSource constructTrident(
       URI uri, String inputFormatClass, String outputFormatClass,
-      List<FieldInfo> fields);
+      String properties, List<FieldInfo> fields);
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b5efe2c4/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
index 0285c97..75cd391 100644
--- 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
@@ -61,13 +61,13 @@ public class DataSourcesRegistry {
 
   public static ISqlTridentDataSource constructTridentDataSource(
       URI uri, String inputFormatClass, String outputFormatClass,
-      List<FieldInfo> fields) {
+      String properties, List<FieldInfo> fields) {
     DataSourcesProvider provider = providers.get(uri.getScheme());
     if (provider == null) {
       return null;
     }
 
-    return provider.constructTrident(uri, inputFormatClass, outputFormatClass, fields);
+    return provider.constructTrident(uri, inputFormatClass, outputFormatClass, properties, fields);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/storm/blob/b5efe2c4/external/storm-kafka/src/jvm/storm/kafka/ByteBufferSerializer.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/storm/kafka/ByteBufferSerializer.java b/external/storm-kafka/src/jvm/storm/kafka/ByteBufferSerializer.java
new file mode 100644
index 0000000..1a7238e
--- /dev/null
+++ b/external/storm-kafka/src/jvm/storm/kafka/ByteBufferSerializer.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 storm.kafka;
+
+import backtype.storm.utils.Utils;
+import org.apache.kafka.common.serialization.Serializer;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+
+public class ByteBufferSerializer implements Serializer<ByteBuffer> {
+  @Override
+  public void configure(Map<String, ?> map, boolean b) {
+
+  }
+
+  @Override
+  public void close() {
+
+  }
+
+  @Override
+  public byte[] serialize(String s, ByteBuffer b) {
+    return Utils.toByteArray(b);
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/b5efe2c4/external/storm-kafka/src/jvm/storm/kafka/IntSerializer.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/jvm/storm/kafka/IntSerializer.java b/external/storm-kafka/src/jvm/storm/kafka/IntSerializer.java
new file mode 100644
index 0000000..07cbd26
--- /dev/null
+++ b/external/storm-kafka/src/jvm/storm/kafka/IntSerializer.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
+ * <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 storm.kafka;
+
+import org.apache.kafka.common.serialization.Serializer;
+
+import java.nio.ByteBuffer;
+import java.nio.IntBuffer;
+import java.util.Map;
+
+public class IntSerializer implements Serializer<Integer> {
+  @Override
+  public void configure(Map<String, ?> map, boolean b) {
+  }
+
+  @Override
+  public byte[] serialize(String topic, Integer val) {
+    byte[] r = new byte[4];
+    IntBuffer b = ByteBuffer.wrap(r).asIntBuffer();
+    b.put(val);
+    return r;
+  }
+
+  @Override
+  public void close() {
+  }
+}


[16/20] storm git commit: [StormSQL] STORM-1222. Support Kafka as external tables in StormSQL.

Posted by sr...@apache.org.
[StormSQL] STORM-1222. Support Kafka as external tables in StormSQL.


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

Branch: refs/heads/master
Commit: b60712f616be4579049d2ffd7e84f9047d20a8e3
Parents: a8894e6
Author: Haohui Mai <wh...@apache.org>
Authored: Thu Nov 19 14:09:26 2015 -0800
Committer: Haohui Mai <wh...@apache.org>
Committed: Fri Dec 4 11:09:28 2015 -0800

----------------------------------------------------------------------
 external/sql/pom.xml                            |   1 +
 external/sql/storm-sql-kafka/pom.xml            | 111 +++++++++++
 .../org/apache/storm/sql/kafka/JsonScheme.java  |  56 ++++++
 .../apache/storm/sql/kafka/JsonSerializer.java  |  58 ++++++
 .../sql/kafka/KafkaDataSourcesProvider.java     | 184 +++++++++++++++++++
 ...apache.storm.sql.runtime.DataSourcesProvider |  16 ++
 .../storm/sql/kafka/TestJsonRepresentation.java |  50 +++++
 .../sql/kafka/TestKafkaDataSourcesProvider.java | 103 +++++++++++
 .../storm/sql/runtime/IOutputSerializer.java    |  31 ++++
 pom.xml                                         |   6 +
 10 files changed, 616 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/b60712f6/external/sql/pom.xml
----------------------------------------------------------------------
diff --git a/external/sql/pom.xml b/external/sql/pom.xml
index e4dae94..7884d39 100644
--- a/external/sql/pom.xml
+++ b/external/sql/pom.xml
@@ -39,5 +39,6 @@
     <modules>
         <module>storm-sql-core</module>
         <module>storm-sql-runtime</module>
+        <module>storm-sql-kafka</module>
     </modules>
 </project>

http://git-wip-us.apache.org/repos/asf/storm/blob/b60712f6/external/sql/storm-sql-kafka/pom.xml
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-kafka/pom.xml b/external/sql/storm-sql-kafka/pom.xml
new file mode 100644
index 0000000..0f6bd19
--- /dev/null
+++ b/external/sql/storm-sql-kafka/pom.xml
@@ -0,0 +1,111 @@
+<?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>0.11.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>kafka_2.10</artifactId>
+            <version>0.8.2.1</version>
+            <!-- use provided scope, so users can pull in whichever scala version they choose -->
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.zookeeper</groupId>
+                    <artifactId>zookeeper</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.kafka</groupId>
+            <artifactId>kafka-clients</artifactId>
+            <version>0.8.2.1</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+        </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/b60712f6/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/JsonScheme.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/JsonScheme.java b/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/JsonScheme.java
new file mode 100644
index 0000000..80037c6
--- /dev/null
+++ b/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/JsonScheme.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
+ * <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 backtype.storm.spout.Scheme;
+import backtype.storm.tuple.Fields;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+public class JsonScheme implements Scheme {
+  private final List<String> fields;
+
+  JsonScheme(List<String> fields) {
+    this.fields = fields;
+  }
+
+  @Override
+  public List<Object> deserialize(byte[] ser) {
+    ObjectMapper mapper = new ObjectMapper();
+    try {
+      @SuppressWarnings("unchecked")
+      HashMap<String, Object> map = mapper.readValue(ser, HashMap.class);
+      ArrayList<Object> list = new ArrayList<>();
+      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);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/b60712f6/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/JsonSerializer.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/JsonSerializer.java b/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/JsonSerializer.java
new file mode 100644
index 0000000..7c5aa57
--- /dev/null
+++ b/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/JsonSerializer.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.kafka;
+
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.google.common.base.Preconditions;
+import org.apache.commons.io.Charsets;
+import org.apache.commons.lang.CharSet;
+import org.apache.storm.sql.runtime.IOutputSerializer;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+import java.util.List;
+
+class JsonSerializer implements IOutputSerializer {
+  private final List<String> fieldNames;
+  private transient final JsonFactory jsonFactory;
+
+  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(Charsets.UTF_8));
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/b60712f6/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/KafkaDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/KafkaDataSourcesProvider.java b/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/KafkaDataSourcesProvider.java
new file mode 100644
index 0000000..e57e4d3
--- /dev/null
+++ b/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/KafkaDataSourcesProvider.java
@@ -0,0 +1,184 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.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 backtype.storm.spout.SchemeAsMultiScheme;
+import com.google.common.base.Preconditions;
+import org.apache.storm.sql.runtime.*;
+import storm.kafka.ZkHosts;
+import storm.kafka.trident.OpaqueTridentKafkaSpout;
+import storm.kafka.trident.TridentKafkaConfig;
+import storm.kafka.trident.TridentKafkaState;
+import storm.kafka.trident.mapper.TridentTupleToKafkaMapper;
+import storm.kafka.trident.selector.KafkaTopicSelector;
+import storm.trident.operation.BaseFunction;
+import storm.trident.operation.Function;
+import storm.trident.operation.TridentCollector;
+import storm.trident.operation.TridentOperationContext;
+import storm.trident.spout.ITridentDataSource;
+import storm.trident.tuple.TridentTuple;
+
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.*;
+
+/**
+ * Create a Kafka spout based on the URI. The URI has the format of
+ * kafka://zkhost:port/broker_path?topic=topic.
+ */
+public class KafkaDataSourcesProvider implements DataSourcesProvider {
+  private static final int DEFAULT_ZK_PORT = 2181;
+  private static class StaticTopicSelector implements KafkaTopicSelector {
+    private final String topic;
+
+    private StaticTopicSelector(String topic) {
+      this.topic = topic;
+    }
+
+    @Override
+    public String getTopic(TridentTuple tuple) {
+      return topic;
+    }
+  }
+
+  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);
+    }
+  }
+
+  static class KafkaTridentSink extends BaseFunction {
+    private transient TridentKafkaState state;
+    private final String topic;
+    private final int primaryKeyIndex;
+    private final List<String> fieldNames;
+
+    private KafkaTridentSink(String topic, int primaryKeyIndex, List<String> fieldNames) {
+      this.topic = topic;
+      this.primaryKeyIndex = primaryKeyIndex;
+      this.fieldNames = fieldNames;
+    }
+
+    @Override
+    public void cleanup() {
+      super.cleanup();
+    }
+
+    @Override
+    public void prepare(Map conf, TridentOperationContext context) {
+      JsonSerializer serializer = new JsonSerializer(fieldNames);
+      SqlKafkaMapper m = new SqlKafkaMapper(primaryKeyIndex, serializer);
+      state = new TridentKafkaState()
+          .withKafkaTopicSelector(new StaticTopicSelector(topic))
+          .withTridentTupleToKafkaMapper(m);
+    }
+
+    @Override
+    public void execute(TridentTuple tuple, TridentCollector collector) {
+      state.updateState(Collections.singletonList(tuple), collector);
+    }
+  }
+
+  private static class KafkaTridentDataSource implements ISqlTridentDataSource {
+    private final TridentKafkaConfig conf;
+    private final String topic;
+    private final int primaryKeyIndex;
+    private final List<String> fields;
+    private KafkaTridentDataSource(TridentKafkaConfig conf, String topic, int primaryKeyIndex,
+                                   List<String> fields) {
+      this.conf = conf;
+      this.topic = topic;
+      this.primaryKeyIndex = primaryKeyIndex;
+      this.fields = fields;
+    }
+
+    @Override
+    public ITridentDataSource getProducer() {
+      return new OpaqueTridentKafkaSpout(conf);
+    }
+
+    @Override
+    public Function getConsumer() {
+      return new KafkaTridentSink(topic, primaryKeyIndex, fields);
+    }
+  }
+
+  @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,
+                                                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");
+    conf.scheme = new SchemeAsMultiScheme(new JsonScheme(fieldNames));
+    return new KafkaTridentDataSource(conf, topic, primaryIndex, fieldNames);
+  }
+
+  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/b60712f6/external/sql/storm-sql-kafka/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-kafka/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider b/external/sql/storm-sql-kafka/src/resources/META-INF/services/org.apache.storm.sql.runtime.DataSourcesProvider
new file mode 100644
index 0000000..7f687cc
--- /dev/null
+++ b/external/sql/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/b60712f6/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestJsonRepresentation.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestJsonRepresentation.java b/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestJsonRepresentation.java
new file mode 100644
index 0000000..d2898e8
--- /dev/null
+++ b/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestJsonRepresentation.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.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 backtype.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(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/b60712f6/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java b/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java
new file mode 100644
index 0000000..531f764
--- /dev/null
+++ b/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.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 kafka.javaapi.producer.Producer;
+import kafka.producer.KeyedMessage;
+import org.apache.storm.sql.kafka.KafkaDataSourcesProvider.KafkaTridentSink;
+import org.apache.storm.sql.runtime.DataSourcesRegistry;
+import org.apache.storm.sql.runtime.FieldInfo;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.ArgumentMatcher;
+import org.mockito.internal.util.reflection.Whitebox;
+import storm.kafka.trident.TridentKafkaState;
+import storm.trident.tuple.TridentTuple;
+
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import static org.mockito.Mockito.*;
+
+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);
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testKafkaSink() {
+    ISqlTridentDataSource ds = DataSourcesRegistry.constructTridentDataSource(
+        URI.create("kafka://mock?topic=foo"), null, null, FIELDS);
+    Assert.assertNotNull(ds);
+    KafkaTridentSink sink = (KafkaTridentSink) ds.getConsumer();
+    sink.prepare(null, null);
+    TridentKafkaState state = (TridentKafkaState) Whitebox.getInternalState(sink, "state");
+    Producer producer = mock(Producer.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<KeyedMessage> {
+    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) {
+      KeyedMessage<Object, ByteBuffer> m = (KeyedMessage<Object,ByteBuffer>)o;
+      if (m.key() != tuple.get(PRIMARY_INDEX)) {
+        return false;
+      }
+      ByteBuffer buf = m.message();
+      ByteBuffer b = SERIALIZER.write(tuple.getValues(), null);
+      return b.equals(buf);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/b60712f6/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
new file mode 100644
index 0000000..b6670d9
--- /dev/null
+++ b/external/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/b60712f6/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 3387583..4c7388d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -228,6 +228,7 @@
         <metrics-clojure.version>2.5.1</metrics-clojure.version>
         <hdrhistogram.version>2.1.7</hdrhistogram.version>
         <calcite.version>1.4.0-incubating</calcite.version>
+        <jackson.version>2.6.3</jackson.version>
     </properties>
 
     <modules>
@@ -694,6 +695,11 @@
               <artifactId>calcite-core</artifactId>
               <version>${calcite.version}</version>
             </dependency>
+            <dependency>
+              <groupId>com.fasterxml.jackson.core</groupId>
+              <artifactId>jackson-databind</artifactId>
+              <version>${jackson.version}</version>
+            </dependency>
         </dependencies>
     </dependencyManagement>
 


[09/20] storm git commit: [StormSQL] STORM-1159. Support nullable operations in StormSQL.

Posted by sr...@apache.org.
[StormSQL] STORM-1159. Support nullable operations in StormSQL.


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

Branch: refs/heads/master
Commit: beeaee739a3ba43d0507de3efd4338f9274c4760
Parents: 31daf26
Author: Haohui Mai <wh...@apache.org>
Authored: Wed Nov 4 18:36:59 2015 -0800
Committer: Haohui Mai <wh...@apache.org>
Committed: Fri Dec 4 11:09:02 2015 -0800

----------------------------------------------------------------------
 .../apache/storm/sql/compiler/ExprCompiler.java | 54 ++++++++++++++------
 .../storm/sql/compiler/TestExprCompiler.java    | 10 ++--
 .../storm/sql/compiler/TestExprSemantic.java    |  9 ++++
 .../storm/sql/compiler/TestRelNodeCompiler.java |  7 ++-
 4 files changed, 59 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/beeaee73/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java
index db97d43..6617ef6 100644
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java
@@ -17,10 +17,7 @@
  */
 package org.apache.storm.sql.compiler;
 
-import com.google.common.base.Joiner;
 import com.google.common.collect.ImmutableMap;
-import org.apache.calcite.adapter.enumerable.CallImplementor;
-import org.apache.calcite.adapter.enumerable.RexImpTable;
 import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexCall;
@@ -42,7 +39,6 @@ import java.io.PrintWriter;
 import java.lang.reflect.Type;
 import java.math.BigDecimal;
 import java.util.AbstractMap;
-import java.util.IdentityHashMap;
 import java.util.Map;
 
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.AND;
@@ -80,7 +76,7 @@ class ExprCompiler implements RexVisitor<String> {
 
   @Override
   public String visitInputRef(RexInputRef rexInputRef) {
-    String name = reserveName(rexInputRef);
+    String name = reserveName();
     String typeName = javaTypeName(rexInputRef);
     pw.print(String.format("%s %s = (%s)(_data.get(%d));\n", typeName, name,
                            typeName, rexInputRef.getIndex()));
@@ -164,9 +160,8 @@ class ExprCompiler implements RexVisitor<String> {
     return ((Class<?>)ty).getCanonicalName();
   }
 
-  private String reserveName(RexNode node) {
-    String name = "t" + ++nameCount;
-    return name;
+  private String reserveName() {
+    return "t" + ++nameCount;
   }
 
   private interface CallExprPrinter {
@@ -223,11 +218,38 @@ class ExprCompiler implements RexVisitor<String> {
             ExprCompiler compiler, RexCall call) {
           int size = call.getOperands().size();
           assert size == 2;
-          String[] ops = new String[size];
-          for (int i = 0; i < size; ++i) {
-            ops[i] = call.getOperands().get(i).accept(compiler);
+          String val = compiler.reserveName();
+          RexNode op0 = call.getOperands().get(0);
+          RexNode op1 = call.getOperands().get(1);
+          boolean lhsNullable = op0.getType().isNullable();
+          boolean rhsNullable = op1.getType().isNullable();
+
+          PrintWriter pw = compiler.pw;
+          String lhs = op0.accept(compiler);
+          pw.print(String.format("final %s %s;\n", compiler.javaTypeName(call), val));
+          if (!lhsNullable) {
+            String rhs = op1.accept(compiler);
+            String calc = foldNullExpr(String.format("%s %s %s", lhs, javaOperator, rhs), "null", rhs);
+            if (!rhsNullable) {
+              pw.print(String.format("%s = %s;\n", val, calc));
+            } else {
+              pw.print(
+                  String.format("%s = %s == null ? null : (%s);\n",
+                      val, rhs, calc));
+            }
+          } else {
+            pw.print(String.format("if (%2$s == null) { %1$s = null; }\n",
+                val, lhs));
+            pw.print("else {\n");
+            String rhs = op1.accept(compiler);
+            String calc = foldNullExpr(String.format("%s %s %s", lhs, javaOperator, rhs), "null", lhs);
+            if (!rhsNullable) {
+              pw.print(String.format("%s = %s;\n}\n", val, calc));
+            } else {
+              pw.print(String.format("%1$s = %2$s == null ? null : (%3$s);\n}\n", val, rhs, calc));
+            }
           }
-          return String.format("%s %s %s", ops[0], javaOperator, ops[1]);
+          return val;
         }
       };
       return new AbstractMap.SimpleImmutableEntry<>(op, trans);
@@ -250,7 +272,7 @@ class ExprCompiler implements RexVisitor<String> {
         public String translate(
             ExprCompiler compiler, RexCall call) {
           assert call.getOperands().size() == 1;
-          String val = compiler.reserveName(call);
+          String val = compiler.reserveName();
           RexNode operand = call.getOperands().get(0);
           boolean nullable = operand.getType().isNullable();
           String op = operand.accept(compiler);
@@ -289,7 +311,7 @@ class ExprCompiler implements RexVisitor<String> {
       @Override
       public String translate(
           ExprCompiler compiler, RexCall call) {
-        String val = compiler.reserveName(call);
+        String val = compiler.reserveName();
         PrintWriter pw = compiler.pw;
         pw.print(String.format("final %s %s;\n", compiler.javaTypeName(call),
                                val));
@@ -331,7 +353,7 @@ class ExprCompiler implements RexVisitor<String> {
       @Override
       public String translate(
           ExprCompiler compiler, RexCall call) {
-        String val = compiler.reserveName(call);
+        String val = compiler.reserveName();
         PrintWriter pw = compiler.pw;
         pw.print(String.format("final %s %s;\n", compiler.javaTypeName(call),
                                val));
@@ -369,7 +391,7 @@ class ExprCompiler implements RexVisitor<String> {
       @Override
       public String translate(
           ExprCompiler compiler, RexCall call) {
-        String val = compiler.reserveName(call);
+        String val = compiler.reserveName();
         PrintWriter pw = compiler.pw;
         RexNode op = call.getOperands().get(0);
         String lhs = op.accept(compiler);

http://git-wip-us.apache.org/repos/asf/storm/blob/beeaee73/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprCompiler.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprCompiler.java
index 9f516d4..f03cff8 100644
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprCompiler.java
+++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprCompiler.java
@@ -28,8 +28,10 @@ import java.io.StringWriter;
 import java.util.ArrayList;
 import java.util.List;
 
+import static org.hamcrest.CoreMatchers.containsString;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
 
 public class TestExprCompiler {
   @Test
@@ -62,7 +64,7 @@ public class TestExprCompiler {
       project.getChildExps().get(0).accept(compiler);
     }
 
-    assertEquals("int t0 = (int)(_data.get(0));\n", sw.toString());
+    assertThat(sw.toString(), containsString("(int)(_data.get(0));"));
   }
 
   @Test
@@ -83,7 +85,9 @@ public class TestExprCompiler {
         res[i] = project.getChildExps().get(i).accept(compiler);
       }
     }
-    assertArrayEquals(new String[]{"1 > 2", "3 + 5", "1 - 1.0E0", "3 + t0"},
-                      res);
+    assertThat(sw.get(0).toString(), containsString("1 > 2"));
+    assertThat(sw.get(1).toString(), containsString("3 + 5"));
+    assertThat(sw.get(2).toString(), containsString("1 - 1.0E0"));
+    assertThat(sw.get(3).toString(), containsString("3 +"));
   }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/beeaee73/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
index 7bde092..8454d7e 100644
--- 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
@@ -58,6 +58,15 @@ public class TestExprSemantic {
   }
 
   @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(

http://git-wip-us.apache.org/repos/asf/storm/blob/beeaee73/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestRelNodeCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestRelNodeCompiler.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestRelNodeCompiler.java
index cedb48b..d820f22 100644
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestRelNodeCompiler.java
+++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestRelNodeCompiler.java
@@ -23,12 +23,15 @@ import org.apache.calcite.rel.core.TableScan;
 import org.apache.calcite.rel.logical.LogicalFilter;
 import org.apache.calcite.rel.logical.LogicalProject;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.hamcrest.CoreMatchers;
 import org.junit.Assert;
 import org.junit.Test;
 
 import java.io.PrintWriter;
 import java.io.StringWriter;
 
+import static org.hamcrest.CoreMatchers.*;
+
 public class TestRelNodeCompiler {
   @Test
   public void testFilter() throws Exception {
@@ -45,7 +48,7 @@ public class TestRelNodeCompiler {
       RelNodeCompiler compiler = new RelNodeCompiler(pw, typeFactory);
       compiler.visitFilter(filter);
       pw.flush();
-      Assert.assertTrue(sw.toString().contains("t0 > 3"));
+      Assert.assertThat(sw.toString(), containsString("> 3"));
     }
 
     try (StringWriter sw = new StringWriter();
@@ -54,7 +57,7 @@ public class TestRelNodeCompiler {
       RelNodeCompiler compiler = new RelNodeCompiler(pw, typeFactory);
       compiler.visitProject(project);
       pw.flush();
-      Assert.assertTrue(sw.toString().contains("t0 + 1"));
+      Assert.assertThat(sw.toString(), containsString("+ 1"));
     }
   }
 }


[19/20] storm git commit: Merge branch 'STORM-1040-merge' of https://github.com/haohui/storm into STORM-1040-V1

Posted by sr...@apache.org.
Merge branch 'STORM-1040-merge' of https://github.com/haohui/storm into STORM-1040-V1


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

Branch: refs/heads/master
Commit: 6e65ae250f77035bc9dac9f0f1025c3ab8965140
Parents: 2b7a758 ff22fd9
Author: Sriharsha Chintalapani <ha...@hortonworks.com>
Authored: Tue Dec 8 09:22:48 2015 -0800
Committer: Sriharsha Chintalapani <ha...@hortonworks.com>
Committed: Tue Dec 8 09:22:48 2015 -0800

----------------------------------------------------------------------
 bin/storm.py                                    |  16 +-
 external/sql/README.md                          | 117 +++++
 external/sql/pom.xml                            |  44 ++
 external/sql/storm-sql-core/pom.xml             | 261 ++++++++++
 .../sql/storm-sql-core/src/codegen/config.fmpp  |  23 +
 .../storm-sql-core/src/codegen/data/Parser.tdd  |  64 +++
 .../src/codegen/includes/license.ftl            |  17 +
 .../src/codegen/includes/parserImpls.ftl        |  86 ++++
 .../src/jvm/org/apache/storm/sql/StormSql.java  |  54 +++
 .../jvm/org/apache/storm/sql/StormSqlImpl.java  | 187 ++++++++
 .../org/apache/storm/sql/StormSqlRunner.java    |  44 ++
 .../apache/storm/sql/compiler/CompilerUtil.java | 168 +++++++
 .../apache/storm/sql/compiler/ExprCompiler.java | 471 +++++++++++++++++++
 .../sql/compiler/PostOrderRelNodeVisitor.java   | 122 +++++
 .../backends/standalone/PlanCompiler.java       | 132 ++++++
 .../backends/standalone/RelNodeCompiler.java    | 111 +++++
 .../compiler/backends/trident/PlanCompiler.java | 201 ++++++++
 .../backends/trident/RelNodeCompiler.java       | 116 +++++
 .../storm/sql/javac/CompilingClassLoader.java   | 225 +++++++++
 .../storm/sql/parser/ColumnConstraint.java      |  42 ++
 .../storm/sql/parser/ColumnDefinition.java      |  44 ++
 .../apache/storm/sql/parser/SqlCreateTable.java | 136 ++++++
 .../apache/storm/sql/parser/SqlDDLKeywords.java |  27 ++
 .../apache/storm/sql/parser/StormParser.java    |  42 ++
 .../apache/storm/sql/parser/UnparseUtil.java    |  60 +++
 .../test/org/apache/storm/sql/TestStormSql.java |  82 ++++
 .../storm/sql/compiler/TestCompilerUtils.java   |  64 +++
 .../storm/sql/compiler/TestExprCompiler.java    |  93 ++++
 .../storm/sql/compiler/TestExprSemantic.java    | 140 ++++++
 .../backends/standalone/TestPlanCompiler.java   |  69 +++
 .../standalone/TestRelNodeCompiler.java         |  62 +++
 .../backends/trident/TestPlanCompiler.java      | 116 +++++
 .../apache/storm/sql/parser/TestSqlParser.java  |  48 ++
 external/sql/storm-sql-kafka/pom.xml            | 111 +++++
 .../org/apache/storm/sql/kafka/JsonScheme.java  |  58 +++
 .../apache/storm/sql/kafka/JsonSerializer.java  |  56 +++
 .../sql/kafka/KafkaDataSourcesProvider.java     | 205 ++++++++
 ...apache.storm.sql.runtime.DataSourcesProvider |  16 +
 .../storm/sql/kafka/TestJsonRepresentation.java |  50 ++
 .../sql/kafka/TestKafkaDataSourcesProvider.java | 114 +++++
 external/sql/storm-sql-runtime/pom.xml          |  73 +++
 .../sql/runtime/AbstractChannelHandler.java     |  44 ++
 .../sql/runtime/AbstractValuesProcessor.java    |  49 ++
 .../storm/sql/runtime/ChannelContext.java       |  30 ++
 .../storm/sql/runtime/ChannelHandler.java       |  39 ++
 .../org/apache/storm/sql/runtime/Channels.java  |  80 ++++
 .../apache/storm/sql/runtime/DataSource.java    |  29 ++
 .../storm/sql/runtime/DataSourcesProvider.java  |  49 ++
 .../storm/sql/runtime/DataSourcesRegistry.java  |  78 +++
 .../org/apache/storm/sql/runtime/FieldInfo.java |  45 ++
 .../storm/sql/runtime/IOutputSerializer.java    |  31 ++
 .../sql/runtime/ISqlTridentDataSource.java      |  30 ++
 .../storm/sql/runtime/StormSqlFunctions.java    |  36 ++
 .../trident/AbstractTridentProcessor.java       |  43 ++
 .../test/org/apache/storm/sql/TestUtils.java    | 163 +++++++
 .../jvm/storm/kafka/ByteBufferSerializer.java   |  41 ++
 .../src/jvm/storm/kafka/IntSerializer.java      |  42 ++
 pom.xml                                         |  17 +
 .../src/jvm/backtype/storm/utils/Utils.java     |  15 +-
 storm-dist/binary/pom.xml                       |  10 +
 storm-dist/binary/src/main/assembly/binary.xml  |  25 +-
 61 files changed, 5058 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/6e65ae25/bin/storm.py
----------------------------------------------------------------------


[02/20] storm git commit: [StormSQL] STORM-1060. Implement PostOrderRelNodeVisitor.

Posted by sr...@apache.org.
[StormSQL] STORM-1060. Implement PostOrderRelNodeVisitor.


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

Branch: refs/heads/master
Commit: b6fa601c821521fb7ffc6ae37c481105edd7b974
Parents: 425280e
Author: Haohui Mai <wh...@apache.org>
Authored: Tue Sep 22 16:12:12 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Fri Dec 4 11:09:01 2015 -0800

----------------------------------------------------------------------
 .../sql/compiler/PostOrderRelNodeVisitor.java   | 127 +++++++++++++++++++
 .../apache/storm/sql/compiler/TestUtils.java    |  92 ++++++++++++++
 2 files changed, 219 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/b6fa601c/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/PostOrderRelNodeVisitor.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/PostOrderRelNodeVisitor.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/PostOrderRelNodeVisitor.java
new file mode 100644
index 0000000..6277e28
--- /dev/null
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/PostOrderRelNodeVisitor.java
@@ -0,0 +1,127 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.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 org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.Calc;
+import org.apache.calcite.rel.core.Collect;
+import org.apache.calcite.rel.core.Correlate;
+import org.apache.calcite.rel.core.Exchange;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.Sample;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.core.Uncollect;
+import org.apache.calcite.rel.core.Window;
+import org.apache.calcite.rel.stream.Delta;
+
+abstract class PostOrderRelNodeVisitor<T> {
+  final T traverse(RelNode n) throws Exception {
+    for (RelNode input : n.getInputs()) {
+      traverse(input);
+    }
+
+    if (n instanceof Aggregate) {
+      return visitAggregate((Aggregate) n);
+    } else if (n instanceof Calc) {
+      return visitCalc((Calc) n);
+    } else if (n instanceof Collect) {
+      return visitCollect((Collect) n);
+    } else if (n instanceof Correlate) {
+      return visitCorrelate((Correlate) n);
+    } else if (n instanceof Delta) {
+      return visitDelta((Delta) n);
+    } else if (n instanceof Exchange) {
+      return visitExchange((Exchange) n);
+    } else if (n instanceof Project) {
+      return visitProject((Project) n);
+    } else if (n instanceof Filter) {
+      return visitFilter((Filter) n);
+    } else if (n instanceof Sample) {
+      return visitSample((Sample) n);
+    } else if (n instanceof Sort) {
+      return visitSort((Sort) n);
+    } else if (n instanceof TableScan) {
+      return visitTableScan((TableScan) n);
+    } else if (n instanceof Uncollect) {
+      return visitUncollect((Uncollect) n);
+    } else if (n instanceof Window) {
+      return visitWindow((Window) n);
+    } else {
+      return defaultValue(n);
+    }
+  }
+
+  T visitAggregate(Aggregate aggregate) throws Exception {
+    return defaultValue(aggregate);
+  }
+
+  T visitCalc(Calc calc) throws Exception {
+    return defaultValue(calc);
+  }
+
+  T visitCollect(Collect collect) throws Exception {
+    return defaultValue(collect);
+  }
+
+  T visitCorrelate(Correlate correlate) throws Exception {
+    return defaultValue(correlate);
+  }
+
+  T visitDelta(Delta delta) throws Exception {
+    return defaultValue(delta);
+  }
+
+  T visitExchange(Exchange exchange) throws Exception {
+    return defaultValue(exchange);
+  }
+
+  T visitProject(Project project) throws Exception {
+    return defaultValue(project);
+  }
+
+  T visitFilter(Filter filter) throws Exception {
+    return defaultValue(filter);
+  }
+
+  T visitSample(Sample sample) throws Exception {
+    return defaultValue(sample);
+  }
+
+  T visitSort(Sort sort) throws Exception {
+    return defaultValue(sort);
+  }
+
+  T visitTableScan(TableScan scan) throws Exception {
+    return defaultValue(scan);
+  }
+
+  T visitUncollect(Uncollect uncollect) throws Exception {
+    return defaultValue(uncollect);
+  }
+
+  T visitWindow(Window window) throws Exception {
+    return defaultValue(window);
+  }
+
+  T defaultValue(RelNode n) {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/b6fa601c/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestUtils.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestUtils.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestUtils.java
new file mode 100644
index 0000000..c16cc49
--- /dev/null
+++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestUtils.java
@@ -0,0 +1,92 @@
+package org.apache.storm.sql.compiler;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.Statistic;
+import org.apache.calcite.schema.Statistics;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.ImmutableBitSet;
+
+import java.util.ArrayList;
+
+class TestUtils {
+  static class TableBuilderInfo {
+    private static class FieldType {
+      private static final int NO_PRECISION = -1;
+      private final String name;
+      private final SqlTypeName type;
+      private final int precision;
+
+      private FieldType(String name, SqlTypeName type, int precision) {
+        this.name = name;
+        this.type = type;
+        this.precision = precision;
+      }
+
+      private FieldType(String name, SqlTypeName type) {
+        this(name, type, NO_PRECISION);
+      }
+    }
+
+    private final ArrayList<FieldType> fields = new ArrayList<>();
+    private final ArrayList<Object[]> rows = new ArrayList<>();
+    private Statistic stats;
+
+    TableBuilderInfo field(String name, SqlTypeName type) {
+      fields.add(new FieldType(name, type));
+      return this;
+    }
+
+    TableBuilderInfo field(String name, SqlTypeName type, int precision) {
+      fields.add(new FieldType(name, type, precision));
+      return this;
+    }
+
+    TableBuilderInfo statistics(Statistic stats) {
+      this.stats = stats;
+      return this;
+    }
+
+    TableBuilderInfo rows(Object[] data) {
+      rows.add(data);
+      return this;
+    }
+
+    Table build() {
+      final Statistic stat = stats;
+      return new Table() {
+        @Override
+        public RelDataType getRowType(
+            RelDataTypeFactory relDataTypeFactory) {
+          RelDataTypeFactory.FieldInfoBuilder b = relDataTypeFactory.builder();
+          for (FieldType f : fields) {
+            if (f.precision == FieldType.NO_PRECISION) {
+              b.add(f.name, f.type);
+            } else {
+              b.add(f.name, f.type, f.precision);
+            }
+          }
+          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.TABLE;
+        }
+      };
+    }
+  }
+
+  static TableBuilderInfo newTable() {
+    return new TableBuilderInfo();
+  }
+}


[11/20] storm git commit: [StormSQL] Implement compiler for relation nodes.

Posted by sr...@apache.org.
[StormSQL] Implement compiler for relation nodes.


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

Branch: refs/heads/master
Commit: bd4f6dc3c1f8cc5fa772fd267cc416d718f8bcd5
Parents: 4e2fe47
Author: Haohui Mai <wh...@apache.org>
Authored: Thu Oct 22 16:45:20 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Fri Dec 4 11:09:02 2015 -0800

----------------------------------------------------------------------
 .../storm/sql/compiler/RelNodeCompiler.java     | 107 +++++++++++++++++++
 .../storm/sql/compiler/TestRelNodeCompiler.java |  70 ++++++++++++
 2 files changed, 177 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/bd4f6dc3/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RelNodeCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RelNodeCompiler.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RelNodeCompiler.java
new file mode 100644
index 0000000..0550035
--- /dev/null
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RelNodeCompiler.java
@@ -0,0 +1,107 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.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.Joiner;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.TableScan;
+
+import java.io.PrintWriter;
+import java.util.Collections;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * Compile RelNodes into individual functions.
+ */
+class RelNodeCompiler extends PostOrderRelNodeVisitor<Void> {
+  private final PrintWriter pw;
+  private final JavaTypeFactory typeFactory;
+
+  public Set<String> getReferredTables() {
+    return Collections.unmodifiableSet(referredTables);
+  }
+
+  private final Set<String> referredTables = new TreeSet<>();
+
+  RelNodeCompiler(PrintWriter pw, JavaTypeFactory typeFactory) {
+    this.pw = pw;
+    this.typeFactory = typeFactory;
+  }
+
+  @Override
+  Void visitFilter(Filter filter) throws Exception {
+    beginFunction(filter);
+    pw.print("  if (_data == null) return null;\n");
+    ExprCompiler compiler = new ExprCompiler(pw, typeFactory);
+    String r = filter.getCondition().accept(compiler);
+    pw.print(String.format("  return %s ? _data : null;\n", r));
+    endFunction();
+    return null;
+  }
+
+  @Override
+  Void visitProject(Project project) throws Exception {
+    beginFunction(project);
+    pw.print("  if (_data == null) return null;\n");
+    ExprCompiler compiler = new ExprCompiler(pw, typeFactory);
+
+    int size = project.getChildExps().size();
+    String[] res = new String[size];
+    for (int i = 0; i < size; ++i) {
+      res[i] = project.getChildExps().get(i).accept(compiler);
+    }
+
+    pw.print(String.format("  return new Values(%s);\n", Joiner.on(',').join
+        (res)));
+    endFunction();
+    return null;
+  }
+
+  @Override
+  Void defaultValue(RelNode n) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  Void visitTableScan(TableScan scan) throws Exception {
+    String tableName = Joiner.on('_').join(scan.getTable().getQualifiedName());
+    referredTables.add(tableName);
+    beginFunction(scan);
+    pw.print(String.format("  return _datasources[TABLE_%s].next();\n",
+                           tableName));
+    endFunction();
+    return null;
+  }
+
+  private void beginFunction(RelNode n) {
+    pw.print(String.format("private Values %s(%s) {\n", getFunctionName(n), n
+        .getInputs().isEmpty() ? "" : "Values _data"));
+  }
+
+  private void endFunction() {
+    pw.print("}\n");
+  }
+
+  static String getFunctionName(RelNode n) {
+    return n.getClass().getSimpleName() + "_" + n.getId();
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/bd4f6dc3/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestRelNodeCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestRelNodeCompiler.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestRelNodeCompiler.java
new file mode 100644
index 0000000..61f5409
--- /dev/null
+++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestRelNodeCompiler.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.compiler;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.rel.core.TableScan;
+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;
+
+public class TestRelNodeCompiler {
+  @Test
+  public void testFilter() throws Exception {
+    String sql = "SELECT ID + 1 FROM FOO WHERE ID > 3";
+    TestUtils.CalciteState state = TestUtils.sqlOverDummyTable(sql);
+    JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(
+        RelDataTypeSystem.DEFAULT);
+    LogicalProject project = (LogicalProject) state.tree;
+    LogicalFilter filter = (LogicalFilter) project.getInput();
+    TableScan scan = (TableScan) filter.getInput();
+
+    try (StringWriter sw = new StringWriter();
+         PrintWriter pw = new PrintWriter(sw)
+    ) {
+      RelNodeCompiler compiler = new RelNodeCompiler(pw, typeFactory);
+      compiler.visitTableScan(scan);
+      pw.flush();
+      Assert.assertTrue(sw.toString().contains("_datasources[TABLE_FOO]"));
+    }
+
+    try (StringWriter sw = new StringWriter();
+         PrintWriter pw = new PrintWriter(sw)
+    ) {
+      RelNodeCompiler compiler = new RelNodeCompiler(pw, typeFactory);
+      compiler.visitFilter(filter);
+      pw.flush();
+      Assert.assertTrue(sw.toString().contains("t0 > 3"));
+    }
+
+    try (StringWriter sw = new StringWriter();
+         PrintWriter pw = new PrintWriter(sw)
+    ) {
+      RelNodeCompiler compiler = new RelNodeCompiler(pw, typeFactory);
+      compiler.visitProject(project);
+      pw.flush();
+      Assert.assertTrue(sw.toString().contains("t0 + 1"));
+    }
+  }
+}


[12/20] storm git commit: [StormSQL] Allow deserializing Java class with custom classloaders in tests.

Posted by sr...@apache.org.
[StormSQL] Allow deserializing Java class with custom classloaders in tests.


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

Branch: refs/heads/master
Commit: 4d8cc419b109a11e1a1f6489aca8b8b528e77ab5
Parents: 3940007
Author: Haohui Mai <wh...@apache.org>
Authored: Wed Nov 11 10:03:07 2015 -0800
Committer: Haohui Mai <wh...@apache.org>
Committed: Fri Dec 4 11:09:27 2015 -0800

----------------------------------------------------------------------
 storm-core/src/jvm/backtype/storm/utils/Utils.java | 15 ++++++++++++++-
 1 file changed, 14 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/4d8cc419/storm-core/src/jvm/backtype/storm/utils/Utils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/utils/Utils.java b/storm-core/src/jvm/backtype/storm/utils/Utils.java
index e0bbb1f..19fe5f1 100644
--- a/storm-core/src/jvm/backtype/storm/utils/Utils.java
+++ b/storm-core/src/jvm/backtype/storm/utils/Utils.java
@@ -38,8 +38,10 @@ import backtype.storm.serialization.DefaultSerializationDelegate;
 import backtype.storm.serialization.SerializationDelegate;
 import clojure.lang.IFn;
 import clojure.lang.RT;
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
 import org.apache.commons.compress.archivers.tar.TarArchiveInputStream;
+import org.apache.commons.io.input.ClassLoaderObjectInputStream;
 import org.apache.commons.lang.StringUtils;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
@@ -118,6 +120,7 @@ public class Utils {
     private static ThreadLocal<TDeserializer> threadDes = new ThreadLocal<TDeserializer>();
 
     private static SerializationDelegate serializationDelegate;
+    private static ClassLoader cl = ClassLoader.getSystemClassLoader();
 
     static {
         Map conf = readStormConfig();
@@ -167,7 +170,7 @@ public class Utils {
     public static <T> T javaDeserialize(byte[] serialized, Class<T> clazz) {
         try {
             ByteArrayInputStream bis = new ByteArrayInputStream(serialized);
-            ObjectInputStream ois = new ObjectInputStream(bis);
+            ObjectInputStream ois = new ClassLoaderObjectInputStream(cl, bis);
             Object ret = ois.readObject();
             ois.close();
             return (T)ret;
@@ -1293,5 +1296,15 @@ public class Utils {
             return defaultValue;
         }
     }
+
+    @VisibleForTesting
+    public static void setClassLoaderForJavaDeSerialize(ClassLoader cl) {
+        Utils.cl = cl;
+    }
+
+    @VisibleForTesting
+    public static void resetClassLoaderForJavaDeSerialize() {
+        Utils.cl = ClassLoader.getSystemClassLoader();
+    }
 }
 


[03/20] storm git commit: [StormSQL] STORM-1045. Parse CREATE TABLE statements.

Posted by sr...@apache.org.
[StormSQL] STORM-1045. Parse CREATE TABLE statements.


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

Branch: refs/heads/master
Commit: 425280e23cc383366a96031ed459833ae837cbf8
Parents: 6babbb0
Author: Haohui Mai <wh...@apache.org>
Authored: Tue Sep 15 09:30:50 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Fri Dec 4 11:09:01 2015 -0800

----------------------------------------------------------------------
 external/sql/README.md                          |  22 ++
 external/sql/pom.xml                            |  42 ++++
 external/sql/storm-sql-core/pom.xml             | 216 +++++++++++++++++++
 .../sql/storm-sql-core/src/codegen/config.fmpp  |  23 ++
 .../storm-sql-core/src/codegen/data/Parser.tdd  |  63 ++++++
 .../src/codegen/includes/license.ftl            |  17 ++
 .../src/codegen/includes/parserImpls.ftl        |  80 +++++++
 .../storm/sql/parser/ColumnConstraint.java      |  36 ++++
 .../storm/sql/parser/ColumnDefinition.java      |  32 +++
 .../apache/storm/sql/parser/SqlCreateTable.java | 105 +++++++++
 .../apache/storm/sql/parser/StormParser.java    |  42 ++++
 .../apache/storm/sql/parser/UnparseUtil.java    |  60 ++++++
 .../apache/storm/sql/parser/TestSqlParser.java  |  42 ++++
 pom.xml                                         |   7 +
 14 files changed, 787 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/425280e2/external/sql/README.md
----------------------------------------------------------------------
diff --git a/external/sql/README.md b/external/sql/README.md
new file mode 100644
index 0000000..078acc6
--- /dev/null
+++ b/external/sql/README.md
@@ -0,0 +1,22 @@
+# Storm SQL
+
+Compile SQL queries to Storm topologies.
+
+## 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.

http://git-wip-us.apache.org/repos/asf/storm/blob/425280e2/external/sql/pom.xml
----------------------------------------------------------------------
diff --git a/external/sql/pom.xml b/external/sql/pom.xml
new file mode 100644
index 0000000..73e7b31
--- /dev/null
+++ b/external/sql/pom.xml
@@ -0,0 +1,42 @@
+<?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>0.11.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>
+    </modules>
+</project>

http://git-wip-us.apache.org/repos/asf/storm/blob/425280e2/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
new file mode 100644
index 0000000..3ca1ced
--- /dev/null
+++ b/external/sql/storm-sql-core/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">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <artifactId>storm</artifactId>
+        <groupId>org.apache.storm</groupId>
+        <version>0.11.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.calcite</groupId>
+            <artifactId>calcite-core</artifactId>
+            <version>${calcite.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <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>
+            <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.19</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/425280e2/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
new file mode 100644
index 0000000..be5a792
--- /dev/null
+++ b/external/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/425280e2/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
new file mode 100644
index 0000000..d026027
--- /dev/null
+++ b/external/sql/storm-sql-core/src/codegen/data/Parser.tdd
@@ -0,0 +1,63 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+# http:# www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT 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.util.*",
+    "org.apache.storm.sql.parser.*",
+    "java.util.*"
+  ]
+
+  # List of keywords.
+  keywords: [
+    "LOCATION",
+    "INPUTFORMAT",
+    "OUTPUTFORMAT",
+    "STORED",
+    "TBLPROPERTIES",
+  ]
+
+  # List of methods for parsing custom SQL statements.
+  statementParserMethods: [
+    "SqlCreateTable()"
+  ]
+
+  # List of methods for parsing custom literals.
+  # Example: ParseJsonLiteral().
+  literalParserMethods: [
+  ]
+
+  # List of methods for parsing custom data types.
+  dataTypeParserMethods: [
+  ]
+
+  # 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
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/425280e2/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
new file mode 100644
index 0000000..7e66353
--- /dev/null
+++ b/external/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/425280e2/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
new file mode 100644
index 0000000..2d87d7f
--- /dev/null
+++ b/external/sql/storm-sql-core/src/codegen/includes/parserImpls.ftl
@@ -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. -->
+
+
+private void ColumnDef(List<ColumnDefinition> list) :
+{
+    SqlParserPos pos;
+    SqlIdentifier name;
+    SqlDataTypeSpec type;
+    ColumnConstraint constraint = null;
+}
+{
+    name = SimpleIdentifier() { pos = getPos(); }
+    type = DataType()
+    [ <PRIMARY> <KEY> { constraint = new ColumnConstraint.PrimaryKey
+    (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 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()
+    [ <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,
+        tbl_properties, select);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/425280e2/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
new file mode 100644
index 0000000..6daf6d3
--- /dev/null
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnConstraint.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.sql.parser;
+
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+public class ColumnConstraint extends SqlLiteral {
+  private ColumnConstraint(
+      Object value, SqlTypeName typeName, SqlParserPos pos) {
+    super(value, typeName, pos);
+  }
+
+  public static class PrimaryKey extends ColumnConstraint {
+    public PrimaryKey(SqlParserPos pos) {
+      super("PRIMARY", SqlTypeName.SYMBOL, pos);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/425280e2/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
new file mode 100644
index 0000000..27f2e57
--- /dev/null
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/ColumnDefinition.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/425280e2/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
new file mode 100644
index 0000000..e81d146
--- /dev/null
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/SqlCreateTable.java
@@ -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
+ * <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.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.util.List;
+
+public class SqlCreateTable extends SqlCall {
+  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]);
+    }
+
+    @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.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 properties;
+  private final SqlNode query;
+
+  public SqlCreateTable(
+      SqlParserPos pos, SqlIdentifier tblName, SqlNodeList fieldList,
+      SqlNode inputFormatClass, SqlNode outputFormatClass, SqlNode location,
+      SqlNode properties, SqlNode query) {
+    super(pos);
+    this.tblName = tblName;
+    this.fieldList = fieldList;
+    this.inputFormatClass = inputFormatClass;
+    this.outputFormatClass = outputFormatClass;
+    this.location = location;
+    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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/425280e2/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
new file mode 100644
index 0000000..9c74f28
--- /dev/null
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/StormParser.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
+ * <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);
+  }
+
+  @VisibleForTesting
+  StormParserImpl impl() {
+    return impl;
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/425280e2/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
new file mode 100644
index 0000000..834fe7c
--- /dev/null
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/parser/UnparseUtil.java
@@ -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.
+ */
+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/425280e2/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
new file mode 100644
index 0000000..41e031d
--- /dev/null
+++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/parser/TestSqlParser.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
+ * <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(expected = ParseException.class)
+  public void testCreateTableWithoutLocation() throws Exception {
+    String sql = "CREATE EXTERNAL TABLE foo (bar INT)";
+    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/425280e2/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index b787e8e..3387583 100644
--- a/pom.xml
+++ b/pom.xml
@@ -227,6 +227,7 @@
         <junit.version>4.11</junit.version>
         <metrics-clojure.version>2.5.1</metrics-clojure.version>
         <hdrhistogram.version>2.1.7</hdrhistogram.version>
+        <calcite.version>1.4.0-incubating</calcite.version>
     </properties>
 
     <modules>
@@ -244,6 +245,7 @@
         <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>
@@ -687,6 +689,11 @@
                 <version>${junit.version}</version>
                 <scope>test</scope>
             </dependency>
+            <dependency>
+              <groupId>org.apache.calcite</groupId>
+              <artifactId>calcite-core</artifactId>
+              <version>${calcite.version}</version>
+            </dependency>
         </dependencies>
     </dependencyManagement>
 


[05/20] storm git commit: [StormSQL] STORM-1149. Support pluggable data sources in CREATE TABLE.

Posted by sr...@apache.org.
[StormSQL] STORM-1149. Support pluggable data sources in CREATE TABLE.


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

Branch: refs/heads/master
Commit: 0bb8e46cea7fa162c1fd16203e1e415e13636fb1
Parents: 915f135
Author: Haohui Mai <wh...@apache.org>
Authored: Mon Nov 2 15:35:29 2015 -0800
Committer: Haohui Mai <wh...@apache.org>
Committed: Fri Dec 4 11:09:02 2015 -0800

----------------------------------------------------------------------
 .../apache/storm/sql/DataSourcesProvider.java   |  47 +++++++++
 .../apache/storm/sql/DataSourcesRegistry.java   |  66 ++++++++++++
 .../src/jvm/org/apache/storm/sql/StormSql.java  |  42 ++++++++
 .../jvm/org/apache/storm/sql/StormSqlImpl.java  | 100 +++++++++++++++++++
 .../storm/sql/parser/ColumnDefinition.java      |  12 +++
 .../apache/storm/sql/parser/SqlCreateTable.java |  25 +++++
 .../apache/storm/sql/parser/StormParser.java    |   2 +-
 .../test/org/apache/storm/sql/TestStormSql.java |  73 ++++++++++++++
 8 files changed, 366 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/0bb8e46c/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/DataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/DataSourcesProvider.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/DataSourcesProvider.java
new file mode 100644
index 0000000..cc4874b
--- /dev/null
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/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
+ * <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 backtype.storm.tuple.Values;
+import org.apache.storm.sql.storm.DataSource;
+
+import java.net.URI;
+import java.util.List;
+import java.util.Map;
+
+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<Map.Entry<String, Class<?>>> fields);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/0bb8e46c/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/DataSourcesRegistry.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/DataSourcesRegistry.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/DataSourcesRegistry.java
new file mode 100644
index 0000000..a841609
--- /dev/null
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/DataSourcesRegistry.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.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.annotations.VisibleForTesting;
+import org.apache.storm.sql.storm.DataSource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URI;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+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<Map.Entry<String, Class<?>>> fields) {
+    DataSourcesProvider provider = providers.get(uri.getScheme());
+    if (provider == null) {
+      return null;
+    }
+
+    return provider.construct(uri, inputFormatClass, outputFormatClass, fields);
+  }
+
+  @VisibleForTesting
+  static Map<String, DataSourcesProvider> providerMap() {
+    return providers;
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/0bb8e46c/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
new file mode 100644
index 0000000..477e633
--- /dev/null
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSql.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
+ * <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 backtype.storm.tuple.Values;
+import org.apache.storm.sql.storm.ChannelHandler;
+import org.apache.storm.sql.storm.DataSource;
+
+import java.util.Map;
+
+/**
+ * The StormSql class provides standalone, interactive interfaces to execute
+ * SQL statements over streaming data.
+ *
+ * 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 {
+  public abstract void execute(Iterable<String> statements,
+      ChannelHandler handler) throws Exception;
+
+  public static StormSql construct() {
+    return new StormSqlImpl();
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/storm/blob/0bb8e46c/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
new file mode 100644
index 0000000..136bc88
--- /dev/null
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlImpl.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
+ * <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.JavaTypeFactoryImpl;
+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.SchemaPlus;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.tools.FrameworkConfig;
+import org.apache.calcite.tools.Frameworks;
+import org.apache.calcite.tools.Planner;
+import org.apache.storm.sql.compiler.PlanCompiler;
+import org.apache.storm.sql.parser.ColumnDefinition;
+import org.apache.storm.sql.parser.SqlCreateTable;
+import org.apache.storm.sql.parser.StormParser;
+import org.apache.storm.sql.storm.ChannelHandler;
+import org.apache.storm.sql.storm.DataSource;
+import org.apache.storm.sql.storm.runtime.AbstractValuesProcessor;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.storm.sql.compiler.CompilerUtil.TableBuilderInfo;
+
+class StormSqlImpl extends StormSql {
+  private final JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(
+      RelDataTypeSystem.DEFAULT);
+  private final SchemaPlus schema = Frameworks.createRootSchema(true);
+
+  @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 {
+        FrameworkConfig config = Frameworks.newConfigBuilder().defaultSchema(
+            schema).build();
+        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);
+      }
+    }
+  }
+
+  private void handleCreateTable(
+      SqlCreateTable n, Map<String, DataSource> dataSources) {
+    TableBuilderInfo builder = new TableBuilderInfo(typeFactory);
+    List<Map.Entry<String, Class<?>>> fields = new ArrayList<>();
+    for (ColumnDefinition col : n.fieldList()) {
+      builder.field(col.name(), col.type());
+      RelDataType dataType = col.type().deriveType(typeFactory);
+      Class<?> javaType = (Class<?>)typeFactory.getJavaClass(dataType);
+      fields.add(new AbstractMap.SimpleImmutableEntry<String, Class<?>>
+                     (col.name(), javaType));
+    }
+    Table table = builder.build();
+    schema.add(n.tableName(), table);
+    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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/0bb8e46c/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
index 27f2e57..3520b86 100644
--- 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
@@ -29,4 +29,16 @@ public class ColumnDefinition extends SqlNodeList {
       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/0bb8e46c/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
index e81d146..8fe4160 100644
--- 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
@@ -29,6 +29,7 @@ import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.util.ImmutableNullableList;
 
+import java.net.URI;
 import java.util.List;
 
 public class SqlCreateTable extends SqlCall {
@@ -102,4 +103,28 @@ public class SqlCreateTable extends SqlCall {
                                     outputFormatClass, location, properties,
                                     query);
   }
+
+  public String tableName() {
+    return tblName.toString();
+  }
+
+  public URI location() {
+    return URI.create(SqlLiteral.stringValue(location));
+  }
+
+  public String inputFormatClass() {
+    return inputFormatClass == null ? null : SqlLiteral.stringValue(
+        inputFormatClass);
+  }
+
+  public String outputFormatClass() {
+    return outputFormatClass == null ? null : SqlLiteral.stringValue
+        (outputFormatClass);
+  }
+
+  @SuppressWarnings("unchecked")
+  public List<ColumnDefinition> fieldList() {
+    return (List<ColumnDefinition>)((List<? extends SqlNode>)fieldList.getList());
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/0bb8e46c/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
index 9c74f28..670901e 100644
--- 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
@@ -36,7 +36,7 @@ public class StormParser {
   }
 
   @VisibleForTesting
-  StormParserImpl impl() {
+  public StormParserImpl impl() {
     return impl;
   }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/0bb8e46c/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
new file mode 100644
index 0000000..07b367f
--- /dev/null
+++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/TestStormSql.java
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.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 backtype.storm.tuple.Values;
+import org.apache.storm.sql.compiler.TestUtils;
+import org.apache.storm.sql.storm.ChannelHandler;
+import org.apache.storm.sql.storm.DataSource;
+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.List;
+import java.util.Map;
+
+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<Map.Entry<String, Class<?>>> fields) {
+      return new TestUtils.MockDataSource();
+    }
+  }
+
+  @BeforeClass
+  public static void setUp() {
+    DataSourcesRegistry.providerMap().put("mock", new MockDataSourceProvider());
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    DataSourcesRegistry.providerMap().remove("mock");
+  }
+
+  @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 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));
+  }
+
+}


[20/20] storm git commit: Added STORM-1040 to CHANGELOG.

Posted by sr...@apache.org.
Added STORM-1040 to CHANGELOG.


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

Branch: refs/heads/master
Commit: 39163bfcee409c32f6a6b952ed39104cd6b0805d
Parents: 6e65ae2
Author: Sriharsha Chintalapani <ha...@hortonworks.com>
Authored: Tue Dec 8 09:37:35 2015 -0800
Committer: Sriharsha Chintalapani <ha...@hortonworks.com>
Committed: Tue Dec 8 09:37:35 2015 -0800

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


http://git-wip-us.apache.org/repos/asf/storm/blob/39163bfc/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index cc8937c..7fea87e 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
 ## 0.11.0
+ * STORM-1040: SQL support for Storm.
  * STORM-1364: Log storm version on daemon start
  * STORM-1375: Blobstore broke Pacemaker
  * STORM-876: Blobstore/DistCache Support


[08/20] storm git commit: [StormSQL] STORM-1173. Support string operations in StormSQL.

Posted by sr...@apache.org.
[StormSQL] STORM-1173. Support string operations in StormSQL.


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

Branch: refs/heads/master
Commit: 39400075b0e83416f84879b8785be1b1cc0c2826
Parents: beeaee7
Author: Haohui Mai <wh...@apache.org>
Authored: Wed Nov 4 15:51:11 2015 -0800
Committer: Haohui Mai <wh...@apache.org>
Committed: Fri Dec 4 11:09:02 2015 -0800

----------------------------------------------------------------------
 .../apache/storm/sql/DataSourcesProvider.java   |   3 +-
 .../apache/storm/sql/DataSourcesRegistry.java   |   3 +-
 .../src/jvm/org/apache/storm/sql/StormSql.java  |   6 +-
 .../jvm/org/apache/storm/sql/StormSqlImpl.java  |   6 +-
 .../apache/storm/sql/compiler/ExprCompiler.java | 205 ++++++++++++-------
 .../apache/storm/sql/compiler/PlanCompiler.java |  14 +-
 .../test/org/apache/storm/sql/TestStormSql.java |   4 +-
 .../storm/sql/compiler/TestExprCompiler.java    |   6 +-
 .../storm/sql/compiler/TestExprSemantic.java    |  28 ++-
 .../storm/sql/compiler/TestPlanCompiler.java    |   6 +-
 .../storm/sql/compiler/TestRelNodeCompiler.java |   2 +-
 .../apache/storm/sql/compiler/TestUtils.java    |   6 +-
 .../sql/runtime/AbstractChannelHandler.java     |  37 ++++
 .../sql/runtime/AbstractValuesProcessor.java    |  49 +++++
 .../storm/sql/runtime/ChannelContext.java       |  30 +++
 .../storm/sql/runtime/ChannelHandler.java       |  39 ++++
 .../org/apache/storm/sql/runtime/Channels.java  |  80 ++++++++
 .../apache/storm/sql/runtime/DataSource.java    |  29 +++
 .../storm/sql/runtime/StormSqlFunctions.java    |  36 ++++
 .../storm/sql/storm/AbstractChannelHandler.java |  35 ----
 .../apache/storm/sql/storm/ChannelContext.java  |  28 ---
 .../apache/storm/sql/storm/ChannelHandler.java  |  37 ----
 .../org/apache/storm/sql/storm/Channels.java    |  78 -------
 .../org/apache/storm/sql/storm/DataSource.java  |  27 ---
 .../storm/runtime/AbstractValuesProcessor.java  |  29 ---
 25 files changed, 477 insertions(+), 346 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/39400075/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/DataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/DataSourcesProvider.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/DataSourcesProvider.java
index cc4874b..46bfa40 100644
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/DataSourcesProvider.java
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/DataSourcesProvider.java
@@ -18,8 +18,7 @@
 
 package org.apache.storm.sql;
 
-import backtype.storm.tuple.Values;
-import org.apache.storm.sql.storm.DataSource;
+import org.apache.storm.sql.runtime.DataSource;
 
 import java.net.URI;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/storm/blob/39400075/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/DataSourcesRegistry.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/DataSourcesRegistry.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/DataSourcesRegistry.java
index a841609..b45d039 100644
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/DataSourcesRegistry.java
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/DataSourcesRegistry.java
@@ -19,12 +19,11 @@
 package org.apache.storm.sql;
 
 import com.google.common.annotations.VisibleForTesting;
-import org.apache.storm.sql.storm.DataSource;
+import org.apache.storm.sql.runtime.DataSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.net.URI;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;

http://git-wip-us.apache.org/repos/asf/storm/blob/39400075/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
index 477e633..6859f8e 100644
--- 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
@@ -17,11 +17,7 @@
  */
 package org.apache.storm.sql;
 
-import backtype.storm.tuple.Values;
-import org.apache.storm.sql.storm.ChannelHandler;
-import org.apache.storm.sql.storm.DataSource;
-
-import java.util.Map;
+import org.apache.storm.sql.runtime.ChannelHandler;
 
 /**
  * The StormSql class provides standalone, interactive interfaces to execute

http://git-wip-us.apache.org/repos/asf/storm/blob/39400075/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
index 136bc88..384b4fa 100644
--- 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
@@ -32,9 +32,9 @@ import org.apache.storm.sql.compiler.PlanCompiler;
 import org.apache.storm.sql.parser.ColumnDefinition;
 import org.apache.storm.sql.parser.SqlCreateTable;
 import org.apache.storm.sql.parser.StormParser;
-import org.apache.storm.sql.storm.ChannelHandler;
-import org.apache.storm.sql.storm.DataSource;
-import org.apache.storm.sql.storm.runtime.AbstractValuesProcessor;
+import org.apache.storm.sql.runtime.ChannelHandler;
+import org.apache.storm.sql.runtime.DataSource;
+import org.apache.storm.sql.runtime.AbstractValuesProcessor;
 
 import java.util.AbstractMap;
 import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/storm/blob/39400075/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java
index 6617ef6..77fdf0c 100644
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java
@@ -17,48 +17,33 @@
  */
 package org.apache.storm.sql.compiler;
 
+import com.google.common.base.Joiner;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import org.apache.calcite.adapter.enumerable.NullPolicy;
 import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.tree.Primitive;
 import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexCorrelVariable;
-import org.apache.calcite.rex.RexDynamicParam;
-import org.apache.calcite.rex.RexFieldAccess;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexLocalRef;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexOver;
-import org.apache.calcite.rex.RexRangeRef;
-import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.*;
+import org.apache.calcite.runtime.SqlFunctions;
 import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.BuiltInMethod;
 import org.apache.calcite.util.NlsString;
 import org.apache.calcite.util.Util;
+import org.apache.storm.sql.runtime.StormSqlFunctions;
 
 import java.io.PrintWriter;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
 import java.lang.reflect.Type;
 import java.math.BigDecimal;
 import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Map;
 
-import static org.apache.calcite.sql.fun.SqlStdOperatorTable.AND;
-import static org.apache.calcite.sql.fun.SqlStdOperatorTable.DIVIDE;
-import static org.apache.calcite.sql.fun.SqlStdOperatorTable.DIVIDE_INTEGER;
-import static org.apache.calcite.sql.fun.SqlStdOperatorTable.GREATER_THAN;
-import static org.apache.calcite.sql.fun.SqlStdOperatorTable.GREATER_THAN_OR_EQUAL;
-import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_FALSE;
-import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_FALSE;
-import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_NULL;
-import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_TRUE;
-import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NULL;
-import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_TRUE;
-import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LESS_THAN;
-import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LESS_THAN_OR_EQUAL;
-import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MINUS;
-import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MULTIPLY;
-import static org.apache.calcite.sql.fun.SqlStdOperatorTable.NOT;
-import static org.apache.calcite.sql.fun.SqlStdOperatorTable.OR;
-import static org.apache.calcite.sql.fun.SqlStdOperatorTable.PLUS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.*;
 
 /**
  * Compile RexNode on top of the Tuple abstraction.
@@ -98,7 +83,7 @@ class ExprCompiler implements RexVisitor<String> {
       case CHAR:
         return CompilerUtil.escapeJavaString(((NlsString) v).getValue(), true);
       case NULL:
-        return "null";
+        return "((" + ((Class<?>)typeFactory.getJavaClass(ty)).getCanonicalName() + ")null)";
       case DOUBLE:
       case BIGINT:
       case DECIMAL:
@@ -164,6 +149,13 @@ class ExprCompiler implements RexVisitor<String> {
     return "t" + ++nameCount;
   }
 
+  // Only generate inline expressions when comparing primitive types
+  private boolean primitiveCompareExpr(SqlOperator op, RelDataType type) {
+    final Primitive primitive = Primitive.ofBoxOr(typeFactory.getJavaClass(type));
+    return primitive != null &&
+        (op == LESS_THAN || op == LESS_THAN_OR_EQUAL || op == GREATER_THAN || op == GREATER_THAN_OR_EQUAL);
+  }
+
   private interface CallExprPrinter {
     String translate(ExprCompiler compiler, RexCall call);
   }
@@ -179,15 +171,25 @@ class ExprCompiler implements RexVisitor<String> {
     private ImpTable() {
       ImmutableMap.Builder<SqlOperator, CallExprPrinter> builder =
           ImmutableMap.builder();
-      builder.put(infixBinary(LESS_THAN, "<"))
-          .put(infixBinary(LESS_THAN_OR_EQUAL, "<="))
-          .put(infixBinary(GREATER_THAN, ">"))
-          .put(infixBinary(GREATER_THAN_OR_EQUAL, ">="))
-          .put(infixBinary(PLUS, "+"))
-          .put(infixBinary(MINUS, "-"))
-          .put(infixBinary(MULTIPLY, "*"))
-          .put(infixBinary(DIVIDE, "/"))
-          .put(infixBinary(DIVIDE_INTEGER, "/"))
+      builder
+          .put(builtInMethod(UPPER, BuiltInMethod.UPPER, NullPolicy.STRICT))
+          .put(builtInMethod(LOWER, BuiltInMethod.LOWER, NullPolicy.STRICT))
+          .put(builtInMethod(INITCAP, BuiltInMethod.INITCAP, NullPolicy.STRICT))
+          .put(builtInMethod(SUBSTRING, BuiltInMethod.SUBSTRING, NullPolicy.STRICT))
+          .put(builtInMethod(CHARACTER_LENGTH, BuiltInMethod.CHAR_LENGTH, NullPolicy.STRICT))
+          .put(builtInMethod(CHAR_LENGTH, BuiltInMethod.CHAR_LENGTH, NullPolicy.STRICT))
+          .put(builtInMethod(CONCAT, BuiltInMethod.STRING_CONCAT, NullPolicy.STRICT))
+          .put(infixBinary(LESS_THAN, "<", "lt"))
+          .put(infixBinary(LESS_THAN_OR_EQUAL, "<=", "le"))
+          .put(infixBinary(GREATER_THAN, ">", "gt"))
+          .put(infixBinary(GREATER_THAN_OR_EQUAL, ">=", "ge"))
+          .put(infixBinary(EQUALS, "==", StormSqlFunctions.class, "eq"))
+          .put(infixBinary(NOT_EQUALS, "<>", StormSqlFunctions.class, "ne"))
+          .put(infixBinary(PLUS, "+", "plus"))
+          .put(infixBinary(MINUS, "-", "minus"))
+          .put(infixBinary(MULTIPLY, "*", "multiply"))
+          .put(infixBinary(DIVIDE, "/", "divide"))
+          .put(infixBinary(DIVIDE_INTEGER, "/", "divide"))
           .put(expect(IS_NULL, null))
           .put(expectNot(IS_NOT_NULL, null))
           .put(expect(IS_TRUE, true))
@@ -210,8 +212,38 @@ class ExprCompiler implements RexVisitor<String> {
       }
     }
 
+    private Map.Entry<SqlOperator, CallExprPrinter> builtInMethod(
+        final SqlOperator op, final BuiltInMethod method, NullPolicy nullPolicy) {
+      if (nullPolicy != NullPolicy.STRICT) {
+        throw new UnsupportedOperationException();
+      }
+      CallExprPrinter printer = new CallExprPrinter() {
+        @Override
+        public String translate(ExprCompiler compiler, RexCall call) {
+          PrintWriter pw = compiler.pw;
+          String val = compiler.reserveName();
+          pw.print(String.format("final %s %s;\n", compiler.javaTypeName(call), val));
+          List<String> args = new ArrayList<>();
+          for (RexNode op : call.getOperands()) {
+            args.add(op.accept(compiler));
+          }
+          pw.print("if (false) {}\n");
+          for (int i = 0; i < args.size(); ++i) {
+            String arg = args.get(i);
+            if (call.getOperands().get(i).getType().isNullable()) {
+              pw.print(String.format("else if (%2$s == null) { %1$s = null; }\n", val, arg));
+            }
+          }
+          String calc = printMethodCall(method.method, args);
+          pw.print(String.format("else { %1$s = %2$s; }\n", val, calc));
+          return val;
+        }
+      };
+      return new AbstractMap.SimpleImmutableEntry<>(op, printer);
+    }
+
     private Map.Entry<SqlOperator, CallExprPrinter> infixBinary
-        (SqlOperator op, final String javaOperator) {
+        (final SqlOperator op, final String javaOperator, final Class<?> clazz, final String backupMethodName) {
       CallExprPrinter trans = new CallExprPrinter() {
         @Override
         public String translate(
@@ -221,40 +253,44 @@ class ExprCompiler implements RexVisitor<String> {
           String val = compiler.reserveName();
           RexNode op0 = call.getOperands().get(0);
           RexNode op1 = call.getOperands().get(1);
+          PrintWriter pw = compiler.pw;
+          if (backupMethodName != null) {
+            if (!compiler.primitiveCompareExpr(op, op0.getType())) {
+              String lhs = op0.accept(compiler);
+              String rhs = op1.accept(compiler);
+              pw.print(String.format("%s %s = %s;\n", compiler.javaTypeName(call), val,
+                  printMethodCall(clazz, backupMethodName, true, Lists.newArrayList(lhs, rhs))));
+              return val;
+            }
+          }
           boolean lhsNullable = op0.getType().isNullable();
           boolean rhsNullable = op1.getType().isNullable();
 
-          PrintWriter pw = compiler.pw;
-          String lhs = op0.accept(compiler);
           pw.print(String.format("final %s %s;\n", compiler.javaTypeName(call), val));
-          if (!lhsNullable) {
-            String rhs = op1.accept(compiler);
-            String calc = foldNullExpr(String.format("%s %s %s", lhs, javaOperator, rhs), "null", rhs);
-            if (!rhsNullable) {
-              pw.print(String.format("%s = %s;\n", val, calc));
-            } else {
-              pw.print(
-                  String.format("%s = %s == null ? null : (%s);\n",
-                      val, rhs, calc));
-            }
-          } else {
-            pw.print(String.format("if (%2$s == null) { %1$s = null; }\n",
-                val, lhs));
-            pw.print("else {\n");
-            String rhs = op1.accept(compiler);
-            String calc = foldNullExpr(String.format("%s %s %s", lhs, javaOperator, rhs), "null", lhs);
-            if (!rhsNullable) {
-              pw.print(String.format("%s = %s;\n}\n", val, calc));
-            } else {
-              pw.print(String.format("%1$s = %2$s == null ? null : (%3$s);\n}\n", val, rhs, calc));
-            }
+          String lhs = op0.accept(compiler);
+          String rhs = op1.accept(compiler);
+          pw.print("if (false) {}\n");
+          if (lhsNullable) {
+            String calc = foldNullExpr(String.format("%s %s %s", lhs, javaOperator, rhs), "null", op1);
+            pw.print(String.format("else if (%2$s == null) { %1$s = %3$s; }\n", val, lhs, calc));
           }
+          if (rhsNullable) {
+            String calc = foldNullExpr(String.format("%s %s %s", lhs, javaOperator, rhs), "null", op0);
+            pw.print(String.format("else if (%2$s == null) { %1$s = %3$s; }\n", val, rhs, calc));
+          }
+          String calc = String.format("%s %s %s", lhs, javaOperator, rhs);
+          pw.print(String.format("else { %1$s = %2$s; }\n", val, calc));
           return val;
         }
       };
       return new AbstractMap.SimpleImmutableEntry<>(op, trans);
     }
 
+    private Map.Entry<SqlOperator, CallExprPrinter> infixBinary
+        (final SqlOperator op, final String javaOperator, final String backupMethodName) {
+      return infixBinary(op, javaOperator, SqlFunctions.class, backupMethodName);
+    }
+
     private Map.Entry<SqlOperator, CallExprPrinter> expect(
         SqlOperator op, final Boolean expect) {
       return expect0(op, expect, false);
@@ -327,17 +363,16 @@ class ExprCompiler implements RexVisitor<String> {
           pw.print(String.format("  %1$s = %2$s;\n}\n", val, rhs));
         } else {
           String foldedLHS = foldNullExpr(
-              String.format("%1$s == null || %1$s", lhs), "true", lhs);
+              String.format("%1$s == null || %1$s", lhs), "true", op0);
           pw.print(String.format("if (%s) {\n", foldedLHS));
           String rhs = op1.accept(compiler);
           String s;
           if (rhsNullable) {
             s = foldNullExpr(
                 String.format("(%2$s != null && !(%2$s)) ? false : %1$s", lhs,
-                              rhs),
-                "null", rhs);
+                    rhs), "null", op1);
           } else {
-            s = String.format("!(%2$s) ? false : %1$s", lhs, rhs);
+            s = String.format("!(%2$s) ? Boolean.FALSE : %1$s", lhs, rhs);
           }
           pw.print(String.format("  %1$s = %2$s;\n", val, s));
           pw.print(String.format("} else { %1$s = false; }\n", val));
@@ -369,16 +404,16 @@ class ExprCompiler implements RexVisitor<String> {
           pw.print(String.format("  %1$s = %2$s;\n}\n", val, rhs));
         } else {
           String foldedLHS = foldNullExpr(
-              String.format("%1$s == null || !(%1$s)", lhs), "true", lhs);
+              String.format("%1$s == null || !(%1$s)", lhs), "true", op0);
           pw.print(String.format("if (%s) {\n", foldedLHS));
           String rhs = op1.accept(compiler);
           String s;
           if (rhsNullable) {
             s = foldNullExpr(
                 String.format("(%2$s != null && %2$s) ? true : %1$s", lhs, rhs),
-                "null", rhs);
+                "null", op1);
           } else {
-            s = String.format("%2$s ? %2$s : %1$s", lhs, rhs);
+            s = String.format("%2$s ? Boolean.valueOf(%2$s) : %1$s", lhs, rhs);
           }
           pw.print(String.format("  %1$s = %2$s;\n", val, s));
           pw.print(String.format("} else { %1$s = true; }\n", val));
@@ -402,20 +437,34 @@ class ExprCompiler implements RexVisitor<String> {
           pw.print(String.format("%1$s = !(%2$s);\n", val, lhs));
         } else {
           String s = foldNullExpr(
-              String.format("%1$s == null ? null : !(%1$s)", lhs), "null", lhs);
+              String.format("%1$s == null ? null : !(%1$s)", lhs), "null", op);
           pw.print(String.format("%1$s = %2$s;\n", val, s));
         }
         return val;
       }
     };
+  }
 
-    private static String foldNullExpr(String notNullExpr, String
-        nullExpr, String op) {
-      if (op.equals("null")) {
-        return nullExpr;
-      } else {
-        return notNullExpr;
-      }
+  private static String foldNullExpr(String notNullExpr, String
+      nullExpr, RexNode op) {
+    if (op instanceof RexLiteral && ((RexLiteral)op).getTypeName() == SqlTypeName.NULL) {
+      return nullExpr;
+    } else {
+      return notNullExpr;
+    }
+  }
+
+  private 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/39400075/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/PlanCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/PlanCompiler.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/PlanCompiler.java
index d006261..1096f5b 100644
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/PlanCompiler.java
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/PlanCompiler.java
@@ -22,7 +22,7 @@ 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.javac.CompilingClassLoader;
-import org.apache.storm.sql.storm.runtime.AbstractValuesProcessor;
+import org.apache.storm.sql.runtime.AbstractValuesProcessor;
 
 import java.io.PrintWriter;
 import java.io.StringWriter;
@@ -38,12 +38,12 @@ public class PlanCompiler {
       "// GENERATED CODE", "package " + PACKAGE_NAME + ";", "",
       "import java.util.Iterator;", "import java.util.Map;",
       "import backtype.storm.tuple.Values;",
-      "import org.apache.storm.sql.storm.AbstractChannelHandler;",
-      "import org.apache.storm.sql.storm.Channels;",
-      "import org.apache.storm.sql.storm.ChannelContext;",
-      "import org.apache.storm.sql.storm.ChannelHandler;",
-      "import org.apache.storm.sql.storm.DataSource;",
-      "import org.apache.storm.sql.storm.runtime.AbstractValuesProcessor;",
+      "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;",
       "public final class Processor extends AbstractValuesProcessor {", "");
   private static final String INITIALIZER_PROLOGUE = NEW_LINE_JOINER.join(
       "  @Override",

http://git-wip-us.apache.org/repos/asf/storm/blob/39400075/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
index 07b367f..e18b9f8 100644
--- 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
@@ -19,8 +19,8 @@ package org.apache.storm.sql;
 
 import backtype.storm.tuple.Values;
 import org.apache.storm.sql.compiler.TestUtils;
-import org.apache.storm.sql.storm.ChannelHandler;
-import org.apache.storm.sql.storm.DataSource;
+import org.apache.storm.sql.runtime.ChannelHandler;
+import org.apache.storm.sql.runtime.DataSource;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;

http://git-wip-us.apache.org/repos/asf/storm/blob/39400075/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprCompiler.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprCompiler.java
index f03cff8..a5f9d67 100644
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprCompiler.java
+++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprCompiler.java
@@ -86,8 +86,8 @@ public class TestExprCompiler {
       }
     }
     assertThat(sw.get(0).toString(), containsString("1 > 2"));
-    assertThat(sw.get(1).toString(), containsString("3 + 5"));
-    assertThat(sw.get(2).toString(), containsString("1 - 1.0E0"));
-    assertThat(sw.get(3).toString(), containsString("3 +"));
+    assertThat(sw.get(1).toString(), containsString("plus(3,5)"));
+    assertThat(sw.get(2).toString(), containsString("minus(1,1.0E0)"));
+    assertThat(sw.get(3).toString(), containsString("plus(3,"));
   }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/39400075/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
index 8454d7e..1d98664 100644
--- 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
@@ -23,9 +23,9 @@ 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.storm.ChannelHandler;
-import org.apache.storm.sql.storm.DataSource;
-import org.apache.storm.sql.storm.runtime.AbstractValuesProcessor;
+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 java.util.ArrayList;
@@ -99,6 +99,28 @@ public class TestExprSemantic {
                             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 testStringMethods() throws Exception {
+    Values v = testExpr(
+        Lists.newArrayList(
+            "UPPER('a')", "LOWER('A')", "INITCAP('foo')",
+            "SUBSTRING('foo', 2)", "CHARACTER_LENGTH('foo')", "CHAR_LENGTH('foo')",
+            "'ab' || 'cd'"
+        ));
+    assertEquals(new Values("A", "a", "Foo", "oo", 3, 3, "abcd"), v);
+  }
+
   private Values testExpr(List<String> exprs) throws Exception {
     String sql = "SELECT " + Joiner.on(',').join(exprs) + " FROM FOO" +
         " WHERE ID > 0 AND ID < 2";

http://git-wip-us.apache.org/repos/asf/storm/blob/39400075/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestPlanCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestPlanCompiler.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestPlanCompiler.java
index 30df0f3..d32fdca 100644
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestPlanCompiler.java
+++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestPlanCompiler.java
@@ -21,9 +21,9 @@ import backtype.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.storm.ChannelHandler;
-import org.apache.storm.sql.storm.DataSource;
-import org.apache.storm.sql.storm.runtime.AbstractValuesProcessor;
+import org.apache.storm.sql.runtime.ChannelHandler;
+import org.apache.storm.sql.runtime.DataSource;
+import org.apache.storm.sql.runtime.AbstractValuesProcessor;
 import org.junit.Assert;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/storm/blob/39400075/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestRelNodeCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestRelNodeCompiler.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestRelNodeCompiler.java
index d820f22..623a2f4 100644
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestRelNodeCompiler.java
+++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestRelNodeCompiler.java
@@ -57,7 +57,7 @@ public class TestRelNodeCompiler {
       RelNodeCompiler compiler = new RelNodeCompiler(pw, typeFactory);
       compiler.visitProject(project);
       pw.flush();
-      Assert.assertThat(sw.toString(), containsString("+ 1"));
+      Assert.assertThat(sw.toString(), containsString("plus("));
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/39400075/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestUtils.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestUtils.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestUtils.java
index 6731c90..5aa4cb0 100644
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestUtils.java
+++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestUtils.java
@@ -15,9 +15,9 @@ 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.storm.ChannelContext;
-import org.apache.storm.sql.storm.ChannelHandler;
-import org.apache.storm.sql.storm.DataSource;
+import org.apache.storm.sql.runtime.ChannelContext;
+import org.apache.storm.sql.runtime.ChannelHandler;
+import org.apache.storm.sql.runtime.DataSource;
 
 import java.util.ArrayList;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/storm/blob/39400075/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
new file mode 100644
index 0000000..73a078c
--- /dev/null
+++ b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractChannelHandler.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.runtime;
+
+import backtype.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) {
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/39400075/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
new file mode 100644
index 0000000..11aa065
--- /dev/null
+++ b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/AbstractValuesProcessor.java
@@ -0,0 +1,49 @@
+/*
+ * *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.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 backtype.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/39400075/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
new file mode 100644
index 0000000..71aba03
--- /dev/null
+++ b/external/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
+ *  * <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 backtype.storm.tuple.Values;
+
+public interface ChannelContext {
+  /**
+   * Emit data to the next stage of the data pipeline.
+   */
+  void emit(Values data);
+  void fireChannelInactive();
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/39400075/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
new file mode 100644
index 0000000..117f312
--- /dev/null
+++ b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ChannelHandler.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;
+
+import backtype.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);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/39400075/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
new file mode 100644
index 0000000..7214f9a
--- /dev/null
+++ b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/Channels.java
@@ -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
+ *  * <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 backtype.storm.tuple.Values;
+
+public class Channels {
+  private static final ChannelContext VOID_CTX = new ChannelContext() {
+    @Override
+    public void emit(Values data) {}
+
+    @Override
+    public void fireChannelInactive() {}
+  };
+
+  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);
+    }
+  }
+
+  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();
+    }
+  }
+
+  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/39400075/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
new file mode 100644
index 0000000..3e80cb2
--- /dev/null
+++ b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSource.java
@@ -0,0 +1,29 @@
+/*
+ * *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance
+ *  * with the License.  You may obtain a copy of the License at
+ *  * <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;
+
+/**
+ * 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/39400075/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
new file mode 100644
index 0000000..62b1019
--- /dev/null
+++ b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/StormSqlFunctions.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
+ *  * <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;
+
+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/39400075/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/AbstractChannelHandler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/AbstractChannelHandler.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/AbstractChannelHandler.java
deleted file mode 100644
index cf110e3..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/AbstractChannelHandler.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.storm;
-
-import backtype.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) {
-
-  }
-}

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

http://git-wip-us.apache.org/repos/asf/storm/blob/39400075/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/ChannelHandler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/ChannelHandler.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/ChannelHandler.java
deleted file mode 100644
index 8cd3a28..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/ChannelHandler.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.storm;
-
-import backtype.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);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/39400075/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/Channels.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/Channels.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/Channels.java
deleted file mode 100644
index b5bb619..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/Channels.java
+++ /dev/null
@@ -1,78 +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.storm;
-
-import backtype.storm.tuple.Values;
-
-public class Channels {
-  private static final ChannelContext VOID_CTX = new ChannelContext() {
-    @Override
-    public void emit(Values data) {}
-
-    @Override
-    public void fireChannelInactive() {}
-  };
-
-  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);
-    }
-  }
-
-  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();
-    }
-  }
-
-  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/39400075/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/DataSource.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/DataSource.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/DataSource.java
deleted file mode 100644
index 84fa6e0..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/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
- * <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.storm;
-
-/**
- * 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/39400075/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/runtime/AbstractValuesProcessor.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/runtime/AbstractValuesProcessor.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/runtime/AbstractValuesProcessor.java
deleted file mode 100644
index bd068be..0000000
--- a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/runtime/AbstractValuesProcessor.java
+++ /dev/null
@@ -1,29 +0,0 @@
-package org.apache.storm.sql.storm.runtime;
-
-import backtype.storm.tuple.Values;
-import org.apache.storm.sql.storm.ChannelHandler;
-import org.apache.storm.sql.storm.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);
-}


[17/20] storm git commit: [StormSQL] STORM-1201. Support distributed deployment in StormSQL.

Posted by sr...@apache.org.
[StormSQL] STORM-1201. Support distributed deployment in StormSQL.


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

Branch: refs/heads/master
Commit: ff22fd970f02a39c73f1ee8bfc49f42212f5534e
Parents: b5efe2c
Author: Haohui Mai <wh...@apache.org>
Authored: Wed Dec 2 01:02:28 2015 -0800
Committer: Haohui Mai <wh...@apache.org>
Committed: Fri Dec 4 11:09:28 2015 -0800

----------------------------------------------------------------------
 bin/storm.py                                    |  16 ++-
 external/sql/README.md                          |  95 ++++++++++++++++
 external/sql/storm-sql-core/pom.xml             |  27 +++++
 .../src/jvm/org/apache/storm/sql/StormSql.java  |  20 +++-
 .../jvm/org/apache/storm/sql/StormSqlImpl.java  | 110 +++++++++++++++++--
 .../org/apache/storm/sql/StormSqlRunner.java    |  44 ++++++++
 .../backends/standalone/PlanCompiler.java       |   1 +
 .../compiler/backends/trident/PlanCompiler.java |  15 ++-
 .../storm/sql/javac/CompilingClassLoader.java   |   4 +
 .../test/org/apache/storm/sql/TestStormSql.java |  10 +-
 .../storm/sql/compiler/TestCompilerUtils.java   |  74 ++++++++-----
 .../org/apache/storm/sql/kafka/JsonScheme.java  |   6 +-
 .../storm/sql/kafka/TestJsonRepresentation.java |   2 +-
 .../sql/kafka/TestKafkaDataSourcesProvider.java |  10 +-
 external/sql/storm-sql-runtime/pom.xml          |   4 -
 .../storm/sql/runtime/DataSourcesRegistry.java  |   2 -
 .../test/org/apache/storm/sql/TestUtils.java    |   6 +
 pom.xml                                         |   4 +
 storm-dist/binary/pom.xml                       |  10 ++
 storm-dist/binary/src/main/assembly/binary.xml  |  25 ++++-
 20 files changed, 419 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/ff22fd97/bin/storm.py
----------------------------------------------------------------------
diff --git a/bin/storm.py b/bin/storm.py
index f943778..1f958f6 100755
--- a/bin/storm.py
+++ b/bin/storm.py
@@ -229,6 +229,18 @@ def jar(jarfile, klass, *args):
         daemon=False,
         jvmopts=JAR_JVM_OPTS + ["-Dstorm.jar=" + jarfile])
 
+def sql(sql_file, topo_nam):
+    """Syntax: [storm sql sql-file topology]
+
+    Compiles the SQL statements into a Trident topology and submits it to Storm.
+    """
+    exec_storm_class(
+        "org.apache.storm.sql.StormSqlRunner",
+        jvmtype="-client",
+        extrajars=[USER_CONF_DIR, STORM_BIN_DIR],
+        args=[sql_file, topo_name],
+        daemon=False)
+
 def kill(*args):
     """Syntax: [storm kill topology-name [-w wait-time-secs]]
 
@@ -300,7 +312,7 @@ def heartbeats(*args):
         args=args,
         jvmtype="-client",
         extrajars=[USER_CONF_DIR, STORM_BIN_DIR])
-    
+
 def activate(*args):
     """Syntax: [storm activate topology-name]
 
@@ -686,7 +698,7 @@ COMMANDS = {"jar": jar, "kill": kill, "shell": shell, "nimbus": nimbus, "ui": ui
             "list": listtopos, "dev-zookeeper": dev_zookeeper, "version": version, "monitor": monitor,
             "upload-credentials": upload_credentials, "pacemaker": pacemaker, "heartbeats": heartbeats, "blobstore": blobstore,
             "get-errors": get_errors, "set_log_level": set_log_level, "kill_workers": kill_workers,
-            "node-health-check": healthcheck}
+            "node-health-check": healthcheck, "sql": sql}
 
 def parse_config(config_list):
     global CONFIG_OPTS

http://git-wip-us.apache.org/repos/asf/storm/blob/ff22fd97/external/sql/README.md
----------------------------------------------------------------------
diff --git a/external/sql/README.md b/external/sql/README.md
index 078acc6..3caba58 100644
--- a/external/sql/README.md
+++ b/external/sql/README.md
@@ -2,6 +2,97 @@
 
 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.
+
+## Supported Features
+
+The following features are supported in the current repository:
+
+* Streaming from and to external data sources
+* Filtering tuples
+* Projections
+
+## 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":"storm.kafka.IntSerializer","value.serializer":"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).
+
+## 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.
+
+## 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":"storm.kafka.IntSerializer","value.serializer":"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":"storm.kafka.IntSerializer","value.serializer":"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. One approach is to put the required jars into the `extlib` directory:
+
+```
+$ cp curator-client-2.5.0.jar curator-framework-2.5.0.jar zookeeper-3.4.6.jar
+ extlib/
+$ cp scala-library-2.10.4.jar kafka-clients-0.8.2.1.jar kafka_2.10-0.8.2.1.jar metrics-core-2.2.0.jar extlib/
+$ cp json-simple-1.1.1.jar extlib/
+$ cp storm-kafka-*.jar storm-sql-kafka-*.jar storm-sql-runtime-*.jar extlib/
+```
+
+The next step is to submit the SQL statements to StormSQL:
+
+```
+$ bin/storm sql order_filtering order_filtering.sql
+```
+
+By now you should be able to see the `order_filtering` topology in the Storm UI.
+
+## Current Limitations
+
+Aggregation, windowing and joining tables are yet to be implemented. Specifying parallelism hints in the topology is not
+yet supported.
+
+Users also need to provide the dependency of the external data sources in the `extlib` directory. Otherwise the topology
+will fail to run because of `ClassNotFoundException`.
+
 ## License
 
 Licensed to the Apache Software Foundation (ASF) under one
@@ -20,3 +111,7 @@ software distributed under the License is distributed on an
 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))

http://git-wip-us.apache.org/repos/asf/storm/blob/ff22fd97/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
index 0713a61..1e0da8c 100644
--- a/external/sql/storm-sql-core/pom.xml
+++ b/external/sql/storm-sql-core/pom.xml
@@ -59,6 +59,33 @@
             <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.codehaus.janino</groupId>
+                    <artifactId>janino</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.codehaus.janino</groupId>
+                    <artifactId>commons-compiler</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.pentaho</groupId>
+                    <artifactId>pentaho-aggdesigner-algorithm</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>commons-lang</groupId>
+            <artifactId>commons-lang</artifactId>
+            <scope>runtime</scope>
         </dependency>
         <dependency>
             <groupId>junit</groupId>

http://git-wip-us.apache.org/repos/asf/storm/blob/ff22fd97/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
index 6859f8e..eb1e452 100644
--- 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
@@ -17,19 +17,35 @@
  */
 package org.apache.storm.sql;
 
+import backtype.storm.StormSubmitter;
+import backtype.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;
+                               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;
 
   public static StormSql construct() {
     return new StormSqlImpl();

http://git-wip-us.apache.org/repos/asf/storm/blob/ff22fd97/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
index 2350422..c2c8bc8 100644
--- 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
@@ -17,6 +17,8 @@
  */
 package org.apache.storm.sql;
 
+import backtype.storm.StormSubmitter;
+import backtype.storm.generated.SubmitOptions;
 import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
 import org.apache.calcite.rel.RelNode;
@@ -28,18 +30,30 @@ import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.tools.FrameworkConfig;
 import org.apache.calcite.tools.Frameworks;
 import org.apache.calcite.tools.Planner;
-import org.apache.storm.sql.parser.ColumnConstraint;
 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.SqlCreateTable;
 import org.apache.storm.sql.parser.StormParser;
 import org.apache.storm.sql.runtime.*;
+import org.apache.storm.sql.runtime.trident.AbstractTridentProcessor;
+import storm.trident.TridentTopology;
 
-import java.util.AbstractMap;
+import java.io.BufferedOutputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
 import java.util.ArrayList;
 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;
 
@@ -72,8 +86,89 @@ class StormSqlImpl extends StormSql {
     }
   }
 
+  @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 {
+        FrameworkConfig config = Frameworks.newConfigBuilder().defaultSchema(
+            schema).build();
+        Planner planner = Frameworks.getPlanner(config);
+        SqlNode parse = planner.parse(sql);
+        SqlNode validate = planner.validate(parse);
+        RelNode tree = planner.convert(validate);
+        org.apache.storm.sql.compiler.backends.trident.PlanCompiler compiler =
+            new org.apache.storm.sql.compiler.backends.trident.PlanCompiler(typeFactory);
+        AbstractTridentProcessor proc = compiler.compile(tree);
+        TridentTopology topo = proc.build(dataSources);
+        Path jarPath = null;
+        try {
+          jarPath = Files.createTempFile("storm-sql", ".jar");
+          System.setProperty("storm.jar", jarPath.toString());
+          packageTopology(jarPath, compiler.getCompilingClassLoader(), proc);
+          StormSubmitter.submitTopologyAs(name, stormConf, topo.build(), opts, progressListener, asUser);
+        } finally {
+          if (jarPath != null) {
+            Files.delete(jarPath);
+          }
+        }
+      }
+    }
+  }
+
+  private void packageTopology(Path jar, CompilingClassLoader cl, 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)) {
+      for (Map.Entry<String, ByteArrayOutputStream> e : cl.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 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()) {
@@ -87,15 +182,6 @@ class StormSqlImpl extends StormSql {
 
     Table table = builder.build();
     schema.add(n.tableName(), table);
-    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);
+    return fields;
   }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/ff22fd97/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
new file mode 100644
index 0000000..970ccd2
--- /dev/null
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/StormSqlRunner.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;
+
+import backtype.storm.generated.SubmitOptions;
+import backtype.storm.generated.TopologyInitialStatus;
+import backtype.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 {
+    public static void main(String[] args) throws Exception {
+        if (args.length != 2) {
+            System.err.println("storm-sql <sql-file> <topo-name>");
+            return;
+        }
+        List<String> stmts = Files.readAllLines(Paths.get(args[0]), StandardCharsets.UTF_8);
+        String topoName = args[1];
+        StormSql sql = StormSql.construct();
+        @SuppressWarnings("unchecked")
+        Map<String, ?> conf = Utils.readStormConfig();
+        SubmitOptions options = new SubmitOptions(TopologyInitialStatus.ACTIVE);
+        sql.submit(topoName, stmts, conf, options, null, null);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/ff22fd97/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
index 46009e9..64bc06e 100644
--- 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
@@ -27,6 +27,7 @@ import org.apache.storm.sql.runtime.AbstractValuesProcessor;
 
 import java.io.PrintWriter;
 import java.io.StringWriter;
+import java.net.URLClassLoader;
 import java.util.ArrayDeque;
 import java.util.HashSet;
 import java.util.Queue;

http://git-wip-us.apache.org/repos/asf/storm/blob/ff22fd97/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/trident/PlanCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/trident/PlanCompiler.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/trident/PlanCompiler.java
index f8bfd12..35e0cb7 100644
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/trident/PlanCompiler.java
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/trident/PlanCompiler.java
@@ -35,6 +35,8 @@ import org.apache.storm.sql.runtime.trident.AbstractTridentProcessor;
 
 import java.io.PrintWriter;
 import java.io.StringWriter;
+import java.net.URL;
+import java.net.URLClassLoader;
 
 public class PlanCompiler {
   private static final Joiner NEW_LINE_JOINER = Joiner.on("\n");
@@ -65,6 +67,7 @@ public class PlanCompiler {
   );
 
   private final JavaTypeFactory typeFactory;
+  private CompilingClassLoader compilingClassLoader;
   public PlanCompiler(JavaTypeFactory typeFactory) {
     this.typeFactory = typeFactory;
   }
@@ -177,10 +180,14 @@ public class PlanCompiler {
 
   public AbstractTridentProcessor compile(RelNode plan) throws Exception {
     String javaCode = generateJavaSource(plan);
-    ClassLoader cl = new CompilingClassLoader(getClass().getClassLoader(),
-                                              PACKAGE_NAME + ".TridentProcessor",
-                                              javaCode, null);
-    return (AbstractTridentProcessor) cl.loadClass(PACKAGE_NAME + ".TridentProcessor").newInstance();
+    compilingClassLoader = new CompilingClassLoader(getClass().getClassLoader(),
+        PACKAGE_NAME + ".TridentProcessor",
+        javaCode, null);
+    return (AbstractTridentProcessor) compilingClassLoader.loadClass(PACKAGE_NAME + ".TridentProcessor").newInstance();
+  }
+
+  public CompilingClassLoader getCompilingClassLoader() {
+    return compilingClassLoader;
   }
 
   private static void printEpilogue(

http://git-wip-us.apache.org/repos/asf/storm/blob/ff22fd97/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
index cf76964..0b7c053 100644
--- 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
@@ -118,6 +118,10 @@ public class CompilingClassLoader extends ClassLoader {
     }
   }
 
+  public Map<String, ByteArrayOutputStream> getClasses() {
+    return byteCodeForClasses;
+  }
+
   /**
    * Override ClassLoader's class resolving method. Don't call this directly, instead use
    * {@link ClassLoader#loadClass(String)}.

http://git-wip-us.apache.org/repos/asf/storm/blob/ff22fd97/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
index add8da5..f145180 100644
--- 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
@@ -17,6 +17,11 @@
  */
 package org.apache.storm.sql;
 
+import backtype.storm.Config;
+import backtype.storm.ILocalCluster;
+import backtype.storm.StormSubmitter;
+import backtype.storm.generated.SubmitOptions;
+import backtype.storm.generated.TopologyInitialStatus;
 import backtype.storm.tuple.Values;
 import org.apache.storm.sql.runtime.*;
 import org.junit.AfterClass;
@@ -26,6 +31,8 @@ import org.junit.Test;
 
 import java.net.URI;
 import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 
 public class TestStormSql {
@@ -45,7 +52,7 @@ public class TestStormSql {
     @Override
     public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
                                                   String properties, List<FieldInfo> fields) {
-      throw new UnsupportedOperationException();
+      return new TestUtils.MockSqlTridentDataSource();
     }
   }
 
@@ -72,5 +79,4 @@ public class TestStormSql {
     Assert.assertEquals(4, values.get(0).get(0));
     Assert.assertEquals(5, values.get(1).get(0));
   }
-
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/ff22fd97/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestCompilerUtils.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestCompilerUtils.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestCompilerUtils.java
index 0e5fa0b..994e419 100644
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestCompilerUtils.java
+++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestCompilerUtils.java
@@ -1,3 +1,20 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.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 org.apache.calcite.adapter.java.JavaTypeFactory;
@@ -13,36 +30,35 @@ import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.tools.*;
 
 public class TestCompilerUtils {
-  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).build();
-    Table table = streamableTable.stream();
-    schema.add("FOO", table);
-    schema.add("BAR", table);
-    FrameworkConfig config = Frameworks.newConfigBuilder().defaultSchema(
-        schema).build();
-    Planner planner = Frameworks.getPlanner(config);
-    SqlNode parse = planner.parse(sql);
-    SqlNode validate = planner.validate(parse);
-    RelNode tree = planner.convert(validate);
-    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 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).build();
+        Table table = streamableTable.stream();
+        schema.add("FOO", table);
+        schema.add("BAR", table);
+        FrameworkConfig config = Frameworks.newConfigBuilder().defaultSchema(
+            schema).build();
+        Planner planner = Frameworks.getPlanner(config);
+        SqlNode parse = planner.parse(sql);
+        SqlNode validate = planner.validate(parse);
+        RelNode tree = planner.convert(validate);
+        return new CalciteState(schema, tree);
     }
 
-    public SchemaPlus schema() { return schema; }
-    public RelNode tree() { return 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; }
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/ff22fd97/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/JsonScheme.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/JsonScheme.java b/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/JsonScheme.java
index 80037c6..1b45b30 100644
--- a/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/JsonScheme.java
+++ b/external/sql/storm-sql-kafka/src/jvm/org/apache/storm/sql/kafka/JsonScheme.java
@@ -19,9 +19,11 @@ package org.apache.storm.sql.kafka;
 
 import backtype.storm.spout.Scheme;
 import backtype.storm.tuple.Fields;
+import backtype.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;
@@ -34,11 +36,11 @@ public class JsonScheme implements Scheme {
   }
 
   @Override
-  public List<Object> deserialize(byte[] ser) {
+  public List<Object> deserialize(ByteBuffer ser) {
     ObjectMapper mapper = new ObjectMapper();
     try {
       @SuppressWarnings("unchecked")
-      HashMap<String, Object> map = mapper.readValue(ser, HashMap.class);
+      HashMap<String, Object> map = mapper.readValue(Utils.toByteArray(ser), HashMap.class);
       ArrayList<Object> list = new ArrayList<>();
       for (String f : fields) {
         list.add(map.get(f));

http://git-wip-us.apache.org/repos/asf/storm/blob/ff22fd97/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestJsonRepresentation.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestJsonRepresentation.java b/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestJsonRepresentation.java
index d2898e8..5973672 100644
--- a/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestJsonRepresentation.java
+++ b/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestJsonRepresentation.java
@@ -34,7 +34,7 @@ public class TestJsonRepresentation {
     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(s.getBytes(Charset.defaultCharset()));
+    List<Object> o = scheme.deserialize(ByteBuffer.wrap(s.getBytes(Charset.defaultCharset())));
     assertArrayEquals(new Object[] {1, "2"}, o.toArray());
   }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/ff22fd97/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java b/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java
index 9380b66..399bb3e 100644
--- a/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java
+++ b/external/sql/storm-sql-kafka/src/test/org/apache/storm/sql/kafka/TestKafkaDataSourcesProvider.java
@@ -38,6 +38,7 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.concurrent.Future;
 
 import static org.mockito.Mockito.*;
 
@@ -65,7 +66,8 @@ public class TestKafkaDataSourcesProvider {
     KafkaTridentSink sink = (KafkaTridentSink) ds.getConsumer();
     sink.prepare(null, null);
     TridentKafkaState state = (TridentKafkaState) Whitebox.getInternalState(sink, "state");
-    Producer producer = mock(Producer.class);
+    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) {
@@ -88,7 +90,7 @@ public class TestKafkaDataSourcesProvider {
     return tupleList;
   }
 
-  private static class KafkaMessageMatcher extends ArgumentMatcher<KeyedMessage> {
+  private static class KafkaMessageMatcher extends ArgumentMatcher<ProducerRecord> {
     private static final int PRIMARY_INDEX = 0;
     private final TridentTuple tuple;
 
@@ -99,11 +101,11 @@ public class TestKafkaDataSourcesProvider {
     @SuppressWarnings("unchecked")
     @Override
     public boolean matches(Object o) {
-      KeyedMessage<Object, ByteBuffer> m = (KeyedMessage<Object,ByteBuffer>)o;
+      ProducerRecord<Object, ByteBuffer> m = (ProducerRecord<Object,ByteBuffer>)o;
       if (m.key() != tuple.get(PRIMARY_INDEX)) {
         return false;
       }
-      ByteBuffer buf = m.message();
+      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/ff22fd97/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
index eb6580a..7126ece 100644
--- a/external/sql/storm-sql-runtime/pom.xml
+++ b/external/sql/storm-sql-runtime/pom.xml
@@ -43,10 +43,6 @@
             <scope>provided</scope>
         </dependency>
         <dependency>
-            <groupId>com.google.guava</groupId>
-            <artifactId>guava</artifactId>
-        </dependency>
-        <dependency>
             <groupId>org.mockito</groupId>
             <artifactId>mockito-all</artifactId>
             <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/storm/blob/ff22fd97/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
index 75cd391..9bb4b54 100644
--- 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
@@ -20,7 +20,6 @@
 
 package org.apache.storm.sql.runtime;
 
-import com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -73,7 +72,6 @@ public class DataSourcesRegistry {
   /**
    * Allow unit tests to inject data sources.
    */
-  @VisibleForTesting
   public static Map<String, DataSourcesProvider> providerMap() {
     return providers;
   }

http://git-wip-us.apache.org/repos/asf/storm/blob/ff22fd97/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
index 46aac4a..58efdf6 100644
--- 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
@@ -19,6 +19,8 @@
  */
 package org.apache.storm.sql;
 
+import backtype.storm.ILocalCluster;
+import backtype.storm.LocalCluster;
 import backtype.storm.task.TopologyContext;
 import backtype.storm.tuple.Fields;
 import backtype.storm.tuple.Values;
@@ -154,4 +156,8 @@ public class TestUtils {
     final long NANOSECONDS_PER_MILLISECOND = 1000000;
     return System.nanoTime() / NANOSECONDS_PER_MILLISECOND;
   }
+
+  public static ILocalCluster newLocalCluster() {
+    return new LocalCluster();
+  }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/ff22fd97/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 4c7388d..3f8e547 100644
--- a/pom.xml
+++ b/pom.xml
@@ -897,6 +897,10 @@
                         <exclude>**/.git/**</exclude>
                         <exclude>**/derby.log</exclude>
                         <exclude>**/src/dev/**</exclude>
+
+                        <!-- StormSQL -->
+                        <exclude>**/src/codegen/config.fmpp</exclude>
+                        <exclude>**/src/codegen/data/Parser.tdd</exclude>
                     </excludes>
                 </configuration>
             </plugin>

http://git-wip-us.apache.org/repos/asf/storm/blob/ff22fd97/storm-dist/binary/pom.xml
----------------------------------------------------------------------
diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml
index c2d709d..7f8f0f2 100644
--- a/storm-dist/binary/pom.xml
+++ b/storm-dist/binary/pom.xml
@@ -36,6 +36,16 @@
             <artifactId>storm-core</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-sql-core</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.storm</groupId>
+            <artifactId>storm-sql-runtime</artifactId>
+            <version>${project.version}</version>
+        </dependency>
     </dependencies>
 
     <build>

http://git-wip-us.apache.org/repos/asf/storm/blob/ff22fd97/storm-dist/binary/src/main/assembly/binary.xml
----------------------------------------------------------------------
diff --git a/storm-dist/binary/src/main/assembly/binary.xml b/storm-dist/binary/src/main/assembly/binary.xml
index 94ee172..b2a6104 100644
--- a/storm-dist/binary/src/main/assembly/binary.xml
+++ b/storm-dist/binary/src/main/assembly/binary.xml
@@ -253,8 +253,29 @@
             </includes>
         </fileSet>
         <fileSet>
-            <directory>${project.basedir}/../../external/storm-cassandra</directory>
-            <outputDirectory>external/storm-cassandra</outputDirectory>
+            <directory>${project.basedir}/../../external/sql/storm-sql-core/target</directory>
+            <outputDirectory>external/sql/storm-sql-core</outputDirectory>
+            <includes>
+                <include>storm*jar</include>
+            </includes>
+        </fileSet>
+        <fileSet>
+            <directory>${project.basedir}/../../external/sql/storm-sql-kafka/target</directory>
+            <outputDirectory>external/sql/storm-sql-kafka</outputDirectory>
+            <includes>
+                <include>storm*jar</include>
+            </includes>
+        </fileSet>
+        <fileSet>
+            <directory>${project.basedir}/../../external/sql/storm-sql-runtime/target</directory>
+            <outputDirectory>external/sql/storm-sql-runtime</outputDirectory>
+            <includes>
+                <include>storm*jar</include>
+            </includes>
+        </fileSet>
+        <fileSet>
+            <directory>${project.basedir}/../../external/sql</directory>
+            <outputDirectory>external/sql</outputDirectory>
             <includes>
                 <include>README.*</include>
             </includes>


[04/20] storm git commit: [StormSQL] Implement nullable semantics for AND, OR and NOT operators.

Posted by sr...@apache.org.
[StormSQL] Implement nullable semantics for AND, OR and NOT operators.


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

Branch: refs/heads/master
Commit: 31daf268b5e536376bfb2a2a77336652d86d5399
Parents: 5f97080
Author: Haohui Mai <wh...@apache.org>
Authored: Wed Nov 4 09:24:32 2015 -0800
Committer: Haohui Mai <wh...@apache.org>
Committed: Fri Dec 4 11:09:02 2015 -0800

----------------------------------------------------------------------
 .../apache/storm/sql/compiler/ExprCompiler.java | 101 +++++++++++++++----
 .../storm/sql/compiler/TestExprSemantic.java    |  49 ++++++---
 2 files changed, 119 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/31daf268/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java
index fd3a614..db97d43 100644
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java
@@ -17,7 +17,9 @@
  */
 package org.apache.storm.sql.compiler;
 
+import com.google.common.base.Joiner;
 import com.google.common.collect.ImmutableMap;
+import org.apache.calcite.adapter.enumerable.CallImplementor;
 import org.apache.calcite.adapter.enumerable.RexImpTable;
 import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.rel.type.RelDataType;
@@ -67,9 +69,9 @@ import static org.apache.calcite.sql.fun.SqlStdOperatorTable.PLUS;
  */
 class ExprCompiler implements RexVisitor<String> {
   private final PrintWriter pw;
-  private final Map<RexNode, String> expr = new IdentityHashMap<>();
   private final JavaTypeFactory typeFactory;
   private static final ImpTable IMP_TABLE = new ImpTable();
+  private int nameCount;
 
   ExprCompiler(PrintWriter pw, JavaTypeFactory typeFactory) {
     this.pw = pw;
@@ -78,14 +80,10 @@ class ExprCompiler implements RexVisitor<String> {
 
   @Override
   public String visitInputRef(RexInputRef rexInputRef) {
-    if (expr.containsKey(rexInputRef)) {
-      return expr.get(rexInputRef);
-    }
     String name = reserveName(rexInputRef);
     String typeName = javaTypeName(rexInputRef);
     pw.print(String.format("%s %s = (%s)(_data.get(%d));\n", typeName, name,
                            typeName, rexInputRef.getIndex()));
-    expr.put(rexInputRef, name);
     return name;
   }
 
@@ -167,8 +165,7 @@ class ExprCompiler implements RexVisitor<String> {
   }
 
   private String reserveName(RexNode node) {
-    String name = "t" + expr.size();
-    expr.put(node, name);
+    String name = "t" + ++nameCount;
     return name;
   }
 
@@ -285,6 +282,9 @@ class ExprCompiler implements RexVisitor<String> {
     }
 
 
+    // If any of the arguments are false, result is false;
+    // else if any arguments are null, result is null;
+    // else true.
     private static final CallExprPrinter AND_EXPR = new CallExprPrinter() {
       @Override
       public String translate(
@@ -293,15 +293,40 @@ class ExprCompiler implements RexVisitor<String> {
         PrintWriter pw = compiler.pw;
         pw.print(String.format("final %s %s;\n", compiler.javaTypeName(call),
                                val));
-        String lhs = call.getOperands().get(0).accept(compiler);
-        pw.print(String.format("if (!(%2$s)) { %1$s = false; }\n", val, lhs));
-        pw.print("else {\n");
-        String rhs = call.getOperands().get(1).accept(compiler);
-        pw.print(String.format("  %1$s = %2$s;\n}\n", val, rhs));
+        RexNode op0 = call.getOperands().get(0);
+        RexNode op1 = call.getOperands().get(1);
+        boolean lhsNullable = op0.getType().isNullable();
+        boolean rhsNullable = op1.getType().isNullable();
+        String lhs = op0.accept(compiler);
+        if (!lhsNullable) {
+          pw.print(String.format("if (!(%2$s)) { %1$s = false; }\n", val, lhs));
+          pw.print("else {\n");
+          String rhs = op1.accept(compiler);
+          pw.print(String.format("  %1$s = %2$s;\n}\n", val, rhs));
+        } else {
+          String foldedLHS = foldNullExpr(
+              String.format("%1$s == null || %1$s", lhs), "true", lhs);
+          pw.print(String.format("if (%s) {\n", foldedLHS));
+          String rhs = op1.accept(compiler);
+          String s;
+          if (rhsNullable) {
+            s = foldNullExpr(
+                String.format("(%2$s != null && !(%2$s)) ? false : %1$s", lhs,
+                              rhs),
+                "null", rhs);
+          } else {
+            s = String.format("!(%2$s) ? false : %1$s", lhs, rhs);
+          }
+          pw.print(String.format("  %1$s = %2$s;\n", val, s));
+          pw.print(String.format("} else { %1$s = false; }\n", val));
+        }
         return val;
       }
     };
 
+    // If any of the arguments are true, result is true;
+    // else if any arguments are null, result is null;
+    // else false.
     private static final CallExprPrinter OR_EXPR = new CallExprPrinter() {
       @Override
       public String translate(
@@ -310,11 +335,32 @@ class ExprCompiler implements RexVisitor<String> {
         PrintWriter pw = compiler.pw;
         pw.print(String.format("final %s %s;\n", compiler.javaTypeName(call),
                                val));
-        String lhs = call.getOperands().get(0).accept(compiler);
-        pw.print(String.format("if (%2$s) { %1$s = true; }\n", val, lhs));
-        pw.print("else {\n");
-        String rhs = call.getOperands().get(1).accept(compiler);
-        pw.print(String.format("  %1$s = %2$s;\n}\n", val, rhs));
+        RexNode op0 = call.getOperands().get(0);
+        RexNode op1 = call.getOperands().get(1);
+        boolean lhsNullable = op0.getType().isNullable();
+        boolean rhsNullable = op1.getType().isNullable();
+        String lhs = op0.accept(compiler);
+        if (!lhsNullable) {
+          pw.print(String.format("if (%2$s) { %1$s = true; }\n", val, lhs));
+          pw.print("else {\n");
+          String rhs = op1.accept(compiler);
+          pw.print(String.format("  %1$s = %2$s;\n}\n", val, rhs));
+        } else {
+          String foldedLHS = foldNullExpr(
+              String.format("%1$s == null || !(%1$s)", lhs), "true", lhs);
+          pw.print(String.format("if (%s) {\n", foldedLHS));
+          String rhs = op1.accept(compiler);
+          String s;
+          if (rhsNullable) {
+            s = foldNullExpr(
+                String.format("(%2$s != null && %2$s) ? true : %1$s", lhs, rhs),
+                "null", rhs);
+          } else {
+            s = String.format("%2$s ? %2$s : %1$s", lhs, rhs);
+          }
+          pw.print(String.format("  %1$s = %2$s;\n", val, s));
+          pw.print(String.format("} else { %1$s = true; }\n", val));
+        }
         return val;
       }
     };
@@ -325,13 +371,30 @@ class ExprCompiler implements RexVisitor<String> {
           ExprCompiler compiler, RexCall call) {
         String val = compiler.reserveName(call);
         PrintWriter pw = compiler.pw;
-        String lhs = call.getOperands().get(0).accept(compiler);
+        RexNode op = call.getOperands().get(0);
+        String lhs = op.accept(compiler);
+        boolean nullable = call.getType().isNullable();
         pw.print(String.format("final %s %s;\n", compiler.javaTypeName(call),
                                val));
-        pw.print(String.format("%1$s = !(%2$s);\n", val, lhs));
+        if (!nullable) {
+          pw.print(String.format("%1$s = !(%2$s);\n", val, lhs));
+        } else {
+          String s = foldNullExpr(
+              String.format("%1$s == null ? null : !(%1$s)", lhs), "null", lhs);
+          pw.print(String.format("%1$s = %2$s;\n", val, s));
+        }
         return val;
       }
     };
+
+    private static String foldNullExpr(String notNullExpr, String
+        nullExpr, String op) {
+      if (op.equals("null")) {
+        return nullExpr;
+      } else {
+        return notNullExpr;
+      }
+    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/31daf268/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
index bda9e00..7bde092 100644
--- 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
@@ -43,8 +43,7 @@ public class TestExprSemantic {
   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)"),
-        "WHERE ID > 0 AND ID < 2");
+                           "NOT (ID > 0 AND ID < 1)"));
     assertEquals(new Values(true, false, true), v);
   }
 
@@ -58,12 +57,42 @@ public class TestExprSemantic {
     assertEquals(new Values(true, false, false, true, false, true, false), v);
   }
 
-  private Values testExpr(List<String> exprs, String additionalCaluse)
-      throws Exception {
-    String sql = "SELECT " + Joiner.on(',').join(exprs) + " FROM FOO";
-    if (additionalCaluse != null) {
-      sql += " " + additionalCaluse;
-    }
+  @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 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);
+  }
+
+  private Values testExpr(List<String> exprs) throws Exception {
+    String sql = "SELECT " + Joiner.on(',').join(exprs) + " FROM FOO" +
+        " WHERE ID > 0 AND ID < 2";
     TestUtils.CalciteState state = TestUtils.sqlOverDummyTable(sql);
     PlanCompiler compiler = new PlanCompiler(typeFactory);
     AbstractValuesProcessor proc = compiler.compile(state.tree);
@@ -75,8 +104,4 @@ public class TestExprSemantic {
     return values.get(0);
   }
 
-  private Values testExpr(List<String> exprs) throws Exception {
-    return testExpr(exprs, null);
-  }
-
 }


[07/20] storm git commit: [StormSQL] Implement IS operators in StormSQL.

Posted by sr...@apache.org.
[StormSQL] Implement IS operators in StormSQL.


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

Branch: refs/heads/master
Commit: 5f9708025e6434b8d4d1457244e45800d10d8c9e
Parents: 0bb8e46
Author: Haohui Mai <wh...@apache.org>
Authored: Tue Nov 3 15:06:22 2015 -0800
Committer: Haohui Mai <wh...@apache.org>
Committed: Fri Dec 4 11:09:02 2015 -0800

----------------------------------------------------------------------
 .../apache/storm/sql/compiler/ExprCompiler.java | 62 +++++++++++++++
 .../storm/sql/compiler/TestExprSemantic.java    | 82 ++++++++++++++++++++
 2 files changed, 144 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/5f970802/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java
index 0d35593..fd3a614 100644
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java
@@ -18,6 +18,7 @@
 package org.apache.storm.sql.compiler;
 
 import com.google.common.collect.ImmutableMap;
+import org.apache.calcite.adapter.enumerable.RexImpTable;
 import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexCall;
@@ -47,6 +48,12 @@ import static org.apache.calcite.sql.fun.SqlStdOperatorTable.DIVIDE;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.DIVIDE_INTEGER;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.GREATER_THAN;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.GREATER_THAN_OR_EQUAL;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_FALSE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_FALSE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_NULL;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_TRUE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NULL;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_TRUE;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LESS_THAN;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LESS_THAN_OR_EQUAL;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MINUS;
@@ -189,6 +196,12 @@ class ExprCompiler implements RexVisitor<String> {
           .put(infixBinary(MULTIPLY, "*"))
           .put(infixBinary(DIVIDE, "/"))
           .put(infixBinary(DIVIDE_INTEGER, "/"))
+          .put(expect(IS_NULL, null))
+          .put(expectNot(IS_NOT_NULL, null))
+          .put(expect(IS_TRUE, true))
+          .put(expectNot(IS_NOT_TRUE, true))
+          .put(expect(IS_FALSE, false))
+          .put(expectNot(IS_NOT_FALSE, false))
           .put(AND, AND_EXPR)
           .put(OR, OR_EXPR)
           .put(NOT, NOT_EXPR);
@@ -223,6 +236,55 @@ class ExprCompiler implements RexVisitor<String> {
       return new AbstractMap.SimpleImmutableEntry<>(op, trans);
     }
 
+    private Map.Entry<SqlOperator, CallExprPrinter> expect(
+        SqlOperator op, final Boolean expect) {
+      return expect0(op, expect, false);
+    }
+
+    private Map.Entry<SqlOperator, CallExprPrinter> expectNot(
+        SqlOperator op, final Boolean expect) {
+      return expect0(op, expect, true);
+    }
+
+    private Map.Entry<SqlOperator, CallExprPrinter> expect0(
+        SqlOperator op, final Boolean expect, final boolean negate) {
+      CallExprPrinter trans = new CallExprPrinter() {
+        @Override
+        public String translate(
+            ExprCompiler compiler, RexCall call) {
+          assert call.getOperands().size() == 1;
+          String val = compiler.reserveName(call);
+          RexNode operand = call.getOperands().get(0);
+          boolean nullable = operand.getType().isNullable();
+          String op = operand.accept(compiler);
+          PrintWriter pw = compiler.pw;
+          if (!nullable) {
+            if (expect == null) {
+              pw.print(String.format("boolean %s = %b;\n", val, !negate));
+            } else {
+              pw.print(String.format("boolean %s = %s == %b;\n", val, op,
+                                     expect ^ negate));
+            }
+          } else {
+            String expr;
+            if (expect == null) {
+              expr = String.format("%s == null", op);
+            } else {
+              expr = String.format("%s == Boolean.%s", op, expect ? "TRUE" :
+                  "FALSE");
+            }
+            if (negate) {
+              expr = String.format("!(%s)", expr);
+            }
+            pw.print(String.format("boolean %s = %s;\n", val, expr));
+          }
+          return val;
+        }
+      };
+      return new AbstractMap.SimpleImmutableEntry<>(op, trans);
+    }
+
+
     private static final CallExprPrinter AND_EXPR = new CallExprPrinter() {
       @Override
       public String translate(

http://git-wip-us.apache.org/repos/asf/storm/blob/5f970802/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
new file mode 100644
index 0000000..bda9e00
--- /dev/null
+++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprSemantic.java
@@ -0,0 +1,82 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.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 backtype.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.storm.ChannelHandler;
+import org.apache.storm.sql.storm.DataSource;
+import org.apache.storm.sql.storm.runtime.AbstractValuesProcessor;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+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)"),
+        "WHERE ID > 0 AND ID < 2");
+    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);
+  }
+
+  private Values testExpr(List<String> exprs, String additionalCaluse)
+      throws Exception {
+    String sql = "SELECT " + Joiner.on(',').join(exprs) + " FROM FOO";
+    if (additionalCaluse != null) {
+      sql += " " + additionalCaluse;
+    }
+    TestUtils.CalciteState state = TestUtils.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);
+  }
+
+  private Values testExpr(List<String> exprs) throws Exception {
+    return testExpr(exprs, null);
+  }
+
+}


[14/20] storm git commit: [StormSQL] STORM-1181. Compile SQLs into Tridient topology and execute them in LocalCluster.

Posted by sr...@apache.org.
[StormSQL] STORM-1181. Compile SQLs into Tridient topology and execute them in LocalCluster.


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

Branch: refs/heads/master
Commit: 7e378c65bc18ebcca3e500ea70f6da3f376003c3
Parents: 82f10eb
Author: Haohui Mai <wh...@apache.org>
Authored: Mon Nov 16 14:48:43 2015 -0800
Committer: Haohui Mai <wh...@apache.org>
Committed: Fri Dec 4 11:09:28 2015 -0800

----------------------------------------------------------------------
 .../compiler/backends/trident/PlanCompiler.java | 194 +++++++++++++++++++
 .../backends/trident/RelNodeCompiler.java       | 116 +++++++++++
 .../backends/trident/TestPlanCompiler.java      | 116 +++++++++++
 external/sql/storm-sql-runtime/pom.xml          |  13 ++
 .../sql/runtime/ISqlTridentDataSource.java      |   3 +-
 .../trident/AbstractTridentProcessor.java       |  43 ++++
 .../test/org/apache/storm/sql/TestUtils.java    |  88 +++++++++
 7 files changed, 572 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/7e378c65/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/trident/PlanCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/trident/PlanCompiler.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/trident/PlanCompiler.java
new file mode 100644
index 0000000..f8bfd12
--- /dev/null
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/trident/PlanCompiler.java
@@ -0,0 +1,194 @@
+/*
+ * *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.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.base.Joiner;
+import com.google.common.base.Preconditions;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.TableModify;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.storm.sql.compiler.CompilerUtil;
+import org.apache.storm.sql.compiler.PostOrderRelNodeVisitor;
+import org.apache.storm.sql.javac.CompilingClassLoader;
+import org.apache.storm.sql.runtime.trident.AbstractTridentProcessor;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+
+public class PlanCompiler {
+  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.List;",
+      "import java.util.Map;",
+      "import backtype.storm.tuple.Fields;",
+      "import backtype.storm.tuple.Values;",
+      "import org.apache.storm.sql.runtime.ISqlTridentDataSource;",
+      "import org.apache.storm.sql.runtime.trident.AbstractTridentProcessor;",
+      "import storm.trident.Stream;",
+      "import storm.trident.TridentTopology;",
+      "import storm.trident.fluent.IAggregatableStream;",
+      "import storm.trident.operation.TridentCollector;",
+      "import storm.trident.operation.BaseFunction;",
+      "import storm.trident.spout.IBatchSpout;",
+      "import storm.trident.tuple.TridentTuple;",
+      "",
+      "public final class TridentProcessor extends AbstractTridentProcessor {",
+      "");
+  private static final String INITIALIZER_PROLOGUE = NEW_LINE_JOINER.join(
+      "  @Override",
+      "  public TridentTopology build(Map<String, ISqlTridentDataSource> _sources) {",
+      "    TridentTopology topo = new TridentTopology();",
+      ""
+  );
+
+  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 static class MainFuncCompiler extends PostOrderRelNodeVisitor<Void> {
+    private final PrintWriter pw;
+    private static final String TABLESCAN_TMPL = NEW_LINE_JOINER.join(
+        "if (!_sources.containsKey(%2$s))",
+        "    throw new RuntimeException(\"Cannot find table \" + %2$s);",
+        "Stream _%1$s = topo.newStream(%2$s, _sources.get(%2$s).getProducer());",
+        ""
+    );
+
+    private static final String TABLEMODIFY_TMPL = NEW_LINE_JOINER.join(
+        "Stream _%1$s = _%3$s.each(new Fields(%4$s), _sources.get(%2$s).getConsumer(), new Fields(%5$s));",
+        ""
+    );
+    private static final String TRANSFORMATION_TMPL = NEW_LINE_JOINER.join(
+        "Stream _%1$s = _%2$s.each(new Fields(%3$s), %1$s, new Fields(%4$s)).toStream().project(new Fields(%4$s));",
+        ""
+    );
+
+    private MainFuncCompiler(PrintWriter pw) {
+      this.pw = pw;
+    }
+
+    @Override
+    public Void defaultValue(RelNode n) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Void visitFilter(Filter filter) throws Exception {
+      visitTransformation(filter);
+      return null;
+    }
+
+    @Override
+    public Void visitTableModify(TableModify modify) throws Exception {
+      Preconditions.checkArgument(modify.isInsert(), "Only INSERT statement is supported.");
+      String name = RelNodeCompiler.getStageName(modify);
+      RelNode input = modify.getInput();
+      String inputName = RelNodeCompiler.getStageName(input);
+      pw.print(String.format(TABLEMODIFY_TMPL, name, CompilerUtil.escapeJavaString(
+          Joiner.on('.').join(modify.getTable().getQualifiedName()), true),
+          inputName, getFieldString(input), getFieldString(modify)));
+      return null;
+    }
+
+    @Override
+    public Void visitTableScan(TableScan scan) throws Exception {
+      String name = RelNodeCompiler.getStageName(scan);
+      pw.print(String.format(TABLESCAN_TMPL, name, CompilerUtil.escapeJavaString(
+          Joiner.on('.').join(scan.getTable().getQualifiedName()), true)));
+      return null;
+    }
+
+    @Override
+    public Void visitProject(Project project) throws Exception {
+      visitTransformation(project);
+      return null;
+    }
+
+    private static String getFieldString(RelNode n) {
+      int id = n.getId();
+      StringBuilder sb = new StringBuilder();
+      boolean first = true;
+      for (String f: n.getRowType().getFieldNames()) {
+        if (!first) {
+          sb.append(", ");
+        }
+        if (n instanceof TableScan) {
+          sb.append(CompilerUtil.escapeJavaString(f, true));
+        } else {
+          sb.append(CompilerUtil.escapeJavaString(String.format("%d_%s", id, f), true));
+        }
+        first = false;
+      }
+      return sb.toString();
+    }
+
+    private void visitTransformation(SingleRel node) {
+      String name = RelNodeCompiler.getStageName(node);
+      RelNode input = node.getInput();
+      String inputName = RelNodeCompiler.getStageName(input);
+      pw.print(String.format(TRANSFORMATION_TMPL, name, inputName,
+          getFieldString(input), getFieldString(node)));
+    }
+  }
+
+  private void printMain(PrintWriter pw, RelNode root) throws Exception {
+    pw.print(INITIALIZER_PROLOGUE);
+    MainFuncCompiler compiler = new MainFuncCompiler(pw);
+    compiler.traverse(root);
+    pw.print(String.format("  this.outputStream = _%s;\n", RelNodeCompiler.getStageName(root)));
+    pw.print("  return topo; \n}\n");
+  }
+
+  public AbstractTridentProcessor compile(RelNode plan) throws Exception {
+    String javaCode = generateJavaSource(plan);
+    ClassLoader cl = new CompilingClassLoader(getClass().getClassLoader(),
+                                              PACKAGE_NAME + ".TridentProcessor",
+                                              javaCode, null);
+    return (AbstractTridentProcessor) cl.loadClass(PACKAGE_NAME + ".TridentProcessor").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/7e378c65/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/trident/RelNodeCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/trident/RelNodeCompiler.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/trident/RelNodeCompiler.java
new file mode 100644
index 0000000..1de39d3
--- /dev/null
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/trident/RelNodeCompiler.java
@@ -0,0 +1,116 @@
+/*
+ * *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.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 backtype.storm.tuple.Fields;
+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.Filter;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.TableModify;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.storm.sql.compiler.ExprCompiler;
+import org.apache.storm.sql.compiler.PostOrderRelNodeVisitor;
+
+import java.io.PrintWriter;
+import java.util.IdentityHashMap;
+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 static final String STAGE_PROLOGUE = NEW_LINE_JOINER.join(
+    "  private static final BaseFunction %1$s = ",
+    "    new BaseFunction() {",
+    "    @Override",
+    "    public void execute(TridentTuple tuple, TridentCollector collector) {",
+    "      List<Object> _data = tuple.getValues();",
+    ""
+  );
+
+  private final IdentityHashMap<RelNode, Fields> outputFields = new IdentityHashMap<>();
+
+  RelNodeCompiler(PrintWriter pw, JavaTypeFactory typeFactory) {
+    this.pw = pw;
+    this.typeFactory = typeFactory;
+  }
+
+  @Override
+  public Void visitFilter(Filter filter) throws Exception {
+    beginStage(filter);
+    ExprCompiler compiler = new ExprCompiler(pw, typeFactory);
+    String r = filter.getCondition().accept(compiler);
+    pw.print(String.format("    if (%s) { collector.emit(_data); }\n", r));
+    endStage();
+    return null;
+  }
+
+  @Override
+  public Void visitProject(Project project) throws Exception {
+    beginStage(project);
+    ExprCompiler compiler = new ExprCompiler(pw, typeFactory);
+
+    int size = project.getChildExps().size();
+    String[] res = new String[size];
+    for (int i = 0; i < size; ++i) {
+      res[i] = project.getChildExps().get(i).accept(compiler);
+    }
+
+    pw.print(String.format("    collector.emit(new Values(%s));\n",
+                           Joiner.on(',').join(res)));
+    endStage();
+    return null;
+  }
+
+  @Override
+  public Void defaultValue(RelNode n) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Void visitTableScan(TableScan scan) throws Exception {
+    return null;
+  }
+
+  @Override
+  public Void visitTableModify(TableModify modify) throws Exception {
+    return null;
+  }
+
+  private void beginStage(RelNode n) {
+    pw.print(String.format(STAGE_PROLOGUE, getStageName(n)));
+  }
+
+  private void endStage() {
+    pw.print("  }\n  };\n");
+  }
+
+  static String getStageName(RelNode n) {
+    return n.getClass().getSimpleName().toUpperCase() + "_" + n.getId();
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7e378c65/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
new file mode 100644
index 0000000..a68ba0c
--- /dev/null
+++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/trident/TestPlanCompiler.java
@@ -0,0 +1,116 @@
+/*
+ * *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.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 backtype.storm.Config;
+import backtype.storm.ILocalCluster;
+import backtype.storm.LocalCluster;
+import backtype.storm.generated.StormTopology;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Values;
+import backtype.storm.utils.Utils;
+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.TestUtils.MockSqlTridentDataSource.CollectDataFunction;
+import org.apache.storm.sql.compiler.TestCompilerUtils;
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import org.apache.storm.sql.runtime.trident.AbstractTridentProcessor;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import storm.trident.TridentTopology;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Callable;
+
+import static org.apache.storm.sql.TestUtils.MockSqlTridentDataSource.CollectDataFunction.*;
+
+public class TestPlanCompiler {
+  private final JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(
+      RelDataTypeSystem.DEFAULT);
+
+  @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);
+    PlanCompiler compiler = new PlanCompiler(typeFactory);
+    final AbstractTridentProcessor proc = compiler.compile(state.tree());
+    final Map<String, ISqlTridentDataSource> data = new HashMap<>();
+    data.put("FOO", new TestUtils.MockSqlTridentDataSource());
+    final TridentTopology topo = proc.build(data);
+    Fields f = proc.outputStream().getOutputFields();
+    proc.outputStream().each(f, new CollectDataFunction(), new Fields()).toStream();
+    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 FROM FOO WHERE ID > 3";
+    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
+    PlanCompiler compiler = new PlanCompiler(typeFactory);
+    final AbstractTridentProcessor proc = compiler.compile(state.tree());
+    final Map<String, ISqlTridentDataSource> data = new HashMap<>();
+    data.put("FOO", new TestUtils.MockSqlTridentDataSource());
+    data.put("BAR", new TestUtils.MockSqlTridentDataSource());
+    final TridentTopology topo = proc.build(data);
+    runTridentTopology(EXPECTED_VALUE_SIZE, proc, topo);
+    Assert.assertArrayEquals(new Values[] { new Values(4)}, getCollectedValues().toArray());
+  }
+
+  private void runTridentTopology(final int expectedValueSize, AbstractTridentProcessor proc,
+                                  TridentTopology topo) throws Exception {
+    final Config conf = new Config();
+    conf.setMaxSpoutPending(20);
+
+    ILocalCluster cluster = new LocalCluster();
+    StormTopology stormTopo = topo.build();
+    try {
+      Utils.setClassLoaderForJavaDeSerialize(proc.getClass().getClassLoader());
+      cluster.submitTopology("storm-sql", conf, stormTopo);
+      waitForCompletion(1000 * 1000, new Callable<Boolean>() {
+        @Override
+        public Boolean call() throws Exception {
+          return getCollectedValues().size() < expectedValueSize;
+        }
+      });
+    } finally {
+      Utils.resetClassLoaderForJavaDeSerialize();
+      cluster.shutdown();
+    }
+  }
+
+  private void waitForCompletion(long timeout, Callable<Boolean> cond) throws Exception {
+    long start = TestUtils.monotonicNow();
+    while (TestUtils.monotonicNow() - start < timeout && cond.call()) {
+      Thread.sleep(100);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7e378c65/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
index a8ba4dc..eb6580a 100644
--- a/external/sql/storm-sql-runtime/pom.xml
+++ b/external/sql/storm-sql-runtime/pom.xml
@@ -60,5 +60,18 @@
     <build>
       <sourceDirectory>src/jvm</sourceDirectory>
       <testSourceDirectory>src/test</testSourceDirectory>
+      <plugins>
+        <plugin>
+          <groupId>org.apache.maven.plugins</groupId>
+          <artifactId>maven-jar-plugin</artifactId>
+          <executions>
+            <execution>
+              <goals>
+                <goal>test-jar</goal>
+              </goals>
+            </execution>
+          </executions>
+        </plugin>
+      </plugins>
     </build>
 </project>

http://git-wip-us.apache.org/repos/asf/storm/blob/7e378c65/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
index 4b2a915..d9e1db7 100644
--- 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
@@ -19,11 +19,12 @@ package org.apache.storm.sql.runtime;
 
 import storm.trident.operation.Function;
 import storm.trident.spout.IBatchSpout;
+import storm.trident.spout.ITridentDataSource;
 
 /**
  * A ISqlTridentDataSource specifies how an external data source produces and consumes data.
  */
 public interface ISqlTridentDataSource {
-  IBatchSpout getProducer();
+  ITridentDataSource getProducer();
   Function getConsumer();
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/7e378c65/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/AbstractTridentProcessor.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/AbstractTridentProcessor.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/AbstractTridentProcessor.java
new file mode 100644
index 0000000..7faa7e4
--- /dev/null
+++ b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/trident/AbstractTridentProcessor.java
@@ -0,0 +1,43 @@
+/*
+ * *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance
+ *  * with the License.  You may obtain a copy of the License at
+ *  * <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;
+
+import org.apache.storm.sql.runtime.ISqlTridentDataSource;
+import storm.trident.Stream;
+import storm.trident.TridentTopology;
+
+import java.util.Map;
+
+public abstract class AbstractTridentProcessor {
+  protected Stream outputStream;
+  /**
+   * @return the output stream of the SQL
+   */
+  public Stream outputStream() {
+    return outputStream;
+  }
+
+  /**
+   * Construct the trident topology based on the SQL.
+   * @param sources the data sources.
+   */
+  public abstract TridentTopology build(Map<String, ISqlTridentDataSource> sources);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/7e378c65/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
index 82347ef..46aac4a 100644
--- 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
@@ -19,13 +19,22 @@
  */
 package org.apache.storm.sql;
 
+import backtype.storm.task.TopologyContext;
+import backtype.storm.tuple.Fields;
 import backtype.storm.tuple.Values;
 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 storm.trident.operation.BaseFunction;
+import storm.trident.operation.Function;
+import storm.trident.operation.TridentCollector;
+import storm.trident.spout.IBatchSpout;
+import storm.trident.tuple.TridentTuple;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 
 public class TestUtils {
   public static class MockDataSource implements DataSource {
@@ -46,6 +55,80 @@ public class TestUtils {
     }
   }
 
+  public static class MockSqlTridentDataSource implements ISqlTridentDataSource {
+    @Override
+    public IBatchSpout getProducer() {
+      return new MockSpout();
+    }
+
+    @Override
+    public Function getConsumer() {
+      return new CollectDataFunction();
+    }
+
+    public static class CollectDataFunction extends BaseFunction {
+      /**
+       * Collect all values in a static variable as the instance will go through serialization and deserialization.
+       */
+      private transient static final List<List<Object> > VALUES = new ArrayList<>();
+      public static List<List<Object>> getCollectedValues() {
+        return VALUES;
+      }
+
+      @Override
+      public void execute(TridentTuple tuple, TridentCollector collector) {
+        VALUES.add(tuple.getValues());
+      }
+    }
+
+    private static class MockSpout implements IBatchSpout {
+      private final ArrayList<Values> RECORDS = new ArrayList<>();
+      private final Fields OUTPUT_FIELDS = new Fields("ID");
+
+      public MockSpout() {
+        for (int i = 0; i < 5; ++i) {
+          RECORDS.add(new Values(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 CollectDataChannelHandler implements ChannelHandler {
     private final List<Values> values;
 
@@ -66,4 +149,9 @@ public class TestUtils {
       throw new RuntimeException(cause);
     }
   }
+
+  public static long monotonicNow() {
+    final long NANOSECONDS_PER_MILLISECOND = 1000000;
+    return System.nanoTime() / NANOSECONDS_PER_MILLISECOND;
+  }
 }


[15/20] storm git commit: [StormSQL] STORM-1200. Support collations of primary keys.

Posted by sr...@apache.org.
[StormSQL] STORM-1200. Support collations of primary keys.


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

Branch: refs/heads/master
Commit: a8894e626ed27867e5f3ed5c4ee997247c8647cb
Parents: 7e378c6
Author: Haohui Mai <wh...@apache.org>
Authored: Wed Nov 11 14:23:14 2015 -0800
Committer: Haohui Mai <wh...@apache.org>
Committed: Fri Dec 4 11:09:28 2015 -0800

----------------------------------------------------------------------
 .../storm-sql-core/src/codegen/data/Parser.tdd  |   1 +
 .../src/codegen/includes/parserImpls.ftl        |  10 +-
 .../jvm/org/apache/storm/sql/StormSqlImpl.java  |   4 +-
 .../apache/storm/sql/compiler/CompilerUtil.java |  72 +++++++---
 .../apache/storm/sql/compiler/PlanCompiler.java | 130 ------------------
 .../storm/sql/compiler/RelNodeCompiler.java     | 101 --------------
 .../backends/standalone/PlanCompiler.java       | 131 +++++++++++++++++++
 .../backends/standalone/RelNodeCompiler.java    | 111 ++++++++++++++++
 .../storm/sql/parser/ColumnConstraint.java      |  10 +-
 .../apache/storm/sql/parser/SqlDDLKeywords.java |  27 ++++
 .../test/org/apache/storm/sql/TestStormSql.java |   2 +-
 .../storm/sql/compiler/TestCompilerUtils.java   |   4 +-
 .../storm/sql/compiler/TestExprSemantic.java    |   1 +
 .../storm/sql/compiler/TestPlanCompiler.java    |  68 ----------
 .../storm/sql/compiler/TestRelNodeCompiler.java |  61 ---------
 .../backends/standalone/TestPlanCompiler.java   |  69 ++++++++++
 .../standalone/TestRelNodeCompiler.java         |  62 +++++++++
 .../apache/storm/sql/parser/TestSqlParser.java  |   6 +
 .../sql/runtime/AbstractChannelHandler.java     |   7 +
 19 files changed, 493 insertions(+), 384 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/a8894e62/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
index d026027..db3a675 100644
--- a/external/sql/storm-sql-core/src/codegen/data/Parser.tdd
+++ b/external/sql/storm-sql-core/src/codegen/data/Parser.tdd
@@ -21,6 +21,7 @@
 
   # List of import statements.
   imports: [
+    "org.apache.calcite.sql.validate.*",
     "org.apache.calcite.util.*",
     "org.apache.storm.sql.parser.*",
     "java.util.*"

http://git-wip-us.apache.org/repos/asf/storm/blob/a8894e62/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
index 2d87d7f..72a8546 100644
--- a/external/sql/storm-sql-core/src/codegen/includes/parserImpls.ftl
+++ b/external/sql/storm-sql-core/src/codegen/includes/parserImpls.ftl
@@ -16,12 +16,18 @@ private void ColumnDef(List<ColumnDefinition> list) :
     SqlIdentifier name;
     SqlDataTypeSpec type;
     ColumnConstraint constraint = null;
+    SqlMonotonicity monotonicity = SqlMonotonicity.NOT_MONOTONIC;
 }
 {
     name = SimpleIdentifier() { pos = getPos(); }
     type = DataType()
-    [ <PRIMARY> <KEY> { constraint = new ColumnConstraint.PrimaryKey
-    (getPos()); } ]
+    [
+      <PRIMARY> <KEY>
+      [ <ASC>   { monotonicity = SqlMonotonicity.INCREASING; }
+      | <DESC>  { monotonicity = SqlMonotonicity.DECREASING; }
+      ]
+      { constraint = new ColumnConstraint.PrimaryKey(monotonicity, getPos()); }
+    ]
     {
         list.add(new ColumnDefinition(name, type, constraint, pos));
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/a8894e62/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
index d951243..2350422 100644
--- 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
@@ -28,8 +28,8 @@ import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.tools.FrameworkConfig;
 import org.apache.calcite.tools.Frameworks;
 import org.apache.calcite.tools.Planner;
-import org.apache.storm.sql.compiler.PlanCompiler;
 import org.apache.storm.sql.parser.ColumnConstraint;
+import org.apache.storm.sql.compiler.backends.standalone.PlanCompiler;
 import org.apache.storm.sql.parser.ColumnDefinition;
 import org.apache.storm.sql.parser.SqlCreateTable;
 import org.apache.storm.sql.parser.StormParser;
@@ -77,7 +77,7 @@ class StormSqlImpl extends StormSql {
     TableBuilderInfo builder = new TableBuilderInfo(typeFactory);
     List<FieldInfo> fields = new ArrayList<>();
     for (ColumnDefinition col : n.fieldList()) {
-      builder.field(col.name(), col.type());
+      builder.field(col.name(), col.type(), col.constraint());
       RelDataType dataType = col.type().deriveType(typeFactory);
       Class<?> javaType = (Class<?>)typeFactory.getJavaClass(dataType);
       ColumnConstraint constraint = col.constraint();

http://git-wip-us.apache.org/repos/asf/storm/blob/a8894e62/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
index 1ef1cb7..30ea0e3 100644
--- 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
@@ -18,20 +18,28 @@
 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.Schema;
-import org.apache.calcite.schema.Statistic;
-import org.apache.calcite.schema.Statistics;
-import org.apache.calcite.schema.Table;
+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.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) {
@@ -66,6 +74,8 @@ public class CompilerUtil {
 
     private final ArrayList<FieldType> fields = new ArrayList<>();
     private final ArrayList<Object[]> rows = new ArrayList<>();
+    private int primaryKey = -1;
+    private SqlMonotonicity primaryKeyMonotonicity;
     private Statistic stats;
 
     public TableBuilderInfo field(String name, SqlTypeName type) {
@@ -74,16 +84,14 @@ public class CompilerUtil {
       return this;
     }
 
-    public TableBuilderInfo field(String name, SqlTypeName type, int
-        precision) {
-      RelDataType dataType = typeFactory.createSqlType(type, precision);
-      fields.add(new FieldType(name, dataType));
-      return this;
-    }
-
-    public TableBuilderInfo field(
-        String name, SqlDataTypeSpec type) {
+    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;
     }
@@ -99,9 +107,9 @@ public class CompilerUtil {
       return this;
     }
 
-    public Table build() {
-      final Statistic stat = stats;
-      return new Table() {
+    public StreamableTable build() {
+      final Statistic stat = buildStatistic();
+      final Table tbl = new Table() {
         @Override
         public RelDataType getRowType(
             RelDataTypeFactory relDataTypeFactory) {
@@ -123,6 +131,38 @@ public class CompilerUtil {
           return Schema.TableType.TABLE;
         }
       };
+
+      return new StreamableTable() {
+        @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.TABLE;
+        }
+      };
+    }
+
+    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/a8894e62/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/PlanCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/PlanCompiler.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/PlanCompiler.java
deleted file mode 100644
index d2d3710..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/PlanCompiler.java
+++ /dev/null
@@ -1,130 +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.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.javac.CompilingClassLoader;
-import org.apache.storm.sql.runtime.AbstractValuesProcessor;
-
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.util.ArrayDeque;
-import java.util.HashSet;
-import java.util.Queue;
-import java.util.Set;
-
-public class PlanCompiler {
-  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 backtype.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;",
-      "public final class Processor extends AbstractValuesProcessor {", "");
-  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) {
-    String lastCtx = "r";
-    Queue<RelNode> q = new ArrayDeque<>();
-    q.add(root);
-    RelNode n;
-    while ((n = q.poll()) != null) {
-      pw.print(
-          String.format("    ChannelContext CTX_%d = Channels.chain(%2$s, %3$s);\n",
-              n.getId(), lastCtx, RelNodeCompiler.getStageName(n)));
-      lastCtx = String.format("CTX_%d", n.getId());
-
-      if (n instanceof TableScan) {
-        tables.add((TableScan)n);
-      }
-
-      for (RelNode i : n.getInputs()) {
-        q.add(i);
-      }
-    }
-  }
-
-  public AbstractValuesProcessor compile(RelNode plan) throws Exception {
-    String javaCode = generateJavaSource(plan);
-    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/a8894e62/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RelNodeCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RelNodeCompiler.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RelNodeCompiler.java
deleted file mode 100644
index eea451f..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RelNodeCompiler.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
- * <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.Joiner;
-import org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Filter;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.core.TableScan;
-
-import java.io.PrintWriter;
-
-/**
- * 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 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) {",
-    ""
-  );
-
-  RelNodeCompiler(PrintWriter pw, JavaTypeFactory typeFactory) {
-    this.pw = pw;
-    this.typeFactory = typeFactory;
-  }
-
-  @Override
-  public Void visitFilter(Filter filter) throws Exception {
-    beginStage(filter);
-    ExprCompiler compiler = new ExprCompiler(pw, typeFactory);
-    String r = filter.getCondition().accept(compiler);
-    pw.print(String.format("    if (%s) { ctx.emit(_data); }\n", r));
-    endStage();
-    return null;
-  }
-
-  @Override
-  public Void visitProject(Project project) throws Exception {
-    beginStage(project);
-    ExprCompiler compiler = new ExprCompiler(pw, typeFactory);
-
-    int size = project.getChildExps().size();
-    String[] res = new String[size];
-    for (int i = 0; i < size; ++i) {
-      res[i] = project.getChildExps().get(i).accept(compiler);
-    }
-
-    pw.print(String.format("    ctx.emit(new Values(%s));\n",
-                           Joiner.on(',').join(res)));
-    endStage();
-    return null;
-  }
-
-  @Override
-  public Void defaultValue(RelNode n) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public Void visitTableScan(TableScan scan) throws Exception {
-    beginStage(scan);
-    pw.print("    ctx.emit(_data);\n");
-    endStage();
-    return null;
-  }
-
-  private void beginStage(RelNode n) {
-    pw.print(String.format(STAGE_PROLOGUE, getStageName(n)));
-  }
-
-  private void endStage() {
-    pw.print("  }\n  };\n");
-  }
-
-  static String getStageName(RelNode n) {
-    return n.getClass().getSimpleName().toUpperCase() + "_" + n.getId();
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/a8894e62/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
new file mode 100644
index 0000000..46009e9
--- /dev/null
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/PlanCompiler.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
+ * <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 java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayDeque;
+import java.util.HashSet;
+import java.util.Queue;
+import java.util.Set;
+
+public class PlanCompiler {
+  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 backtype.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;",
+      "public final class Processor extends AbstractValuesProcessor {", "");
+  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) {
+    String lastCtx = "r";
+    Queue<RelNode> q = new ArrayDeque<>();
+    q.add(root);
+    RelNode n;
+    while ((n = q.poll()) != null) {
+      pw.print(
+          String.format("    ChannelContext CTX_%d = Channels.chain(%2$s, %3$s);\n",
+              n.getId(), lastCtx, RelNodeCompiler.getStageName(n)));
+      lastCtx = String.format("CTX_%d", n.getId());
+
+      if (n instanceof TableScan) {
+        tables.add((TableScan)n);
+      }
+
+      for (RelNode i : n.getInputs()) {
+        q.add(i);
+      }
+    }
+  }
+
+  public AbstractValuesProcessor compile(RelNode plan) throws Exception {
+    String javaCode = generateJavaSource(plan);
+    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/a8894e62/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
new file mode 100644
index 0000000..6d51a11
--- /dev/null
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/backends/standalone/RelNodeCompiler.java
@@ -0,0 +1,111 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.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.Filter;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.stream.Delta;
+import org.apache.storm.sql.compiler.ExprCompiler;
+import org.apache.storm.sql.compiler.PostOrderRelNodeVisitor;
+
+import java.io.PrintWriter;
+
+/**
+ * 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 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 STAGE_PASSTHROUGH = NEW_LINE_JOINER.join(
+      "  private static final ChannelHandler %1$s = AbstractChannelHandler.PASS_THROUGH;",
+      "");
+
+  RelNodeCompiler(PrintWriter pw, JavaTypeFactory typeFactory) {
+    this.pw = pw;
+    this.typeFactory = typeFactory;
+  }
+
+  @Override
+  public Void visitDelta(Delta delta) throws Exception {
+    pw.print(String.format(STAGE_PASSTHROUGH, getStageName(delta)));
+    return null;
+  }
+
+  @Override
+  public Void visitFilter(Filter filter) throws Exception {
+    beginStage(filter);
+    ExprCompiler compiler = new ExprCompiler(pw, typeFactory);
+    String r = filter.getCondition().accept(compiler);
+    pw.print(String.format("    if (%s) { ctx.emit(_data); }\n", r));
+    endStage();
+    return null;
+  }
+
+  @Override
+  public Void visitProject(Project project) throws Exception {
+    beginStage(project);
+    ExprCompiler compiler = new ExprCompiler(pw, typeFactory);
+
+    int size = project.getChildExps().size();
+    String[] res = new String[size];
+    for (int i = 0; i < size; ++i) {
+      res[i] = project.getChildExps().get(i).accept(compiler);
+    }
+
+    pw.print(String.format("    ctx.emit(new Values(%s));\n",
+                           Joiner.on(',').join(res)));
+    endStage();
+    return null;
+  }
+
+  @Override
+  public Void defaultValue(RelNode n) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Void visitTableScan(TableScan scan) throws Exception {
+    pw.print(String.format(STAGE_PASSTHROUGH, getStageName(scan)));
+    return null;
+  }
+
+  private void beginStage(RelNode n) {
+    pw.print(String.format(STAGE_PROLOGUE, getStageName(n)));
+  }
+
+  private void endStage() {
+    pw.print("  }\n  };\n");
+  }
+
+  static String getStageName(RelNode n) {
+    return n.getClass().getSimpleName().toUpperCase() + "_" + n.getId();
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/a8894e62/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
index 6daf6d3..c67d8e7 100644
--- 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
@@ -21,6 +21,7 @@ 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(
@@ -29,8 +30,13 @@ public class ColumnConstraint extends SqlLiteral {
   }
 
   public static class PrimaryKey extends ColumnConstraint {
-    public PrimaryKey(SqlParserPos pos) {
-      super("PRIMARY", SqlTypeName.SYMBOL, pos);
+    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/a8894e62/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
new file mode 100644
index 0000000..3112e53
--- /dev/null
+++ b/external/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/a8894e62/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
index 9facd8a..b238e18 100644
--- 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
@@ -63,7 +63,7 @@ public class TestStormSql {
   public void testExternalDataSource() throws Exception {
     List<String> stmt = new ArrayList<>();
     stmt.add("CREATE EXTERNAL TABLE FOO (ID INT) LOCATION 'mock:///foo'");
-    stmt.add("SELECT ID + 1 FROM FOO WHERE ID > 2");
+    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);

http://git-wip-us.apache.org/repos/asf/storm/blob/a8894e62/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestCompilerUtils.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestCompilerUtils.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestCompilerUtils.java
index c582fdc..0e5fa0b 100644
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestCompilerUtils.java
+++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestCompilerUtils.java
@@ -5,6 +5,7 @@ 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.sql.SqlNode;
 import org.apache.calcite.sql.parser.SqlParseException;
@@ -17,8 +18,9 @@ public class TestCompilerUtils {
     SchemaPlus schema = Frameworks.createRootSchema(true);
     JavaTypeFactory typeFactory = new JavaTypeFactoryImpl
         (RelDataTypeSystem.DEFAULT);
-    Table table = new CompilerUtil.TableBuilderInfo(typeFactory)
+    StreamableTable streamableTable = new CompilerUtil.TableBuilderInfo(typeFactory)
         .field("ID", SqlTypeName.INTEGER).build();
+    Table table = streamableTable.stream();
     schema.add("FOO", table);
     schema.add("BAR", table);
     FrameworkConfig config = Frameworks.newConfigBuilder().defaultSchema(

http://git-wip-us.apache.org/repos/asf/storm/blob/a8894e62/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
index 40bb884..febfdb5 100644
--- 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
@@ -24,6 +24,7 @@ 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;

http://git-wip-us.apache.org/repos/asf/storm/blob/a8894e62/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestPlanCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestPlanCompiler.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestPlanCompiler.java
deleted file mode 100644
index 2b3a2d3..0000000
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestPlanCompiler.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.compiler;
-
-import backtype.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.ChannelHandler;
-import org.apache.storm.sql.runtime.DataSource;
-import org.apache.storm.sql.runtime.AbstractValuesProcessor;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.ArrayList;
-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));
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/a8894e62/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestRelNodeCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestRelNodeCompiler.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestRelNodeCompiler.java
deleted file mode 100644
index 99083cb..0000000
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestRelNodeCompiler.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.compiler;
-
-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 static org.hamcrest.CoreMatchers.*;
-
-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);
-      compiler.visitFilter(filter);
-      pw.flush();
-      Assert.assertThat(sw.toString(), containsString("> 3"));
-    }
-
-    try (StringWriter sw = new StringWriter();
-         PrintWriter pw = new PrintWriter(sw)
-    ) {
-      RelNodeCompiler compiler = new RelNodeCompiler(pw, typeFactory);
-      compiler.visitProject(project);
-      pw.flush();
-      Assert.assertThat(sw.toString(), containsString("plus("));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/a8894e62/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
new file mode 100644
index 0000000..8d7fc65
--- /dev/null
+++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestPlanCompiler.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.compiler.backends.standalone;
+
+import backtype.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.compiler.TestCompilerUtils;
+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.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));
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/a8894e62/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
new file mode 100644
index 0000000..76eba1d
--- /dev/null
+++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/backends/standalone/TestRelNodeCompiler.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.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.apache.storm.sql.compiler.TestCompilerUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+
+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);
+      compiler.visitFilter(filter);
+      pw.flush();
+      Assert.assertThat(sw.toString(), containsString("> 3"));
+    }
+
+    try (StringWriter sw = new StringWriter();
+         PrintWriter pw = new PrintWriter(sw)
+    ) {
+      RelNodeCompiler compiler = new RelNodeCompiler(pw, typeFactory);
+      compiler.visitProject(project);
+      pw.flush();
+      Assert.assertThat(sw.toString(), containsString("plus("));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/a8894e62/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
index 41e031d..b957565 100644
--- 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
@@ -29,6 +29,12 @@ public class TestSqlParser {
     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)";

http://git-wip-us.apache.org/repos/asf/storm/blob/a8894e62/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
index 73a078c..892d2e4 100644
--- 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
@@ -34,4 +34,11 @@ public abstract class AbstractChannelHandler implements ChannelHandler {
   public void exceptionCaught(Throwable cause) {
 
   }
+
+  public static final AbstractChannelHandler PASS_THROUGH = new AbstractChannelHandler() {
+    @Override
+    public void dataReceived(ChannelContext ctx, Values data) {
+      ctx.emit(data);
+    }
+  };
 }


[10/20] storm git commit: [StormSQL] Compile logical plans to Java code.

Posted by sr...@apache.org.
[StormSQL] Compile logical plans to Java code.


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

Branch: refs/heads/master
Commit: 0a257f12527a4aec6c477147ff8390ac08c92136
Parents: bd4f6dc
Author: Haohui Mai <wh...@apache.org>
Authored: Thu Oct 22 16:52:59 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Fri Dec 4 11:09:02 2015 -0800

----------------------------------------------------------------------
 external/sql/pom.xml                            |   1 +
 external/sql/storm-sql-core/pom.xml             |  11 +
 .../apache/storm/sql/compiler/CompilerUtil.java |  95 +++++++-
 .../apache/storm/sql/compiler/PlanCompiler.java | 131 +++++++++++
 .../storm/sql/compiler/RelNodeCompiler.java     |  45 ++--
 .../storm/sql/javac/CompilingClassLoader.java   | 221 +++++++++++++++++++
 .../storm/sql/compiler/TestPlanCompiler.java    |  53 +++++
 .../storm/sql/compiler/TestRelNodeCompiler.java |  10 -
 .../apache/storm/sql/compiler/TestUtils.java    | 125 ++++-------
 external/sql/storm-sql-runtime/pom.xml          |  65 ++++++
 .../storm/sql/storm/AbstractChannelHandler.java |  35 +++
 .../apache/storm/sql/storm/ChannelContext.java  |  28 +++
 .../apache/storm/sql/storm/ChannelHandler.java  |  37 ++++
 .../org/apache/storm/sql/storm/Channels.java    |  78 +++++++
 .../org/apache/storm/sql/storm/DataSource.java  |  27 +++
 .../storm/runtime/AbstractValuesProcessor.java  |  29 +++
 16 files changed, 881 insertions(+), 110 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/0a257f12/external/sql/pom.xml
----------------------------------------------------------------------
diff --git a/external/sql/pom.xml b/external/sql/pom.xml
index 73e7b31..e4dae94 100644
--- a/external/sql/pom.xml
+++ b/external/sql/pom.xml
@@ -38,5 +38,6 @@
 
     <modules>
         <module>storm-sql-core</module>
+        <module>storm-sql-runtime</module>
     </modules>
 </project>

http://git-wip-us.apache.org/repos/asf/storm/blob/0a257f12/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
index 3ca1ced..bcace6c 100644
--- a/external/sql/storm-sql-core/pom.xml
+++ b/external/sql/storm-sql-core/pom.xml
@@ -43,6 +43,12 @@
             <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.calcite</groupId>
             <artifactId>calcite-core</artifactId>
             <version>${calcite.version}</version>
@@ -52,6 +58,11 @@
             <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>

http://git-wip-us.apache.org/repos/asf/storm/blob/0a257f12/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
index 5e7453a..1a48052 100644
--- 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
@@ -17,9 +17,22 @@
  */
 package org.apache.storm.sql.compiler;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.Statistic;
+import org.apache.calcite.schema.Statistics;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Util;
 
-class CompilerUtil {
+import java.util.ArrayList;
+
+public class CompilerUtil {
   static String escapeJavaString(String s, boolean nullMeansNull) {
       if(s == null) {
         return nullMeansNull ? "null" : "\"\"";
@@ -32,4 +45,84 @@ class CompilerUtil {
         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 Statistic stats;
+
+    public TableBuilderInfo field(String name, SqlTypeName type) {
+      RelDataType dataType = typeFactory.createSqlType(type);
+      fields.add(new FieldType(name, dataType));
+      return this;
+    }
+
+    public TableBuilderInfo field(String name, SqlTypeName type, int
+        precision) {
+      RelDataType dataType = typeFactory.createSqlType(type, precision);
+      fields.add(new FieldType(name, dataType));
+      return this;
+    }
+
+    public TableBuilderInfo field(
+        String name, SqlDataTypeSpec type) {
+      RelDataType dataType = type.deriveType(typeFactory);
+      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 Table build() {
+      final Statistic stat = stats;
+      return 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.TABLE;
+        }
+      };
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/0a257f12/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/PlanCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/PlanCompiler.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/PlanCompiler.java
new file mode 100644
index 0000000..d006261
--- /dev/null
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/PlanCompiler.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
+ * <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.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.javac.CompilingClassLoader;
+import org.apache.storm.sql.storm.runtime.AbstractValuesProcessor;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.ArrayDeque;
+import java.util.HashSet;
+import java.util.Queue;
+import java.util.Set;
+
+public class PlanCompiler {
+  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 backtype.storm.tuple.Values;",
+      "import org.apache.storm.sql.storm.AbstractChannelHandler;",
+      "import org.apache.storm.sql.storm.Channels;",
+      "import org.apache.storm.sql.storm.ChannelContext;",
+      "import org.apache.storm.sql.storm.ChannelHandler;",
+      "import org.apache.storm.sql.storm.DataSource;",
+      "import org.apache.storm.sql.storm.runtime.AbstractValuesProcessor;",
+      "public final class Processor extends AbstractValuesProcessor {", "");
+  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) {
+    String lastCtx = "r";
+    Queue<RelNode> q = new ArrayDeque<>();
+    q.add(root);
+    RelNode n;
+    while ((n = q.poll()) != null) {
+      pw.print(
+          String.format("    ChannelContext CTX_%d = Channels.chain(%2$s, " +
+                            "%3$s);\n", n.getId(), lastCtx, RelNodeCompiler
+              .getStageName(n)));
+      lastCtx = String.format("CTX_%d", n.getId());
+
+      if (n instanceof TableScan) {
+        tables.add((TableScan)n);
+      }
+
+      for (RelNode i : n.getInputs()) {
+        q.add(i);
+      }
+    }
+  }
+
+  public AbstractValuesProcessor compile(RelNode plan) throws Exception {
+    String javaCode = generateJavaSource(plan);
+    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/0a257f12/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RelNodeCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RelNodeCompiler.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RelNodeCompiler.java
index 0550035..5a21fba2 100644
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RelNodeCompiler.java
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RelNodeCompiler.java
@@ -33,8 +33,17 @@ import java.util.TreeSet;
  * 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 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) {",
+    ""
+  );
 
   public Set<String> getReferredTables() {
     return Collections.unmodifiableSet(referredTables);
@@ -49,19 +58,17 @@ class RelNodeCompiler extends PostOrderRelNodeVisitor<Void> {
 
   @Override
   Void visitFilter(Filter filter) throws Exception {
-    beginFunction(filter);
-    pw.print("  if (_data == null) return null;\n");
+    beginStage(filter);
     ExprCompiler compiler = new ExprCompiler(pw, typeFactory);
     String r = filter.getCondition().accept(compiler);
-    pw.print(String.format("  return %s ? _data : null;\n", r));
-    endFunction();
+    pw.print(String.format("    if (%s) { ctx.emit(_data); }\n", r));
+    endStage();
     return null;
   }
 
   @Override
   Void visitProject(Project project) throws Exception {
-    beginFunction(project);
-    pw.print("  if (_data == null) return null;\n");
+    beginStage(project);
     ExprCompiler compiler = new ExprCompiler(pw, typeFactory);
 
     int size = project.getChildExps().size();
@@ -70,9 +77,9 @@ class RelNodeCompiler extends PostOrderRelNodeVisitor<Void> {
       res[i] = project.getChildExps().get(i).accept(compiler);
     }
 
-    pw.print(String.format("  return new Values(%s);\n", Joiner.on(',').join
-        (res)));
-    endFunction();
+    pw.print(String.format("    ctx.emit(new Values(%s));\n",
+                           Joiner.on(',').join(res)));
+    endStage();
     return null;
   }
 
@@ -85,23 +92,21 @@ class RelNodeCompiler extends PostOrderRelNodeVisitor<Void> {
   Void visitTableScan(TableScan scan) throws Exception {
     String tableName = Joiner.on('_').join(scan.getTable().getQualifiedName());
     referredTables.add(tableName);
-    beginFunction(scan);
-    pw.print(String.format("  return _datasources[TABLE_%s].next();\n",
-                           tableName));
-    endFunction();
+    beginStage(scan);
+    pw.print("    ctx.emit(_data);\n");
+    endStage();
     return null;
   }
 
-  private void beginFunction(RelNode n) {
-    pw.print(String.format("private Values %s(%s) {\n", getFunctionName(n), n
-        .getInputs().isEmpty() ? "" : "Values _data"));
+  private void beginStage(RelNode n) {
+    pw.print(String.format(STAGE_PROLOGUE, getStageName(n)));
   }
 
-  private void endFunction() {
-    pw.print("}\n");
+  private void endStage() {
+    pw.print("  }\n  };\n");
   }
 
-  static String getFunctionName(RelNode n) {
-    return n.getClass().getSimpleName() + "_" + n.getId();
+  static String getStageName(RelNode n) {
+    return n.getClass().getSimpleName().toUpperCase() + "_" + n.getId();
   }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/0a257f12/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
new file mode 100644
index 0000000..cf76964
--- /dev/null
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/javac/CompilingClassLoader.java
@@ -0,0 +1,221 @@
+/*
+ * 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);
+    }
+  }
+
+  /**
+   * 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/0a257f12/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestPlanCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestPlanCompiler.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestPlanCompiler.java
new file mode 100644
index 0000000..8d22a53
--- /dev/null
+++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestPlanCompiler.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
+ * <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 backtype.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.storm.ChannelHandler;
+import org.apache.storm.sql.storm.DataSource;
+import org.apache.storm.sql.storm.runtime.AbstractValuesProcessor;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.ArrayList;
+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";
+    TestUtils.CalciteState state = TestUtils.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());
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/0a257f12/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestRelNodeCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestRelNodeCompiler.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestRelNodeCompiler.java
index 61f5409..cedb48b 100644
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestRelNodeCompiler.java
+++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestRelNodeCompiler.java
@@ -38,16 +38,6 @@ public class TestRelNodeCompiler {
         RelDataTypeSystem.DEFAULT);
     LogicalProject project = (LogicalProject) state.tree;
     LogicalFilter filter = (LogicalFilter) project.getInput();
-    TableScan scan = (TableScan) filter.getInput();
-
-    try (StringWriter sw = new StringWriter();
-         PrintWriter pw = new PrintWriter(sw)
-    ) {
-      RelNodeCompiler compiler = new RelNodeCompiler(pw, typeFactory);
-      compiler.visitTableScan(scan);
-      pw.flush();
-      Assert.assertTrue(sw.toString().contains("_datasources[TABLE_FOO]"));
-    }
 
     try (StringWriter sw = new StringWriter();
          PrintWriter pw = new PrintWriter(sw)

http://git-wip-us.apache.org/repos/asf/storm/blob/0a257f12/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestUtils.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestUtils.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestUtils.java
index ae4300a..6731c90 100644
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestUtils.java
+++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestUtils.java
@@ -1,13 +1,11 @@
 package org.apache.storm.sql.compiler;
 
-import com.google.common.collect.ImmutableList;
+import backtype.storm.tuple.Values;
+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.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.schema.Schema;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.schema.Statistic;
-import org.apache.calcite.schema.Statistics;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.parser.SqlParseException;
@@ -17,15 +15,21 @@ 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.calcite.util.ImmutableBitSet;
+import org.apache.storm.sql.storm.ChannelContext;
+import org.apache.storm.sql.storm.ChannelHandler;
+import org.apache.storm.sql.storm.DataSource;
 
 import java.util.ArrayList;
+import java.util.List;
 
-class TestUtils {
+public class TestUtils {
   static CalciteState sqlOverDummyTable(String sql)
       throws RelConversionException, ValidationException, SqlParseException {
     SchemaPlus schema = Frameworks.createRootSchema(true);
-    Table table = newTable().field("ID", SqlTypeName.INTEGER).build();
+    JavaTypeFactory typeFactory = new JavaTypeFactoryImpl
+        (RelDataTypeSystem.DEFAULT);
+    Table table = new CompilerUtil.TableBuilderInfo(typeFactory)
+        .field("ID", SqlTypeName.INTEGER).build();
     schema.add("FOO", table);
     FrameworkConfig config = Frameworks.newConfigBuilder().defaultSchema(
         schema).build();
@@ -36,90 +40,53 @@ class TestUtils {
     return new CalciteState(schema, tree);
   }
 
-  static class TableBuilderInfo {
-    private static class FieldType {
-      private static final int NO_PRECISION = -1;
-      private final String name;
-      private final SqlTypeName type;
-      private final int precision;
-
-      private FieldType(String name, SqlTypeName type, int precision) {
-        this.name = name;
-        this.type = type;
-        this.precision = precision;
-      }
+  static class CalciteState {
+    final SchemaPlus schema;
+    final RelNode tree;
 
-      private FieldType(String name, SqlTypeName type) {
-        this(name, type, NO_PRECISION);
-      }
+    private CalciteState(SchemaPlus schema, RelNode tree) {
+      this.schema = schema;
+      this.tree = tree;
     }
+  }
 
-    private final ArrayList<FieldType> fields = new ArrayList<>();
-    private final ArrayList<Object[]> rows = new ArrayList<>();
-    private Statistic stats;
+  public static class MockDataSource implements DataSource {
+    private final ArrayList<Values> RECORDS = new ArrayList<>();
 
-    TableBuilderInfo field(String name, SqlTypeName type) {
-      fields.add(new FieldType(name, type));
-      return this;
+    public MockDataSource() {
+      for (int i = 0; i < 5; ++i) {
+        RECORDS.add(new Values(i));
+      }
     }
 
-    TableBuilderInfo field(String name, SqlTypeName type, int precision) {
-      fields.add(new FieldType(name, type, precision));
-      return this;
+    @Override
+    public void open(ChannelContext ctx) {
+      for (Values v : RECORDS) {
+        ctx.emit(v);
+      }
+      ctx.fireChannelInactive();
     }
+  }
 
-    TableBuilderInfo statistics(Statistic stats) {
-      this.stats = stats;
-      return this;
-    }
+  public static class CollectDataChannelHandler implements ChannelHandler {
+    private final List<Values> values;
 
-    TableBuilderInfo rows(Object[] data) {
-      rows.add(data);
-      return this;
+    public CollectDataChannelHandler(List<Values> values) {
+      this.values = values;
     }
 
-    Table build() {
-      final Statistic stat = stats;
-      return new Table() {
-        @Override
-        public RelDataType getRowType(
-            RelDataTypeFactory relDataTypeFactory) {
-          RelDataTypeFactory.FieldInfoBuilder b = relDataTypeFactory.builder();
-          for (FieldType f : fields) {
-            if (f.precision == FieldType.NO_PRECISION) {
-              b.add(f.name, f.type);
-            } else {
-              b.add(f.name, f.type, f.precision);
-            }
-          }
-          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.TABLE;
-        }
-      };
+    @Override
+    public void dataReceived(ChannelContext ctx, Values data) {
+      values.add(data);
     }
-  }
 
-  static TableBuilderInfo newTable() {
-    return new TableBuilderInfo();
-  }
+    @Override
+    public void channelInactive(ChannelContext ctx) {}
 
-  static class CalciteState {
-    final SchemaPlus schema;
-    final RelNode tree;
-
-    private CalciteState(SchemaPlus schema, RelNode tree) {
-      this.schema = schema;
-      this.tree = tree;
+    @Override
+    public void exceptionCaught(Throwable cause) {
+      throw new RuntimeException(cause);
     }
   }
+
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/0a257f12/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
new file mode 100644
index 0000000..62f2d95
--- /dev/null
+++ b/external/sql/storm-sql-runtime/pom.xml
@@ -0,0 +1,65 @@
+<?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>0.11.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>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+            <scope>test</scope>
+        </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>
+    </build>
+</project>

http://git-wip-us.apache.org/repos/asf/storm/blob/0a257f12/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/AbstractChannelHandler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/AbstractChannelHandler.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/AbstractChannelHandler.java
new file mode 100644
index 0000000..cf110e3
--- /dev/null
+++ b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/AbstractChannelHandler.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.storm;
+
+import backtype.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) {
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/0a257f12/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/ChannelContext.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/ChannelContext.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/ChannelContext.java
new file mode 100644
index 0000000..a2806b2
--- /dev/null
+++ b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/ChannelContext.java
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <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.storm;
+
+import backtype.storm.tuple.Values;
+
+public interface ChannelContext {
+  /**
+   * Emit data to the next stage of the data pipeline.
+   */
+  void emit(Values data);
+  void fireChannelInactive();
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/0a257f12/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/ChannelHandler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/ChannelHandler.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/ChannelHandler.java
new file mode 100644
index 0000000..8cd3a28
--- /dev/null
+++ b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/ChannelHandler.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.storm;
+
+import backtype.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);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/0a257f12/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/Channels.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/Channels.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/Channels.java
new file mode 100644
index 0000000..b5bb619
--- /dev/null
+++ b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/Channels.java
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <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.storm;
+
+import backtype.storm.tuple.Values;
+
+public class Channels {
+  private static final ChannelContext VOID_CTX = new ChannelContext() {
+    @Override
+    public void emit(Values data) {}
+
+    @Override
+    public void fireChannelInactive() {}
+  };
+
+  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);
+    }
+  }
+
+  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();
+    }
+  }
+
+  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/0a257f12/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/DataSource.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/DataSource.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/DataSource.java
new file mode 100644
index 0000000..84fa6e0
--- /dev/null
+++ b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/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
+ * <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.storm;
+
+/**
+ * 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/0a257f12/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/runtime/AbstractValuesProcessor.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/runtime/AbstractValuesProcessor.java b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/runtime/AbstractValuesProcessor.java
new file mode 100644
index 0000000..bd068be
--- /dev/null
+++ b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/storm/runtime/AbstractValuesProcessor.java
@@ -0,0 +1,29 @@
+package org.apache.storm.sql.storm.runtime;
+
+import backtype.storm.tuple.Values;
+import org.apache.storm.sql.storm.ChannelHandler;
+import org.apache.storm.sql.storm.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);
+}


[06/20] storm git commit: [StormSQL] STORM-1130. Support AND, OR, NOT operators in StormSQL.

Posted by sr...@apache.org.
[StormSQL] STORM-1130. Support AND, OR, NOT operators in StormSQL.


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

Branch: refs/heads/master
Commit: 915f135438c08a58bbca6d4ff27386b05254e27f
Parents: 0a257f1
Author: Haohui Mai <wh...@apache.org>
Authored: Mon Oct 26 17:29:20 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Fri Dec 4 11:09:02 2015 -0800

----------------------------------------------------------------------
 .../apache/storm/sql/compiler/ExprCompiler.java | 75 +++++++++++++++++---
 .../storm/sql/compiler/TestExprCompiler.java    |  1 -
 .../storm/sql/compiler/TestPlanCompiler.java    | 14 ++++
 3 files changed, 81 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/915f1354/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java
index b9d8f88..0d35593 100644
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java
@@ -42,6 +42,7 @@ import java.util.AbstractMap;
 import java.util.IdentityHashMap;
 import java.util.Map;
 
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.AND;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.DIVIDE;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.DIVIDE_INTEGER;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.GREATER_THAN;
@@ -50,6 +51,8 @@ import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LESS_THAN;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LESS_THAN_OR_EQUAL;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MINUS;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MULTIPLY;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.NOT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.OR;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.PLUS;
 
 /**
@@ -71,8 +74,10 @@ class ExprCompiler implements RexVisitor<String> {
     if (expr.containsKey(rexInputRef)) {
       return expr.get(rexInputRef);
     }
-    String name = printExpr(rexInputRef, String.format("_data.get(%d)",
-                            rexInputRef.getIndex()));
+    String name = reserveName(rexInputRef);
+    String typeName = javaTypeName(rexInputRef);
+    pw.print(String.format("%s %s = (%s)(_data.get(%d));\n", typeName, name,
+                           typeName, rexInputRef.getIndex()));
     expr.put(rexInputRef, name);
     return name;
   }
@@ -149,12 +154,14 @@ class ExprCompiler implements RexVisitor<String> {
     throw new UnsupportedOperationException();
   }
 
-  private String printExpr(RexNode node, String definition) {
-    String name = "t" + expr.size();
+  private String javaTypeName(RexNode node) {
     Type ty = typeFactory.getJavaClass(node.getType());
-    String typeName = ((Class<?>)ty).getCanonicalName();
-    pw.append(
-        String.format("%s %s = (%s)(%s);\n", typeName, name, typeName, definition));
+    return ((Class<?>)ty).getCanonicalName();
+  }
+
+  private String reserveName(RexNode node) {
+    String name = "t" + expr.size();
+    expr.put(node, name);
     return name;
   }
 
@@ -181,7 +188,10 @@ class ExprCompiler implements RexVisitor<String> {
           .put(infixBinary(MINUS, "-"))
           .put(infixBinary(MULTIPLY, "*"))
           .put(infixBinary(DIVIDE, "/"))
-          .put(infixBinary(DIVIDE_INTEGER, "/"));
+          .put(infixBinary(DIVIDE_INTEGER, "/"))
+          .put(AND, AND_EXPR)
+          .put(OR, OR_EXPR)
+          .put(NOT, NOT_EXPR);
       this.translators = builder.build();
     }
 
@@ -212,5 +222,54 @@ class ExprCompiler implements RexVisitor<String> {
       };
       return new AbstractMap.SimpleImmutableEntry<>(op, trans);
     }
+
+    private static final CallExprPrinter AND_EXPR = new CallExprPrinter() {
+      @Override
+      public String translate(
+          ExprCompiler compiler, RexCall call) {
+        String val = compiler.reserveName(call);
+        PrintWriter pw = compiler.pw;
+        pw.print(String.format("final %s %s;\n", compiler.javaTypeName(call),
+                               val));
+        String lhs = call.getOperands().get(0).accept(compiler);
+        pw.print(String.format("if (!(%2$s)) { %1$s = false; }\n", val, lhs));
+        pw.print("else {\n");
+        String rhs = call.getOperands().get(1).accept(compiler);
+        pw.print(String.format("  %1$s = %2$s;\n}\n", val, rhs));
+        return val;
+      }
+    };
+
+    private static final CallExprPrinter OR_EXPR = new CallExprPrinter() {
+      @Override
+      public String translate(
+          ExprCompiler compiler, RexCall call) {
+        String val = compiler.reserveName(call);
+        PrintWriter pw = compiler.pw;
+        pw.print(String.format("final %s %s;\n", compiler.javaTypeName(call),
+                               val));
+        String lhs = call.getOperands().get(0).accept(compiler);
+        pw.print(String.format("if (%2$s) { %1$s = true; }\n", val, lhs));
+        pw.print("else {\n");
+        String rhs = call.getOperands().get(1).accept(compiler);
+        pw.print(String.format("  %1$s = %2$s;\n}\n", val, rhs));
+        return val;
+      }
+    };
+
+    private static final CallExprPrinter NOT_EXPR = new CallExprPrinter() {
+      @Override
+      public String translate(
+          ExprCompiler compiler, RexCall call) {
+        String val = compiler.reserveName(call);
+        PrintWriter pw = compiler.pw;
+        String lhs = call.getOperands().get(0).accept(compiler);
+        pw.print(String.format("final %s %s;\n", compiler.javaTypeName(call),
+                               val));
+        pw.print(String.format("%1$s = !(%2$s);\n", val, lhs));
+        return val;
+      }
+    };
   }
+
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/915f1354/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprCompiler.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprCompiler.java
index 6409d63..9f516d4 100644
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprCompiler.java
+++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprCompiler.java
@@ -86,5 +86,4 @@ public class TestExprCompiler {
     assertArrayEquals(new String[]{"1 > 2", "3 + 5", "1 - 1.0E0", "3 + t0"},
                       res);
   }
-
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/915f1354/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestPlanCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestPlanCompiler.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestPlanCompiler.java
index 8d22a53..30df0f3 100644
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestPlanCompiler.java
+++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestPlanCompiler.java
@@ -50,4 +50,18 @@ public class TestPlanCompiler {
     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";
+    TestUtils.CalciteState state = TestUtils.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));
+  }
 }


[13/20] storm git commit: [StormSQL] Refactor to support compiling StormSQL to Trident topology.

Posted by sr...@apache.org.
[StormSQL] Refactor to support compiling StormSQL to Trident topology.


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

Branch: refs/heads/master
Commit: 82f10ebfd00ba71b5c95b95e917d856c5eff9c9f
Parents: 4d8cc41
Author: Haohui Mai <wh...@apache.org>
Authored: Fri Nov 6 15:50:31 2015 -0800
Committer: Haohui Mai <wh...@apache.org>
Committed: Fri Dec 4 11:09:28 2015 -0800

----------------------------------------------------------------------
 external/sql/storm-sql-core/pom.xml             |  7 ++
 .../apache/storm/sql/DataSourcesProvider.java   | 46 ----------
 .../apache/storm/sql/DataSourcesRegistry.java   | 65 --------------
 .../jvm/org/apache/storm/sql/StormSqlImpl.java  | 13 +--
 .../apache/storm/sql/compiler/CompilerUtil.java |  2 +-
 .../apache/storm/sql/compiler/ExprCompiler.java |  4 +-
 .../apache/storm/sql/compiler/PlanCompiler.java |  5 +-
 .../sql/compiler/PostOrderRelNodeVisitor.java   | 51 +++++------
 .../storm/sql/compiler/RelNodeCompiler.java     | 19 +---
 .../test/org/apache/storm/sql/TestStormSql.java | 13 +--
 .../storm/sql/compiler/TestCompilerUtils.java   | 46 ++++++++++
 .../storm/sql/compiler/TestExprCompiler.java    |  6 +-
 .../storm/sql/compiler/TestExprSemantic.java    |  3 +-
 .../storm/sql/compiler/TestPlanCompiler.java    |  5 +-
 .../storm/sql/compiler/TestRelNodeCompiler.java |  4 +-
 .../apache/storm/sql/compiler/TestUtils.java    | 92 --------------------
 external/sql/storm-sql-runtime/pom.xml          |  1 -
 .../storm/sql/runtime/DataSourcesProvider.java  | 50 +++++++++++
 .../storm/sql/runtime/DataSourcesRegistry.java  | 80 +++++++++++++++++
 .../org/apache/storm/sql/runtime/FieldInfo.java | 45 ++++++++++
 .../sql/runtime/ISqlTridentDataSource.java      | 29 ++++++
 .../test/org/apache/storm/sql/TestUtils.java    | 69 +++++++++++++++
 22 files changed, 382 insertions(+), 273 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/82f10ebf/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
index bcace6c..0713a61 100644
--- a/external/sql/storm-sql-core/pom.xml
+++ b/external/sql/storm-sql-core/pom.xml
@@ -49,6 +49,13 @@
             <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>

http://git-wip-us.apache.org/repos/asf/storm/blob/82f10ebf/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/DataSourcesProvider.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/DataSourcesProvider.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/DataSourcesProvider.java
deleted file mode 100644
index 46bfa40..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/DataSourcesProvider.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 org.apache.storm.sql.runtime.DataSource;
-
-import java.net.URI;
-import java.util.List;
-import java.util.Map;
-
-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<Map.Entry<String, Class<?>>> fields);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/82f10ebf/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/DataSourcesRegistry.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/DataSourcesRegistry.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/DataSourcesRegistry.java
deleted file mode 100644
index b45d039..0000000
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/DataSourcesRegistry.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;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.storm.sql.runtime.DataSource;
-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.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<Map.Entry<String, Class<?>>> fields) {
-    DataSourcesProvider provider = providers.get(uri.getScheme());
-    if (provider == null) {
-      return null;
-    }
-
-    return provider.construct(uri, inputFormatClass, outputFormatClass, fields);
-  }
-
-  @VisibleForTesting
-  static Map<String, DataSourcesProvider> providerMap() {
-    return providers;
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/82f10ebf/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
index 384b4fa..d951243 100644
--- 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
@@ -29,12 +29,11 @@ import org.apache.calcite.tools.FrameworkConfig;
 import org.apache.calcite.tools.Frameworks;
 import org.apache.calcite.tools.Planner;
 import org.apache.storm.sql.compiler.PlanCompiler;
+import org.apache.storm.sql.parser.ColumnConstraint;
 import org.apache.storm.sql.parser.ColumnDefinition;
 import org.apache.storm.sql.parser.SqlCreateTable;
 import org.apache.storm.sql.parser.StormParser;
-import org.apache.storm.sql.runtime.ChannelHandler;
-import org.apache.storm.sql.runtime.DataSource;
-import org.apache.storm.sql.runtime.AbstractValuesProcessor;
+import org.apache.storm.sql.runtime.*;
 
 import java.util.AbstractMap;
 import java.util.ArrayList;
@@ -76,14 +75,16 @@ class StormSqlImpl extends StormSql {
   private void handleCreateTable(
       SqlCreateTable n, Map<String, DataSource> dataSources) {
     TableBuilderInfo builder = new TableBuilderInfo(typeFactory);
-    List<Map.Entry<String, Class<?>>> fields = new ArrayList<>();
+    List<FieldInfo> fields = new ArrayList<>();
     for (ColumnDefinition col : n.fieldList()) {
       builder.field(col.name(), col.type());
       RelDataType dataType = col.type().deriveType(typeFactory);
       Class<?> javaType = (Class<?>)typeFactory.getJavaClass(dataType);
-      fields.add(new AbstractMap.SimpleImmutableEntry<String, Class<?>>
-                     (col.name(), javaType));
+      ColumnConstraint constraint = col.constraint();
+      boolean isPrimary = constraint != null && constraint instanceof ColumnConstraint.PrimaryKey;
+      fields.add(new FieldInfo(col.name(), javaType, isPrimary));
     }
+
     Table table = builder.build();
     schema.add(n.tableName(), table);
     DataSource ds = DataSourcesRegistry.construct(n.location(), n

http://git-wip-us.apache.org/repos/asf/storm/blob/82f10ebf/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
index 1a48052..1ef1cb7 100644
--- 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
@@ -33,7 +33,7 @@ import org.apache.calcite.util.Util;
 import java.util.ArrayList;
 
 public class CompilerUtil {
-  static String escapeJavaString(String s, boolean nullMeansNull) {
+  public static String escapeJavaString(String s, boolean nullMeansNull) {
       if(s == null) {
         return nullMeansNull ? "null" : "\"\"";
       } else {

http://git-wip-us.apache.org/repos/asf/storm/blob/82f10ebf/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java
index 77fdf0c..01024f0 100644
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/ExprCompiler.java
@@ -48,13 +48,13 @@ import static org.apache.calcite.sql.fun.SqlStdOperatorTable.*;
 /**
  * Compile RexNode on top of the Tuple abstraction.
  */
-class ExprCompiler implements RexVisitor<String> {
+public class ExprCompiler implements RexVisitor<String> {
   private final PrintWriter pw;
   private final JavaTypeFactory typeFactory;
   private static final ImpTable IMP_TABLE = new ImpTable();
   private int nameCount;
 
-  ExprCompiler(PrintWriter pw, JavaTypeFactory typeFactory) {
+  public ExprCompiler(PrintWriter pw, JavaTypeFactory typeFactory) {
     this.pw = pw;
     this.typeFactory = typeFactory;
   }

http://git-wip-us.apache.org/repos/asf/storm/blob/82f10ebf/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/PlanCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/PlanCompiler.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/PlanCompiler.java
index 1096f5b..d2d3710 100644
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/PlanCompiler.java
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/PlanCompiler.java
@@ -96,9 +96,8 @@ public class PlanCompiler {
     RelNode n;
     while ((n = q.poll()) != null) {
       pw.print(
-          String.format("    ChannelContext CTX_%d = Channels.chain(%2$s, " +
-                            "%3$s);\n", n.getId(), lastCtx, RelNodeCompiler
-              .getStageName(n)));
+          String.format("    ChannelContext CTX_%d = Channels.chain(%2$s, %3$s);\n",
+              n.getId(), lastCtx, RelNodeCompiler.getStageName(n)));
       lastCtx = String.format("CTX_%d", n.getId());
 
       if (n instanceof TableScan) {

http://git-wip-us.apache.org/repos/asf/storm/blob/82f10ebf/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/PostOrderRelNodeVisitor.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/PostOrderRelNodeVisitor.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/PostOrderRelNodeVisitor.java
index 6277e28..bb7c8d1 100644
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/PostOrderRelNodeVisitor.java
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/PostOrderRelNodeVisitor.java
@@ -18,22 +18,11 @@
 package org.apache.storm.sql.compiler;
 
 import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Aggregate;
-import org.apache.calcite.rel.core.Calc;
-import org.apache.calcite.rel.core.Collect;
-import org.apache.calcite.rel.core.Correlate;
-import org.apache.calcite.rel.core.Exchange;
-import org.apache.calcite.rel.core.Filter;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.core.Sample;
-import org.apache.calcite.rel.core.Sort;
-import org.apache.calcite.rel.core.TableScan;
-import org.apache.calcite.rel.core.Uncollect;
-import org.apache.calcite.rel.core.Window;
+import org.apache.calcite.rel.core.*;
 import org.apache.calcite.rel.stream.Delta;
 
-abstract class PostOrderRelNodeVisitor<T> {
-  final T traverse(RelNode n) throws Exception {
+public abstract class PostOrderRelNodeVisitor<T> {
+  public final T traverse(RelNode n) throws Exception {
     for (RelNode input : n.getInputs()) {
       traverse(input);
     }
@@ -58,6 +47,8 @@ abstract class PostOrderRelNodeVisitor<T> {
       return visitSample((Sample) n);
     } else if (n instanceof Sort) {
       return visitSort((Sort) n);
+    } else if (n instanceof TableModify) {
+      return visitTableModify((TableModify) n);
     } else if (n instanceof TableScan) {
       return visitTableScan((TableScan) n);
     } else if (n instanceof Uncollect) {
@@ -69,59 +60,63 @@ abstract class PostOrderRelNodeVisitor<T> {
     }
   }
 
-  T visitAggregate(Aggregate aggregate) throws Exception {
+  public T visitAggregate(Aggregate aggregate) throws Exception {
     return defaultValue(aggregate);
   }
 
-  T visitCalc(Calc calc) throws Exception {
+  public T visitCalc(Calc calc) throws Exception {
     return defaultValue(calc);
   }
 
-  T visitCollect(Collect collect) throws Exception {
+  public T visitCollect(Collect collect) throws Exception {
     return defaultValue(collect);
   }
 
-  T visitCorrelate(Correlate correlate) throws Exception {
+  public T visitCorrelate(Correlate correlate) throws Exception {
     return defaultValue(correlate);
   }
 
-  T visitDelta(Delta delta) throws Exception {
+  public T visitDelta(Delta delta) throws Exception {
     return defaultValue(delta);
   }
 
-  T visitExchange(Exchange exchange) throws Exception {
+  public T visitExchange(Exchange exchange) throws Exception {
     return defaultValue(exchange);
   }
 
-  T visitProject(Project project) throws Exception {
+  public T visitProject(Project project) throws Exception {
     return defaultValue(project);
   }
 
-  T visitFilter(Filter filter) throws Exception {
+  public T visitFilter(Filter filter) throws Exception {
     return defaultValue(filter);
   }
 
-  T visitSample(Sample sample) throws Exception {
+  public T visitSample(Sample sample) throws Exception {
     return defaultValue(sample);
   }
 
-  T visitSort(Sort sort) throws Exception {
+  public T visitSort(Sort sort) throws Exception {
     return defaultValue(sort);
   }
 
-  T visitTableScan(TableScan scan) throws Exception {
+  public T visitTableModify(TableModify modify) throws Exception {
+    return defaultValue(modify);
+  }
+
+  public T visitTableScan(TableScan scan) throws Exception {
     return defaultValue(scan);
   }
 
-  T visitUncollect(Uncollect uncollect) throws Exception {
+  public T visitUncollect(Uncollect uncollect) throws Exception {
     return defaultValue(uncollect);
   }
 
-  T visitWindow(Window window) throws Exception {
+  public T visitWindow(Window window) throws Exception {
     return defaultValue(window);
   }
 
-  T defaultValue(RelNode n) {
+  public T defaultValue(RelNode n) {
     return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/82f10ebf/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RelNodeCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RelNodeCompiler.java b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RelNodeCompiler.java
index 5a21fba2..eea451f 100644
--- a/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RelNodeCompiler.java
+++ b/external/sql/storm-sql-core/src/jvm/org/apache/storm/sql/compiler/RelNodeCompiler.java
@@ -25,9 +25,6 @@ import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.core.TableScan;
 
 import java.io.PrintWriter;
-import java.util.Collections;
-import java.util.Set;
-import java.util.TreeSet;
 
 /**
  * Compile RelNodes into individual functions.
@@ -45,19 +42,13 @@ class RelNodeCompiler extends PostOrderRelNodeVisitor<Void> {
     ""
   );
 
-  public Set<String> getReferredTables() {
-    return Collections.unmodifiableSet(referredTables);
-  }
-
-  private final Set<String> referredTables = new TreeSet<>();
-
   RelNodeCompiler(PrintWriter pw, JavaTypeFactory typeFactory) {
     this.pw = pw;
     this.typeFactory = typeFactory;
   }
 
   @Override
-  Void visitFilter(Filter filter) throws Exception {
+  public Void visitFilter(Filter filter) throws Exception {
     beginStage(filter);
     ExprCompiler compiler = new ExprCompiler(pw, typeFactory);
     String r = filter.getCondition().accept(compiler);
@@ -67,7 +58,7 @@ class RelNodeCompiler extends PostOrderRelNodeVisitor<Void> {
   }
 
   @Override
-  Void visitProject(Project project) throws Exception {
+  public Void visitProject(Project project) throws Exception {
     beginStage(project);
     ExprCompiler compiler = new ExprCompiler(pw, typeFactory);
 
@@ -84,14 +75,12 @@ class RelNodeCompiler extends PostOrderRelNodeVisitor<Void> {
   }
 
   @Override
-  Void defaultValue(RelNode n) {
+  public Void defaultValue(RelNode n) {
     throw new UnsupportedOperationException();
   }
 
   @Override
-  Void visitTableScan(TableScan scan) throws Exception {
-    String tableName = Joiner.on('_').join(scan.getTable().getQualifiedName());
-    referredTables.add(tableName);
+  public Void visitTableScan(TableScan scan) throws Exception {
     beginStage(scan);
     pw.print("    ctx.emit(_data);\n");
     endStage();

http://git-wip-us.apache.org/repos/asf/storm/blob/82f10ebf/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
index e18b9f8..9facd8a 100644
--- 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
@@ -18,9 +18,7 @@
 package org.apache.storm.sql;
 
 import backtype.storm.tuple.Values;
-import org.apache.storm.sql.compiler.TestUtils;
-import org.apache.storm.sql.runtime.ChannelHandler;
-import org.apache.storm.sql.runtime.DataSource;
+import org.apache.storm.sql.runtime.*;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -29,7 +27,6 @@ import org.junit.Test;
 import java.net.URI;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
 
 public class TestStormSql {
   private static class MockDataSourceProvider implements DataSourcesProvider {
@@ -41,9 +38,15 @@ public class TestStormSql {
     @Override
     public DataSource construct(
         URI uri, String inputFormatClass, String outputFormatClass,
-        List<Map.Entry<String, Class<?>>> fields) {
+        List<FieldInfo> fields) {
       return new TestUtils.MockDataSource();
     }
+
+    @Override
+    public ISqlTridentDataSource constructTrident(URI uri, String inputFormatClass, String outputFormatClass,
+         List<FieldInfo> fields) {
+      throw new UnsupportedOperationException();
+    }
   }
 
   @BeforeClass

http://git-wip-us.apache.org/repos/asf/storm/blob/82f10ebf/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestCompilerUtils.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestCompilerUtils.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestCompilerUtils.java
new file mode 100644
index 0000000..c582fdc
--- /dev/null
+++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestCompilerUtils.java
@@ -0,0 +1,46 @@
+package org.apache.storm.sql.compiler;
+
+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.Table;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.tools.*;
+
+public class TestCompilerUtils {
+  public static CalciteState sqlOverDummyTable(String sql)
+      throws RelConversionException, ValidationException, SqlParseException {
+    SchemaPlus schema = Frameworks.createRootSchema(true);
+    JavaTypeFactory typeFactory = new JavaTypeFactoryImpl
+        (RelDataTypeSystem.DEFAULT);
+    Table table = new CompilerUtil.TableBuilderInfo(typeFactory)
+        .field("ID", SqlTypeName.INTEGER).build();
+    schema.add("FOO", table);
+    schema.add("BAR", table);
+    FrameworkConfig config = Frameworks.newConfigBuilder().defaultSchema(
+        schema).build();
+    Planner planner = Frameworks.getPlanner(config);
+    SqlNode parse = planner.parse(sql);
+    SqlNode validate = planner.validate(parse);
+    RelNode tree = planner.convert(validate);
+    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; }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/82f10ebf/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprCompiler.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprCompiler.java
index a5f9d67..017aa25 100644
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprCompiler.java
+++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestExprCompiler.java
@@ -37,7 +37,7 @@ public class TestExprCompiler {
   @Test
   public void testLiteral() throws Exception {
     String sql = "SELECT 1,1.0,TRUE,'FOO' FROM FOO";
-    TestUtils.CalciteState state = TestUtils.sqlOverDummyTable(sql);
+    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
     JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
     LogicalProject project = (LogicalProject) state.tree;
     String[] res = new String[project.getChildExps().size()];
@@ -55,7 +55,7 @@ public class TestExprCompiler {
   @Test
   public void testInputRef() throws Exception {
     String sql = "SELECT ID FROM FOO";
-    TestUtils.CalciteState state = TestUtils.sqlOverDummyTable(sql);
+    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
     JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
     LogicalProject project = (LogicalProject) state.tree;
     StringWriter sw = new StringWriter();
@@ -70,7 +70,7 @@ public class TestExprCompiler {
   @Test
   public void testCallExpr() throws Exception {
     String sql = "SELECT 1>2, 3+5, 1-1.0, 3+ID FROM FOO";
-    TestUtils.CalciteState state = TestUtils.sqlOverDummyTable(sql);
+    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
     JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
     LogicalProject project = (LogicalProject) state.tree;
     String[] res = new String[project.getChildExps().size()];

http://git-wip-us.apache.org/repos/asf/storm/blob/82f10ebf/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
index 1d98664..40bb884 100644
--- 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
@@ -23,6 +23,7 @@ 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.runtime.ChannelHandler;
 import org.apache.storm.sql.runtime.DataSource;
 import org.apache.storm.sql.runtime.AbstractValuesProcessor;
@@ -124,7 +125,7 @@ public class TestExprSemantic {
   private Values testExpr(List<String> exprs) throws Exception {
     String sql = "SELECT " + Joiner.on(',').join(exprs) + " FROM FOO" +
         " WHERE ID > 0 AND ID < 2";
-    TestUtils.CalciteState state = TestUtils.sqlOverDummyTable(sql);
+    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
     PlanCompiler compiler = new PlanCompiler(typeFactory);
     AbstractValuesProcessor proc = compiler.compile(state.tree);
     Map<String, DataSource> data = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/storm/blob/82f10ebf/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestPlanCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestPlanCompiler.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestPlanCompiler.java
index d32fdca..2b3a2d3 100644
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestPlanCompiler.java
+++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestPlanCompiler.java
@@ -21,6 +21,7 @@ import backtype.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.ChannelHandler;
 import org.apache.storm.sql.runtime.DataSource;
 import org.apache.storm.sql.runtime.AbstractValuesProcessor;
@@ -39,7 +40,7 @@ public class TestPlanCompiler {
   @Test
   public void testCompile() throws Exception {
     String sql = "SELECT ID + 1 FROM FOO WHERE ID > 2";
-    TestUtils.CalciteState state = TestUtils.sqlOverDummyTable(sql);
+    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
     PlanCompiler compiler = new PlanCompiler(typeFactory);
     AbstractValuesProcessor proc = compiler.compile(state.tree);
     Map<String, DataSource> data = new HashMap<>();
@@ -54,7 +55,7 @@ public class TestPlanCompiler {
   @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";
-    TestUtils.CalciteState state = TestUtils.sqlOverDummyTable(sql);
+    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
     PlanCompiler compiler = new PlanCompiler(typeFactory);
     AbstractValuesProcessor proc = compiler.compile(state.tree);
     Map<String, DataSource> data = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/storm/blob/82f10ebf/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestRelNodeCompiler.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestRelNodeCompiler.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestRelNodeCompiler.java
index 623a2f4..99083cb 100644
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestRelNodeCompiler.java
+++ b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestRelNodeCompiler.java
@@ -19,11 +19,9 @@ package org.apache.storm.sql.compiler;
 
 import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.calcite.rel.core.TableScan;
 import org.apache.calcite.rel.logical.LogicalFilter;
 import org.apache.calcite.rel.logical.LogicalProject;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.hamcrest.CoreMatchers;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -36,7 +34,7 @@ public class TestRelNodeCompiler {
   @Test
   public void testFilter() throws Exception {
     String sql = "SELECT ID + 1 FROM FOO WHERE ID > 3";
-    TestUtils.CalciteState state = TestUtils.sqlOverDummyTable(sql);
+    TestCompilerUtils.CalciteState state = TestCompilerUtils.sqlOverDummyTable(sql);
     JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(
         RelDataTypeSystem.DEFAULT);
     LogicalProject project = (LogicalProject) state.tree;

http://git-wip-us.apache.org/repos/asf/storm/blob/82f10ebf/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestUtils.java
----------------------------------------------------------------------
diff --git a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestUtils.java b/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestUtils.java
deleted file mode 100644
index 5aa4cb0..0000000
--- a/external/sql/storm-sql-core/src/test/org/apache/storm/sql/compiler/TestUtils.java
+++ /dev/null
@@ -1,92 +0,0 @@
-package org.apache.storm.sql.compiler;
-
-import backtype.storm.tuple.Values;
-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.Table;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.parser.SqlParseException;
-import org.apache.calcite.sql.type.SqlTypeName;
-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.runtime.ChannelContext;
-import org.apache.storm.sql.runtime.ChannelHandler;
-import org.apache.storm.sql.runtime.DataSource;
-
-import java.util.ArrayList;
-import java.util.List;
-
-public class TestUtils {
-  static CalciteState sqlOverDummyTable(String sql)
-      throws RelConversionException, ValidationException, SqlParseException {
-    SchemaPlus schema = Frameworks.createRootSchema(true);
-    JavaTypeFactory typeFactory = new JavaTypeFactoryImpl
-        (RelDataTypeSystem.DEFAULT);
-    Table table = new CompilerUtil.TableBuilderInfo(typeFactory)
-        .field("ID", SqlTypeName.INTEGER).build();
-    schema.add("FOO", table);
-    FrameworkConfig config = Frameworks.newConfigBuilder().defaultSchema(
-        schema).build();
-    Planner planner = Frameworks.getPlanner(config);
-    SqlNode parse = planner.parse(sql);
-    SqlNode validate = planner.validate(parse);
-    RelNode tree = planner.convert(validate);
-    return new CalciteState(schema, tree);
-  }
-
-  static class CalciteState {
-    final SchemaPlus schema;
-    final RelNode tree;
-
-    private CalciteState(SchemaPlus schema, RelNode tree) {
-      this.schema = schema;
-      this.tree = tree;
-    }
-  }
-
-  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));
-      }
-    }
-
-    @Override
-    public void open(ChannelContext ctx) {
-      for (Values v : RECORDS) {
-        ctx.emit(v);
-      }
-      ctx.fireChannelInactive();
-    }
-  }
-
-  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);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/82f10ebf/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
index 62f2d95..a8ba4dc 100644
--- a/external/sql/storm-sql-runtime/pom.xml
+++ b/external/sql/storm-sql-runtime/pom.xml
@@ -45,7 +45,6 @@
         <dependency>
             <groupId>com.google.guava</groupId>
             <artifactId>guava</artifactId>
-            <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>org.mockito</groupId>

http://git-wip-us.apache.org/repos/asf/storm/blob/82f10ebf/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
new file mode 100644
index 0000000..eaabc8d
--- /dev/null
+++ b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesProvider.java
@@ -0,0 +1,50 @@
+/*
+ * *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.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.net.URI;
+import java.util.List;
+import java.util.Map;
+
+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,
+      List<FieldInfo> fields);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/82f10ebf/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
new file mode 100644
index 0000000..0285c97
--- /dev/null
+++ b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/DataSourcesRegistry.java
@@ -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
+ *  * <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 com.google.common.annotations.VisibleForTesting;
+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.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,
+      List<FieldInfo> fields) {
+    DataSourcesProvider provider = providers.get(uri.getScheme());
+    if (provider == null) {
+      return null;
+    }
+
+    return provider.constructTrident(uri, inputFormatClass, outputFormatClass, fields);
+  }
+
+  /**
+   * Allow unit tests to inject data sources.
+   */
+  @VisibleForTesting
+  public static Map<String, DataSourcesProvider> providerMap() {
+    return providers;
+  }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/82f10ebf/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
new file mode 100644
index 0000000..cb1176b
--- /dev/null
+++ b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/FieldInfo.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.runtime;
+
+/**
+ * Describe each column of the field
+ */
+public class FieldInfo {
+  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/82f10ebf/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
new file mode 100644
index 0000000..4b2a915
--- /dev/null
+++ b/external/sql/storm-sql-runtime/src/jvm/org/apache/storm/sql/runtime/ISqlTridentDataSource.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <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 storm.trident.operation.Function;
+import storm.trident.spout.IBatchSpout;
+
+/**
+ * A ISqlTridentDataSource specifies how an external data source produces and consumes data.
+ */
+public interface ISqlTridentDataSource {
+  IBatchSpout getProducer();
+  Function getConsumer();
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/82f10ebf/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
new file mode 100644
index 0000000..82347ef
--- /dev/null
+++ b/external/sql/storm-sql-runtime/src/test/org/apache/storm/sql/TestUtils.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;
+
+import backtype.storm.tuple.Values;
+import org.apache.storm.sql.runtime.ChannelContext;
+import org.apache.storm.sql.runtime.ChannelHandler;
+import org.apache.storm.sql.runtime.DataSource;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class TestUtils {
+  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));
+      }
+    }
+
+    @Override
+    public void open(ChannelContext ctx) {
+      for (Values v : RECORDS) {
+        ctx.emit(v);
+      }
+      ctx.fireChannelInactive();
+    }
+  }
+
+  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);
+    }
+  }
+}