[ 
https://issues.apache.org/jira/browse/BEAM-2281?focusedWorklogId=111102&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-111102
 ]

ASF GitHub Bot logged work on BEAM-2281:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 12/Jun/18 15:54
            Start Date: 12/Jun/18 15:54
    Worklog Time Spent: 10m 
      Work Description: kennknowles closed pull request #5596: [BEAM-2281] 
Improve SQL expression testing so SQL syntax tests suffice; delete extraneous 
unit tests
URL: https://github.com/apache/beam/pull/5596
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlStringFunctionsIntegrationTest.java
 
b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslStringOperatorsTest.java
similarity index 61%
rename from 
sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlStringFunctionsIntegrationTest.java
rename to 
sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslStringOperatorsTest.java
index 4d4c1793637..b50454b2286 100644
--- 
a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlStringFunctionsIntegrationTest.java
+++ 
b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslStringOperatorsTest.java
@@ -15,13 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.beam.sdk.extensions.sql.integrationtest;
+package org.apache.beam.sdk.extensions.sql;
 
+import 
org.apache.beam.sdk.extensions.sql.integrationtest.BeamSqlBuiltinFunctionsIntegrationTestBase;
 import org.junit.Test;
 
 /** Integration test for string functions. */
-public class BeamSqlStringFunctionsIntegrationTest
-    extends BeamSqlBuiltinFunctionsIntegrationTestBase {
+public class BeamSqlDslStringOperatorsTest extends 
BeamSqlBuiltinFunctionsIntegrationTestBase {
   @Test
   public void testStringFunctions() throws Exception {
     ExpressionChecker checker =
@@ -29,18 +29,26 @@ public void testStringFunctions() throws Exception {
             .addExpr("'hello' || ' world'", "hello world")
             .addExpr("CHAR_LENGTH('hello')", 5)
             .addExpr("CHARACTER_LENGTH('hello')", 5)
-            .addExpr("UPPER('hello')", "HELLO")
+            .addExpr("INITCAP('hello world')", "Hello World")
             .addExpr("LOWER('HELLO')", "hello")
             .addExpr("POSITION('world' IN 'helloworld')", 6)
             .addExpr("POSITION('world' IN 'helloworldworld' FROM 7)", 11)
+            .addExpr("POSITION('world' IN 'hello')", 0)
             .addExpr("TRIM(' hello ')", "hello")
-            .addExpr("TRIM(LEADING ' ' FROM ' hello ')", "hello ")
-            .addExpr("TRIM(TRAILING ' ' FROM ' hello ')", " hello")
+            .addExpr("TRIM(LEADING 'eh' FROM 'hehe__hehe')", "__hehe")
+            .addExpr("TRIM(TRAILING 'eh' FROM 'hehe__hehe')", "hehe__")
+            .addExpr("TRIM(BOTH 'eh' FROM 'hehe__hehe')", "__")
             .addExpr("TRIM(BOTH ' ' FROM ' hello ')", "hello")
             .addExpr("OVERLAY('w3333333rce' PLACING 'resou' FROM 3)", 
"w3resou3rce")
+            .addExpr("OVERLAY('w3333333rce' PLACING 'resou' FROM 3 FOR 4)", 
"w3resou33rce")
+            .addExpr("OVERLAY('w3333333rce' PLACING 'resou' FROM 3 FOR 5)", 
"w3resou3rce")
+            .addExpr("OVERLAY('w3333333rce' PLACING 'resou' FROM 3 FOR 7)", 
"w3resouce")
             .addExpr("SUBSTRING('hello' FROM 2)", "ello")
+            .addExpr("SUBSTRING('hello' FROM -1)", "o")
             .addExpr("SUBSTRING('hello' FROM 2 FOR 2)", "el")
-            .addExpr("INITCAP('hello world')", "Hello World");
+            .addExpr("SUBSTRING('hello' FROM 2 FOR 100)", "ello")
+            .addExpr("SUBSTRING('hello' FROM -3 for 2)", "ll")
+            .addExpr("UPPER('hello')", "HELLO");
 
     checker.buildRunAndCheck();
   }
diff --git 
a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/StringOperatorsTest.java
 
b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/StringOperatorsTest.java
deleted file mode 100644
index 6e6c61e9b9c..00000000000
--- 
a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/interpreter/operator/StringOperatorsTest.java
+++ /dev/null
@@ -1,516 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.beam.sdk.extensions.sql.impl.interpreter.operator;
-
-import static org.hamcrest.Matchers.equalTo;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-
-import com.google.common.collect.ImmutableList;
-import 
org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutorTestBase;
-import org.apache.calcite.sql.fun.SqlTrimFunction;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.junit.Test;
-import org.junit.experimental.runners.Enclosed;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** Test of BeamSqlUpperExpression. */
-@RunWith(Enclosed.class)
-public class StringOperatorsTest extends BeamSqlFnExecutorTestBase {
-
-  /** Tests for UPPER. */
-  @RunWith(JUnit4.class)
-  public static class UpperTest {
-    @Test
-    public void testUpper() {
-      assertThat(
-          StringOperators.UPPER
-              .apply(ImmutableList.of(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, 
"hello")))
-              .getValue(),
-          equalTo("HELLO"));
-    }
-  }
-
-  /** Tests for LOWER. */
-  @RunWith(JUnit4.class)
-  public static class LowerTest {
-    @Test
-    public void testLower() {
-      assertThat(
-          StringOperators.LOWER
-              .apply(ImmutableList.of(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, 
"HELLo")))
-              .getValue(),
-          equalTo("hello"));
-    }
-  }
-
-  /** Tests for TRIM. */
-  @RunWith(JUnit4.class)
-  public static class TrimTest {
-
-    @Test
-    public void testAcceptOne() {
-      assertTrue(
-          StringOperators.TRIM.accept(
-              ImmutableList.of(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, " 
hello "))));
-    }
-
-    @Test
-    public void testAcceptThree() {
-      assertTrue(
-          StringOperators.TRIM.accept(
-              ImmutableList.of(
-                  BeamSqlPrimitive.of(SqlTypeName.SYMBOL, 
SqlTrimFunction.Flag.BOTH),
-                  BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "he"),
-                  BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hehe__hehe"))));
-    }
-
-    @Test
-    public void testRejectTwo() {
-      assertFalse(
-          StringOperators.TRIM.accept(
-              ImmutableList.of(
-                  BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "he"),
-                  BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hehe__hehe"))));
-    }
-
-    @Test
-    public void testLeading() throws Exception {
-      assertThat(
-          StringOperators.TRIM
-              .apply(
-                  ImmutableList.of(
-                      BeamSqlPrimitive.of(SqlTypeName.SYMBOL, 
SqlTrimFunction.Flag.LEADING),
-                      BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "eh"),
-                      BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hehe__hehe")))
-              .getValue(),
-          equalTo("__hehe"));
-    }
-
-    @Test
-    public void testTrailing() {
-      assertThat(
-          StringOperators.TRIM
-              .apply(
-                  ImmutableList.of(
-                      BeamSqlPrimitive.of(SqlTypeName.SYMBOL, 
SqlTrimFunction.Flag.TRAILING),
-                      BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "eh"),
-                      BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hehe__hehe")))
-              .getValue(),
-          equalTo("hehe__"));
-    }
-
-    @Test
-    public void testBoth() {
-      assertThat(
-          StringOperators.TRIM
-              .apply(
-                  ImmutableList.of(
-                      BeamSqlPrimitive.of(SqlTypeName.SYMBOL, 
SqlTrimFunction.Flag.BOTH),
-                      BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "eh"),
-                      BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hehe__hehe")))
-              .getValue(),
-          equalTo("__"));
-    }
-
-    @Test
-    public void testDefault() {
-      assertThat(
-          StringOperators.TRIM
-              .apply(ImmutableList.of(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, 
" hello ")))
-              .getValue(),
-          equalTo("hello"));
-    }
-  }
-
-  /** Tests for CHAR_LENGTH. */
-  @RunWith(JUnit4.class)
-  public static class CharLengthTest {
-
-    @Test
-    public void testSimple() {
-      assertThat(
-          StringOperators.CHAR_LENGTH
-              .apply(ImmutableList.of(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, 
"hello")))
-              .getValue(),
-          equalTo(5));
-    }
-
-    @Test
-    public void testAccept() {
-      assertTrue(
-          StringOperators.CHAR_LENGTH.accept(
-              ImmutableList.of(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, 
"hello"))));
-    }
-
-    @Test
-    public void testRejectNonString() {
-      assertFalse(
-          StringOperators.CHAR_LENGTH.accept(
-              ImmutableList.of(BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1))));
-    }
-
-    @Test
-    public void testRejectTooManyArgs() {
-      assertFalse(
-          StringOperators.CHAR_LENGTH.accept(
-              ImmutableList.of(
-                  BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"),
-                  BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"))));
-    }
-  }
-
-  /** Tests for Concat operator. */
-  @RunWith(JUnit4.class)
-  public static class BeamSqlConcatExpressionTest extends 
BeamSqlFnExecutorTestBase {
-
-    @Test
-    public void accept() throws Exception {
-      assertTrue(
-          StringOperators.CONCAT.accept(
-              ImmutableList.of(
-                  BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"),
-                  BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"))));
-    }
-
-    @Test
-    public void rejectNonString() throws Exception {
-      assertFalse(
-          StringOperators.CONCAT.accept(
-              ImmutableList.of(
-                  BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1),
-                  BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"))));
-    }
-
-    @Test
-    public void rejectTooMany() throws Exception {
-      assertFalse(
-          StringOperators.CONCAT.accept(
-              ImmutableList.of(
-                  BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"),
-                  BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"),
-                  BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"))));
-    }
-
-    @Test
-    public void testApply() throws Exception {
-      assertThat(
-          StringOperators.CONCAT
-              .apply(
-                  ImmutableList.of(
-                      BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"),
-                      BeamSqlPrimitive.of(SqlTypeName.VARCHAR, " world")))
-              .getValue(),
-          equalTo("hello world"));
-    }
-  }
-
-  /** Test for SUBSTRING. */
-  public static class SubstringTest {
-
-    @Test
-    public void testAcceptTwoArgs() throws Exception {
-      assertTrue(
-          StringOperators.SUBSTRING.accept(
-              ImmutableList.of(
-                  BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"),
-                  BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1))));
-    }
-
-    @Test
-    public void testAcceptThreeArgs() {
-      assertTrue(
-          StringOperators.SUBSTRING.accept(
-              ImmutableList.of(
-                  BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"),
-                  BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1),
-                  BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2))));
-    }
-
-    @Test
-    public void testApplyWhole() {
-      assertThat(
-          StringOperators.SUBSTRING
-              .apply(
-                  ImmutableList.of(
-                      BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"),
-                      BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1)))
-              .getValue(),
-          equalTo("hello"));
-    }
-
-    @Test
-    public void testApplySubstring() {
-      assertThat(
-          StringOperators.SUBSTRING
-              .apply(
-                  ImmutableList.of(
-                      BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"),
-                      BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1),
-                      BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2)))
-              .getValue(),
-          equalTo("he"));
-    }
-
-    @Test
-    public void testApplyExactLength() {
-      assertThat(
-          StringOperators.SUBSTRING
-              .apply(
-                  ImmutableList.of(
-                      BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"),
-                      BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1),
-                      BeamSqlPrimitive.of(SqlTypeName.INTEGER, 5)))
-              .getValue(),
-          equalTo("hello"));
-    }
-
-    @Test
-    public void testApplyExceedsLength() {
-      assertThat(
-          StringOperators.SUBSTRING
-              .apply(
-                  ImmutableList.of(
-                      BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"),
-                      BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1),
-                      BeamSqlPrimitive.of(SqlTypeName.INTEGER, 100)))
-              .getValue(),
-          equalTo("hello"));
-    }
-
-    @Test
-    public void testApplyNegativeLength() {
-      assertThat(
-          StringOperators.SUBSTRING
-              .apply(
-                  ImmutableList.of(
-                      BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"),
-                      BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1),
-                      BeamSqlPrimitive.of(SqlTypeName.INTEGER, 0)))
-              .getValue(),
-          equalTo(""));
-    }
-
-    @Test
-    public void testApplyNegativeStartpoint() {
-      assertThat(
-          StringOperators.SUBSTRING
-              .apply(
-                  ImmutableList.of(
-                      BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"),
-                      BeamSqlPrimitive.of(SqlTypeName.INTEGER, -1)))
-              .getValue(),
-          equalTo("o"));
-    }
-  }
-
-  /** Test for POSITION. */
-  public static class PositionTest {
-    @Test
-    public void testAcceptTwoArgs() throws Exception {
-      assertTrue(
-          StringOperators.POSITION.accept(
-              ImmutableList.of(
-                  BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"),
-                  BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"))));
-    }
-
-    @Test
-    public void testAcceptTrheeArgs() throws Exception {
-      assertTrue(
-          StringOperators.POSITION.accept(
-              ImmutableList.of(
-                  BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"),
-                  BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"),
-                  BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1))));
-    }
-
-    @Test
-    public void testReject() {
-      assertFalse(
-          StringOperators.POSITION.accept(
-              ImmutableList.of(
-                  BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1),
-                  BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"))));
-    }
-
-    @Test
-    public void testRejectTwoMany() {
-      assertFalse(
-          StringOperators.POSITION.accept(
-              ImmutableList.of(
-                  BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"),
-                  BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"),
-                  BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1),
-                  BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1))));
-    }
-
-    @Test
-    public void testBasic() {
-      assertThat(
-          StringOperators.POSITION
-              .apply(
-                  ImmutableList.of(
-                      BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"),
-                      BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello")))
-              .getValue(),
-          equalTo(6));
-    }
-
-    @Test
-    public void testThreeArgs() {
-      assertThat(
-          StringOperators.POSITION
-              .apply(
-                  ImmutableList.of(
-                      BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"),
-                      BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "worldhello"),
-                      BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1)))
-              .getValue(),
-          equalTo(6));
-    }
-
-    @Test
-    public void testThreeArgsNotFound() {
-      assertThat(
-          StringOperators.POSITION
-              .apply(
-                  ImmutableList.of(
-                      BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"),
-                      BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "world"),
-                      BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1)))
-              .getValue(),
-          equalTo(0));
-    }
-  }
-
-  /** Test for BeamSqlOverlayExpression. */
-  public static class OverlayTest {
-
-    @Test
-    public void acceptThreeArgs() {
-      assertTrue(
-          StringOperators.OVERLAY.accept(
-              ImmutableList.of(
-                  BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"),
-                  BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"),
-                  BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1))));
-    }
-
-    @Test
-    public void acceptFourArgs() {
-      assertTrue(
-          StringOperators.OVERLAY.accept(
-              ImmutableList.of(
-                  BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"),
-                  BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "hello"),
-                  BeamSqlPrimitive.of(SqlTypeName.INTEGER, 1),
-                  BeamSqlPrimitive.of(SqlTypeName.INTEGER, 2))));
-    }
-
-    @Test
-    public void testOverlayBasic() {
-      assertThat(
-          StringOperators.OVERLAY
-              .apply(
-                  ImmutableList.of(
-                      BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "w3333333rce"),
-                      BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "resou"),
-                      BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3)))
-              .getValue(),
-          equalTo("w3resou3rce"));
-    }
-
-    @Test
-    public void testOverlayFourArgs() {
-      assertThat(
-          StringOperators.OVERLAY
-              .apply(
-                  ImmutableList.of(
-                      BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "w3333333rce"),
-                      BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "resou"),
-                      BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3),
-                      BeamSqlPrimitive.of(SqlTypeName.INTEGER, 4)))
-              .getValue(),
-          equalTo("w3resou33rce"));
-    }
-
-    @Test
-    public void testOverlayFourArgs2() {
-      assertThat(
-          StringOperators.OVERLAY
-              .apply(
-                  ImmutableList.of(
-                      BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "w3333333rce"),
-                      BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "resou"),
-                      BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3),
-                      BeamSqlPrimitive.of(SqlTypeName.INTEGER, 5)))
-              .getValue(),
-          equalTo("w3resou3rce"));
-    }
-
-    @Test
-    public void testOverlayBigGap() {
-      assertThat(
-          StringOperators.OVERLAY
-              .apply(
-                  ImmutableList.of(
-                      BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "w3333333rce"),
-                      BeamSqlPrimitive.of(SqlTypeName.VARCHAR, "resou"),
-                      BeamSqlPrimitive.of(SqlTypeName.INTEGER, 3),
-                      BeamSqlPrimitive.of(SqlTypeName.INTEGER, 7)))
-              .getValue(),
-          equalTo("w3resouce"));
-    }
-  }
-
-  /** Test of BeamSqlInitCapExpression. */
-  @RunWith(JUnit4.class)
-  public static class InitCapTest {
-
-    @Test
-    public void testTwoWords() {
-      assertThat(
-          StringOperators.INIT_CAP
-              .apply(ImmutableList.of(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, 
"hello world")))
-              .getValue(),
-          equalTo("Hello World"));
-    }
-
-    @Test
-    public void testTwoWordsWonky() {
-      assertThat(
-          StringOperators.INIT_CAP
-              .apply(ImmutableList.of(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, 
"hEllO wOrld")))
-              .getValue(),
-          equalTo("Hello World"));
-    }
-
-    @Test
-    public void testTwoWordsSpacedOut() {
-      assertThat(
-          StringOperators.INIT_CAP
-              .apply(ImmutableList.of(BeamSqlPrimitive.of(SqlTypeName.VARCHAR, 
"hello     world")))
-              .getValue(),
-          equalTo("Hello     World"));
-    }
-  }
-}
diff --git 
a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java
 
b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java
index 25c86cc9c65..9990a59e66e 100644
--- 
a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java
+++ 
b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java
@@ -18,10 +18,6 @@
 
 package org.apache.beam.sdk.extensions.sql.integrationtest;
 
-import static java.util.stream.Collectors.toList;
-import static org.apache.beam.sdk.schemas.Schema.toSchema;
-
-import com.google.common.base.Joiner;
 import com.google.common.collect.ImmutableMap;
 import java.math.BigDecimal;
 import java.util.ArrayList;
@@ -126,37 +122,28 @@ public ExpressionChecker addExpr(String expression, 
Object expectedValue) {
       return this;
     }
 
-    private String getSql() {
-      List<String> expStrs = new ArrayList<>();
-      for (Pair<String, Object> pair : exps) {
-        expStrs.add(pair.getKey());
-      }
-      return "SELECT " + Joiner.on(",\n  ").join(expStrs) + " FROM 
PCOLLECTION";
-    }
-
     /** Build the corresponding SQL, compile to Beam Pipeline, run it, and 
check the result. */
     public void buildRunAndCheck() {
       PCollection<Row> inputCollection = getTestPCollection();
-      System.out.println("SQL:>\n" + getSql());
-      try {
+
+      for (Pair<String, Object> testCase : exps) {
+        String expression = testCase.left;
+        Object expectedValue = testCase.right;
+        String sql = String.format("SELECT %s FROM PCOLLECTION", expression);
         Schema schema =
-            exps.stream()
-                .map(
-                    exp ->
-                        Schema.Field.of(
-                            exp.getKey(),
-                            
FieldType.of(JAVA_CLASS_TO_FIELDTYPE.get(exp.getValue().getClass()))))
-                .collect(toSchema());
+            Schema.builder()
+                .addField(
+                    expression, 
FieldType.of(JAVA_CLASS_TO_FIELDTYPE.get(expectedValue.getClass())))
+                .build();
 
-        List<Object> values = 
exps.stream().map(Pair::getValue).collect(toList());
+        PCollection<Row> output =
+            inputCollection.apply(testCase.toString(), 
SqlTransform.query(sql));
 
-        PCollection<Row> rows = 
inputCollection.apply(SqlTransform.query(getSql()));
-        PAssert.that(rows)
-            
.containsInAnyOrder(TestUtils.RowsBuilder.of(schema).addRows(values).getRows());
-        inputCollection.getPipeline().run();
-      } catch (Exception e) {
-        throw new RuntimeException(e);
+        PAssert.that(output)
+            
.containsInAnyOrder(TestUtils.RowsBuilder.of(schema).addRows(expectedValue).getRows());
       }
+
+      inputCollection.getPipeline().run();
     }
   }
 }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 111102)
    Time Spent: 3h 20m  (was: 3h 10m)

> call SqlFunctions in operator implementation
> --------------------------------------------
>
>                 Key: BEAM-2281
>                 URL: https://issues.apache.org/jira/browse/BEAM-2281
>             Project: Beam
>          Issue Type: Improvement
>          Components: dsl-sql
>            Reporter: Xu Mingmin
>            Assignee: Kenneth Knowles
>            Priority: Major
>          Time Spent: 3h 20m
>  Remaining Estimate: 0h
>
> Calcite has a collections of functions in 
> {{org.apache.calcite.runtime.SqlFunctions}}. It sounds a good source to 
> leverage when adding operators as {{BeamSqlExpression}}. 
> [~xumingming] [~app-tarush], any comments?



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to