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);
+ }
+ }
+}