You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by ta...@apache.org on 2017/07/12 17:08:34 UTC

[1/2] beam git commit: [BEAM-2564] add integration test for string functions

Repository: beam
Updated Branches:
  refs/heads/DSL_SQL 8defe6f21 -> 53d27e6c4


[BEAM-2564] add integration test for string functions


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

Branch: refs/heads/DSL_SQL
Commit: 3a170744861dec12a33a19358f9ce89dbd64491c
Parents: 8defe6f
Author: James Xu <xu...@gmail.com>
Authored: Mon Jul 10 19:58:21 2017 +0800
Committer: Tyler Akidau <ta...@apache.org>
Committed: Wed Jul 12 10:06:58 2017 -0700

----------------------------------------------------------------------
 .../operator/string/BeamSqlTrimExpression.java  | 12 +--
 .../BeamSqlStringFunctionsIntegrationTest.java  | 92 ++++++++++++++++++++
 .../sql/interpreter/BeamSqlFnExecutorTest.java  |  3 +-
 .../string/BeamSqlTrimExpressionTest.java       |  9 +-
 4 files changed, 105 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/3a170744/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlTrimExpression.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlTrimExpression.java b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlTrimExpression.java
index d7c8a6a..ac4d060 100644
--- a/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlTrimExpression.java
+++ b/dsls/sql/src/main/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlTrimExpression.java
@@ -19,10 +19,10 @@
 package org.apache.beam.dsls.sql.interpreter.operator.string;
 
 import java.util.List;
-
 import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
 import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlPrimitive;
 import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.calcite.sql.fun.SqlTrimFunction;
 import org.apache.calcite.sql.type.SqlTypeName;
 
 /**
@@ -48,7 +48,7 @@ public class BeamSqlTrimExpression extends BeamSqlExpression {
 
     if (operands.size() == 3
         && (
-        !SqlTypeName.CHAR_TYPES.contains(opType(0))
+        SqlTypeName.SYMBOL != opType(0)
             || !SqlTypeName.CHAR_TYPES.contains(opType(1))
             || !SqlTypeName.CHAR_TYPES.contains(opType(2)))
         ) {
@@ -63,16 +63,16 @@ public class BeamSqlTrimExpression extends BeamSqlExpression {
       return BeamSqlPrimitive.of(SqlTypeName.VARCHAR,
           opValueEvaluated(0, inputRecord).toString().trim());
     } else {
-      String type = opValueEvaluated(0, inputRecord);
+      SqlTrimFunction.Flag type = opValueEvaluated(0, inputRecord);
       String targetStr = opValueEvaluated(1, inputRecord);
       String containingStr = opValueEvaluated(2, inputRecord);
 
       switch (type) {
-        case "LEADING":
+        case LEADING:
           return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, leadingTrim(containingStr, targetStr));
-        case "TRAILING":
+        case TRAILING:
           return BeamSqlPrimitive.of(SqlTypeName.VARCHAR, trailingTrim(containingStr, targetStr));
-        case "BOTH":
+        case BOTH:
         default:
           return BeamSqlPrimitive.of(SqlTypeName.VARCHAR,
               trailingTrim(leadingTrim(containingStr, targetStr), targetStr));

http://git-wip-us.apache.org/repos/asf/beam/blob/3a170744/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlStringFunctionsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlStringFunctionsIntegrationTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlStringFunctionsIntegrationTest.java
new file mode 100644
index 0000000..11465f5
--- /dev/null
+++ b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/integrationtest/BeamSqlStringFunctionsIntegrationTest.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.dsls.sql.integrationtest;
+
+import java.sql.Types;
+import org.apache.beam.dsls.sql.BeamSqlCli;
+import org.apache.beam.dsls.sql.BeamSqlEnv;
+import org.apache.beam.dsls.sql.TestUtils;
+import org.apache.beam.dsls.sql.schema.BeamSqlRow;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.PCollection;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * Integration test for string functions.
+ */
+public class BeamSqlStringFunctionsIntegrationTest {
+  static BeamSqlEnv sqlEnv = new BeamSqlEnv();
+
+  @Rule
+  public final TestPipeline pipeline = TestPipeline.create();
+
+  @Test
+  public void testStringFunctions() throws Exception {
+    String sql = "SELECT "
+        + "'hello' || ' world' as concat,"
+        + "CHAR_LENGTH('hello') as cl,"
+        + "CHARACTER_LENGTH('hello') as cl1,"
+        + "UPPER('hello') as up,"
+        + "LOWER('HELLO') as lo,"
+        + "POSITION('world' IN 'helloworld') as po,"
+        + "POSITION('world' IN 'helloworldworld' FROM 7) as po1,"
+        + "TRIM(' hello ') as tr,"
+        + "TRIM(LEADING ' ' FROM ' hello ') as tr1,"
+        + "TRIM(TRAILING ' ' FROM ' hello ') as tr2,"
+        + "TRIM(BOTH ' ' FROM ' hello ') as tr3,"
+        + "OVERLAY('w3333333rce' PLACING 'resou' FROM 3) as ol,"
+        + "SUBSTRING('hello' FROM 2) as ss,"
+        + "SUBSTRING('hello' FROM 2 FOR 2) as ss1,"
+        + "INITCAP('hello world') as ss1"
+    ;
+
+    PCollection<BeamSqlRow> rows = BeamSqlCli.compilePipeline(sql, pipeline, sqlEnv);
+    PAssert.that(rows).containsInAnyOrder(
+        TestUtils.RowsBuilder.of(
+            // 1 -> 5
+            Types.VARCHAR, "concat",
+            Types.INTEGER, "cl",
+            Types.INTEGER, "cl1",
+            Types.VARCHAR, "up",
+            Types.VARCHAR, "lo",
+            // 6 -> 10
+            Types.INTEGER, "po",
+            Types.INTEGER, "po1",
+            Types.VARCHAR, "tr",
+            Types.VARCHAR, "tr1",
+            Types.VARCHAR, "tr2",
+            // 11 -> 15
+            Types.VARCHAR, "tr3",
+            Types.VARCHAR, "ol",
+            Types.VARCHAR, "ss",
+            Types.VARCHAR, "ss1",
+            Types.VARCHAR, "ic"
+        ).addRows(
+            // 1 -> 5(lo)
+            "hello world", 5, 5, "HELLO", "hello",
+            // 6 -> 10()
+            5, 10, "hello", "hello ", " hello",
+            // 11 -> 15
+            "hello", "w3resou3rce", "ello", "el", "Hello World"
+        ).getRows());
+    pipeline.run();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/3a170744/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTest.java
index d97acc7..1aa4f1c 100644
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTest.java
+++ b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/BeamSqlFnExecutorTest.java
@@ -64,6 +64,7 @@ import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.fun.SqlTrimFunction;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.junit.Assert;
 import org.junit.Test;
@@ -292,7 +293,7 @@ public class BeamSqlFnExecutorTest extends BeamSqlFnExecutorTestBase {
 
     rexNode = rexBuilder.makeCall(SqlStdOperatorTable.TRIM,
         Arrays.asList(
-            rexBuilder.makeLiteral("BOTH"),
+            rexBuilder.makeFlag(SqlTrimFunction.Flag.BOTH),
             rexBuilder.makeLiteral("HELLO"),
             rexBuilder.makeLiteral("HELLO")
         )

http://git-wip-us.apache.org/repos/asf/beam/blob/3a170744/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlTrimExpressionTest.java
----------------------------------------------------------------------
diff --git a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlTrimExpressionTest.java b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlTrimExpressionTest.java
index 8c595f3..9ae9212 100644
--- a/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlTrimExpressionTest.java
+++ b/dsls/sql/src/test/java/org/apache/beam/dsls/sql/interpreter/operator/string/BeamSqlTrimExpressionTest.java
@@ -28,6 +28,7 @@ import java.util.List;
 import org.apache.beam.dsls.sql.interpreter.BeamSqlFnExecutorTestBase;
 import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlExpression;
 import org.apache.beam.dsls.sql.interpreter.operator.BeamSqlPrimitive;
+import org.apache.calcite.sql.fun.SqlTrimFunction;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.junit.Test;
 
@@ -43,7 +44,7 @@ public class BeamSqlTrimExpressionTest extends BeamSqlFnExecutorTestBase {
     assertTrue(new BeamSqlTrimExpression(operands).accept());
 
     operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "LEADING"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, SqlTrimFunction.Flag.BOTH));
     operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "he"));
     operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hehe__hehe"));
     assertTrue(new BeamSqlTrimExpression(operands).accept());
@@ -57,21 +58,21 @@ public class BeamSqlTrimExpressionTest extends BeamSqlFnExecutorTestBase {
   @Test public void evaluate() throws Exception {
     List<BeamSqlExpression> operands = new ArrayList<>();
 
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "LEADING"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, SqlTrimFunction.Flag.LEADING));
     operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "he"));
     operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hehe__hehe"));
     assertEquals("__hehe",
         new BeamSqlTrimExpression(operands).evaluate(record).getValue());
 
     operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "TRAILING"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, SqlTrimFunction.Flag.TRAILING));
     operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "he"));
     operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hehe__hehe"));
     assertEquals("hehe__",
         new BeamSqlTrimExpression(operands).evaluate(record).getValue());
 
     operands.clear();
-    operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "BOTH"));
+    operands.add(BeamSqlPrimitive.of(SqlTypeName.SYMBOL, SqlTrimFunction.Flag.BOTH));
     operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "he"));
     operands.add(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "__"));
     assertEquals("__",


[2/2] beam git commit: [BEAM-2564] This closes #3532

Posted by ta...@apache.org.
[BEAM-2564] This closes #3532


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

Branch: refs/heads/DSL_SQL
Commit: 53d27e6c45ab7bbd6877c6674868e8a2e3f9a971
Parents: 8defe6f 3a17074
Author: Tyler Akidau <ta...@apache.org>
Authored: Wed Jul 12 10:07:39 2017 -0700
Committer: Tyler Akidau <ta...@apache.org>
Committed: Wed Jul 12 10:07:39 2017 -0700

----------------------------------------------------------------------
 .../operator/string/BeamSqlTrimExpression.java  | 12 +--
 .../BeamSqlStringFunctionsIntegrationTest.java  | 92 ++++++++++++++++++++
 .../sql/interpreter/BeamSqlFnExecutorTest.java  |  3 +-
 .../string/BeamSqlTrimExpressionTest.java       |  9 +-
 4 files changed, 105 insertions(+), 11 deletions(-)
----------------------------------------------------------------------