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:53 UTC

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

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